You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@rya.apache.org by ejwhite922 <gi...@git.apache.org> on 2016/11/28 21:58:49 UTC

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

GitHub user ejwhite922 opened a pull request:

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

    RYA-119 Added MongoDB Column Visibility (called Document Visibility).

    ## Description
    This adds a new field to each document called documentVisibility which
    uses a boolean expression to determine if the user can access the document.
    The boolean expression is in Disjunctive Normal Formal so that the expression's grouping is simplified enough that MongoDB can run Set operations on it to determine if
    the document is viewable.  The expression is stored as an array in MongoDB.
    
    The classes in the package "org.apache.rya.mongodb.document.visibility" are pretty much a copy of the classes that are used by "org.apache.accumulo.core.security.ColumnVisibility" so we don't need to have an Accumulo dependency in a MongoDB component.  If anyone is against that then let me know.
    
    ### Tests
    Unit Tests
    
    ### Links
    [Jira](https://issues.apache.org/jira/browse/RYA-119)
    
    ### Checklist
    - [x] Code Review
    - [ ] Squash Commits
    
    #### People To Review
    @pujav65 
    @amihalik 
    @isper3at 
    @DLotts

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

    $ git pull https://github.com/ejwhite922/incubator-rya RYA-119_MongoDBColumnVisibility

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

    https://github.com/apache/incubator-rya/pull/124.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 #124
    
----
commit 2c14d78a82562ea556fdd7e652185605feeb5544
Author: ejwhite922 <er...@sparta.com>
Date:   2016-11-02T13:51:32Z

    RYA-119 Added MongoDB Column Visibility (called Document Visibility).
    This adds a new field to each document called documentVisibility which
    uses a boolean expression to determine if the user can access the document.
    The boolean expression is in Disjunctive Normal Formal so that it can be
    stored as an array that MongoDB can run Set operations on to determine if
    the document is viewable.

----


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90357217
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/iter/RyaStatementBindingSetCursorIterator.java ---
    @@ -22,104 +44,104 @@
     
     import info.aduna.iteration.CloseableIteration;
     
    -import java.util.Collection;
    -import java.util.Iterator;
    -import java.util.Map.Entry;
    -
    -import org.apache.rya.api.RdfCloudTripleStoreUtils;
    -import org.apache.rya.api.domain.RyaStatement;
    -import org.apache.rya.api.persist.RyaDAOException;
    -import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    -
    -import org.openrdf.query.BindingSet;
    -
    -import com.google.common.collect.Multimap;
    -import com.mongodb.DBCollection;
    -import com.mongodb.DBCursor;
    -import com.mongodb.DBObject;
    -
     public class RyaStatementBindingSetCursorIterator implements CloseableIteration<Entry<RyaStatement, BindingSet>, RyaDAOException> {
    -
    -	private DBCollection coll;
    -	private Multimap<DBObject, BindingSet> rangeMap;
    -	private Iterator<DBObject> queryIterator;
    -	private Long maxResults;
    -	private DBCursor resultCursor;
    -	private RyaStatement currentStatement;
    -	private Collection<BindingSet> currentBindingSetCollection;
    -	private Iterator<BindingSet> currentBindingSetIterator;
    -	private MongoDBStorageStrategy strategy;
    -
    -	public RyaStatementBindingSetCursorIterator(DBCollection coll,
    -			Multimap<DBObject, BindingSet> rangeMap, MongoDBStorageStrategy strategy) {
    -		this.coll = coll;
    -		this.rangeMap = rangeMap;
    -		this.queryIterator = rangeMap.keySet().iterator();
    -		this.strategy = strategy;
    -	}
    -
    -	@Override
    -	public boolean hasNext() {
    -		if (!currentBindingSetIteratorIsValid()) {
    -			findNextResult();
    -		}
    -		return currentBindingSetIteratorIsValid();
    -	}
    -
    -	@Override
    -	public Entry<RyaStatement, BindingSet> next() {
    -		if (!currentBindingSetIteratorIsValid()) {
    -			findNextResult();
    -		}
    -		if (currentBindingSetIteratorIsValid()) {
    -			BindingSet currentBindingSet = currentBindingSetIterator.next();
    -			return new RdfCloudTripleStoreUtils.CustomEntry<RyaStatement, BindingSet>(currentStatement, currentBindingSet);
    -		}
    -		return null;
    -	}
    -	
    -	private boolean currentBindingSetIteratorIsValid() {
    -		return (currentBindingSetIterator != null) && currentBindingSetIterator.hasNext();
    -	}
    -
    -	private void findNextResult() {
    -		if (!currentResultCursorIsValid()) {
    -			findNextValidResultCursor();
    -		}
    -		if (currentResultCursorIsValid()) {
    -			// convert to Rya Statement
    -			DBObject queryResult = resultCursor.next();
    -			currentStatement = strategy.deserializeDBObject(queryResult);
    -			currentBindingSetIterator = currentBindingSetCollection.iterator();
    -		}
    -	}
    -
    -	private void findNextValidResultCursor() {
    -		while (queryIterator.hasNext()){
    -			DBObject currentQuery = queryIterator.next();
    -			resultCursor = coll.find(currentQuery);
    -			currentBindingSetCollection = rangeMap.get(currentQuery);
    -			if (resultCursor.hasNext()) return;
    -		}
    -	}
    -	
    -	private boolean currentResultCursorIsValid() {
    -		return (resultCursor != null) && resultCursor.hasNext();
    -	}
    -
    -
    -	public void setMaxResults(Long maxResults) {
    -		this.maxResults = maxResults;
    -	}
    -
    -	@Override
    -	public void close() throws RyaDAOException {
    -		// TODO don't know what to do here
    -	}
    -
    -	@Override
    -	public void remove() throws RyaDAOException {
    -		next();
    -	}
    +    private static final Logger log = Logger.getLogger(RyaStatementBindingSetCursorIterator.class);
    +
    +    private final DBCollection coll;
    +    private final Multimap<DBObject, BindingSet> rangeMap;
    +    private final Iterator<DBObject> queryIterator;
    +    private Long maxResults;
    +    private Iterator<DBObject> resultsIterator;
    +    private RyaStatement currentStatement;
    +    private Collection<BindingSet> currentBindingSetCollection;
    +    private Iterator<BindingSet> currentBindingSetIterator;
    +    private final MongoDBStorageStrategy<RyaStatement> strategy;
    +    private final Authorizations auths;
    +
    +    public RyaStatementBindingSetCursorIterator(final DBCollection coll,
    --- End diff --
    
    Done


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89904319
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    + */
    +public class Authorizations implements Iterable<byte[]>, Serializable, AuthorizationContainer {
    +
    +  private static final long serialVersionUID = 1L;
    +
    +  private final Set<ByteSequence> auths = new HashSet<ByteSequence>();
    +  private final List<byte[]> authsList = new ArrayList<byte[]>(); // sorted order
    +
    +  /**
    +   * An empty set of authorizations.
    +   */
    +  public static final Authorizations EMPTY = new Authorizations();
    +
    +  private static final boolean[] validAuthChars = new boolean[256];
    +
    +  /**
    +   * A special header string used when serializing instances of this class.
    +   *
    +   * @see #serialize()
    +   */
    +  public static final String HEADER = "!AUTH1:";
    +
    +  static {
    +    for (int i = 0; i < 256; i++) {
    +      validAuthChars[i] = false;
    +    }
    +
    +    for (int i = 'a'; i <= 'z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = 'A'; i <= 'Z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = '0'; i <= '9'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    validAuthChars['_'] = true;
    +    validAuthChars['-'] = true;
    +    validAuthChars[':'] = true;
    +    validAuthChars['.'] = true;
    +    validAuthChars['/'] = true;
    +  }
    +
    +  static final boolean isValidAuthChar(final byte b) {
    +    return validAuthChars[0xff & b];
    +  }
    +
    +  private void checkAuths() {
    +    final Set<ByteSequence> sortedAuths = new TreeSet<ByteSequence>(auths);
    +
    +    for (final ByteSequence bs : sortedAuths) {
    +      if (bs.length() == 0) {
    +        throw new IllegalArgumentException("Empty authorization");
    +      }
    +
    +      authsList.add(bs.toArray());
    +    }
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a collection of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does
    +   * not verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          collection of authorizations, as strings encoded in UTF-8
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final Collection<byte[]> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    for (final byte[] auth : authorizations) {
    +		auths.add(new ArrayByteSequence(auth));
    +	}
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a list of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does not
    +   * verify that each encoded string is valid UTF-8.
    --- End diff --
    
    why can't we check encoding?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89903387
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/AuthorizationContainer.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.mongodb.document.visibility;
    +
    +/**
    + * An interface for classes that contain a collection of authorizations.
    + */
    +public interface AuthorizationContainer {
    +  /**
    +   * Checks whether this object contains the given authorization.
    +   *
    +   * @param auth
    +   *          authorization, as a string encoded in UTF-8
    +   * @return true if authorization is in this collection
    +   */
    +  boolean contains(ByteSequence auth);
    --- End diff --
    
    rename to isAuthorized(), or something along those lines.  makes more readable over contains()


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90343756
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    + */
    +public class Authorizations implements Iterable<byte[]>, Serializable, AuthorizationContainer {
    +
    +  private static final long serialVersionUID = 1L;
    +
    +  private final Set<ByteSequence> auths = new HashSet<ByteSequence>();
    +  private final List<byte[]> authsList = new ArrayList<byte[]>(); // sorted order
    +
    +  /**
    +   * An empty set of authorizations.
    +   */
    +  public static final Authorizations EMPTY = new Authorizations();
    +
    +  private static final boolean[] validAuthChars = new boolean[256];
    +
    +  /**
    +   * A special header string used when serializing instances of this class.
    +   *
    +   * @see #serialize()
    +   */
    +  public static final String HEADER = "!AUTH1:";
    +
    +  static {
    +    for (int i = 0; i < 256; i++) {
    +      validAuthChars[i] = false;
    +    }
    +
    +    for (int i = 'a'; i <= 'z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = 'A'; i <= 'Z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = '0'; i <= '9'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    validAuthChars['_'] = true;
    +    validAuthChars['-'] = true;
    +    validAuthChars[':'] = true;
    +    validAuthChars['.'] = true;
    +    validAuthChars['/'] = true;
    +  }
    +
    +  static final boolean isValidAuthChar(final byte b) {
    +    return validAuthChars[0xff & b];
    +  }
    +
    +  private void checkAuths() {
    +    final Set<ByteSequence> sortedAuths = new TreeSet<ByteSequence>(auths);
    +
    +    for (final ByteSequence bs : sortedAuths) {
    +      if (bs.length() == 0) {
    +        throw new IllegalArgumentException("Empty authorization");
    +      }
    +
    +      authsList.add(bs.toArray());
    +    }
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a collection of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does
    +   * not verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          collection of authorizations, as strings encoded in UTF-8
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final Collection<byte[]> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    for (final byte[] auth : authorizations) {
    +		auths.add(new ArrayByteSequence(auth));
    +	}
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a list of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does not
    +   * verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          list of authorizations, as strings encoded in UTF-8 and placed in buffers
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final List<ByteBuffer> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    for (final ByteBuffer buffer : authorizations) {
    +      auths.add(new ArrayByteSequence(ByteBufferUtil.toBytes(buffer)));
    +    }
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Constructs an authorizations object from a serialized form. This is NOT a constructor for a set of authorizations of size one. Warning: This method does
    +   * not verify that the encoded serialized form is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          a serialized authorizations string produced by {@link #getAuthorizationsArray()} or {@link #serialize()}, converted to UTF-8 bytes
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   */
    +  public Authorizations(final byte[] authorizations) {
    +
    +    ArgumentChecker.notNull(authorizations);
    +
    +    String authsString = new String(authorizations, UTF_8);
    +    if (authsString.startsWith(HEADER)) {
    +      // it's the new format
    +      authsString = authsString.substring(HEADER.length());
    +      if (authsString.length() > 0) {
    +        for (final String encAuth : authsString.split(",")) {
    +          final byte[] auth = Base64.decodeBase64(encAuth.getBytes(UTF_8));
    +          auths.add(new ArrayByteSequence(auth));
    +        }
    +        checkAuths();
    +      }
    +    } else {
    +      // it's the old format
    +      if (authorizations.length > 0) {
    +		setAuthorizations(authsString.split(","));
    --- End diff --
    
    same as above


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89905342
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    + */
    +public class Authorizations implements Iterable<byte[]>, Serializable, AuthorizationContainer {
    +
    +  private static final long serialVersionUID = 1L;
    +
    +  private final Set<ByteSequence> auths = new HashSet<ByteSequence>();
    +  private final List<byte[]> authsList = new ArrayList<byte[]>(); // sorted order
    +
    +  /**
    +   * An empty set of authorizations.
    +   */
    +  public static final Authorizations EMPTY = new Authorizations();
    +
    +  private static final boolean[] validAuthChars = new boolean[256];
    +
    +  /**
    +   * A special header string used when serializing instances of this class.
    +   *
    +   * @see #serialize()
    +   */
    +  public static final String HEADER = "!AUTH1:";
    +
    +  static {
    +    for (int i = 0; i < 256; i++) {
    +      validAuthChars[i] = false;
    +    }
    +
    +    for (int i = 'a'; i <= 'z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = 'A'; i <= 'Z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = '0'; i <= '9'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    validAuthChars['_'] = true;
    +    validAuthChars['-'] = true;
    +    validAuthChars[':'] = true;
    +    validAuthChars['.'] = true;
    +    validAuthChars['/'] = true;
    +  }
    +
    +  static final boolean isValidAuthChar(final byte b) {
    +    return validAuthChars[0xff & b];
    +  }
    +
    +  private void checkAuths() {
    +    final Set<ByteSequence> sortedAuths = new TreeSet<ByteSequence>(auths);
    +
    +    for (final ByteSequence bs : sortedAuths) {
    +      if (bs.length() == 0) {
    +        throw new IllegalArgumentException("Empty authorization");
    +      }
    +
    +      authsList.add(bs.toArray());
    +    }
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a collection of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does
    +   * not verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          collection of authorizations, as strings encoded in UTF-8
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final Collection<byte[]> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    for (final byte[] auth : authorizations) {
    +		auths.add(new ArrayByteSequence(auth));
    +	}
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a list of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does not
    +   * verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          list of authorizations, as strings encoded in UTF-8 and placed in buffers
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final List<ByteBuffer> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    for (final ByteBuffer buffer : authorizations) {
    +      auths.add(new ArrayByteSequence(ByteBufferUtil.toBytes(buffer)));
    +    }
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Constructs an authorizations object from a serialized form. This is NOT a constructor for a set of authorizations of size one. Warning: This method does
    +   * not verify that the encoded serialized form is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          a serialized authorizations string produced by {@link #getAuthorizationsArray()} or {@link #serialize()}, converted to UTF-8 bytes
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   */
    +  public Authorizations(final byte[] authorizations) {
    +
    +    ArgumentChecker.notNull(authorizations);
    +
    +    String authsString = new String(authorizations, UTF_8);
    +    if (authsString.startsWith(HEADER)) {
    +      // it's the new format
    +      authsString = authsString.substring(HEADER.length());
    +      if (authsString.length() > 0) {
    +        for (final String encAuth : authsString.split(",")) {
    +          final byte[] auth = Base64.decodeBase64(encAuth.getBytes(UTF_8));
    +          auths.add(new ArrayByteSequence(auth));
    +        }
    +        checkAuths();
    +      }
    +    } else {
    +      // it's the old format
    +      if (authorizations.length > 0) {
    +		setAuthorizations(authsString.split(","));
    +	}
    +    }
    +  }
    +
    +  /**
    +   * Constructs an empty set of authorizations.
    +   *
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations() {}
    +
    +  /**
    +   * Constructs an authorizations object from a set of human-readable authorizations.
    +   *
    +   * @param authorizations
    +   *          array of authorizations
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   */
    +  public Authorizations(final String... authorizations) {
    +    setAuthorizations(authorizations);
    +  }
    +
    +  private void setAuthorizations(final String... authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    auths.clear();
    +    for (String str : authorizations) {
    +      str = str.trim();
    +      auths.add(new ArrayByteSequence(str.getBytes(UTF_8)));
    +    }
    +
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Returns a serialized form of these authorizations.
    +   *
    +   * @return serialized form of these authorizations, as a string encoded in UTF-8
    +   * @see #serialize()
    +   */
    +  public byte[] getAuthorizationsArray() {
    +    return serialize().getBytes(UTF_8);
    +  }
    +
    +  /**
    +   * Gets the authorizations in sorted order. The returned list is not modifiable.
    +   *
    +   * @return authorizations, each as a string encoded in UTF-8
    +   * @see #Authorizations(Collection)
    +   */
    +  public List<byte[]> getAuthorizations() {
    +    final ArrayList<byte[]> copy = new ArrayList<byte[]>(authsList.size());
    +    for (final byte[] auth : authsList) {
    +      final byte[] bytes = new byte[auth.length];
    +      System.arraycopy(auth, 0, bytes, 0, auth.length);
    +      copy.add(bytes);
    +    }
    +    return Collections.unmodifiableList(copy);
    +  }
    +
    +  /**
    +   * Gets the authorizations in sorted order. The returned list is not modifiable.
    +   *
    +   * @return authorizations, each as a string encoded in UTF-8 and within a buffer
    +   */
    +  public List<ByteBuffer> getAuthorizationsBB() {
    +    final ArrayList<ByteBuffer> copy = new ArrayList<ByteBuffer>(authsList.size());
    +    for (final byte[] auth : authsList) {
    +      final byte[] bytes = new byte[auth.length];
    +      System.arraycopy(auth, 0, bytes, 0, auth.length);
    +      copy.add(ByteBuffer.wrap(bytes));
    +    }
    +    return Collections.unmodifiableList(copy);
    +  }
    +
    +  /**
    +   * Gets the authorizations in sorted order. The returned list is not modifiable.
    +   *
    +   * @return authorizations, each as a string encoded in UTF-8
    +   */
    +  public List<String> getAuthorizationsStrings() {
    +    final List<String> copy = new ArrayList<>(authsList.size());
    +    for (final byte[] auth : authsList) {
    +      copy.add(new String(auth, Charsets.UTF_8));
    +    }
    +    return Collections.unmodifiableList(copy);
    +  }
    +
    +  /**
    +   * Gets the authorizations in sorted order.
    +   *
    +   * @return authorizations, each as a string encoded in UTF-8
    +   */
    +  public String[] getAuthorizationsStringArray() {
    +    return getAuthorizationsStrings().toArray(new String[0]);
    +  }
    +
    +  @Override
    +  public String toString() {
    +    final StringBuilder sb = new StringBuilder();
    +    String sep = "";
    +    for (final ByteSequence auth : auths) {
    +      sb.append(sep);
    +      sep = ",";
    +      sb.append(new String(auth.toArray(), UTF_8));
    +    }
    +
    +    return sb.toString();
    +  }
    +
    +  /**
    +   * Checks whether this object contains the given authorization.
    +   *
    +   * @param auth
    +   *          authorization, as a string encoded in UTF-8
    +   * @return true if authorization is in this collection
    +   */
    +  public boolean contains(final byte[] auth) {
    +    return auths.contains(new ArrayByteSequence(auth));
    +  }
    +
    +  /**
    +   * Checks whether this object contains the given authorization. Warning: This method does not verify that the encoded string is valid UTF-8.
    +   *
    +   * @param auth
    +   *          authorization, as a string encoded in UTF-8
    +   * @return true if authorization is in this collection
    +   */
    +  @Override
    +  public boolean contains(final ByteSequence auth) {
    +    return auths.contains(auth);
    +  }
    +
    +  /**
    +   * Checks whether this object contains the given authorization.
    +   *
    +   * @param auth
    +   *          authorization
    +   * @return true if authorization is in this collection
    +   */
    +  public boolean contains(final String auth) {
    +    return auths.contains(new ArrayByteSequence(auth));
    +  }
    +
    +  @Override
    +  public boolean equals(final Object o) {
    +    if (o == null) {
    +      return false;
    +    }
    +
    +    if (o instanceof Authorizations) {
    +      final Authorizations ao = (Authorizations) o;
    +
    +      return auths.equals(ao.auths);
    +    }
    +
    +    return false;
    +  }
    +
    +  @Override
    +  public int hashCode() {
    +    int result = 0;
    +    for (final ByteSequence b : auths) {
    +		result += b.hashCode();
    +	}
    +    return result;
    +  }
    +
    +  /**
    +   * Gets the size of this collection of authorizations.
    +   *
    +   * @return collection size
    +   */
    +  public int size() {
    +    return auths.size();
    +  }
    +
    +  /**
    +   * Checks if this collection of authorizations is empty.
    +   *
    +   * @return true if this collection contains no authorizations
    +   */
    +  public boolean isEmpty() {
    +    return auths.isEmpty();
    +  }
    +
    +  @Override
    +  public Iterator<byte[]> iterator() {
    +    return getAuthorizations().iterator();
    +  }
    +
    +  /**
    +   * Returns a serialized form of these authorizations. Convert the returned string to UTF-8 bytes to deserialize with {@link #Authorizations(byte[])}.
    +   *
    +   * @return serialized form of authorizations
    +   */
    +  public String serialize() {
    --- End diff --
    
    Override annotation?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89897743
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDbRdfConstants.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.mongodb;
    +
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
    +
    +/**
    + * Interface MongoDbRdfConstants.
    + */
    +public interface MongoDbRdfConstants {
    +    public static final Authorizations ALL_AUTHORIZATIONS = Authorizations.EMPTY;
    --- End diff --
    
    doc what each constant is and what its for


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89906381
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/DocumentVisibility.java ---
    @@ -0,0 +1,584 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static java.nio.charset.StandardCharsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.List;
    +import java.util.TreeSet;
    +
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.io.WritableComparator;
    +
    +/**
    + * Validate the document visibility is a valid expression and set the visibility for a Mutation. See {@link DocumentVisibility#DocumentVisibility(byte[])} for the
    + * definition of an expression.
    + *
    + * <p>
    + * The expression is a sequence of characters from the set [A-Za-z0-9_-.] along with the binary operators "&amp;" and "|" indicating that both operands are
    + * necessary, or the either is necessary. The following are valid expressions for visibility:
    + *
    + * <pre>
    + * A
    + * A|B
    + * (A|B)&amp;(C|D)
    + * orange|(red&amp;yellow)
    + * </pre>
    + *
    + * <p>
    + * The following are not valid expressions for visibility:
    + *
    + * <pre>
    + * A|B&amp;C
    + * A=B
    + * A|B|
    + * A&amp;|B
    + * ()
    + * )
    + * dog|!cat
    + * </pre>
    + *
    + * <p>
    + * In addition to the base set of visibilities, any character can be used in the expression if it is quoted. If the quoted term contains '&quot;' or '\', then
    + * escape the character with '\'. The {@link #quote(String)} method can be used to properly quote and escape terms automatically. The following is an example of
    + * a quoted term:
    + *
    + * <pre>
    + * &quot;A#C&quot; &amp; B
    + * </pre>
    + */
    +public class DocumentVisibility {
    +
    +  Node node = null;
    +  private byte[] expression;
    +
    +  /**
    +   * Accessor for the underlying byte string.
    +   *
    +   * @return byte array representation of a visibility expression
    +   */
    +  public byte[] getExpression() {
    +    return expression;
    +  }
    +
    +  /**
    +   * The node types in a parse tree for a visibility expression.
    +   */
    +  public static enum NodeType {
    +    EMPTY, TERM, OR, AND,
    +  }
    +
    +  /**
    +   * All empty nodes are equal and represent the same value.
    +   */
    +  private static final Node EMPTY_NODE = new Node("".getBytes(), NodeType.EMPTY, 0);
    +
    +  /**
    +   * A node in the parse tree for a visibility expression.
    +   */
    +  public static class Node {
    +    /**
    +     * An empty list of nodes.
    +     */
    +    public final static List<Node> EMPTY = Collections.emptyList();
    +    NodeType type;
    +    int start;
    +    int end;
    +    List<Node> children = EMPTY;
    +    byte[] expression;
    +
    +    public Node(final byte[] expression, final NodeType type, final int start) {
    +      this.type = type;
    +      this.start = start;
    +      this.end = start + 1;
    +      this.expression = expression;
    +    }
    +
    +    public Node(final byte[] expression, final int start, final int end) {
    +      this.type = NodeType.TERM;
    +      this.start = start;
    +      this.end = end;
    +      this.expression = expression;
    +    }
    +
    +    public void add(final Node child) {
    +      if (children == EMPTY) {
    +		children = new ArrayList<>();
    +	}
    +
    +      children.add(child);
    +    }
    +
    +    public NodeType getType() {
    +      return type;
    +    }
    +
    +    public List<Node> getChildren() {
    +      return children;
    +    }
    +
    +    public int getTermStart() {
    +      return start;
    +    }
    +
    +    public int getTermEnd() {
    +      return end;
    +    }
    +
    +    public byte[] getExpression() {
    +      return expression;
    +    }
    +
    +    public ByteSequence getTerm(final byte expression[]) {
    +      if (type != NodeType.TERM) {
    +		throw new RuntimeException();
    +	}
    +
    +      if (expression[start] == '"') {
    +        // its a quoted term
    +        final int qStart = start + 1;
    +        final int qEnd = end - 1;
    +
    +        return new ArrayByteSequence(expression, qStart, qEnd - qStart);
    +      }
    +      return new ArrayByteSequence(expression, start, end - start);
    +    }
    +  }
    +
    +  /**
    +   * A node comparator. Nodes sort according to node type, terms sort lexicographically. AND and OR nodes sort by number of children, or if the same by
    +   * corresponding children.
    +   */
    +  public static class NodeComparator implements Comparator<Node>, Serializable {
    +
    +    private static final long serialVersionUID = 1L;
    +    byte[] text;
    +
    +    /**
    +     * Creates a new comparator.
    +     *
    +     * @param text
    +     *          expression string, encoded in UTF-8
    +     */
    +    public NodeComparator(final byte[] text) {
    +      this.text = text;
    +    }
    +
    +    @Override
    +    public int compare(final Node a, final Node b) {
    +      int diff = a.type.ordinal() - b.type.ordinal();
    +      if (diff != 0) {
    +		return diff;
    +	}
    +      switch (a.type) {
    +        case EMPTY:
    +          return 0; // All empty nodes are the same
    +        case TERM:
    +          return WritableComparator.compareBytes(text, a.start, a.end - a.start, text, b.start, b.end - b.start);
    +        case OR:
    +        case AND:
    +          diff = a.children.size() - b.children.size();
    +          if (diff != 0) {
    +			return diff;
    +		}
    +          for (int i = 0; i < a.children.size(); i++) {
    +            diff = compare(a.children.get(i), b.children.get(i));
    +            if (diff != 0) {
    +				return diff;
    +			}
    +          }
    +      }
    +      return 0;
    +    }
    +  }
    +
    +  /*
    +   * Convience method that delegates to normalize with a new NodeComparator constructed using the supplied expression.
    +   */
    +  public static Node normalize(final Node root, final byte[] expression) {
    +    return normalize(root, expression, new NodeComparator(expression));
    +  }
    +
    +  // @formatter:off
    +  /*
    +   * Walks an expression's AST in order to:
    +   *  1) roll up expressions with the same operant (`a&(b&c) becomes a&b&c`)
    +   *  2) sorts labels lexicographically (permutations of `a&b&c` are re-ordered to appear as `a&b&c`)
    +   *  3) dedupes labels (`a&b&a` becomes `a&b`)
    +   */
    +  // @formatter:on
    +  public static Node normalize(final Node root, final byte[] expression, final NodeComparator comparator) {
    +    if (root.type != NodeType.TERM) {
    +      final TreeSet<Node> rolledUp = new TreeSet<>(comparator);
    +      final java.util.Iterator<Node> itr = root.children.iterator();
    +      while (itr.hasNext()) {
    +        final Node c = normalize(itr.next(), expression, comparator);
    +        if (c.type == root.type) {
    +          rolledUp.addAll(c.children);
    +          itr.remove();
    +        }
    +      }
    +      rolledUp.addAll(root.children);
    +      root.children.clear();
    +      root.children.addAll(rolledUp);
    +
    +      // need to promote a child if it's an only child
    +      if (root.children.size() == 1) {
    +        return root.children.get(0);
    +      }
    +    }
    +
    +    return root;
    +  }
    +
    +  /*
    +   * Walks an expression's AST and appends a string representation to a supplied StringBuilder. This method adds parens where necessary.
    +   */
    +  public static void stringify(final Node root, final byte[] expression, final StringBuilder out) {
    +    if (root.type == NodeType.TERM) {
    +      out.append(new String(expression, root.start, root.end - root.start, UTF_8));
    +    } else {
    +      String sep = "";
    +      for (final Node c : root.children) {
    +        out.append(sep);
    +        final boolean parens = (c.type != NodeType.TERM && root.type != c.type);
    +        if (parens) {
    +			out.append("(");
    +		}
    +        stringify(c, expression, out);
    +        if (parens) {
    +			out.append(")");
    +		}
    +        sep = root.type == NodeType.AND ? "&" : "|";
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Generates a byte[] that represents a normalized, but logically equivalent, form of this evaluator's expression.
    +   *
    +   * @return normalized expression in byte[] form
    +   */
    +  public byte[] flatten() {
    +    final Node normRoot = normalize(node, expression);
    +    final StringBuilder builder = new StringBuilder(expression.length);
    +    stringify(normRoot, expression, builder);
    +    return builder.toString().getBytes(UTF_8);
    +  }
    +
    +  private static class DocumentVisibilityParser {
    --- End diff --
    
    this needs to be heavily documented.  Its not very clear how this is getting parsed, from what to what


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90068005
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    + */
    +public class Authorizations implements Iterable<byte[]>, Serializable, AuthorizationContainer {
    +
    +  private static final long serialVersionUID = 1L;
    +
    +  private final Set<ByteSequence> auths = new HashSet<ByteSequence>();
    +  private final List<byte[]> authsList = new ArrayList<byte[]>(); // sorted order
    +
    +  /**
    +   * An empty set of authorizations.
    +   */
    +  public static final Authorizations EMPTY = new Authorizations();
    --- End diff --
    
    same as above.  The constant class had a empty document visibility.   It also referenced this constant to let be known it was for ALL_AUTHORIZATIONS like in Accumulo Rya.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r94593591
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRyaDAO.java ---
    @@ -146,15 +153,18 @@ public void destroy() throws RyaDAOException {
         public void add(final RyaStatement statement) throws RyaDAOException {
             // add it to the collection
             try {
    -            coll.insert(storageStrategy.serialize(statement));
    -            for(final RyaSecondaryIndexer index: secondaryIndexers) {
    -                index.storeStatement(statement);
    +            final boolean canAdd = DocumentVisibilityUtil.doesUserHaveDocumentAccess(auths, statement.getColumnVisibility());
    --- End diff --
    
    you should throw an exception if it can't be added -- either that or log the fact that there was an auth mismatch


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

[GitHub] incubator-rya issue #124: RYA-119 Added MongoDB Column Visibility (called Do...

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

    https://github.com/apache/incubator-rya/pull/124
  
    Nothing was done with the indexers.  But they use the MongoDB Rya DAO don't they?  So, they should be utilizing document visibility if they are and they're configured with authorizations.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r94812004
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/operators/aggregation/PipelineOperators.java ---
    @@ -0,0 +1,92 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.mongodb.document.operators.aggregation;
    +
    +import static org.apache.rya.mongodb.document.operators.query.ConditionalOperators.cond;
    +
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.BasicDBObjectBuilder;
    +
    +/**
    + * Utility methods for pipeline operators.
    + */
    +public final class PipelineOperators {
    --- End diff --
    
    For all the *Operators, I think it would be helpful to make the functions return the actual BasicDBObjectBuilder.  That way you can supply each function with a builder and it appends the operation to it, making it a lot easier to spin up more complicated queries.
    
    If you have time to do this, awesome, if not we can open a ticket


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90343484
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/ArgumentChecker.java ---
    @@ -0,0 +1,112 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +/**
    + * This class provides methods to check arguments of a variable number for null values, or anything else that might be required on a routine basis. These
    + * methods should be used for early failures as close to the end user as possible, so things do not fail later on the server side, when they are harder to
    + * debug.
    + *
    + * Methods are created for a specific number of arguments, due to the poor performance of array allocation for varargs methods.
    + */
    +public class ArgumentChecker {
    --- End diff --
    
    Removed class. It's taken from accumulo-core dependency now


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89909627
  
    --- Diff: dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOTest.java ---
    @@ -28,102 +30,279 @@
     import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
     import org.apache.rya.api.domain.RyaURI;
     import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.api.persist.query.RyaQuery;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.bson.Document;
    +import org.calrissian.mango.collect.CloseableIterable;
     import org.junit.Before;
     import org.junit.Test;
     
    -import com.mongodb.DB;
    -import com.mongodb.DBCollection;
     import com.mongodb.MongoException;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.MongoDatabase;
     
     public class MongoDBRyaDAOTest extends MongoRyaTestBase {
     
    -	private MongoDBRyaDAO dao;
    -	private MongoDBRdfConfiguration configuration;
    +    private MongoDBRyaDAO dao;
    +    private MongoDBRdfConfiguration configuration;
     
    -	@Before
    -	public void setUp() throws IOException, RyaDAOException{
    -		final Configuration conf = new Configuration();
    +    @Before
    +    public void setUp() throws IOException, RyaDAOException{
    +        final Configuration conf = new Configuration();
             conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, "test");
             conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
             conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya_");
             configuration = new MongoDBRdfConfiguration(conf);
             final int port = mongoClient.getServerAddressList().get(0).getPort();
             configuration.set(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT, ""+port);
    -		dao = new MongoDBRyaDAO(configuration, mongoClient);
    -	}
    +        dao = new MongoDBRyaDAO(configuration, mongoClient);
    +    }
     
     
    -	@Test
    -	public void testDeleteWildcard() throws RyaDAOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		dao.delete(builder.build(), configuration);
    -	}
    +    @Test
    +    public void testDeleteWildcard() throws RyaDAOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        dao.delete(builder.build(), configuration);
    +    }
     
     
    -	@Test
    -	public void testAdd() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    +    @Test
    +    public void testAdd() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(builder.build());
    +        dao.add(builder.build());
     
             assertEquals(coll.count(),1);
     
    -	}
    +    }
     
    -	@Test
    -	public void testDelete() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    -		final RyaStatement statement = builder.build();
    +    @Test
    +    public void testDelete() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
    +        final RyaStatement statement = builder.build();
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(statement);
    +        dao.add(statement);
     
             assertEquals(coll.count(),1);
     
             dao.delete(statement, configuration);
     
             assertEquals(coll.count(),0);
     
    -	}
    +    }
     
    -	@Test
    -	public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    -		builder.setContext(new RyaURI("http://context.com"));
    -		final RyaStatement statement = builder.build();
    +    @Test
    +    public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
    +        builder.setContext(new RyaURI("http://context.com"));
    +        final RyaStatement statement = builder.build();
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(statement);
    +        dao.add(statement);
     
             assertEquals(coll.count(),1);
     
    -		final RyaStatementBuilder builder2 = new RyaStatementBuilder();
    -		builder2.setPredicate(new RyaURI("http://temp.com"));
    -		builder2.setObject(new RyaURI("http://object.com"));
    -		builder2.setContext(new RyaURI("http://context3.com"));
    -		final RyaStatement query = builder2.build();
    +        final RyaStatementBuilder builder2 = new RyaStatementBuilder();
    +        builder2.setPredicate(new RyaURI("http://temp.com"));
    +        builder2.setObject(new RyaURI("http://object.com"));
    +        builder2.setContext(new RyaURI("http://context3.com"));
    +        final RyaStatement query = builder2.build();
     
             dao.delete(query, configuration);
     
             assertEquals(coll.count(),1);
     
    -	}
    +    }
     
    +    @Test
    +    public void testVisibility() throws RyaDAOException, MongoException, IOException {
    +        // Doc requires "A" and user has "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A", new Authorizations("B")));
    +
    +        // Doc requires "A" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A")));
    +
    +        // Doc requires "A" and "B" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or "B" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and "B" and user has "A" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A&B", new Authorizations("A")));
    +
    +        // Doc requires "A" and "B" and "C" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A&B&C", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and "B" and "C" and user has "A" and "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A&B&C", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and "B" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" or "B" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A")));
    +
    +        // Doc requires "A" or "B" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A", Authorizations.EMPTY));
    +
    +        // Doc requires "A" and "B" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A&B", Authorizations.EMPTY));
    +
    +        // Doc requires "A" or "B" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A|B", Authorizations.EMPTY));
    --- End diff --
    
    this seems wrong....if Doc requries A or B, shouldn't this be false?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r94828180
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/operators/aggregation/PipelineOperators.java ---
    @@ -0,0 +1,92 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.mongodb.document.operators.aggregation;
    +
    +import static org.apache.rya.mongodb.document.operators.query.ConditionalOperators.cond;
    +
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.BasicDBObjectBuilder;
    +
    +/**
    + * Utility methods for pipeline operators.
    + */
    +public final class PipelineOperators {
    --- End diff --
    
    Added this in.  The classes now provide 2 methods for each expression (one returns the builder the other returns the DBObject so either one can be used)


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89908511
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/iter/AggregationUtil.java ---
    @@ -0,0 +1,322 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.mongodb.iter;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.LinkedList;
    +import java.util.List;
    +
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +
    +import com.google.common.collect.Lists;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +
    +/**
    + * Utility methods for MongoDB aggregation.
    + */
    +public final class AggregationUtil {
    +    /**
    +     * Private constructor to prevent instantiation.
    +     */
    +    private AggregationUtil() {
    +    }
    +
    +    /**
    +     * Creates a MongoDB $redact aggregation pipeline that only include
    +     * documents whose document visibility match the provided authorizations.
    +     * All other documents are excluded.
    +     * @param authorizations the {@link Authorization}s to include in the
    +     * $redact. Only documents that match the authorizations will be returned.
    +     * @return the {@link List} of {@link DBObject}s that represents the $redact
    +     * aggregation pipeline.
    +     */
    +    public static List<DBObject> createRedactPipeline(final Authorizations authorizations) {
    +        if (MongoDbRdfConstants.ALL_AUTHORIZATIONS.equals(authorizations)) {
    +            return Lists.newArrayList();
    +        }
    +        final List<String> authAndList = authorizations.getAuthorizationsStrings();
    +
    +        // Generate all combinations of the authorization strings without repetition.
    +        final List<List<String>> authOrList = createCombinations(authorizations.getAuthorizationsStrings());
    +
    +        final String documentVisibilityField = "$" + SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +
    +        final BasicDBObject setIsSubset =
    +            setIsSubsetNullSafe(
    +                documentVisibilityField,
    +                authAndList.toArray()
    +            );
    +
    +        final BasicDBObject setIntersectionExists =
    +            gt(
    +                size(
    +                    setIntersection(
    +                        documentVisibilityField,
    +                        authOrList.toArray()
    +                    )
    +                ),
    +                0
    +            );
    +
    +        final BasicDBObject orExpression = or(setIsSubset, setIntersectionExists);
    +
    +        final List<DBObject> pipeline = new ArrayList<>();
    +        pipeline.add(
    +            redact(
    +               orExpression,
    +               "$$DESCEND",
    +               "$$PRUNE"
    +            )
    +        );
    +
    +        return pipeline;
    +    }
    +
    +    /**
    +     * Creates all combinations of the values that are of the size of value
    +     * array or smaller without repetition.
    +     * @param values the {@link List} of values to create combinations from.
    +     * @return the {@link List} of combinations.
    +     */
    +    public static <T> List<List<T>> createCombinations(final List<T> values) {
    +        final List<List<T>> allCombinations = new ArrayList<>();
    +        for (int i = 1; i <= values.size(); i++) {
    +            allCombinations.addAll(createCombinations(values, i));
    +        }
    +        return allCombinations;
    +    }
    +
    +    /**
    +     * Creates all combinations of the values that are of the specified size
    +     * without repetition.
    +     * @param values the {@link List} of values to create combinations from.
    +     * @param size the size of the combinations.
    +     * @return the {@link List} of combinations.
    +     */
    +    public static <T> List<List<T>> createCombinations(final List<T> values, final int size) {
    +        if (0 == size) {
    +            return Collections.singletonList(Collections.<T> emptyList());
    +        }
    +
    +        if (values.isEmpty()) {
    +            return Collections.emptyList();
    +        }
    +
    +        final List<List<T>> combination = new LinkedList<List<T>>();
    +
    +        final T actual = values.iterator().next();
    +
    +        final List<T> subSet = new LinkedList<T>(values);
    +        subSet.remove(actual);
    +
    +        final List<List<T>> subSetCombination = createCombinations(subSet, size - 1);
    +
    +        for (final List<T> set : subSetCombination) {
    +            final List<T> newSet = new LinkedList<T>(set);
    +            newSet.add(0, actual);
    +            combination.add(newSet);
    +        }
    +
    +        combination.addAll(createCombinations(subSet, size));
    +
    +        return combination;
    +    }
    +
    +    /**
    +     * Creates an "if-then-else" MongoDB expression.
    +     * @param ifStatement the "if" statement {@link BasicDBObject}.
    +     * @param thenResult the {@link Object} to return when the
    +     * {@code ifStatement} is {@code true}.
    +     * @param elseResult the {@link Object} to return when the
    +     * {@code ifStatement} is {@code false}.
    +     * @return the "if" expression {@link BasicDBObject}.
    +     */
    +    public static BasicDBObject ifThenElse(final BasicDBObject ifStatement, final Object thenResult, final Object elseResult) {
    --- End diff --
    
    this doesn't seem aggregation specific, why is this needed?


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

[GitHub] incubator-rya issue #124: RYA-119 Added MongoDB Column Visibility (called Do...

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

    https://github.com/apache/incubator-rya/pull/124
  
    asfbot build


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90067343
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/ArgumentChecker.java ---
    @@ -0,0 +1,112 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +/**
    + * This class provides methods to check arguments of a variable number for null values, or anything else that might be required on a routine basis. These
    + * methods should be used for early failures as close to the end user as possible, so things do not fail later on the server side, when they are harder to
    + * debug.
    + *
    + * Methods are created for a specific number of arguments, due to the poor performance of array allocation for varargs methods.
    + */
    +public class ArgumentChecker {
    --- End diff --
    
    This is a straight copy from accumulo-core so we wouldn't need to add that dependency for mongodb components.  All the classes in the org.apache.rya.mongodb.document.visibility package are from accumulo-core and I didn't really modify any of them beside the ColumnVisibility class.
    
    I could add the accumulo-core dependency and we'd have less classes to maintain.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89903996
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    + */
    +public class Authorizations implements Iterable<byte[]>, Serializable, AuthorizationContainer {
    +
    +  private static final long serialVersionUID = 1L;
    +
    +  private final Set<ByteSequence> auths = new HashSet<ByteSequence>();
    +  private final List<byte[]> authsList = new ArrayList<byte[]>(); // sorted order
    +
    +  /**
    +   * An empty set of authorizations.
    +   */
    +  public static final Authorizations EMPTY = new Authorizations();
    +
    +  private static final boolean[] validAuthChars = new boolean[256];
    +
    +  /**
    +   * A special header string used when serializing instances of this class.
    +   *
    +   * @see #serialize()
    +   */
    +  public static final String HEADER = "!AUTH1:";
    +
    +  static {
    +    for (int i = 0; i < 256; i++) {
    +      validAuthChars[i] = false;
    +    }
    +
    +    for (int i = 'a'; i <= 'z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = 'A'; i <= 'Z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = '0'; i <= '9'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    validAuthChars['_'] = true;
    +    validAuthChars['-'] = true;
    +    validAuthChars[':'] = true;
    +    validAuthChars['.'] = true;
    +    validAuthChars['/'] = true;
    +  }
    +
    +  static final boolean isValidAuthChar(final byte b) {
    +    return validAuthChars[0xff & b];
    +  }
    +
    +  private void checkAuths() {
    +    final Set<ByteSequence> sortedAuths = new TreeSet<ByteSequence>(auths);
    +
    +    for (final ByteSequence bs : sortedAuths) {
    +      if (bs.length() == 0) {
    +        throw new IllegalArgumentException("Empty authorization");
    +      }
    +
    +      authsList.add(bs.toArray());
    +    }
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a collection of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does
    +   * not verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          collection of authorizations, as strings encoded in UTF-8
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final Collection<byte[]> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    --- End diff --
    
    why not use Preconditions?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90343654
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/ArrayByteSequence.java ---
    @@ -0,0 +1,126 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +
    +public class ArrayByteSequence extends ByteSequence implements Serializable {
    --- End diff --
    
    Removed class. It's taken from accumulo-core dependency now.  All class from org.apache.rya.mongodb.document.visibility now uses the accumulo-core classes they were copied from.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89909207
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/iter/RyaStatementCursorIterator.java ---
    @@ -22,83 +40,83 @@
     
     import info.aduna.iteration.CloseableIteration;
     
    -import java.util.Iterator;
    -import java.util.Map.Entry;
    -import java.util.Set;
    -
    -import org.apache.rya.api.RdfCloudTripleStoreUtils;
    -import org.apache.rya.api.domain.RyaStatement;
    -import org.apache.rya.api.persist.RyaDAOException;
    -import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    -
    -import org.calrissian.mango.collect.CloseableIterable;
    -import org.openrdf.query.BindingSet;
    -
    -import com.mongodb.DBCollection;
    -import com.mongodb.DBCursor;
    -import com.mongodb.DBObject;
    -
     public class RyaStatementCursorIterator implements CloseableIteration<RyaStatement, RyaDAOException> {
    -
    -	private DBCollection coll;
    -	private Iterator<DBObject> queryIterator;
    -	private DBCursor currentCursor;
    -	private MongoDBStorageStrategy strategy;
    -	private Long maxResults;
    -
    -	public RyaStatementCursorIterator(DBCollection coll, Set<DBObject> queries, MongoDBStorageStrategy strategy) {
    -		this.coll = coll;
    -		this.queryIterator = queries.iterator();
    -		this.strategy = strategy;
    -	}
    -
    -	@Override
    -	public boolean hasNext() {
    -		if (!currentCursorIsValid()) {
    -			findNextValidCursor();
    -		}
    -		return currentCursorIsValid();
    -	}
    -
    -	@Override
    -	public RyaStatement next() {
    -		if (!currentCursorIsValid()) {
    -			findNextValidCursor();
    -		}
    -		if (currentCursorIsValid()) {
    -			// convert to Rya Statement
    -			DBObject queryResult = currentCursor.next();
    -			RyaStatement statement = strategy.deserializeDBObject(queryResult);
    -			return statement;
    -		}
    -		return null;
    -	}
    -	
    -	private void findNextValidCursor() {
    -		while (queryIterator.hasNext()){
    -			DBObject currentQuery = queryIterator.next();
    -			currentCursor = coll.find(currentQuery);
    -			if (currentCursor.hasNext()) break;
    -		}
    -	}
    -	
    -	private boolean currentCursorIsValid() {
    -		return (currentCursor != null) && currentCursor.hasNext();
    -	}
    -
    -
    -	public void setMaxResults(Long maxResults) {
    -		this.maxResults = maxResults;
    -	}
    -
    -	@Override
    -	public void close() throws RyaDAOException {
    -		// TODO don't know what to do here
    -	}
    -
    -	@Override
    -	public void remove() throws RyaDAOException {
    -		next();
    -	}
    -
    +    private static final Logger log = Logger.getLogger(RyaStatementCursorIterator.class);
    +
    +    private final DBCollection coll;
    +    private final Iterator<DBObject> queryIterator;
    +    private Iterator<DBObject> resultsIterator;
    +    private final MongoDBStorageStrategy<RyaStatement> strategy;
    +    private Long maxResults;
    +    private final Authorizations auths;
    +
    +    public RyaStatementCursorIterator(final DBCollection coll, final Set<DBObject> queries, final MongoDBStorageStrategy<RyaStatement> strategy, final MongoDBRdfConfiguration conf) {
    --- End diff --
    
    same, don't pass the config when all you need is auths


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89904345
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    + */
    +public class Authorizations implements Iterable<byte[]>, Serializable, AuthorizationContainer {
    +
    +  private static final long serialVersionUID = 1L;
    +
    +  private final Set<ByteSequence> auths = new HashSet<ByteSequence>();
    +  private final List<byte[]> authsList = new ArrayList<byte[]>(); // sorted order
    +
    +  /**
    +   * An empty set of authorizations.
    +   */
    +  public static final Authorizations EMPTY = new Authorizations();
    +
    +  private static final boolean[] validAuthChars = new boolean[256];
    +
    +  /**
    +   * A special header string used when serializing instances of this class.
    +   *
    +   * @see #serialize()
    +   */
    +  public static final String HEADER = "!AUTH1:";
    +
    +  static {
    +    for (int i = 0; i < 256; i++) {
    +      validAuthChars[i] = false;
    +    }
    +
    +    for (int i = 'a'; i <= 'z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = 'A'; i <= 'Z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = '0'; i <= '9'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    validAuthChars['_'] = true;
    +    validAuthChars['-'] = true;
    +    validAuthChars[':'] = true;
    +    validAuthChars['.'] = true;
    +    validAuthChars['/'] = true;
    +  }
    +
    +  static final boolean isValidAuthChar(final byte b) {
    +    return validAuthChars[0xff & b];
    +  }
    +
    +  private void checkAuths() {
    +    final Set<ByteSequence> sortedAuths = new TreeSet<ByteSequence>(auths);
    +
    +    for (final ByteSequence bs : sortedAuths) {
    +      if (bs.length() == 0) {
    +        throw new IllegalArgumentException("Empty authorization");
    +      }
    +
    +      authsList.add(bs.toArray());
    +    }
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a collection of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does
    +   * not verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          collection of authorizations, as strings encoded in UTF-8
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final Collection<byte[]> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    for (final byte[] auth : authorizations) {
    +		auths.add(new ArrayByteSequence(auth));
    +	}
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a list of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does not
    +   * verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          list of authorizations, as strings encoded in UTF-8 and placed in buffers
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final List<ByteBuffer> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    for (final ByteBuffer buffer : authorizations) {
    +      auths.add(new ArrayByteSequence(ByteBufferUtil.toBytes(buffer)));
    +    }
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Constructs an authorizations object from a serialized form. This is NOT a constructor for a set of authorizations of size one. Warning: This method does
    +   * not verify that the encoded serialized form is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          a serialized authorizations string produced by {@link #getAuthorizationsArray()} or {@link #serialize()}, converted to UTF-8 bytes
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   */
    +  public Authorizations(final byte[] authorizations) {
    +
    +    ArgumentChecker.notNull(authorizations);
    +
    +    String authsString = new String(authorizations, UTF_8);
    +    if (authsString.startsWith(HEADER)) {
    +      // it's the new format
    +      authsString = authsString.substring(HEADER.length());
    +      if (authsString.length() > 0) {
    +        for (final String encAuth : authsString.split(",")) {
    +          final byte[] auth = Base64.decodeBase64(encAuth.getBytes(UTF_8));
    +          auths.add(new ArrayByteSequence(auth));
    +        }
    +        checkAuths();
    +      }
    +    } else {
    +      // it's the old format
    +      if (authorizations.length > 0) {
    +		setAuthorizations(authsString.split(","));
    --- End diff --
    
    odd spacing


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89904761
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    + */
    +public class Authorizations implements Iterable<byte[]>, Serializable, AuthorizationContainer {
    +
    +  private static final long serialVersionUID = 1L;
    +
    +  private final Set<ByteSequence> auths = new HashSet<ByteSequence>();
    +  private final List<byte[]> authsList = new ArrayList<byte[]>(); // sorted order
    +
    +  /**
    +   * An empty set of authorizations.
    +   */
    +  public static final Authorizations EMPTY = new Authorizations();
    +
    +  private static final boolean[] validAuthChars = new boolean[256];
    +
    +  /**
    +   * A special header string used when serializing instances of this class.
    +   *
    +   * @see #serialize()
    +   */
    +  public static final String HEADER = "!AUTH1:";
    +
    +  static {
    +    for (int i = 0; i < 256; i++) {
    +      validAuthChars[i] = false;
    +    }
    +
    +    for (int i = 'a'; i <= 'z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = 'A'; i <= 'Z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = '0'; i <= '9'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    validAuthChars['_'] = true;
    +    validAuthChars['-'] = true;
    +    validAuthChars[':'] = true;
    +    validAuthChars['.'] = true;
    +    validAuthChars['/'] = true;
    +  }
    +
    +  static final boolean isValidAuthChar(final byte b) {
    +    return validAuthChars[0xff & b];
    +  }
    +
    +  private void checkAuths() {
    +    final Set<ByteSequence> sortedAuths = new TreeSet<ByteSequence>(auths);
    +
    +    for (final ByteSequence bs : sortedAuths) {
    +      if (bs.length() == 0) {
    +        throw new IllegalArgumentException("Empty authorization");
    +      }
    +
    +      authsList.add(bs.toArray());
    +    }
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a collection of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does
    +   * not verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          collection of authorizations, as strings encoded in UTF-8
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final Collection<byte[]> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    for (final byte[] auth : authorizations) {
    +		auths.add(new ArrayByteSequence(auth));
    +	}
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a list of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does not
    +   * verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          list of authorizations, as strings encoded in UTF-8 and placed in buffers
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final List<ByteBuffer> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    for (final ByteBuffer buffer : authorizations) {
    +      auths.add(new ArrayByteSequence(ByteBufferUtil.toBytes(buffer)));
    +    }
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Constructs an authorizations object from a serialized form. This is NOT a constructor for a set of authorizations of size one. Warning: This method does
    +   * not verify that the encoded serialized form is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          a serialized authorizations string produced by {@link #getAuthorizationsArray()} or {@link #serialize()}, converted to UTF-8 bytes
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   */
    +  public Authorizations(final byte[] authorizations) {
    +
    +    ArgumentChecker.notNull(authorizations);
    +
    +    String authsString = new String(authorizations, UTF_8);
    +    if (authsString.startsWith(HEADER)) {
    +      // it's the new format
    +      authsString = authsString.substring(HEADER.length());
    +      if (authsString.length() > 0) {
    +        for (final String encAuth : authsString.split(",")) {
    +          final byte[] auth = Base64.decodeBase64(encAuth.getBytes(UTF_8));
    +          auths.add(new ArrayByteSequence(auth));
    +        }
    +        checkAuths();
    +      }
    +    } else {
    +      // it's the old format
    +      if (authorizations.length > 0) {
    +		setAuthorizations(authsString.split(","));
    +	}
    +    }
    +  }
    +
    +  /**
    +   * Constructs an empty set of authorizations.
    +   *
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations() {}
    +
    +  /**
    +   * Constructs an authorizations object from a set of human-readable authorizations.
    +   *
    +   * @param authorizations
    +   *          array of authorizations
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   */
    +  public Authorizations(final String... authorizations) {
    +    setAuthorizations(authorizations);
    +  }
    +
    +  private void setAuthorizations(final String... authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    auths.clear();
    +    for (String str : authorizations) {
    +      str = str.trim();
    +      auths.add(new ArrayByteSequence(str.getBytes(UTF_8)));
    +    }
    +
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Returns a serialized form of these authorizations.
    --- End diff --
    
    if this is the serialized form, put it in the Serialize.serialize()


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89905766
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/ByteBufferUtil.java ---
    @@ -0,0 +1,84 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.List;
    +
    +import org.apache.hadoop.io.Text;
    +
    +public class ByteBufferUtil {
    --- End diff --
    
    doc


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90358101
  
    --- Diff: dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOTest.java ---
    @@ -28,102 +30,279 @@
     import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
     import org.apache.rya.api.domain.RyaURI;
     import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.api.persist.query.RyaQuery;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.bson.Document;
    +import org.calrissian.mango.collect.CloseableIterable;
     import org.junit.Before;
     import org.junit.Test;
     
    -import com.mongodb.DB;
    -import com.mongodb.DBCollection;
     import com.mongodb.MongoException;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.MongoDatabase;
     
     public class MongoDBRyaDAOTest extends MongoRyaTestBase {
     
    -	private MongoDBRyaDAO dao;
    -	private MongoDBRdfConfiguration configuration;
    +    private MongoDBRyaDAO dao;
    +    private MongoDBRdfConfiguration configuration;
     
    -	@Before
    -	public void setUp() throws IOException, RyaDAOException{
    -		final Configuration conf = new Configuration();
    +    @Before
    +    public void setUp() throws IOException, RyaDAOException{
    +        final Configuration conf = new Configuration();
             conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, "test");
             conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
             conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya_");
             configuration = new MongoDBRdfConfiguration(conf);
             final int port = mongoClient.getServerAddressList().get(0).getPort();
             configuration.set(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT, ""+port);
    -		dao = new MongoDBRyaDAO(configuration, mongoClient);
    -	}
    +        dao = new MongoDBRyaDAO(configuration, mongoClient);
    +    }
     
     
    -	@Test
    -	public void testDeleteWildcard() throws RyaDAOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		dao.delete(builder.build(), configuration);
    -	}
    +    @Test
    +    public void testDeleteWildcard() throws RyaDAOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        dao.delete(builder.build(), configuration);
    +    }
     
     
    -	@Test
    -	public void testAdd() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    +    @Test
    +    public void testAdd() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(builder.build());
    +        dao.add(builder.build());
     
             assertEquals(coll.count(),1);
     
    -	}
    +    }
     
    -	@Test
    -	public void testDelete() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    -		final RyaStatement statement = builder.build();
    +    @Test
    +    public void testDelete() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
    +        final RyaStatement statement = builder.build();
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(statement);
    +        dao.add(statement);
     
             assertEquals(coll.count(),1);
     
             dao.delete(statement, configuration);
     
             assertEquals(coll.count(),0);
     
    -	}
    +    }
     
    -	@Test
    -	public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    -		builder.setContext(new RyaURI("http://context.com"));
    -		final RyaStatement statement = builder.build();
    +    @Test
    +    public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
    +        builder.setContext(new RyaURI("http://context.com"));
    +        final RyaStatement statement = builder.build();
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(statement);
    +        dao.add(statement);
     
             assertEquals(coll.count(),1);
     
    -		final RyaStatementBuilder builder2 = new RyaStatementBuilder();
    -		builder2.setPredicate(new RyaURI("http://temp.com"));
    -		builder2.setObject(new RyaURI("http://object.com"));
    -		builder2.setContext(new RyaURI("http://context3.com"));
    -		final RyaStatement query = builder2.build();
    +        final RyaStatementBuilder builder2 = new RyaStatementBuilder();
    +        builder2.setPredicate(new RyaURI("http://temp.com"));
    +        builder2.setObject(new RyaURI("http://object.com"));
    +        builder2.setContext(new RyaURI("http://context3.com"));
    +        final RyaStatement query = builder2.build();
     
             dao.delete(query, configuration);
     
             assertEquals(coll.count(),1);
     
    -	}
    +    }
     
    +    @Test
    +    public void testVisibility() throws RyaDAOException, MongoException, IOException {
    +        // Doc requires "A" and user has "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A", new Authorizations("B")));
    +
    +        // Doc requires "A" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A")));
    +
    +        // Doc requires "A" and "B" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or "B" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and "B" and user has "A" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A&B", new Authorizations("A")));
    +
    +        // Doc requires "A" and "B" and "C" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A&B&C", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and "B" and "C" and user has "A" and "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A&B&C", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and "B" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" or "B" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A")));
    +
    +        // Doc requires "A" or "B" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A", Authorizations.EMPTY));
    +
    +        // Doc requires "A" and "B" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A&B", Authorizations.EMPTY));
    +
    +        // Doc requires "A" or "B" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A|B", Authorizations.EMPTY));
    +
    +        // Doc has no requirement and user has "" = User can view
    +        assertTrue(testVisibilityStatement("", Authorizations.EMPTY));
    +
    +        // Doc has no requirement and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("", new Authorizations("A")));
    +
    +        // Doc has no requirement and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or ("B" and "C") and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|(B&C)", new Authorizations("A")));
    +
    +        // Doc requires "A" or ("B" and "C") and user has "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|(B&C)", new Authorizations("B", "C")));
    +
    +        // Doc requires "A" and ("B" or "C") and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A&(B|C)", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and ("B" or "C") and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A&(B|C)", new Authorizations("A", "C")));
    +
    +        // Doc requires "(A|B)&(C|(D&E))" and user has "A" and "C" = User can view
    +        assertTrue(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("A", "C")));
    +
    +        // Doc requires "(A|B)&(C|(D&E))" and user has "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("B", "C")));
    +
    +        // Doc requires "(A|B)&(C|(D&E))" and user has "A" and "D" and "E" = User can view
    +        assertTrue(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("A", "D", "E")));
    +
    +        // Doc requires "(A|B)&(C|(D&E))" and user has "B" and "D" and "E" = User can view
    +        assertTrue(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("B", "D", "E")));
    +
    +        // Doc requires "(A|B)&(C|(D&E))" and user has "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("B")));
    +
    +        // Doc requires "A|(B&C&(D|E))" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|(B&C&(D|E))", new Authorizations("A")));
    +
    +        // Doc requires "A|(B&C&(D|E))" and user has "B" and "C" and "D" = User can view
    +        assertTrue(testVisibilityStatement("A|(B&C&(D|E))", new Authorizations("B", "C", "D")));
    +
    +        // Doc requires "A|(B&C&(D|E))" and user has "B" and "C" and "E" = User can view
    +        assertTrue(testVisibilityStatement("A|(B&C&(D|E))", new Authorizations("B", "C", "E")));
    +
    +        // Doc requires "A|(B&C&(D|E))" and user has "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A|(B&C&(D|E))", new Authorizations("B")));
    +
    +        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("A")));
    +
    +        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "E" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("E")));
    +
    +        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "E" and "F" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("E", "F")));
    +
    +        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "I" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("I")));
    +
    +        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "A" and "I" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("A", "I")));
    +
    +        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "E" and "F" and "G" and "H" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("E", "F", "G", "H")));
    +
    +        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "E" and "F" and "G" and "H" and "I" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("E", "F", "G", "H", "I")));
    +    }
    +
    +    /**
    +     * Generates a test statement with the provided document visibility to
    +     * determine if the specified user authorization can view the statement.
    +     * @param documentVisibility the document visibility boolean expression
    +     * string.
    +     * @param userAuthorizations the user authorization strings.
    +     * @return {@code true} if provided authorization could access the document
    +     * in the collection. {@code false} otherwise.
    +     * @throws RyaDAOException
    +     */
    +    private boolean testVisibilityStatement(final String documentVisibility, final Authorizations userAuthorizations) throws RyaDAOException {
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    --- End diff --
    
    Moved all of these to MongoDBRyaDAOIT


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90067506
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/ArrayByteSequence.java ---
    @@ -0,0 +1,126 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +
    +public class ArrayByteSequence extends ByteSequence implements Serializable {
    --- End diff --
    
    Copied from accumulo-core because ColumnVisibility uses it.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89909939
  
    --- Diff: dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOTest.java ---
    @@ -28,102 +30,279 @@
     import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
     import org.apache.rya.api.domain.RyaURI;
     import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.api.persist.query.RyaQuery;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.bson.Document;
    +import org.calrissian.mango.collect.CloseableIterable;
     import org.junit.Before;
     import org.junit.Test;
     
    -import com.mongodb.DB;
    -import com.mongodb.DBCollection;
     import com.mongodb.MongoException;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.MongoDatabase;
     
     public class MongoDBRyaDAOTest extends MongoRyaTestBase {
     
    -	private MongoDBRyaDAO dao;
    -	private MongoDBRdfConfiguration configuration;
    +    private MongoDBRyaDAO dao;
    +    private MongoDBRdfConfiguration configuration;
     
    -	@Before
    -	public void setUp() throws IOException, RyaDAOException{
    -		final Configuration conf = new Configuration();
    +    @Before
    +    public void setUp() throws IOException, RyaDAOException{
    +        final Configuration conf = new Configuration();
             conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, "test");
             conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
             conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya_");
             configuration = new MongoDBRdfConfiguration(conf);
             final int port = mongoClient.getServerAddressList().get(0).getPort();
             configuration.set(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT, ""+port);
    -		dao = new MongoDBRyaDAO(configuration, mongoClient);
    -	}
    +        dao = new MongoDBRyaDAO(configuration, mongoClient);
    +    }
     
     
    -	@Test
    -	public void testDeleteWildcard() throws RyaDAOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		dao.delete(builder.build(), configuration);
    -	}
    +    @Test
    +    public void testDeleteWildcard() throws RyaDAOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        dao.delete(builder.build(), configuration);
    +    }
     
     
    -	@Test
    -	public void testAdd() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    +    @Test
    +    public void testAdd() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(builder.build());
    +        dao.add(builder.build());
     
             assertEquals(coll.count(),1);
     
    -	}
    +    }
     
    -	@Test
    -	public void testDelete() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    -		final RyaStatement statement = builder.build();
    +    @Test
    +    public void testDelete() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
    +        final RyaStatement statement = builder.build();
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(statement);
    +        dao.add(statement);
     
             assertEquals(coll.count(),1);
     
             dao.delete(statement, configuration);
     
             assertEquals(coll.count(),0);
     
    -	}
    +    }
     
    -	@Test
    -	public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    -		builder.setContext(new RyaURI("http://context.com"));
    -		final RyaStatement statement = builder.build();
    +    @Test
    +    public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
    +        builder.setContext(new RyaURI("http://context.com"));
    +        final RyaStatement statement = builder.build();
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(statement);
    +        dao.add(statement);
     
             assertEquals(coll.count(),1);
     
    -		final RyaStatementBuilder builder2 = new RyaStatementBuilder();
    -		builder2.setPredicate(new RyaURI("http://temp.com"));
    -		builder2.setObject(new RyaURI("http://object.com"));
    -		builder2.setContext(new RyaURI("http://context3.com"));
    -		final RyaStatement query = builder2.build();
    +        final RyaStatementBuilder builder2 = new RyaStatementBuilder();
    +        builder2.setPredicate(new RyaURI("http://temp.com"));
    +        builder2.setObject(new RyaURI("http://object.com"));
    +        builder2.setContext(new RyaURI("http://context3.com"));
    +        final RyaStatement query = builder2.build();
     
             dao.delete(query, configuration);
     
             assertEquals(coll.count(),1);
     
    -	}
    +    }
     
    +    @Test
    +    public void testVisibility() throws RyaDAOException, MongoException, IOException {
    +        // Doc requires "A" and user has "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A", new Authorizations("B")));
    +
    +        // Doc requires "A" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A")));
    +
    +        // Doc requires "A" and "B" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or "B" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and "B" and user has "A" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A&B", new Authorizations("A")));
    +
    +        // Doc requires "A" and "B" and "C" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A&B&C", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and "B" and "C" and user has "A" and "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A&B&C", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and "B" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" or "B" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A")));
    +
    +        // Doc requires "A" or "B" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A", Authorizations.EMPTY));
    +
    +        // Doc requires "A" and "B" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A&B", Authorizations.EMPTY));
    +
    +        // Doc requires "A" or "B" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A|B", Authorizations.EMPTY));
    +
    +        // Doc has no requirement and user has "" = User can view
    +        assertTrue(testVisibilityStatement("", Authorizations.EMPTY));
    +
    +        // Doc has no requirement and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("", new Authorizations("A")));
    +
    +        // Doc has no requirement and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or ("B" and "C") and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|(B&C)", new Authorizations("A")));
    +
    +        // Doc requires "A" or ("B" and "C") and user has "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|(B&C)", new Authorizations("B", "C")));
    +
    +        // Doc requires "A" and ("B" or "C") and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A&(B|C)", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and ("B" or "C") and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A&(B|C)", new Authorizations("A", "C")));
    --- End diff --
    
    there are some missing assertFalse combinations that need to be added for completeness


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89897938
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBStorageStrategy.java ---
    @@ -96,12 +103,19 @@ public DBObject getQuery(final RyaStatement stmt) {
     
         @Override
         public RyaStatement deserializeDBObject(final DBObject queryResult) {
    -        final Map result = queryResult.toMap();
    +        final Map<?, ?> result = queryResult.toMap();
    --- End diff --
    
    this seems dangerous, do we not know types here?


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

[GitHub] incubator-rya issue #124: RYA-119 Added MongoDB Column Visibility (called Do...

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

    https://github.com/apache/incubator-rya/pull/124
  
    @ejwhite922, is this good to merge?  It looked fine to me -- great job


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90357458
  
    --- Diff: dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOTest.java ---
    @@ -28,102 +30,279 @@
     import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
     import org.apache.rya.api.domain.RyaURI;
     import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.api.persist.query.RyaQuery;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.bson.Document;
    +import org.calrissian.mango.collect.CloseableIterable;
     import org.junit.Before;
     import org.junit.Test;
     
    -import com.mongodb.DB;
    -import com.mongodb.DBCollection;
     import com.mongodb.MongoException;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.MongoDatabase;
     
     public class MongoDBRyaDAOTest extends MongoRyaTestBase {
     
    -	private MongoDBRyaDAO dao;
    -	private MongoDBRdfConfiguration configuration;
    +    private MongoDBRyaDAO dao;
    +    private MongoDBRdfConfiguration configuration;
     
    -	@Before
    -	public void setUp() throws IOException, RyaDAOException{
    -		final Configuration conf = new Configuration();
    +    @Before
    +    public void setUp() throws IOException, RyaDAOException{
    +        final Configuration conf = new Configuration();
             conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, "test");
             conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
             conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya_");
             configuration = new MongoDBRdfConfiguration(conf);
             final int port = mongoClient.getServerAddressList().get(0).getPort();
             configuration.set(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT, ""+port);
    -		dao = new MongoDBRyaDAO(configuration, mongoClient);
    -	}
    +        dao = new MongoDBRyaDAO(configuration, mongoClient);
    +    }
     
     
    -	@Test
    -	public void testDeleteWildcard() throws RyaDAOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		dao.delete(builder.build(), configuration);
    -	}
    +    @Test
    +    public void testDeleteWildcard() throws RyaDAOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        dao.delete(builder.build(), configuration);
    +    }
     
     
    -	@Test
    -	public void testAdd() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    +    @Test
    +    public void testAdd() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(builder.build());
    +        dao.add(builder.build());
     
             assertEquals(coll.count(),1);
     
    -	}
    +    }
     
    -	@Test
    -	public void testDelete() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    -		final RyaStatement statement = builder.build();
    +    @Test
    +    public void testDelete() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
    +        final RyaStatement statement = builder.build();
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(statement);
    +        dao.add(statement);
     
             assertEquals(coll.count(),1);
     
             dao.delete(statement, configuration);
     
             assertEquals(coll.count(),0);
     
    -	}
    +    }
     
    -	@Test
    -	public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    -		builder.setContext(new RyaURI("http://context.com"));
    -		final RyaStatement statement = builder.build();
    +    @Test
    +    public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
    +        builder.setContext(new RyaURI("http://context.com"));
    +        final RyaStatement statement = builder.build();
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(statement);
    +        dao.add(statement);
     
             assertEquals(coll.count(),1);
     
    -		final RyaStatementBuilder builder2 = new RyaStatementBuilder();
    -		builder2.setPredicate(new RyaURI("http://temp.com"));
    -		builder2.setObject(new RyaURI("http://object.com"));
    -		builder2.setContext(new RyaURI("http://context3.com"));
    -		final RyaStatement query = builder2.build();
    +        final RyaStatementBuilder builder2 = new RyaStatementBuilder();
    +        builder2.setPredicate(new RyaURI("http://temp.com"));
    +        builder2.setObject(new RyaURI("http://object.com"));
    +        builder2.setContext(new RyaURI("http://context3.com"));
    +        final RyaStatement query = builder2.build();
     
             dao.delete(query, configuration);
     
             assertEquals(coll.count(),1);
     
    -	}
    +    }
     
    +    @Test
    +    public void testVisibility() throws RyaDAOException, MongoException, IOException {
    +        // Doc requires "A" and user has "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A", new Authorizations("B")));
    +
    +        // Doc requires "A" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A")));
    +
    +        // Doc requires "A" and "B" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or "B" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and "B" and user has "A" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A&B", new Authorizations("A")));
    +
    +        // Doc requires "A" and "B" and "C" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A&B&C", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and "B" and "C" and user has "A" and "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A&B&C", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and "B" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" or "B" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A")));
    +
    +        // Doc requires "A" or "B" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A", Authorizations.EMPTY));
    +
    +        // Doc requires "A" and "B" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A&B", Authorizations.EMPTY));
    +
    +        // Doc requires "A" or "B" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A|B", Authorizations.EMPTY));
    +
    +        // Doc has no requirement and user has "" = User can view
    +        assertTrue(testVisibilityStatement("", Authorizations.EMPTY));
    +
    +        // Doc has no requirement and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("", new Authorizations("A")));
    +
    +        // Doc has no requirement and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or ("B" and "C") and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|(B&C)", new Authorizations("A")));
    +
    +        // Doc requires "A" or ("B" and "C") and user has "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|(B&C)", new Authorizations("B", "C")));
    +
    +        // Doc requires "A" and ("B" or "C") and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A&(B|C)", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and ("B" or "C") and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A&(B|C)", new Authorizations("A", "C")));
    --- End diff --
    
    Added extra test cases


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90330643
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/util/DocumentVisibilityUtil.java ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.mongodb.document.util;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.log4j.Logger;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.apache.rya.mongodb.document.visibility.ByteSequence;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.Node;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.NodeType;
    +import org.apache.rya.mongodb.document.visibility.VisibilityEvaluator;
    +import org.apache.rya.mongodb.document.visibility.VisibilityParseException;
    +
    +import com.google.common.base.Charsets;
    +import com.google.common.collect.Lists;
    +import com.mongodb.BasicDBList;
    +
    +/**
    + * Utility methods for converting boolean expressions between an Accumulo column
    + * visibility string style and a multidimensional array that can be used
    + * in MongoDB expressions.
    --- End diff --
    
    Done.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89895771
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRyaDAO.java ---
    @@ -111,26 +117,27 @@ public MongoDBRdfConfiguration getConf() {
     
         @Override
         public void init() throws RyaDAOException {
    -            secondaryIndexers = conf.getAdditionalIndexers();
    -            for(final MongoSecondaryIndex index: secondaryIndexers) {
    -                index.setConf(conf);
    -                index.setClient(mongoClient);
    -            }
    +        secondaryIndexers = conf.getAdditionalIndexers();
    +        for(final MongoSecondaryIndex index: secondaryIndexers) {
    +            index.setConf(conf);
    +            index.setClient(mongoClient);
    +        }
     
    -            db = mongoClient.getDB(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -            coll = db.getCollection(conf.getTriplesCollectionName());
    -            nameSpaceManager = new SimpleMongoDBNamespaceManager(db.getCollection(conf.getNameSpacesCollectionName()));
    -            queryEngine = new MongoDBQueryEngine(conf, mongoClient);
    -            storageStrategy = new SimpleMongoDBStorageStrategy();
    -            storageStrategy.createIndices(coll);
    -            for(final MongoSecondaryIndex index: secondaryIndexers) {
    -                index.init();
    -            }
    +        db = mongoClient.getDB(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        coll = db.getCollection(conf.getTriplesCollectionName());
    +        nameSpaceManager = new SimpleMongoDBNamespaceManager(db.getCollection(conf.getNameSpacesCollectionName()));
    +        queryEngine = new MongoDBQueryEngine(conf, mongoClient);
    +        storageStrategy = new SimpleMongoDBStorageStrategy();
    +        storageStrategy.createIndices(coll);
    +        for(final MongoSecondaryIndex index: secondaryIndexers) {
    +            index.init();
    +        }
         }
     
         @Override
         public boolean isInitialized() throws RyaDAOException {
             return true;
    +
    --- End diff --
    
    nit: remove this spare line


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89897518
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRyaDAO.java ---
    @@ -164,26 +174,31 @@ public void add(final Iterator<RyaStatement> statement) throws RyaDAOException {
             final List<DBObject> dbInserts = new ArrayList<DBObject>();
             while (statement.hasNext()){
                 final RyaStatement ryaStatement = statement.next();
    -            final DBObject insert = storageStrategy.serialize(ryaStatement);
    -            dbInserts.add(insert);
    -
    -            try {
    -                for (final RyaSecondaryIndexer index : secondaryIndexers) {
    -                    index.storeStatement(ryaStatement);
    +            final boolean canAdd = DocumentVisibilityUtil.doesUserHaveDocumentAccess(auths, ryaStatement.getColumnVisibility());
    +            if (canAdd) {
    +                final DBObject insert = storageStrategy.serialize(ryaStatement);
    +                dbInserts.add(insert);
    +
    +                try {
    +                    for (final RyaSecondaryIndexer index : secondaryIndexers) {
    +                        index.storeStatement(ryaStatement);
    +                    }
    +                } catch (final IOException e) {
    +                    log.error("Failed to add: " + ryaStatement.toString() + " to the indexer");
                     }
    -            } catch (final IOException e) {
    -                log.error("Failed to add: " + ryaStatement.toString() + " to the indexer");
                 }
    -
             }
             coll.insert(dbInserts, new InsertOptions().continueOnError(true));
         }
     
         @Override
         public void delete(final RyaStatement statement, final MongoDBRdfConfiguration conf)
                 throws RyaDAOException {
    -        final DBObject obj = storageStrategy.getQuery(statement);
    -        coll.remove(obj);
    +        final boolean canDelete = DocumentVisibilityUtil.doesUserHaveDocumentAccess(auths, statement.getColumnVisibility());
    +        if (canDelete) {
    +            final DBObject obj = storageStrategy.getQuery(statement);
    +            coll.remove(obj);
    --- End diff --
    
    I think @dlotts added some code here for removing statements from the indexers.  make sure that doesn't get erased here.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89907166
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/TextUtil.java ---
    @@ -0,0 +1,30 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import org.apache.hadoop.io.Text;
    +
    +public final class TextUtil {
    +  public static byte[] getBytes(final Text text) {
    --- End diff --
    
    why not use Text's toString().getBytes(); or decode()?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89904458
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    + */
    +public class Authorizations implements Iterable<byte[]>, Serializable, AuthorizationContainer {
    +
    +  private static final long serialVersionUID = 1L;
    +
    +  private final Set<ByteSequence> auths = new HashSet<ByteSequence>();
    +  private final List<byte[]> authsList = new ArrayList<byte[]>(); // sorted order
    +
    +  /**
    +   * An empty set of authorizations.
    +   */
    +  public static final Authorizations EMPTY = new Authorizations();
    +
    +  private static final boolean[] validAuthChars = new boolean[256];
    +
    +  /**
    +   * A special header string used when serializing instances of this class.
    +   *
    +   * @see #serialize()
    +   */
    +  public static final String HEADER = "!AUTH1:";
    +
    +  static {
    +    for (int i = 0; i < 256; i++) {
    +      validAuthChars[i] = false;
    +    }
    +
    +    for (int i = 'a'; i <= 'z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = 'A'; i <= 'Z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = '0'; i <= '9'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    validAuthChars['_'] = true;
    +    validAuthChars['-'] = true;
    +    validAuthChars[':'] = true;
    +    validAuthChars['.'] = true;
    +    validAuthChars['/'] = true;
    +  }
    +
    +  static final boolean isValidAuthChar(final byte b) {
    +    return validAuthChars[0xff & b];
    +  }
    +
    +  private void checkAuths() {
    +    final Set<ByteSequence> sortedAuths = new TreeSet<ByteSequence>(auths);
    +
    +    for (final ByteSequence bs : sortedAuths) {
    +      if (bs.length() == 0) {
    +        throw new IllegalArgumentException("Empty authorization");
    +      }
    +
    +      authsList.add(bs.toArray());
    +    }
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a collection of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does
    +   * not verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          collection of authorizations, as strings encoded in UTF-8
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final Collection<byte[]> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    for (final byte[] auth : authorizations) {
    +		auths.add(new ArrayByteSequence(auth));
    +	}
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a list of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does not
    +   * verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          list of authorizations, as strings encoded in UTF-8 and placed in buffers
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final List<ByteBuffer> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    for (final ByteBuffer buffer : authorizations) {
    +      auths.add(new ArrayByteSequence(ByteBufferUtil.toBytes(buffer)));
    +    }
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Constructs an authorizations object from a serialized form. This is NOT a constructor for a set of authorizations of size one. Warning: This method does
    +   * not verify that the encoded serialized form is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          a serialized authorizations string produced by {@link #getAuthorizationsArray()} or {@link #serialize()}, converted to UTF-8 bytes
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   */
    +  public Authorizations(final byte[] authorizations) {
    +
    +    ArgumentChecker.notNull(authorizations);
    +
    +    String authsString = new String(authorizations, UTF_8);
    +    if (authsString.startsWith(HEADER)) {
    +      // it's the new format
    +      authsString = authsString.substring(HEADER.length());
    +      if (authsString.length() > 0) {
    +        for (final String encAuth : authsString.split(",")) {
    +          final byte[] auth = Base64.decodeBase64(encAuth.getBytes(UTF_8));
    +          auths.add(new ArrayByteSequence(auth));
    +        }
    +        checkAuths();
    +      }
    +    } else {
    +      // it's the old format
    +      if (authorizations.length > 0) {
    +		setAuthorizations(authsString.split(","));
    +	}
    +    }
    +  }
    +
    +  /**
    +   * Constructs an empty set of authorizations.
    +   *
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations() {}
    --- End diff --
    
    why an empty constructor?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89905264
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    + */
    +public class Authorizations implements Iterable<byte[]>, Serializable, AuthorizationContainer {
    --- End diff --
    
    AuthorizationContainer only has 1 contains, but you end up with 3 here, making it vague which to use for AuthorizationContainer


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89900455
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/util/DocumentVisibilityUtil.java ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.mongodb.document.util;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.log4j.Logger;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.apache.rya.mongodb.document.visibility.ByteSequence;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.Node;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.NodeType;
    +import org.apache.rya.mongodb.document.visibility.VisibilityEvaluator;
    +import org.apache.rya.mongodb.document.visibility.VisibilityParseException;
    +
    +import com.google.common.base.Charsets;
    +import com.google.common.collect.Lists;
    +import com.mongodb.BasicDBList;
    +
    +/**
    + * Utility methods for converting boolean expressions between an Accumulo column
    + * visibility string style and a multidimensional array that can be used
    + * in MongoDB expressions.
    --- End diff --
    
    reword this to make it more db agnostic.  From string representation to a mongo-friendly form.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89908036
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/VisibilityParseException.java ---
    @@ -0,0 +1,49 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.text.ParseException;
    +
    +/**
    + * An exception thrown when a visibility string cannot be parsed.
    --- End diff --
    
    doc where this is used


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89903566
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    --- End diff --
    
    Doc how this is to be used, how to use it, etc.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89895899
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRyaDAO.java ---
    @@ -146,15 +153,18 @@ public void destroy() throws RyaDAOException {
         public void add(final RyaStatement statement) throws RyaDAOException {
             // add it to the collection
             try {
    -            coll.insert(storageStrategy.serialize(statement));
    -            for(final RyaSecondaryIndexer index: secondaryIndexers) {
    -                index.storeStatement(statement);
    +            final boolean canAdd = DocumentVisibilityUtil.doesUserHaveDocumentAccess(auths, statement.getColumnVisibility());
    --- End diff --
    
    I'll check later, but should this be doesUserHaveCollectionAccess?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90357417
  
    --- Diff: dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOTest.java ---
    @@ -28,102 +30,279 @@
     import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
     import org.apache.rya.api.domain.RyaURI;
     import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.api.persist.query.RyaQuery;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.bson.Document;
    +import org.calrissian.mango.collect.CloseableIterable;
     import org.junit.Before;
     import org.junit.Test;
     
    -import com.mongodb.DB;
    -import com.mongodb.DBCollection;
     import com.mongodb.MongoException;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.MongoDatabase;
     
     public class MongoDBRyaDAOTest extends MongoRyaTestBase {
     
    -	private MongoDBRyaDAO dao;
    -	private MongoDBRdfConfiguration configuration;
    +    private MongoDBRyaDAO dao;
    +    private MongoDBRdfConfiguration configuration;
     
    -	@Before
    -	public void setUp() throws IOException, RyaDAOException{
    -		final Configuration conf = new Configuration();
    +    @Before
    +    public void setUp() throws IOException, RyaDAOException{
    +        final Configuration conf = new Configuration();
             conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, "test");
             conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
             conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya_");
             configuration = new MongoDBRdfConfiguration(conf);
             final int port = mongoClient.getServerAddressList().get(0).getPort();
             configuration.set(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT, ""+port);
    -		dao = new MongoDBRyaDAO(configuration, mongoClient);
    -	}
    +        dao = new MongoDBRyaDAO(configuration, mongoClient);
    +    }
     
     
    -	@Test
    -	public void testDeleteWildcard() throws RyaDAOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		dao.delete(builder.build(), configuration);
    -	}
    +    @Test
    +    public void testDeleteWildcard() throws RyaDAOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        dao.delete(builder.build(), configuration);
    +    }
     
     
    -	@Test
    -	public void testAdd() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    +    @Test
    +    public void testAdd() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(builder.build());
    +        dao.add(builder.build());
     
             assertEquals(coll.count(),1);
     
    -	}
    +    }
     
    -	@Test
    -	public void testDelete() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    -		final RyaStatement statement = builder.build();
    +    @Test
    +    public void testDelete() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
    +        final RyaStatement statement = builder.build();
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(statement);
    +        dao.add(statement);
     
             assertEquals(coll.count(),1);
     
             dao.delete(statement, configuration);
     
             assertEquals(coll.count(),0);
     
    -	}
    +    }
     
    -	@Test
    -	public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    -		builder.setContext(new RyaURI("http://context.com"));
    -		final RyaStatement statement = builder.build();
    +    @Test
    +    public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
    +        builder.setContext(new RyaURI("http://context.com"));
    +        final RyaStatement statement = builder.build();
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(statement);
    +        dao.add(statement);
     
             assertEquals(coll.count(),1);
     
    -		final RyaStatementBuilder builder2 = new RyaStatementBuilder();
    -		builder2.setPredicate(new RyaURI("http://temp.com"));
    -		builder2.setObject(new RyaURI("http://object.com"));
    -		builder2.setContext(new RyaURI("http://context3.com"));
    -		final RyaStatement query = builder2.build();
    +        final RyaStatementBuilder builder2 = new RyaStatementBuilder();
    +        builder2.setPredicate(new RyaURI("http://temp.com"));
    +        builder2.setObject(new RyaURI("http://object.com"));
    +        builder2.setContext(new RyaURI("http://context3.com"));
    +        final RyaStatement query = builder2.build();
     
             dao.delete(query, configuration);
     
             assertEquals(coll.count(),1);
     
    -	}
    +    }
     
    +    @Test
    +    public void testVisibility() throws RyaDAOException, MongoException, IOException {
    +        // Doc requires "A" and user has "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A", new Authorizations("B")));
    +
    +        // Doc requires "A" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A")));
    +
    +        // Doc requires "A" and "B" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or "B" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and "B" and user has "A" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A&B", new Authorizations("A")));
    +
    +        // Doc requires "A" and "B" and "C" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A&B&C", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and "B" and "C" and user has "A" and "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A&B&C", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and "B" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" or "B" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A")));
    +
    +        // Doc requires "A" or "B" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A", Authorizations.EMPTY));
    +
    +        // Doc requires "A" and "B" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A&B", Authorizations.EMPTY));
    +
    +        // Doc requires "A" or "B" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A|B", Authorizations.EMPTY));
    --- End diff --
    
    Added extra test cases


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r94617786
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRyaDAO.java ---
    @@ -146,15 +153,18 @@ public void destroy() throws RyaDAOException {
         public void add(final RyaStatement statement) throws RyaDAOException {
             // add it to the collection
             try {
    -            coll.insert(storageStrategy.serialize(statement));
    -            for(final RyaSecondaryIndexer index: secondaryIndexers) {
    -                index.storeStatement(statement);
    +            final boolean canAdd = DocumentVisibilityUtil.doesUserHaveDocumentAccess(auths, statement.getColumnVisibility());
    --- End diff --
    
    Added exceptions for add and delete if they don't have the required authorizations.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r94812679
  
    --- Diff: dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/document/visibility/DocumentVisibilityAdapterTest.java ---
    @@ -0,0 +1,167 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibilityAdapter.MalformedDocumentVisibilityException;
    +import org.junit.Test;
    +
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.util.JSON;
    +
    +/**
    + * Tests the methods of {@link DocumentVisibilityAdapter}.
    + */
    +public class DocumentVisibilityAdapterTest {
    --- End diff --
    
    yes they do :+1: 


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89905435
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/BadArgumentException.java ---
    @@ -0,0 +1,27 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import java.util.regex.PatternSyntaxException;
    --- End diff --
    
    why is this a PatternSyntaxException


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90331288
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/util/DocumentVisibilityUtil.java ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.mongodb.document.util;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.log4j.Logger;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.apache.rya.mongodb.document.visibility.ByteSequence;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.Node;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.NodeType;
    +import org.apache.rya.mongodb.document.visibility.VisibilityEvaluator;
    +import org.apache.rya.mongodb.document.visibility.VisibilityParseException;
    +
    +import com.google.common.base.Charsets;
    +import com.google.common.collect.Lists;
    +import com.mongodb.BasicDBList;
    +
    +/**
    + * Utility methods for converting boolean expressions between an Accumulo column
    + * visibility string style and a multidimensional array that can be used
    + * in MongoDB expressions.
    + */
    +public final class DocumentVisibilityUtil {
    +    private static final Logger log = Logger.getLogger(DocumentVisibilityUtil.class);
    +
    +    /**
    +     * Private constructor to prevent instantiation.
    +     */
    +    private DocumentVisibilityUtil() {
    +    }
    +
    +    /**
    +     * Converts a boolean string expression into a multidimensional
    +     * array representation of the boolean expression.
    +     * @param booleanString the boolean string expression.
    +     * @return the multidimensional array representation of the boolean
    +     * expression.
    +     */
    +    public static Object[] toMultidimensionalArray(final String booleanString) {
    +        final DocumentVisibility dv = new DocumentVisibility(booleanString);
    +        return toMultidimensionalArray(dv);
    +    }
    +
    +    /**
    +     * Converts a {@link DocumentVisibility} object into a multidimensional
    +     * array representation of the boolean expression.
    +     * @param dv the {@link DocumentVisibility}. (not {@code null})
    +     * @return the multidimensional array representation of the boolean
    +     * expression.
    +     */
    +    public static Object[] toMultidimensionalArray(final DocumentVisibility dv) {
    +        checkNotNull(dv);
    +        final byte[] expression = dv.flatten();
    +        final DocumentVisibility flattenedDv = DisjunctiveNormalFormConverter.createDnfDocumentVisibility(expression);
    +        final Object[] result = toMultidimensionalArray(flattenedDv.getParseTree(), expression);
    +        return result;
    +    }
    +
    +    /**
    +     * Converts a {@link Node} and its corresponding expression into a
    +     * multidimensional array representation of the boolean expression.
    +     * @param node the {@link Node}. (not {@code null})
    +     * @param expression the expression byte array.
    +     * @return the multidimensional array representation of the boolean
    +     * expression.
    +     */
    +    public static Object[] toMultidimensionalArray(final Node node, final byte[] expression) {
    +        checkNotNull(node);
    +        final List<Object> array = new ArrayList<>();
    +
    +        if (node.getChildren().isEmpty() && node.getType() == NodeType.TERM) {
    +            final String data = getTermNodeData(node);
    +            array.add(data);
    +        }
    +
    +        log.trace("Children size: " + node.getChildren().size() + " Type: " + node.getType());
    +        for (final Node child : node.getChildren()) {
    +            switch (child.getType()) {
    +                case EMPTY:
    +                case TERM:
    +                    String data;
    +                    if (child.getType() == NodeType.TERM) {
    +                        data = getTermNodeData(child);
    +                    } else {
    +                        data = "";
    +                    }
    +                    if (node.getType() == NodeType.OR) {
    +                        array.add(Lists.newArrayList(data).toArray(new Object[0]));
    +                    } else {
    +                        array.add(data);
    +                    }
    +                    break;
    +                case OR:
    +                case AND:
    +                    array.add(toMultidimensionalArray(child, expression));
    +                    break;
    +                default:
    +                    break;
    +            }
    +        }
    +
    +        return array.toArray(new Object[0]);
    +    }
    +
    +    public static String nodeToBooleanString(final Node node) {
    +        boolean isFirst = true;
    +        final StringBuilder sb = new StringBuilder();
    +        if (node.getType() == NodeType.TERM) {
    +            final String data = getTermNodeData(node);
    +            sb.append(data);
    +        }
    +        if (node.getType() == NodeType.AND) {
    +            sb.append("(");
    +        }
    +        for (final Node child : node.getChildren()) {
    --- End diff --
    
    No.  But this is the node's children not the children's children.  We need to handle a node's children.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90067552
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/AuthorizationContainer.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.mongodb.document.visibility;
    +
    +/**
    + * An interface for classes that contain a collection of authorizations.
    + */
    +public interface AuthorizationContainer {
    +  /**
    +   * Checks whether this object contains the given authorization.
    +   *
    +   * @param auth
    +   *          authorization, as a string encoded in UTF-8
    +   * @return true if authorization is in this collection
    +   */
    +  boolean contains(ByteSequence auth);
    --- End diff --
    
    same as above


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90066459
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/util/DocumentVisibilityUtil.java ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.mongodb.document.util;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.log4j.Logger;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.apache.rya.mongodb.document.visibility.ByteSequence;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.Node;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.NodeType;
    +import org.apache.rya.mongodb.document.visibility.VisibilityEvaluator;
    +import org.apache.rya.mongodb.document.visibility.VisibilityParseException;
    +
    +import com.google.common.base.Charsets;
    +import com.google.common.collect.Lists;
    +import com.mongodb.BasicDBList;
    +
    +/**
    + * Utility methods for converting boolean expressions between an Accumulo column
    + * visibility string style and a multidimensional array that can be used
    + * in MongoDB expressions.
    + */
    +public final class DocumentVisibilityUtil {
    +    private static final Logger log = Logger.getLogger(DocumentVisibilityUtil.class);
    +
    +    /**
    +     * Private constructor to prevent instantiation.
    +     */
    +    private DocumentVisibilityUtil() {
    +    }
    +
    +    /**
    +     * Converts a boolean string expression into a multidimensional
    +     * array representation of the boolean expression.
    +     * @param booleanString the boolean string expression.
    +     * @return the multidimensional array representation of the boolean
    +     * expression.
    +     */
    +    public static Object[] toMultidimensionalArray(final String booleanString) {
    +        final DocumentVisibility dv = new DocumentVisibility(booleanString);
    +        return toMultidimensionalArray(dv);
    +    }
    +
    +    /**
    +     * Converts a {@link DocumentVisibility} object into a multidimensional
    +     * array representation of the boolean expression.
    +     * @param dv the {@link DocumentVisibility}. (not {@code null})
    +     * @return the multidimensional array representation of the boolean
    +     * expression.
    +     */
    +    public static Object[] toMultidimensionalArray(final DocumentVisibility dv) {
    +        checkNotNull(dv);
    +        final byte[] expression = dv.flatten();
    +        final DocumentVisibility flattenedDv = DisjunctiveNormalFormConverter.createDnfDocumentVisibility(expression);
    +        final Object[] result = toMultidimensionalArray(flattenedDv.getParseTree(), expression);
    +        return result;
    +    }
    +
    +    /**
    +     * Converts a {@link Node} and its corresponding expression into a
    +     * multidimensional array representation of the boolean expression.
    +     * @param node the {@link Node}. (not {@code null})
    +     * @param expression the expression byte array.
    +     * @return the multidimensional array representation of the boolean
    +     * expression.
    +     */
    +    public static Object[] toMultidimensionalArray(final Node node, final byte[] expression) {
    +        checkNotNull(node);
    +        final List<Object> array = new ArrayList<>();
    +
    +        if (node.getChildren().isEmpty() && node.getType() == NodeType.TERM) {
    +            final String data = getTermNodeData(node);
    +            array.add(data);
    +        }
    +
    +        log.trace("Children size: " + node.getChildren().size() + " Type: " + node.getType());
    +        for (final Node child : node.getChildren()) {
    +            switch (child.getType()) {
    +                case EMPTY:
    +                case TERM:
    +                    String data;
    +                    if (child.getType() == NodeType.TERM) {
    +                        data = getTermNodeData(child);
    +                    } else {
    +                        data = "";
    +                    }
    +                    if (node.getType() == NodeType.OR) {
    +                        array.add(Lists.newArrayList(data).toArray(new Object[0]));
    +                    } else {
    +                        array.add(data);
    +                    }
    +                    break;
    +                case OR:
    +                case AND:
    +                    array.add(toMultidimensionalArray(child, expression));
    +                    break;
    +                default:
    +                    break;
    +            }
    +        }
    +
    +        return array.toArray(new Object[0]);
    +    }
    +
    +    public static String nodeToBooleanString(final Node node) {
    +        boolean isFirst = true;
    +        final StringBuilder sb = new StringBuilder();
    +        if (node.getType() == NodeType.TERM) {
    +            final String data = getTermNodeData(node);
    +            sb.append(data);
    +        }
    +        if (node.getType() == NodeType.AND) {
    +            sb.append("(");
    +        }
    +        for (final Node child : node.getChildren()) {
    +            if (isFirst) {
    +                isFirst = false;
    +            } else {
    +                if (node.getType() == NodeType.OR) {
    +                    sb.append("|");
    +                } else if (node.getType() == NodeType.AND) {
    +                    sb.append("&");
    +                }
    +            }
    +            switch (child.getType()) {
    +                case EMPTY:
    +                    sb.append("");
    +                    break;
    +                case TERM:
    +                    final String data = getTermNodeData(child);
    +                    sb.append(data);
    +                    break;
    +                case OR:
    +                    sb.append("(");
    +                    sb.append(nodeToBooleanString(child));
    +                    sb.append(")");
    +                    break;
    +                case AND:
    +                    sb.append(nodeToBooleanString(child));
    +                    break;
    +                default:
    +                    break;
    +            }
    +        }
    +        if (node.getType() == NodeType.AND) {
    +            sb.append(")");
    +        }
    +
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Converts a multidimensional array object representation of the document
    +     * visibility boolean expression into a string.
    +     * @param object the multidimensional array object representing the
    +     * document visibility boolean expression.
    +     * @return the boolean string expression.
    +     */
    +    public static String multidimensionalArrayToBooleanString(final Object[] object) {
    +        final String booleanString = multidimensionalArrayToBooleanStringInternal(object);
    +
    +        // Simplify and clean up the formatting.
    +        final DocumentVisibility dv = DisjunctiveNormalFormConverter.createDnfDocumentVisibility(booleanString);
    +        final byte[] bytes = dv.flatten();
    +        final String result = new String(bytes, Charsets.UTF_8);
    +
    +        return result;
    +    }
    +
    +    private static String multidimensionalArrayToBooleanStringInternal(final Object[] object) {
    +        final StringBuilder sb = new StringBuilder();
    +
    +        int count = 0;
    +        boolean isAnd = false;
    +        for (final Object child : object) {
    +            if (child instanceof String) {
    +                isAnd = true;
    +                if (count > 0) {
    +                    sb.append("&");
    +                }
    +                sb.append(child);
    +            } else if (child instanceof Object[]) {
    +                if (count > 0 && isAnd) {
    +                    sb.append("&");
    +                }
    +                final Object[] obj = (Object[]) child;
    +                sb.append("(");
    +                sb.append(multidimensionalArrayToBooleanStringInternal(obj));
    +                sb.append(")");
    +            }
    +
    +            if (object.length > 1 && count + 1 < object.length && !isAnd) {
    +                sb.append("|");
    +            }
    +            count++;
    +        }
    +
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Conditionally adds quotes around a string.
    +     * @param data the string to add quotes to.
    +     * @param addQuotes {@code true} to add quotes. {@code false} to leave the
    +     * string as is.
    +     * @return the quoted string if {@code addQuotes} is {@code true}.
    +     * Otherwise, returns the string as is.
    +     */
    +    public static String addQuotes(final String data, final boolean addQuotes) {
    +        if (addQuotes) {
    +            return "\"" + data + "\"";
    +        } else {
    +            return data;
    +        }
    +    }
    +
    +    /**
    +     * Returns the term node's data.
    +     * @param node the {@link Node}.
    +     * @return the term node's data.
    +     */
    +    public static String getTermNodeData(final Node node) {
    +        final boolean isQuotedTerm = node.getExpression()[node.getTermStart()] == '"';
    +        final ByteSequence bs = node.getTerm(node.getExpression());
    +        final String data = addQuotes(new String(bs.toArray(), Charsets.UTF_8), isQuotedTerm);
    +        return data;
    +    }
    +
    +    /**
    +     * Checks if the user's authorizations allows them to have access to the
    +     * provided document based on its document visibility.
    +     * @param authorizations the {@link Authorizations}.
    +     * @param documentVisibility the document visibility byte expression.
    +     * @return {@code true} if the user has access to the document.
    +     * {@code false} otherwise.
    +     */
    +    public static boolean doesUserHaveDocumentAccess(final Authorizations authorizations, final byte[] documentVisibilityExpression) {
    +        final byte[] expression = documentVisibilityExpression != null ? documentVisibilityExpression : MongoDbRdfConstants.EMPTY_DV.getExpression();
    +        final DocumentVisibility documentVisibility = new DocumentVisibility(expression);
    +        return doesUserHaveDocumentAccess(authorizations, documentVisibility);
    +    }
    +
    +    /**
    +     * Checks if the user's authorizations allows them to have access to the
    +     * provided document based on its document visibility.
    +     * @param authorizations the {@link Authorizations}.
    +     * @param documentVisibility the {@link DocumentVisibility}.
    +     * @return {@code true} if the user has access to the document.
    +     * {@code false} otherwise.
    +     */
    +    public static boolean doesUserHaveDocumentAccess(final Authorizations authorizations, final DocumentVisibility documentVisibility) {
    +        return doesUserHaveDocumentAccess(authorizations, documentVisibility, true);
    +    }
    +
    +    /**
    +     * Checks if the user's authorizations allows them to have access to the
    +     * provided document based on its document visibility.
    +     * @param authorizations the {@link Authorizations}.
    +     * @param documentVisibility the {@link DocumentVisibility}.
    +     * @param doesEmptyAccessPass {@code true} if an empty authorization pass
    +     * allows access to everything. {@code false} otherwise.
    +     * @return {@code true} if the user has access to the document.
    +     * {@code false} otherwise.
    +     */
    +    public static boolean doesUserHaveDocumentAccess(final Authorizations authorizations, final DocumentVisibility documentVisibility, final boolean doesEmptyAccessPass) {
    +        final Authorizations userAuths = authorizations != null ? authorizations : MongoDbRdfConstants.ALL_AUTHORIZATIONS;
    +        final VisibilityEvaluator visibilityEvaluator = new VisibilityEvaluator(userAuths);
    +        boolean accept = false;
    +        if (doesEmptyAccessPass && MongoDbRdfConstants.ALL_AUTHORIZATIONS.equals(userAuths)) {
    +            accept = true;
    +        } else {
    +            try {
    +                accept = visibilityEvaluator.evaluate(documentVisibility);
    +            } catch (final VisibilityParseException e) {
    +                log.error("Could not parse document visibility.");
    +            }
    +        }
    +
    +        return accept;
    +    }
    +
    +    public static Object[] convertBasicDBListToObjectArray(final BasicDBList basicDbList) {
    --- End diff --
    
    Done.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89908817
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/iter/RyaStatementBindingSetCursorIterator.java ---
    @@ -1,5 +1,27 @@
     package org.apache.rya.mongodb.iter;
     
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map.Entry;
    +
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.RdfCloudTripleStoreUtils;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.openrdf.query.BindingSet;
    +
    +import com.google.common.collect.Multimap;
    +import com.mongodb.AggregationOutput;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBCollection;
    +import com.mongodb.DBObject;
    +
     /*
    --- End diff --
    
    move this back above the package declaration


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89909295
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/iter/RyaStatementCursorIterator.java ---
    @@ -22,83 +40,83 @@
     
     import info.aduna.iteration.CloseableIteration;
     
    -import java.util.Iterator;
    -import java.util.Map.Entry;
    -import java.util.Set;
    -
    -import org.apache.rya.api.RdfCloudTripleStoreUtils;
    -import org.apache.rya.api.domain.RyaStatement;
    -import org.apache.rya.api.persist.RyaDAOException;
    -import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    -
    -import org.calrissian.mango.collect.CloseableIterable;
    -import org.openrdf.query.BindingSet;
    -
    -import com.mongodb.DBCollection;
    -import com.mongodb.DBCursor;
    -import com.mongodb.DBObject;
    -
     public class RyaStatementCursorIterator implements CloseableIteration<RyaStatement, RyaDAOException> {
    -
    -	private DBCollection coll;
    -	private Iterator<DBObject> queryIterator;
    -	private DBCursor currentCursor;
    -	private MongoDBStorageStrategy strategy;
    -	private Long maxResults;
    -
    -	public RyaStatementCursorIterator(DBCollection coll, Set<DBObject> queries, MongoDBStorageStrategy strategy) {
    -		this.coll = coll;
    -		this.queryIterator = queries.iterator();
    -		this.strategy = strategy;
    -	}
    -
    -	@Override
    -	public boolean hasNext() {
    -		if (!currentCursorIsValid()) {
    -			findNextValidCursor();
    -		}
    -		return currentCursorIsValid();
    -	}
    -
    -	@Override
    -	public RyaStatement next() {
    -		if (!currentCursorIsValid()) {
    -			findNextValidCursor();
    -		}
    -		if (currentCursorIsValid()) {
    -			// convert to Rya Statement
    -			DBObject queryResult = currentCursor.next();
    -			RyaStatement statement = strategy.deserializeDBObject(queryResult);
    -			return statement;
    -		}
    -		return null;
    -	}
    -	
    -	private void findNextValidCursor() {
    -		while (queryIterator.hasNext()){
    -			DBObject currentQuery = queryIterator.next();
    -			currentCursor = coll.find(currentQuery);
    -			if (currentCursor.hasNext()) break;
    -		}
    -	}
    -	
    -	private boolean currentCursorIsValid() {
    -		return (currentCursor != null) && currentCursor.hasNext();
    -	}
    -
    -
    -	public void setMaxResults(Long maxResults) {
    -		this.maxResults = maxResults;
    -	}
    -
    -	@Override
    -	public void close() throws RyaDAOException {
    -		// TODO don't know what to do here
    -	}
    -
    -	@Override
    -	public void remove() throws RyaDAOException {
    -		next();
    -	}
    -
    +    private static final Logger log = Logger.getLogger(RyaStatementCursorIterator.class);
    +
    +    private final DBCollection coll;
    +    private final Iterator<DBObject> queryIterator;
    +    private Iterator<DBObject> resultsIterator;
    +    private final MongoDBStorageStrategy<RyaStatement> strategy;
    +    private Long maxResults;
    +    private final Authorizations auths;
    +
    +    public RyaStatementCursorIterator(final DBCollection coll, final Set<DBObject> queries, final MongoDBStorageStrategy<RyaStatement> strategy, final MongoDBRdfConfiguration conf) {
    +        this.coll = coll;
    +        this.queryIterator = queries.iterator();
    +        this.strategy = strategy;
    +        if (conf != null) {
    +            this.auths = conf.getAuthorizations();
    +        } else {
    +            auths = MongoDbRdfConstants.ALL_AUTHORIZATIONS;
    +        }
    +    }
    +
    +    @Override
    +    public boolean hasNext() {
    +        if (!currentCursorIsValid()) {
    +            findNextValidCursor();
    +        }
    +        return currentCursorIsValid();
    +    }
    +
    +    @Override
    +    public RyaStatement next() {
    +        if (!currentCursorIsValid()) {
    +            findNextValidCursor();
    +        }
    +        if (currentCursorIsValid()) {
    +            // convert to Rya Statement
    +            final DBObject queryResult = resultsIterator.next();
    +            final RyaStatement statement = strategy.deserializeDBObject(queryResult);
    +            return statement;
    +        }
    +        return null;
    +    }
    +
    +    private void findNextValidCursor() {
    +        while (queryIterator.hasNext()){
    +            final DBObject currentQuery = queryIterator.next();
    +
    +            // Executing redact aggregation to only return documents the user
    +            // has access to.
    +            final List<DBObject> pipeline = new ArrayList<>();
    +            pipeline.add(new BasicDBObject("$match", currentQuery));
    +            pipeline.addAll(AggregationUtil.createRedactPipeline(auths));
    +            log.debug(pipeline);
    +            final AggregationOutput output = coll.aggregate(pipeline);
    +            resultsIterator = output.results().iterator();
    +            if (resultsIterator.hasNext()) {
    +                break;
    +            }
    +        }
    +    }
    +
    +    private boolean currentCursorIsValid() {
    +        return (resultsIterator != null) && resultsIterator.hasNext();
    +    }
    +
    +
    +    public void setMaxResults(final Long maxResults) {
    +        this.maxResults = maxResults;
    +    }
    +
    +    @Override
    +    public void close() throws RyaDAOException {
    +        // TODO don't know what to do here
    --- End diff --
    
    why are we implementing ClosableIterable when we ignore close()?.....not this problem though


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90068092
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    + */
    +public class Authorizations implements Iterable<byte[]>, Serializable, AuthorizationContainer {
    +
    +  private static final long serialVersionUID = 1L;
    +
    +  private final Set<ByteSequence> auths = new HashSet<ByteSequence>();
    +  private final List<byte[]> authsList = new ArrayList<byte[]>(); // sorted order
    +
    +  /**
    +   * An empty set of authorizations.
    +   */
    +  public static final Authorizations EMPTY = new Authorizations();
    +
    +  private static final boolean[] validAuthChars = new boolean[256];
    +
    +  /**
    +   * A special header string used when serializing instances of this class.
    +   *
    +   * @see #serialize()
    +   */
    +  public static final String HEADER = "!AUTH1:";
    +
    +  static {
    +    for (int i = 0; i < 256; i++) {
    +      validAuthChars[i] = false;
    +    }
    +
    +    for (int i = 'a'; i <= 'z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = 'A'; i <= 'Z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = '0'; i <= '9'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    validAuthChars['_'] = true;
    +    validAuthChars['-'] = true;
    +    validAuthChars[':'] = true;
    +    validAuthChars['.'] = true;
    +    validAuthChars['/'] = true;
    +  }
    +
    +  static final boolean isValidAuthChar(final byte b) {
    +    return validAuthChars[0xff & b];
    +  }
    +
    +  private void checkAuths() {
    +    final Set<ByteSequence> sortedAuths = new TreeSet<ByteSequence>(auths);
    +
    +    for (final ByteSequence bs : sortedAuths) {
    +      if (bs.length() == 0) {
    +        throw new IllegalArgumentException("Empty authorization");
    +      }
    +
    +      authsList.add(bs.toArray());
    +    }
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a collection of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does
    +   * not verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          collection of authorizations, as strings encoded in UTF-8
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final Collection<byte[]> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    --- End diff --
    
    same as above


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89908758
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/iter/AggregationUtil.java ---
    @@ -0,0 +1,322 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.mongodb.iter;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.LinkedList;
    +import java.util.List;
    +
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +
    +import com.google.common.collect.Lists;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +
    +/**
    + * Utility methods for MongoDB aggregation.
    + */
    +public final class AggregationUtil {
    +    /**
    +     * Private constructor to prevent instantiation.
    +     */
    +    private AggregationUtil() {
    +    }
    +
    +    /**
    +     * Creates a MongoDB $redact aggregation pipeline that only include
    +     * documents whose document visibility match the provided authorizations.
    +     * All other documents are excluded.
    +     * @param authorizations the {@link Authorization}s to include in the
    +     * $redact. Only documents that match the authorizations will be returned.
    +     * @return the {@link List} of {@link DBObject}s that represents the $redact
    +     * aggregation pipeline.
    +     */
    +    public static List<DBObject> createRedactPipeline(final Authorizations authorizations) {
    +        if (MongoDbRdfConstants.ALL_AUTHORIZATIONS.equals(authorizations)) {
    +            return Lists.newArrayList();
    +        }
    +        final List<String> authAndList = authorizations.getAuthorizationsStrings();
    +
    +        // Generate all combinations of the authorization strings without repetition.
    +        final List<List<String>> authOrList = createCombinations(authorizations.getAuthorizationsStrings());
    +
    +        final String documentVisibilityField = "$" + SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +
    +        final BasicDBObject setIsSubset =
    +            setIsSubsetNullSafe(
    +                documentVisibilityField,
    +                authAndList.toArray()
    +            );
    +
    +        final BasicDBObject setIntersectionExists =
    +            gt(
    +                size(
    +                    setIntersection(
    +                        documentVisibilityField,
    +                        authOrList.toArray()
    +                    )
    +                ),
    +                0
    +            );
    +
    +        final BasicDBObject orExpression = or(setIsSubset, setIntersectionExists);
    +
    +        final List<DBObject> pipeline = new ArrayList<>();
    +        pipeline.add(
    +            redact(
    +               orExpression,
    +               "$$DESCEND",
    +               "$$PRUNE"
    +            )
    +        );
    +
    +        return pipeline;
    +    }
    +
    +    /**
    +     * Creates all combinations of the values that are of the size of value
    +     * array or smaller without repetition.
    +     * @param values the {@link List} of values to create combinations from.
    +     * @return the {@link List} of combinations.
    +     */
    +    public static <T> List<List<T>> createCombinations(final List<T> values) {
    +        final List<List<T>> allCombinations = new ArrayList<>();
    +        for (int i = 1; i <= values.size(); i++) {
    +            allCombinations.addAll(createCombinations(values, i));
    +        }
    +        return allCombinations;
    +    }
    +
    +    /**
    +     * Creates all combinations of the values that are of the specified size
    +     * without repetition.
    +     * @param values the {@link List} of values to create combinations from.
    +     * @param size the size of the combinations.
    +     * @return the {@link List} of combinations.
    +     */
    +    public static <T> List<List<T>> createCombinations(final List<T> values, final int size) {
    +        if (0 == size) {
    +            return Collections.singletonList(Collections.<T> emptyList());
    +        }
    +
    +        if (values.isEmpty()) {
    +            return Collections.emptyList();
    +        }
    +
    +        final List<List<T>> combination = new LinkedList<List<T>>();
    +
    +        final T actual = values.iterator().next();
    +
    +        final List<T> subSet = new LinkedList<T>(values);
    +        subSet.remove(actual);
    +
    +        final List<List<T>> subSetCombination = createCombinations(subSet, size - 1);
    +
    +        for (final List<T> set : subSetCombination) {
    +            final List<T> newSet = new LinkedList<T>(set);
    +            newSet.add(0, actual);
    +            combination.add(newSet);
    +        }
    +
    +        combination.addAll(createCombinations(subSet, size));
    +
    +        return combination;
    +    }
    +
    +    /**
    +     * Creates an "if-then-else" MongoDB expression.
    +     * @param ifStatement the "if" statement {@link BasicDBObject}.
    +     * @param thenResult the {@link Object} to return when the
    +     * {@code ifStatement} is {@code true}.
    +     * @param elseResult the {@link Object} to return when the
    +     * {@code ifStatement} is {@code false}.
    +     * @return the "if" expression {@link BasicDBObject}.
    +     */
    +    public static BasicDBObject ifThenElse(final BasicDBObject ifStatement, final Object thenResult, final Object elseResult) {
    --- End diff --
    
    I have concerns with these mongo-adapting helper functions being here, they are not isolated to aggregation.   I feel like a lot of these exist in the BDObjectBuilder


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89905453
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/BadArgumentException.java ---
    @@ -0,0 +1,27 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import java.util.regex.PatternSyntaxException;
    +
    +public final class BadArgumentException extends PatternSyntaxException {
    --- End diff --
    
    Doc


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89910082
  
    --- Diff: dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOTest.java ---
    @@ -28,102 +30,279 @@
     import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
     import org.apache.rya.api.domain.RyaURI;
     import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.api.persist.query.RyaQuery;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.bson.Document;
    +import org.calrissian.mango.collect.CloseableIterable;
     import org.junit.Before;
     import org.junit.Test;
     
    -import com.mongodb.DB;
    -import com.mongodb.DBCollection;
     import com.mongodb.MongoException;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.MongoDatabase;
     
     public class MongoDBRyaDAOTest extends MongoRyaTestBase {
     
    -	private MongoDBRyaDAO dao;
    -	private MongoDBRdfConfiguration configuration;
    +    private MongoDBRyaDAO dao;
    +    private MongoDBRdfConfiguration configuration;
     
    -	@Before
    -	public void setUp() throws IOException, RyaDAOException{
    -		final Configuration conf = new Configuration();
    +    @Before
    +    public void setUp() throws IOException, RyaDAOException{
    +        final Configuration conf = new Configuration();
             conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, "test");
             conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
             conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya_");
             configuration = new MongoDBRdfConfiguration(conf);
             final int port = mongoClient.getServerAddressList().get(0).getPort();
             configuration.set(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT, ""+port);
    -		dao = new MongoDBRyaDAO(configuration, mongoClient);
    -	}
    +        dao = new MongoDBRyaDAO(configuration, mongoClient);
    +    }
     
     
    -	@Test
    -	public void testDeleteWildcard() throws RyaDAOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		dao.delete(builder.build(), configuration);
    -	}
    +    @Test
    +    public void testDeleteWildcard() throws RyaDAOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        dao.delete(builder.build(), configuration);
    +    }
     
     
    -	@Test
    -	public void testAdd() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    +    @Test
    +    public void testAdd() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(builder.build());
    +        dao.add(builder.build());
     
             assertEquals(coll.count(),1);
     
    -	}
    +    }
     
    -	@Test
    -	public void testDelete() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    -		final RyaStatement statement = builder.build();
    +    @Test
    +    public void testDelete() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
    +        final RyaStatement statement = builder.build();
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(statement);
    +        dao.add(statement);
     
             assertEquals(coll.count(),1);
     
             dao.delete(statement, configuration);
     
             assertEquals(coll.count(),0);
     
    -	}
    +    }
     
    -	@Test
    -	public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    -		builder.setContext(new RyaURI("http://context.com"));
    -		final RyaStatement statement = builder.build();
    +    @Test
    +    public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
    +        builder.setContext(new RyaURI("http://context.com"));
    +        final RyaStatement statement = builder.build();
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(statement);
    +        dao.add(statement);
     
             assertEquals(coll.count(),1);
     
    -		final RyaStatementBuilder builder2 = new RyaStatementBuilder();
    -		builder2.setPredicate(new RyaURI("http://temp.com"));
    -		builder2.setObject(new RyaURI("http://object.com"));
    -		builder2.setContext(new RyaURI("http://context3.com"));
    -		final RyaStatement query = builder2.build();
    +        final RyaStatementBuilder builder2 = new RyaStatementBuilder();
    +        builder2.setPredicate(new RyaURI("http://temp.com"));
    +        builder2.setObject(new RyaURI("http://object.com"));
    +        builder2.setContext(new RyaURI("http://context3.com"));
    +        final RyaStatement query = builder2.build();
     
             dao.delete(query, configuration);
     
             assertEquals(coll.count(),1);
     
    -	}
    +    }
     
    +    @Test
    +    public void testVisibility() throws RyaDAOException, MongoException, IOException {
    +        // Doc requires "A" and user has "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A", new Authorizations("B")));
    +
    +        // Doc requires "A" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A")));
    +
    +        // Doc requires "A" and "B" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or "B" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and "B" and user has "A" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A&B", new Authorizations("A")));
    +
    +        // Doc requires "A" and "B" and "C" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A&B&C", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and "B" and "C" and user has "A" and "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A&B&C", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and "B" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" or "B" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A")));
    +
    +        // Doc requires "A" or "B" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A", Authorizations.EMPTY));
    +
    +        // Doc requires "A" and "B" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A&B", Authorizations.EMPTY));
    +
    +        // Doc requires "A" or "B" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A|B", Authorizations.EMPTY));
    +
    +        // Doc has no requirement and user has "" = User can view
    +        assertTrue(testVisibilityStatement("", Authorizations.EMPTY));
    +
    +        // Doc has no requirement and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("", new Authorizations("A")));
    +
    +        // Doc has no requirement and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or ("B" and "C") and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|(B&C)", new Authorizations("A")));
    +
    +        // Doc requires "A" or ("B" and "C") and user has "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|(B&C)", new Authorizations("B", "C")));
    +
    +        // Doc requires "A" and ("B" or "C") and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A&(B|C)", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and ("B" or "C") and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A&(B|C)", new Authorizations("A", "C")));
    +
    +        // Doc requires "(A|B)&(C|(D&E))" and user has "A" and "C" = User can view
    +        assertTrue(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("A", "C")));
    +
    +        // Doc requires "(A|B)&(C|(D&E))" and user has "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("B", "C")));
    +
    +        // Doc requires "(A|B)&(C|(D&E))" and user has "A" and "D" and "E" = User can view
    +        assertTrue(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("A", "D", "E")));
    +
    +        // Doc requires "(A|B)&(C|(D&E))" and user has "B" and "D" and "E" = User can view
    +        assertTrue(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("B", "D", "E")));
    +
    +        // Doc requires "(A|B)&(C|(D&E))" and user has "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("(A|B)&(C|(D&E))", new Authorizations("B")));
    +
    +        // Doc requires "A|(B&C&(D|E))" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|(B&C&(D|E))", new Authorizations("A")));
    +
    +        // Doc requires "A|(B&C&(D|E))" and user has "B" and "C" and "D" = User can view
    +        assertTrue(testVisibilityStatement("A|(B&C&(D|E))", new Authorizations("B", "C", "D")));
    +
    +        // Doc requires "A|(B&C&(D|E))" and user has "B" and "C" and "E" = User can view
    +        assertTrue(testVisibilityStatement("A|(B&C&(D|E))", new Authorizations("B", "C", "E")));
    +
    +        // Doc requires "A|(B&C&(D|E))" and user has "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A|(B&C&(D|E))", new Authorizations("B")));
    +
    +        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("A")));
    +
    +        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "E" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("E")));
    +
    +        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "E" and "F" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("E", "F")));
    +
    +        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "I" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("I")));
    +
    +        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "A" and "I" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("A", "I")));
    +
    +        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "E" and "F" and "G" and "H" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("E", "F", "G", "H")));
    +
    +        // Doc requires "A|B|C|D|(E&F&G&H)" and user has "E" and "F" and "G" and "H" and "I" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C|D|(E&F&G&H)", new Authorizations("E", "F", "G", "H", "I")));
    +    }
    +
    +    /**
    +     * Generates a test statement with the provided document visibility to
    +     * determine if the specified user authorization can view the statement.
    +     * @param documentVisibility the document visibility boolean expression
    +     * string.
    +     * @param userAuthorizations the user authorization strings.
    +     * @return {@code true} if provided authorization could access the document
    +     * in the collection. {@code false} otherwise.
    +     * @throws RyaDAOException
    +     */
    +    private boolean testVisibilityStatement(final String documentVisibility, final Authorizations userAuthorizations) throws RyaDAOException {
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    --- End diff --
    
    hitting mongo makes this an integration test.  can you move these out to another file for visiblity integration tests?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

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


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90357571
  
    --- Diff: dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOTest.java ---
    @@ -28,102 +30,279 @@
     import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
     import org.apache.rya.api.domain.RyaURI;
     import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.api.persist.query.RyaQuery;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.bson.Document;
    +import org.calrissian.mango.collect.CloseableIterable;
     import org.junit.Before;
     import org.junit.Test;
     
    -import com.mongodb.DB;
    -import com.mongodb.DBCollection;
     import com.mongodb.MongoException;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.MongoDatabase;
     
     public class MongoDBRyaDAOTest extends MongoRyaTestBase {
     
    -	private MongoDBRyaDAO dao;
    -	private MongoDBRdfConfiguration configuration;
    +    private MongoDBRyaDAO dao;
    +    private MongoDBRdfConfiguration configuration;
     
    -	@Before
    -	public void setUp() throws IOException, RyaDAOException{
    -		final Configuration conf = new Configuration();
    +    @Before
    +    public void setUp() throws IOException, RyaDAOException{
    +        final Configuration conf = new Configuration();
             conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, "test");
             conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
             conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya_");
             configuration = new MongoDBRdfConfiguration(conf);
             final int port = mongoClient.getServerAddressList().get(0).getPort();
             configuration.set(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT, ""+port);
    -		dao = new MongoDBRyaDAO(configuration, mongoClient);
    -	}
    +        dao = new MongoDBRyaDAO(configuration, mongoClient);
    +    }
     
     
    -	@Test
    -	public void testDeleteWildcard() throws RyaDAOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		dao.delete(builder.build(), configuration);
    -	}
    +    @Test
    +    public void testDeleteWildcard() throws RyaDAOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        dao.delete(builder.build(), configuration);
    +    }
     
     
    -	@Test
    -	public void testAdd() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    +    @Test
    +    public void testAdd() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(builder.build());
    +        dao.add(builder.build());
     
             assertEquals(coll.count(),1);
     
    -	}
    +    }
     
    -	@Test
    -	public void testDelete() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    -		final RyaStatement statement = builder.build();
    +    @Test
    +    public void testDelete() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
    +        final RyaStatement statement = builder.build();
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(statement);
    +        dao.add(statement);
     
             assertEquals(coll.count(),1);
     
             dao.delete(statement, configuration);
     
             assertEquals(coll.count(),0);
     
    -	}
    +    }
     
    -	@Test
    -	public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    -		builder.setContext(new RyaURI("http://context.com"));
    -		final RyaStatement statement = builder.build();
    +    @Test
    +    public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
    +        builder.setContext(new RyaURI("http://context.com"));
    +        final RyaStatement statement = builder.build();
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(statement);
    +        dao.add(statement);
     
             assertEquals(coll.count(),1);
     
    -		final RyaStatementBuilder builder2 = new RyaStatementBuilder();
    -		builder2.setPredicate(new RyaURI("http://temp.com"));
    -		builder2.setObject(new RyaURI("http://object.com"));
    -		builder2.setContext(new RyaURI("http://context3.com"));
    -		final RyaStatement query = builder2.build();
    +        final RyaStatementBuilder builder2 = new RyaStatementBuilder();
    +        builder2.setPredicate(new RyaURI("http://temp.com"));
    +        builder2.setObject(new RyaURI("http://object.com"));
    +        builder2.setContext(new RyaURI("http://context3.com"));
    +        final RyaStatement query = builder2.build();
     
             dao.delete(query, configuration);
     
             assertEquals(coll.count(),1);
     
    -	}
    +    }
     
    +    @Test
    +    public void testVisibility() throws RyaDAOException, MongoException, IOException {
    +        // Doc requires "A" and user has "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A", new Authorizations("B")));
    +
    +        // Doc requires "A" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A")));
    +
    +        // Doc requires "A" and "B" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or "B" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and "B" and user has "A" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A&B", new Authorizations("A")));
    +
    +        // Doc requires "A" and "B" and "C" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A&B&C", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and "B" and "C" and user has "A" and "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A&B&C", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and "B" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" or "B" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A")));
    +
    +        // Doc requires "A" or "B" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A", Authorizations.EMPTY));
    +
    +        // Doc requires "A" and "B" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A&B", Authorizations.EMPTY));
    +
    +        // Doc requires "A" or "B" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A|B", Authorizations.EMPTY));
    --- End diff --
    
    Switched to MongoDbRdfConstants.ALL_AUTHORIZATIONS so it makes more sense.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89908937
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/iter/RyaStatementBindingSetCursorIterator.java ---
    @@ -22,104 +44,104 @@
     
     import info.aduna.iteration.CloseableIteration;
     
    -import java.util.Collection;
    -import java.util.Iterator;
    -import java.util.Map.Entry;
    -
    -import org.apache.rya.api.RdfCloudTripleStoreUtils;
    -import org.apache.rya.api.domain.RyaStatement;
    -import org.apache.rya.api.persist.RyaDAOException;
    -import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    -
    -import org.openrdf.query.BindingSet;
    -
    -import com.google.common.collect.Multimap;
    -import com.mongodb.DBCollection;
    -import com.mongodb.DBCursor;
    -import com.mongodb.DBObject;
    -
     public class RyaStatementBindingSetCursorIterator implements CloseableIteration<Entry<RyaStatement, BindingSet>, RyaDAOException> {
    -
    -	private DBCollection coll;
    -	private Multimap<DBObject, BindingSet> rangeMap;
    -	private Iterator<DBObject> queryIterator;
    -	private Long maxResults;
    -	private DBCursor resultCursor;
    -	private RyaStatement currentStatement;
    -	private Collection<BindingSet> currentBindingSetCollection;
    -	private Iterator<BindingSet> currentBindingSetIterator;
    -	private MongoDBStorageStrategy strategy;
    -
    -	public RyaStatementBindingSetCursorIterator(DBCollection coll,
    -			Multimap<DBObject, BindingSet> rangeMap, MongoDBStorageStrategy strategy) {
    -		this.coll = coll;
    -		this.rangeMap = rangeMap;
    -		this.queryIterator = rangeMap.keySet().iterator();
    -		this.strategy = strategy;
    -	}
    -
    -	@Override
    -	public boolean hasNext() {
    -		if (!currentBindingSetIteratorIsValid()) {
    -			findNextResult();
    -		}
    -		return currentBindingSetIteratorIsValid();
    -	}
    -
    -	@Override
    -	public Entry<RyaStatement, BindingSet> next() {
    -		if (!currentBindingSetIteratorIsValid()) {
    -			findNextResult();
    -		}
    -		if (currentBindingSetIteratorIsValid()) {
    -			BindingSet currentBindingSet = currentBindingSetIterator.next();
    -			return new RdfCloudTripleStoreUtils.CustomEntry<RyaStatement, BindingSet>(currentStatement, currentBindingSet);
    -		}
    -		return null;
    -	}
    -	
    -	private boolean currentBindingSetIteratorIsValid() {
    -		return (currentBindingSetIterator != null) && currentBindingSetIterator.hasNext();
    -	}
    -
    -	private void findNextResult() {
    -		if (!currentResultCursorIsValid()) {
    -			findNextValidResultCursor();
    -		}
    -		if (currentResultCursorIsValid()) {
    -			// convert to Rya Statement
    -			DBObject queryResult = resultCursor.next();
    -			currentStatement = strategy.deserializeDBObject(queryResult);
    -			currentBindingSetIterator = currentBindingSetCollection.iterator();
    -		}
    -	}
    -
    -	private void findNextValidResultCursor() {
    -		while (queryIterator.hasNext()){
    -			DBObject currentQuery = queryIterator.next();
    -			resultCursor = coll.find(currentQuery);
    -			currentBindingSetCollection = rangeMap.get(currentQuery);
    -			if (resultCursor.hasNext()) return;
    -		}
    -	}
    -	
    -	private boolean currentResultCursorIsValid() {
    -		return (resultCursor != null) && resultCursor.hasNext();
    -	}
    -
    -
    -	public void setMaxResults(Long maxResults) {
    -		this.maxResults = maxResults;
    -	}
    -
    -	@Override
    -	public void close() throws RyaDAOException {
    -		// TODO don't know what to do here
    -	}
    -
    -	@Override
    -	public void remove() throws RyaDAOException {
    -		next();
    -	}
    +    private static final Logger log = Logger.getLogger(RyaStatementBindingSetCursorIterator.class);
    +
    +    private final DBCollection coll;
    +    private final Multimap<DBObject, BindingSet> rangeMap;
    +    private final Iterator<DBObject> queryIterator;
    +    private Long maxResults;
    +    private Iterator<DBObject> resultsIterator;
    +    private RyaStatement currentStatement;
    +    private Collection<BindingSet> currentBindingSetCollection;
    +    private Iterator<BindingSet> currentBindingSetIterator;
    +    private final MongoDBStorageStrategy<RyaStatement> strategy;
    +    private final Authorizations auths;
    +
    +    public RyaStatementBindingSetCursorIterator(final DBCollection coll,
    --- End diff --
    
    don't pass in the whole config, just pass in the Auths


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89903895
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    + */
    +public class Authorizations implements Iterable<byte[]>, Serializable, AuthorizationContainer {
    +
    +  private static final long serialVersionUID = 1L;
    +
    +  private final Set<ByteSequence> auths = new HashSet<ByteSequence>();
    +  private final List<byte[]> authsList = new ArrayList<byte[]>(); // sorted order
    +
    +  /**
    +   * An empty set of authorizations.
    +   */
    +  public static final Authorizations EMPTY = new Authorizations();
    +
    +  private static final boolean[] validAuthChars = new boolean[256];
    +
    +  /**
    +   * A special header string used when serializing instances of this class.
    +   *
    +   * @see #serialize()
    +   */
    +  public static final String HEADER = "!AUTH1:";
    +
    +  static {
    --- End diff --
    
    replace this with a regex pattern?  then just do a match on it?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90357174
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/iter/AggregationUtil.java ---
    @@ -0,0 +1,322 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.mongodb.iter;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.LinkedList;
    +import java.util.List;
    +
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +
    +import com.google.common.collect.Lists;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +
    +/**
    + * Utility methods for MongoDB aggregation.
    + */
    +public final class AggregationUtil {
    +    /**
    +     * Private constructor to prevent instantiation.
    +     */
    +    private AggregationUtil() {
    +    }
    +
    +    /**
    +     * Creates a MongoDB $redact aggregation pipeline that only include
    +     * documents whose document visibility match the provided authorizations.
    +     * All other documents are excluded.
    +     * @param authorizations the {@link Authorization}s to include in the
    +     * $redact. Only documents that match the authorizations will be returned.
    +     * @return the {@link List} of {@link DBObject}s that represents the $redact
    +     * aggregation pipeline.
    +     */
    +    public static List<DBObject> createRedactPipeline(final Authorizations authorizations) {
    +        if (MongoDbRdfConstants.ALL_AUTHORIZATIONS.equals(authorizations)) {
    +            return Lists.newArrayList();
    +        }
    +        final List<String> authAndList = authorizations.getAuthorizationsStrings();
    +
    +        // Generate all combinations of the authorization strings without repetition.
    +        final List<List<String>> authOrList = createCombinations(authorizations.getAuthorizationsStrings());
    +
    +        final String documentVisibilityField = "$" + SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +
    +        final BasicDBObject setIsSubset =
    +            setIsSubsetNullSafe(
    +                documentVisibilityField,
    +                authAndList.toArray()
    +            );
    +
    +        final BasicDBObject setIntersectionExists =
    +            gt(
    +                size(
    +                    setIntersection(
    +                        documentVisibilityField,
    +                        authOrList.toArray()
    +                    )
    +                ),
    +                0
    +            );
    +
    +        final BasicDBObject orExpression = or(setIsSubset, setIntersectionExists);
    +
    +        final List<DBObject> pipeline = new ArrayList<>();
    +        pipeline.add(
    +            redact(
    +               orExpression,
    +               "$$DESCEND",
    +               "$$PRUNE"
    +            )
    +        );
    +
    +        return pipeline;
    +    }
    +
    +    /**
    +     * Creates all combinations of the values that are of the size of value
    +     * array or smaller without repetition.
    +     * @param values the {@link List} of values to create combinations from.
    +     * @return the {@link List} of combinations.
    +     */
    +    public static <T> List<List<T>> createCombinations(final List<T> values) {
    +        final List<List<T>> allCombinations = new ArrayList<>();
    +        for (int i = 1; i <= values.size(); i++) {
    +            allCombinations.addAll(createCombinations(values, i));
    +        }
    +        return allCombinations;
    +    }
    +
    +    /**
    +     * Creates all combinations of the values that are of the specified size
    +     * without repetition.
    +     * @param values the {@link List} of values to create combinations from.
    +     * @param size the size of the combinations.
    +     * @return the {@link List} of combinations.
    +     */
    +    public static <T> List<List<T>> createCombinations(final List<T> values, final int size) {
    +        if (0 == size) {
    +            return Collections.singletonList(Collections.<T> emptyList());
    +        }
    +
    +        if (values.isEmpty()) {
    +            return Collections.emptyList();
    +        }
    +
    +        final List<List<T>> combination = new LinkedList<List<T>>();
    +
    +        final T actual = values.iterator().next();
    +
    +        final List<T> subSet = new LinkedList<T>(values);
    +        subSet.remove(actual);
    +
    +        final List<List<T>> subSetCombination = createCombinations(subSet, size - 1);
    +
    +        for (final List<T> set : subSetCombination) {
    +            final List<T> newSet = new LinkedList<T>(set);
    +            newSet.add(0, actual);
    +            combination.add(newSet);
    +        }
    +
    +        combination.addAll(createCombinations(subSet, size));
    +
    +        return combination;
    +    }
    +
    +    /**
    +     * Creates an "if-then-else" MongoDB expression.
    +     * @param ifStatement the "if" statement {@link BasicDBObject}.
    +     * @param thenResult the {@link Object} to return when the
    +     * {@code ifStatement} is {@code true}.
    +     * @param elseResult the {@link Object} to return when the
    +     * {@code ifStatement} is {@code false}.
    +     * @return the "if" expression {@link BasicDBObject}.
    +     */
    +    public static BasicDBObject ifThenElse(final BasicDBObject ifStatement, final Object thenResult, final Object elseResult) {
    --- End diff --
    
    These functions aren't available in BasicDBObjectBuilder.  I'll make the methods that aren't aggregation specific private.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90055877
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDbRdfConstants.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.mongodb;
    +
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
    +
    +/**
    + * Interface MongoDbRdfConstants.
    + */
    +public interface MongoDbRdfConstants {
    +    public static final Authorizations ALL_AUTHORIZATIONS = Authorizations.EMPTY;
    --- End diff --
    
    Added docs


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89905934
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/ByteSequence.java ---
    @@ -0,0 +1,100 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import org.apache.hadoop.io.WritableComparator;
    +
    +public abstract class ByteSequence implements Comparable<ByteSequence> {
    --- End diff --
    
    doc.  Whats the different between a ByteSequence, a byte[], a ByteBuffer?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89907937
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/VisibilityEvaluator.java ---
    @@ -0,0 +1,152 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import java.util.ArrayList;
    +
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.Node;
    +
    +/**
    + * A class which evaluates visibility expressions against a set of authorizations.
    + */
    +public class VisibilityEvaluator {
    +  private final AuthorizationContainer auths;
    +
    +  /**
    +   * Creates a new {@link Authorizations} object with escaped forms of the authorizations in the given object.
    +   *
    +   * @param auths
    +   *          original authorizations
    +   * @return authorizations object with escaped authorization strings
    +   * @see #escape(byte[], boolean)
    +   */
    +  static Authorizations escape(final Authorizations auths) {
    +    final ArrayList<byte[]> retAuths = new ArrayList<byte[]>(auths.getAuthorizations().size());
    +
    +    for (final byte[] auth : auths.getAuthorizations()) {
    +		retAuths.add(escape(auth, false));
    +	}
    +
    +    return new Authorizations(retAuths);
    +  }
    +
    +  /**
    +   * Properly escapes an authorization string. The string can be quoted if desired.
    +   *
    +   * @param auth
    +   *          authorization string, as UTF-8 encoded bytes
    +   * @param quote
    +   *          true to wrap escaped authorization in quotes
    +   * @return escaped authorization string
    +   */
    +  public static byte[] escape(byte[] auth, final boolean quote) {
    +    int escapeCount = 0;
    +
    +    for (final byte element : auth) {
    +		if (element == '"' || element == '\\') {
    +			escapeCount++;
    +		}
    +	}
    +
    +    if (escapeCount > 0 || quote) {
    +      final byte[] escapedAuth = new byte[auth.length + escapeCount + (quote ? 2 : 0)];
    +      int index = quote ? 1 : 0;
    +      for (final byte element : auth) {
    +        if (element == '"' || element == '\\') {
    +			escapedAuth[index++] = '\\';
    +		}
    +        escapedAuth[index++] = element;
    +      }
    +
    +      if (quote) {
    +        escapedAuth[0] = '"';
    +        escapedAuth[escapedAuth.length - 1] = '"';
    +      }
    +
    +      auth = escapedAuth;
    +    }
    +    return auth;
    +  }
    +
    +//  /**
    --- End diff --
    
    remove doc'd code


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89906753
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/DocumentVisibility.java ---
    @@ -0,0 +1,584 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static java.nio.charset.StandardCharsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.List;
    +import java.util.TreeSet;
    +
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.io.WritableComparator;
    +
    +/**
    + * Validate the document visibility is a valid expression and set the visibility for a Mutation. See {@link DocumentVisibility#DocumentVisibility(byte[])} for the
    + * definition of an expression.
    + *
    + * <p>
    + * The expression is a sequence of characters from the set [A-Za-z0-9_-.] along with the binary operators "&amp;" and "|" indicating that both operands are
    + * necessary, or the either is necessary. The following are valid expressions for visibility:
    + *
    + * <pre>
    + * A
    + * A|B
    + * (A|B)&amp;(C|D)
    + * orange|(red&amp;yellow)
    + * </pre>
    + *
    + * <p>
    + * The following are not valid expressions for visibility:
    + *
    + * <pre>
    + * A|B&amp;C
    + * A=B
    + * A|B|
    + * A&amp;|B
    + * ()
    + * )
    + * dog|!cat
    + * </pre>
    + *
    + * <p>
    + * In addition to the base set of visibilities, any character can be used in the expression if it is quoted. If the quoted term contains '&quot;' or '\', then
    + * escape the character with '\'. The {@link #quote(String)} method can be used to properly quote and escape terms automatically. The following is an example of
    + * a quoted term:
    + *
    + * <pre>
    + * &quot;A#C&quot; &amp; B
    + * </pre>
    + */
    +public class DocumentVisibility {
    +
    +  Node node = null;
    +  private byte[] expression;
    +
    +  /**
    +   * Accessor for the underlying byte string.
    +   *
    +   * @return byte array representation of a visibility expression
    +   */
    +  public byte[] getExpression() {
    +    return expression;
    +  }
    +
    +  /**
    +   * The node types in a parse tree for a visibility expression.
    +   */
    +  public static enum NodeType {
    +    EMPTY, TERM, OR, AND,
    +  }
    +
    +  /**
    +   * All empty nodes are equal and represent the same value.
    +   */
    +  private static final Node EMPTY_NODE = new Node("".getBytes(), NodeType.EMPTY, 0);
    +
    +  /**
    +   * A node in the parse tree for a visibility expression.
    +   */
    +  public static class Node {
    +    /**
    +     * An empty list of nodes.
    +     */
    +    public final static List<Node> EMPTY = Collections.emptyList();
    +    NodeType type;
    +    int start;
    +    int end;
    +    List<Node> children = EMPTY;
    +    byte[] expression;
    +
    +    public Node(final byte[] expression, final NodeType type, final int start) {
    +      this.type = type;
    +      this.start = start;
    +      this.end = start + 1;
    +      this.expression = expression;
    +    }
    +
    +    public Node(final byte[] expression, final int start, final int end) {
    +      this.type = NodeType.TERM;
    +      this.start = start;
    +      this.end = end;
    +      this.expression = expression;
    +    }
    +
    +    public void add(final Node child) {
    +      if (children == EMPTY) {
    +		children = new ArrayList<>();
    +	}
    +
    +      children.add(child);
    +    }
    +
    +    public NodeType getType() {
    +      return type;
    +    }
    +
    +    public List<Node> getChildren() {
    +      return children;
    +    }
    +
    +    public int getTermStart() {
    +      return start;
    +    }
    +
    +    public int getTermEnd() {
    +      return end;
    +    }
    +
    +    public byte[] getExpression() {
    +      return expression;
    +    }
    +
    +    public ByteSequence getTerm(final byte expression[]) {
    +      if (type != NodeType.TERM) {
    +		throw new RuntimeException();
    +	}
    +
    +      if (expression[start] == '"') {
    +        // its a quoted term
    +        final int qStart = start + 1;
    +        final int qEnd = end - 1;
    +
    +        return new ArrayByteSequence(expression, qStart, qEnd - qStart);
    +      }
    +      return new ArrayByteSequence(expression, start, end - start);
    +    }
    +  }
    +
    +  /**
    +   * A node comparator. Nodes sort according to node type, terms sort lexicographically. AND and OR nodes sort by number of children, or if the same by
    +   * corresponding children.
    +   */
    +  public static class NodeComparator implements Comparator<Node>, Serializable {
    +
    +    private static final long serialVersionUID = 1L;
    +    byte[] text;
    +
    +    /**
    +     * Creates a new comparator.
    +     *
    +     * @param text
    +     *          expression string, encoded in UTF-8
    +     */
    +    public NodeComparator(final byte[] text) {
    +      this.text = text;
    +    }
    +
    +    @Override
    +    public int compare(final Node a, final Node b) {
    +      int diff = a.type.ordinal() - b.type.ordinal();
    +      if (diff != 0) {
    +		return diff;
    +	}
    +      switch (a.type) {
    +        case EMPTY:
    +          return 0; // All empty nodes are the same
    +        case TERM:
    +          return WritableComparator.compareBytes(text, a.start, a.end - a.start, text, b.start, b.end - b.start);
    +        case OR:
    +        case AND:
    +          diff = a.children.size() - b.children.size();
    +          if (diff != 0) {
    +			return diff;
    +		}
    +          for (int i = 0; i < a.children.size(); i++) {
    +            diff = compare(a.children.get(i), b.children.get(i));
    +            if (diff != 0) {
    +				return diff;
    +			}
    +          }
    +      }
    +      return 0;
    +    }
    +  }
    +
    +  /*
    +   * Convience method that delegates to normalize with a new NodeComparator constructed using the supplied expression.
    +   */
    +  public static Node normalize(final Node root, final byte[] expression) {
    +    return normalize(root, expression, new NodeComparator(expression));
    +  }
    +
    +  // @formatter:off
    +  /*
    +   * Walks an expression's AST in order to:
    +   *  1) roll up expressions with the same operant (`a&(b&c) becomes a&b&c`)
    +   *  2) sorts labels lexicographically (permutations of `a&b&c` are re-ordered to appear as `a&b&c`)
    +   *  3) dedupes labels (`a&b&a` becomes `a&b`)
    +   */
    +  // @formatter:on
    +  public static Node normalize(final Node root, final byte[] expression, final NodeComparator comparator) {
    +    if (root.type != NodeType.TERM) {
    +      final TreeSet<Node> rolledUp = new TreeSet<>(comparator);
    +      final java.util.Iterator<Node> itr = root.children.iterator();
    +      while (itr.hasNext()) {
    +        final Node c = normalize(itr.next(), expression, comparator);
    +        if (c.type == root.type) {
    +          rolledUp.addAll(c.children);
    +          itr.remove();
    +        }
    +      }
    +      rolledUp.addAll(root.children);
    +      root.children.clear();
    +      root.children.addAll(rolledUp);
    +
    +      // need to promote a child if it's an only child
    +      if (root.children.size() == 1) {
    +        return root.children.get(0);
    +      }
    +    }
    +
    +    return root;
    +  }
    +
    +  /*
    +   * Walks an expression's AST and appends a string representation to a supplied StringBuilder. This method adds parens where necessary.
    +   */
    +  public static void stringify(final Node root, final byte[] expression, final StringBuilder out) {
    +    if (root.type == NodeType.TERM) {
    +      out.append(new String(expression, root.start, root.end - root.start, UTF_8));
    +    } else {
    +      String sep = "";
    +      for (final Node c : root.children) {
    +        out.append(sep);
    +        final boolean parens = (c.type != NodeType.TERM && root.type != c.type);
    +        if (parens) {
    +			out.append("(");
    +		}
    +        stringify(c, expression, out);
    +        if (parens) {
    +			out.append(")");
    +		}
    +        sep = root.type == NodeType.AND ? "&" : "|";
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Generates a byte[] that represents a normalized, but logically equivalent, form of this evaluator's expression.
    +   *
    +   * @return normalized expression in byte[] form
    +   */
    +  public byte[] flatten() {
    +    final Node normRoot = normalize(node, expression);
    +    final StringBuilder builder = new StringBuilder(expression.length);
    +    stringify(normRoot, expression, builder);
    +    return builder.toString().getBytes(UTF_8);
    +  }
    +
    +  private static class DocumentVisibilityParser {
    +    private int index = 0;
    +    private int parens = 0;
    +
    +    public DocumentVisibilityParser() {}
    +
    +    Node parse(final byte[] expression) {
    +      if (expression.length > 0) {
    +        final Node node = parse_(expression);
    +        if (node == null) {
    +          throw new BadArgumentException("operator or missing parens", new String(expression, UTF_8), index - 1);
    +        }
    +        if (parens != 0) {
    +          throw new BadArgumentException("parenthesis mis-match", new String(expression, UTF_8), index - 1);
    +        }
    +        return node;
    +      }
    +      return null;
    +    }
    +
    +    Node processTerm(final int start, final int end, final Node expr, final byte[] expression) {
    +      if (start != end) {
    +        if (expr != null) {
    +			throw new BadArgumentException("expression needs | or &", new String(expression, UTF_8), start);
    +		}
    +        return new Node(expression, start, end);
    +      }
    +      if (expr == null) {
    +		throw new BadArgumentException("empty term", new String(expression, UTF_8), start);
    +	}
    +      return expr;
    +    }
    +
    +    Node parse_(final byte[] expression) {
    +      Node result = null;
    +      Node expr = null;
    +      final int wholeTermStart = index;
    +      int subtermStart = index;
    +      boolean subtermComplete = false;
    +
    +      while (index < expression.length) {
    +        switch (expression[index++]) {
    +          case '&': {
    +            expr = processTerm(subtermStart, index - 1, expr, expression);
    +            if (result != null) {
    +              if (!result.type.equals(NodeType.AND)) {
    +				throw new BadArgumentException("cannot mix & and |", new String(expression, UTF_8), index - 1);
    +			}
    +            } else {
    +              result = new Node(expression, NodeType.AND, wholeTermStart);
    +            }
    +            result.add(expr);
    +            expr = null;
    +            subtermStart = index;
    +            subtermComplete = false;
    +            break;
    +          }
    +          case '|': {
    +            expr = processTerm(subtermStart, index - 1, expr, expression);
    +            if (result != null) {
    +              if (!result.type.equals(NodeType.OR)) {
    +				throw new BadArgumentException("cannot mix | and &", new String(expression, UTF_8), index - 1);
    +			}
    +            } else {
    +              result = new Node(expression, NodeType.OR, wholeTermStart);
    +            }
    +            result.add(expr);
    +            expr = null;
    +            subtermStart = index;
    +            subtermComplete = false;
    +            break;
    +          }
    +          case '(': {
    +            parens++;
    +            if (subtermStart != index - 1 || expr != null) {
    +				throw new BadArgumentException("expression needs & or |", new String(expression, UTF_8), index - 1);
    +			}
    +            expr = parse_(expression);
    +            subtermStart = index;
    +            subtermComplete = false;
    +            break;
    +          }
    +          case ')': {
    +            parens--;
    +            final Node child = processTerm(subtermStart, index - 1, expr, expression);
    +            if (child == null && result == null) {
    +				throw new BadArgumentException("empty expression not allowed", new String(expression, UTF_8), index);
    +			}
    +            if (result == null) {
    +				return child;
    +			}
    +            if (result.type == child.type) {
    +				for (final Node c : child.children) {
    +					result.add(c);
    +				}
    +			} else {
    +				result.add(child);
    +			}
    +            result.end = index - 1;
    +            return result;
    +          }
    +          case '"': {
    +            if (subtermStart != index - 1) {
    +				throw new BadArgumentException("expression needs & or |", new String(expression, UTF_8), index - 1);
    +			}
    +
    +            while (index < expression.length && expression[index] != '"') {
    +              if (expression[index] == '\\') {
    +                index++;
    +                if (expression[index] != '\\' && expression[index] != '"') {
    +					throw new BadArgumentException("invalid escaping within quotes", new String(expression, UTF_8), index - 1);
    +				}
    +              }
    +              index++;
    +            }
    +
    +            if (index == expression.length) {
    +				throw new BadArgumentException("unclosed quote", new String(expression, UTF_8), subtermStart);
    +			}
    +
    +            if (subtermStart + 1 == index) {
    +				throw new BadArgumentException("empty term", new String(expression, UTF_8), subtermStart);
    +			}
    +
    +            index++;
    +
    +            subtermComplete = true;
    +
    +            break;
    +          }
    +          default: {
    +            if (subtermComplete) {
    +				throw new BadArgumentException("expression needs & or |", new String(expression, UTF_8), index - 1);
    +			}
    +
    +            final byte c = expression[index - 1];
    +            if (!Authorizations.isValidAuthChar(c)) {
    +				throw new BadArgumentException("bad character (" + c + ")", new String(expression, UTF_8), index - 1);
    +			}
    +          }
    +        }
    +      }
    +      final Node child = processTerm(subtermStart, index, expr, expression);
    +      if (result != null) {
    +        result.add(child);
    +        result.end = index;
    +      } else {
    +		result = child;
    +	}
    +      if (result.type != NodeType.TERM) {
    +		if (result.children.size() < 2) {
    +			throw new BadArgumentException("missing term", new String(expression, UTF_8), index);
    +		}
    +	}
    +      return result;
    +    }
    +  }
    +
    +  private void validate(final byte[] expression) {
    +    if (expression != null && expression.length > 0) {
    +      final DocumentVisibilityParser p = new DocumentVisibilityParser();
    +      node = p.parse(expression);
    +    } else {
    +      node = EMPTY_NODE;
    +    }
    +    this.expression = expression;
    +  }
    +
    +  /**
    +   * Creates an empty visibility. Normally, elements with empty visibility can be seen by everyone. Though, one could change this behavior with filters.
    +   *
    +   * @see #DocumentVisibility(String)
    +   */
    +  public DocumentVisibility() {
    +    this(new byte[] {});
    +  }
    +
    +  /**
    +   * Creates a document visibility for a Mutation.
    +   *
    +   * @param expression
    +   *          An expression of the rights needed to see this mutation. The expression syntax is defined at the class-level documentation
    +   */
    +  public DocumentVisibility(final String expression) {
    +    this(expression.getBytes(UTF_8));
    +  }
    +
    +  /**
    +   * Creates a document visibility for a Mutation.
    +   *
    +   * @param expression
    +   *          visibility expression
    +   * @see #DocumentVisibility(String)
    +   */
    +  public DocumentVisibility(final Text expression) {
    +    this(TextUtil.getBytes(expression));
    +  }
    +
    +  /**
    +   * Creates a document visibility for a Mutation from a string already encoded in UTF-8 bytes.
    +   *
    +   * @param expression
    +   *          visibility expression, encoded as UTF-8 bytes
    +   * @see #DocumentVisibility(String)
    +   */
    +  public DocumentVisibility(final byte[] expression) {
    +    validate(expression);
    +  }
    +
    +  @Override
    +  public String toString() {
    +    return "[" + new String(expression, UTF_8) + "]";
    +  }
    +
    +  /**
    +   * See {@link #equals(DocumentVisibility)}
    +   */
    +  @Override
    +  public boolean equals(final Object obj) {
    +    if (obj instanceof DocumentVisibility) {
    +		return equals(obj);
    +	}
    +    return false;
    +  }
    +
    +  /**
    +   * Compares two DocumentVisibilities for string equivalence, not as a meaningful comparison of terms and conditions.
    +   *
    +   * @param otherLe
    +   *          other document visibility
    +   * @return true if this visibility equals the other via string comparison
    +   */
    +  public boolean equals(final DocumentVisibility otherLe) {
    +    return Arrays.equals(expression, otherLe.expression);
    +  }
    +
    +  @Override
    +  public int hashCode() {
    +    return Arrays.hashCode(expression);
    +  }
    +
    +  /**
    +   * Gets the parse tree for this document visibility.
    +   *
    +   * @return parse tree node
    +   */
    +  public Node getParseTree() {
    +    return node;
    +  }
    +
    +  /**
    +   * Properly quotes terms in a document visibility expression. If no quoting is needed, then nothing is done.
    +   *
    +   * <p>
    +   * Examples of using quote :
    +   *
    +   * <pre>
    +   * import static org.apache.rya.mongodb.document.visibility.DocumentVisibility.quote;
    +   *   .
    +   *   .
    +   *   .
    +   * DocumentVisibility dv = new DocumentVisibility(quote(&quot;A#C&quot;) + &quot;&amp;&quot; + quote(&quot;FOO&quot;));
    +   * </pre>
    +   *
    +   * @param term
    +   *          term to quote
    +   * @return quoted term (unquoted if unnecessary)
    +   */
    +  public static String quote(final String term) {
    --- End diff --
    
    why not create a new class level String of a UTF_8 quote?  UTF_8_QUOTE + "some auth" + UTF_8_QUOTE


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

[GitHub] incubator-rya issue #124: RYA-119 Added MongoDB Column Visibility (called Do...

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

    https://github.com/apache/incubator-rya/pull/124
  
    asfbot build


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90058015
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBStorageStrategy.java ---
    @@ -96,12 +103,19 @@ public DBObject getQuery(final RyaStatement stmt) {
     
         @Override
         public RyaStatement deserializeDBObject(final DBObject queryResult) {
    -        final Map result = queryResult.toMap();
    +        final Map<?, ?> result = queryResult.toMap();
    --- End diff --
    
    edit last sentence above. Map< ?, ?> gets rid of the warning


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89901044
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/util/DocumentVisibilityUtil.java ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.mongodb.document.util;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.log4j.Logger;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.apache.rya.mongodb.document.visibility.ByteSequence;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.Node;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.NodeType;
    +import org.apache.rya.mongodb.document.visibility.VisibilityEvaluator;
    +import org.apache.rya.mongodb.document.visibility.VisibilityParseException;
    +
    +import com.google.common.base.Charsets;
    +import com.google.common.collect.Lists;
    +import com.mongodb.BasicDBList;
    +
    +/**
    + * Utility methods for converting boolean expressions between an Accumulo column
    + * visibility string style and a multidimensional array that can be used
    + * in MongoDB expressions.
    + */
    +public final class DocumentVisibilityUtil {
    +    private static final Logger log = Logger.getLogger(DocumentVisibilityUtil.class);
    +
    +    /**
    +     * Private constructor to prevent instantiation.
    +     */
    +    private DocumentVisibilityUtil() {
    +    }
    +
    +    /**
    +     * Converts a boolean string expression into a multidimensional
    +     * array representation of the boolean expression.
    +     * @param booleanString the boolean string expression.
    +     * @return the multidimensional array representation of the boolean
    +     * expression.
    +     */
    +    public static Object[] toMultidimensionalArray(final String booleanString) {
    +        final DocumentVisibility dv = new DocumentVisibility(booleanString);
    +        return toMultidimensionalArray(dv);
    +    }
    +
    +    /**
    +     * Converts a {@link DocumentVisibility} object into a multidimensional
    +     * array representation of the boolean expression.
    +     * @param dv the {@link DocumentVisibility}. (not {@code null})
    +     * @return the multidimensional array representation of the boolean
    +     * expression.
    +     */
    +    public static Object[] toMultidimensionalArray(final DocumentVisibility dv) {
    +        checkNotNull(dv);
    +        final byte[] expression = dv.flatten();
    +        final DocumentVisibility flattenedDv = DisjunctiveNormalFormConverter.createDnfDocumentVisibility(expression);
    +        final Object[] result = toMultidimensionalArray(flattenedDv.getParseTree(), expression);
    +        return result;
    +    }
    +
    +    /**
    +     * Converts a {@link Node} and its corresponding expression into a
    +     * multidimensional array representation of the boolean expression.
    +     * @param node the {@link Node}. (not {@code null})
    +     * @param expression the expression byte array.
    +     * @return the multidimensional array representation of the boolean
    +     * expression.
    +     */
    +    public static Object[] toMultidimensionalArray(final Node node, final byte[] expression) {
    +        checkNotNull(node);
    +        final List<Object> array = new ArrayList<>();
    +
    +        if (node.getChildren().isEmpty() && node.getType() == NodeType.TERM) {
    +            final String data = getTermNodeData(node);
    +            array.add(data);
    +        }
    +
    +        log.trace("Children size: " + node.getChildren().size() + " Type: " + node.getType());
    +        for (final Node child : node.getChildren()) {
    +            switch (child.getType()) {
    +                case EMPTY:
    +                case TERM:
    +                    String data;
    +                    if (child.getType() == NodeType.TERM) {
    +                        data = getTermNodeData(child);
    +                    } else {
    +                        data = "";
    +                    }
    +                    if (node.getType() == NodeType.OR) {
    +                        array.add(Lists.newArrayList(data).toArray(new Object[0]));
    +                    } else {
    +                        array.add(data);
    +                    }
    +                    break;
    +                case OR:
    +                case AND:
    +                    array.add(toMultidimensionalArray(child, expression));
    +                    break;
    +                default:
    +                    break;
    +            }
    +        }
    +
    +        return array.toArray(new Object[0]);
    +    }
    +
    +    public static String nodeToBooleanString(final Node node) {
    +        boolean isFirst = true;
    +        final StringBuilder sb = new StringBuilder();
    +        if (node.getType() == NodeType.TERM) {
    +            final String data = getTermNodeData(node);
    +            sb.append(data);
    +        }
    +        if (node.getType() == NodeType.AND) {
    +            sb.append("(");
    +        }
    +        for (final Node child : node.getChildren()) {
    --- End diff --
    
    do you have to worry about children's children?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89900078
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/util/DisjunctiveNormalFormConverter.java ---
    @@ -0,0 +1,270 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.mongodb.document.util;
    +
    +import static java.nio.charset.StandardCharsets.UTF_8;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.LinkedHashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.Node;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.NodeType;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * Utility for converting document visibility boolean expressions into
    + * Disjunctive Normal Form.
    + */
    +public final class DisjunctiveNormalFormConverter {
    +    private static final Logger log = Logger.getLogger(DisjunctiveNormalFormConverter.class);
    +
    +    /**
    +     * Private constructor to prevent instantiation.
    +     */
    +    private DisjunctiveNormalFormConverter() {
    --- End diff --
    
    why does this need to be a singleton?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r90059667
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRyaDAO.java ---
    @@ -164,26 +174,31 @@ public void add(final Iterator<RyaStatement> statement) throws RyaDAOException {
             final List<DBObject> dbInserts = new ArrayList<DBObject>();
             while (statement.hasNext()){
                 final RyaStatement ryaStatement = statement.next();
    -            final DBObject insert = storageStrategy.serialize(ryaStatement);
    -            dbInserts.add(insert);
    -
    -            try {
    -                for (final RyaSecondaryIndexer index : secondaryIndexers) {
    -                    index.storeStatement(ryaStatement);
    +            final boolean canAdd = DocumentVisibilityUtil.doesUserHaveDocumentAccess(auths, ryaStatement.getColumnVisibility());
    +            if (canAdd) {
    +                final DBObject insert = storageStrategy.serialize(ryaStatement);
    +                dbInserts.add(insert);
    +
    +                try {
    +                    for (final RyaSecondaryIndexer index : secondaryIndexers) {
    +                        index.storeStatement(ryaStatement);
    +                    }
    +                } catch (final IOException e) {
    +                    log.error("Failed to add: " + ryaStatement.toString() + " to the indexer");
                     }
    -            } catch (final IOException e) {
    -                log.error("Failed to add: " + ryaStatement.toString() + " to the indexer");
                 }
    -
             }
             coll.insert(dbInserts, new InsertOptions().continueOnError(true));
         }
     
         @Override
         public void delete(final RyaStatement statement, final MongoDBRdfConfiguration conf)
                 throws RyaDAOException {
    -        final DBObject obj = storageStrategy.getQuery(statement);
    -        coll.remove(obj);
    +        final boolean canDelete = DocumentVisibilityUtil.doesUserHaveDocumentAccess(auths, statement.getColumnVisibility());
    +        if (canDelete) {
    +            final DBObject obj = storageStrategy.getQuery(statement);
    +            coll.remove(obj);
    --- End diff --
    
    it looks like the fix was only implemented for batch remove


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90053582
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRyaDAO.java ---
    @@ -111,26 +117,27 @@ public MongoDBRdfConfiguration getConf() {
     
         @Override
         public void init() throws RyaDAOException {
    -            secondaryIndexers = conf.getAdditionalIndexers();
    -            for(final MongoSecondaryIndex index: secondaryIndexers) {
    -                index.setConf(conf);
    -                index.setClient(mongoClient);
    -            }
    +        secondaryIndexers = conf.getAdditionalIndexers();
    +        for(final MongoSecondaryIndex index: secondaryIndexers) {
    +            index.setConf(conf);
    +            index.setClient(mongoClient);
    +        }
     
    -            db = mongoClient.getDB(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -            coll = db.getCollection(conf.getTriplesCollectionName());
    -            nameSpaceManager = new SimpleMongoDBNamespaceManager(db.getCollection(conf.getNameSpacesCollectionName()));
    -            queryEngine = new MongoDBQueryEngine(conf, mongoClient);
    -            storageStrategy = new SimpleMongoDBStorageStrategy();
    -            storageStrategy.createIndices(coll);
    -            for(final MongoSecondaryIndex index: secondaryIndexers) {
    -                index.init();
    -            }
    +        db = mongoClient.getDB(conf.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        coll = db.getCollection(conf.getTriplesCollectionName());
    +        nameSpaceManager = new SimpleMongoDBNamespaceManager(db.getCollection(conf.getNameSpacesCollectionName()));
    +        queryEngine = new MongoDBQueryEngine(conf, mongoClient);
    +        storageStrategy = new SimpleMongoDBStorageStrategy();
    +        storageStrategy.createIndices(coll);
    +        for(final MongoSecondaryIndex index: secondaryIndexers) {
    +            index.init();
    +        }
         }
     
         @Override
         public boolean isInitialized() throws RyaDAOException {
             return true;
    +
    --- End diff --
    
    Done


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89909687
  
    --- Diff: dao/mongodb.rya/src/test/java/org/apache/rya/mongodb/MongoDBRyaDAOTest.java ---
    @@ -28,102 +30,279 @@
     import org.apache.rya.api.domain.RyaStatement.RyaStatementBuilder;
     import org.apache.rya.api.domain.RyaURI;
     import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.api.persist.query.RyaQuery;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.bson.Document;
    +import org.calrissian.mango.collect.CloseableIterable;
     import org.junit.Before;
     import org.junit.Test;
     
    -import com.mongodb.DB;
    -import com.mongodb.DBCollection;
     import com.mongodb.MongoException;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.MongoDatabase;
     
     public class MongoDBRyaDAOTest extends MongoRyaTestBase {
     
    -	private MongoDBRyaDAO dao;
    -	private MongoDBRdfConfiguration configuration;
    +    private MongoDBRyaDAO dao;
    +    private MongoDBRdfConfiguration configuration;
     
    -	@Before
    -	public void setUp() throws IOException, RyaDAOException{
    -		final Configuration conf = new Configuration();
    +    @Before
    +    public void setUp() throws IOException, RyaDAOException{
    +        final Configuration conf = new Configuration();
             conf.set(MongoDBRdfConfiguration.MONGO_DB_NAME, "test");
             conf.set(MongoDBRdfConfiguration.MONGO_COLLECTION_PREFIX, "rya_");
             conf.set(RdfCloudTripleStoreConfiguration.CONF_TBL_PREFIX, "rya_");
             configuration = new MongoDBRdfConfiguration(conf);
             final int port = mongoClient.getServerAddressList().get(0).getPort();
             configuration.set(MongoDBRdfConfiguration.MONGO_INSTANCE_PORT, ""+port);
    -		dao = new MongoDBRyaDAO(configuration, mongoClient);
    -	}
    +        dao = new MongoDBRyaDAO(configuration, mongoClient);
    +    }
     
     
    -	@Test
    -	public void testDeleteWildcard() throws RyaDAOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		dao.delete(builder.build(), configuration);
    -	}
    +    @Test
    +    public void testDeleteWildcard() throws RyaDAOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        dao.delete(builder.build(), configuration);
    +    }
     
     
    -	@Test
    -	public void testAdd() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    +    @Test
    +    public void testAdd() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(builder.build());
    +        dao.add(builder.build());
     
             assertEquals(coll.count(),1);
     
    -	}
    +    }
     
    -	@Test
    -	public void testDelete() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    -		final RyaStatement statement = builder.build();
    +    @Test
    +    public void testDelete() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
    +        final RyaStatement statement = builder.build();
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(statement);
    +        dao.add(statement);
     
             assertEquals(coll.count(),1);
     
             dao.delete(statement, configuration);
     
             assertEquals(coll.count(),0);
     
    -	}
    +    }
     
    -	@Test
    -	public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
    -		final RyaStatementBuilder builder = new RyaStatementBuilder();
    -		builder.setPredicate(new RyaURI("http://temp.com"));
    -		builder.setSubject(new RyaURI("http://subject.com"));
    -		builder.setObject(new RyaURI("http://object.com"));
    -		builder.setContext(new RyaURI("http://context.com"));
    -		final RyaStatement statement = builder.build();
    +    @Test
    +    public void testDeleteWildcardSubjectWithContext() throws RyaDAOException, MongoException, IOException {
    +        final RyaStatementBuilder builder = new RyaStatementBuilder();
    +        builder.setPredicate(new RyaURI("http://temp.com"));
    +        builder.setSubject(new RyaURI("http://subject.com"));
    +        builder.setObject(new RyaURI("http://object.com"));
    +        builder.setContext(new RyaURI("http://context.com"));
    +        final RyaStatement statement = builder.build();
     
    -		final DB db = mongoClient.getDB(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    -        final DBCollection coll = db.getCollection(configuration.getTriplesCollectionName());
    +        final MongoDatabase db = mongoClient.getDatabase(configuration.get(MongoDBRdfConfiguration.MONGO_DB_NAME));
    +        final MongoCollection<Document> coll = db.getCollection(configuration.getTriplesCollectionName());
     
    -		dao.add(statement);
    +        dao.add(statement);
     
             assertEquals(coll.count(),1);
     
    -		final RyaStatementBuilder builder2 = new RyaStatementBuilder();
    -		builder2.setPredicate(new RyaURI("http://temp.com"));
    -		builder2.setObject(new RyaURI("http://object.com"));
    -		builder2.setContext(new RyaURI("http://context3.com"));
    -		final RyaStatement query = builder2.build();
    +        final RyaStatementBuilder builder2 = new RyaStatementBuilder();
    +        builder2.setPredicate(new RyaURI("http://temp.com"));
    +        builder2.setObject(new RyaURI("http://object.com"));
    +        builder2.setContext(new RyaURI("http://context3.com"));
    +        final RyaStatement query = builder2.build();
     
             dao.delete(query, configuration);
     
             assertEquals(coll.count(),1);
     
    -	}
    +    }
     
    +    @Test
    +    public void testVisibility() throws RyaDAOException, MongoException, IOException {
    +        // Doc requires "A" and user has "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A", new Authorizations("B")));
    +
    +        // Doc requires "A" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A")));
    +
    +        // Doc requires "A" and "B" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or "B" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and "B" and user has "A" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A&B", new Authorizations("A")));
    +
    +        // Doc requires "A" and "B" and "C" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A&B&C", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and "B" and "C" and user has "A" and "B" = User CANNOT view
    +        assertFalse(testVisibilityStatement("A&B&C", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" and "B" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A&B", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" or "B" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" and "B" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A", "B")));
    +
    +        // Doc requires "A" or "B" or "C" and user has "A" = User can view
    +        assertTrue(testVisibilityStatement("A|B|C", new Authorizations("A")));
    +
    +        // Doc requires "A" or "B" and user has "A" and "B" and "C" = User can view
    +        assertTrue(testVisibilityStatement("A|B", new Authorizations("A", "B", "C")));
    +
    +        // Doc requires "A" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A", Authorizations.EMPTY));
    +
    +        // Doc requires "A" and "B" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A&B", Authorizations.EMPTY));
    +
    +        // Doc requires "A" or "B" and user has "" = User can view
    +        assertTrue(testVisibilityStatement("A|B", Authorizations.EMPTY));
    --- End diff --
    
    and all the other ones....


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90068055
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    + */
    +public class Authorizations implements Iterable<byte[]>, Serializable, AuthorizationContainer {
    +
    +  private static final long serialVersionUID = 1L;
    +
    +  private final Set<ByteSequence> auths = new HashSet<ByteSequence>();
    +  private final List<byte[]> authsList = new ArrayList<byte[]>(); // sorted order
    +
    +  /**
    +   * An empty set of authorizations.
    +   */
    +  public static final Authorizations EMPTY = new Authorizations();
    +
    +  private static final boolean[] validAuthChars = new boolean[256];
    +
    +  /**
    +   * A special header string used when serializing instances of this class.
    +   *
    +   * @see #serialize()
    +   */
    +  public static final String HEADER = "!AUTH1:";
    +
    +  static {
    --- End diff --
    
    same as above


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90067685
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    + */
    +public class Authorizations implements Iterable<byte[]>, Serializable, AuthorizationContainer {
    --- End diff --
    
    same as above


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r94812491
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/DocumentVisibilityAdapter.java ---
    @@ -0,0 +1,137 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import org.apache.log4j.Logger;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.util.DocumentVisibilityConversionException;
    +import org.apache.rya.mongodb.document.util.DocumentVisibilityUtil;
    +
    +import com.mongodb.BasicDBList;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.BasicDBObjectBuilder;
    +import com.mongodb.DBObject;
    +
    +import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
    +import edu.umd.cs.findbugs.annotations.NonNull;
    +
    +/**
    + * Serializes the document visibility field of a Rya Statement for use in
    + * MongoDB.
    + * The {@link DBObject} will look like:
    + * <pre>
    + * {@code
    + * {
    + *   "documentVisibility": &lt;array&gt;,
    + * }
    + * </pre>
    + */
    +@DefaultAnnotation(NonNull.class)
    +public class DocumentVisibilityAdapter {
    +    private static final Logger log = Logger.getLogger(DocumentVisibilityAdapter.class);
    +
    +    public static final String DOCUMENT_VISIBILITY_KEY = SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +
    +    /**
    +     * Serializes a document visibility expression byte array to a MongoDB
    +     * {@link DBObject}.
    +     * @param expression the document visibility expression byte array to be
    +     * serialized.
    +     * @return The MongoDB {@link DBObject}.
    +     */
    +    public static BasicDBObject toDBObject(final byte[] expression) {
    --- End diff --
    
    generally, we don't make the adapters static.  It makes writing unit tests for them a lot easier.  Do tests exist for the adapter?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89909168
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/iter/RyaStatementCursorIterator.java ---
    @@ -1,5 +1,23 @@
     package org.apache.rya.mongodb.iter;
     
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.log4j.Logger;
    +import org.apache.rya.api.domain.RyaStatement;
    +import org.apache.rya.api.persist.RyaDAOException;
    +import org.apache.rya.mongodb.MongoDBRdfConfiguration;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +
    +import com.mongodb.AggregationOutput;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBCollection;
    +import com.mongodb.DBObject;
    +
     /*
    --- End diff --
    
    same


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90343835
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    + */
    +public class Authorizations implements Iterable<byte[]>, Serializable, AuthorizationContainer {
    +
    +  private static final long serialVersionUID = 1L;
    +
    +  private final Set<ByteSequence> auths = new HashSet<ByteSequence>();
    +  private final List<byte[]> authsList = new ArrayList<byte[]>(); // sorted order
    +
    +  /**
    +   * An empty set of authorizations.
    +   */
    +  public static final Authorizations EMPTY = new Authorizations();
    +
    +  private static final boolean[] validAuthChars = new boolean[256];
    +
    +  /**
    +   * A special header string used when serializing instances of this class.
    +   *
    +   * @see #serialize()
    +   */
    +  public static final String HEADER = "!AUTH1:";
    +
    +  static {
    +    for (int i = 0; i < 256; i++) {
    +      validAuthChars[i] = false;
    +    }
    +
    +    for (int i = 'a'; i <= 'z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = 'A'; i <= 'Z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = '0'; i <= '9'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    validAuthChars['_'] = true;
    +    validAuthChars['-'] = true;
    +    validAuthChars[':'] = true;
    +    validAuthChars['.'] = true;
    +    validAuthChars['/'] = true;
    +  }
    +
    +  static final boolean isValidAuthChar(final byte b) {
    +    return validAuthChars[0xff & b];
    +  }
    +
    +  private void checkAuths() {
    +    final Set<ByteSequence> sortedAuths = new TreeSet<ByteSequence>(auths);
    +
    +    for (final ByteSequence bs : sortedAuths) {
    +      if (bs.length() == 0) {
    +        throw new IllegalArgumentException("Empty authorization");
    +      }
    +
    +      authsList.add(bs.toArray());
    +    }
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a collection of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does
    +   * not verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          collection of authorizations, as strings encoded in UTF-8
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final Collection<byte[]> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    for (final byte[] auth : authorizations) {
    +		auths.add(new ArrayByteSequence(auth));
    +	}
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a list of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does not
    +   * verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          list of authorizations, as strings encoded in UTF-8 and placed in buffers
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final List<ByteBuffer> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    for (final ByteBuffer buffer : authorizations) {
    +      auths.add(new ArrayByteSequence(ByteBufferUtil.toBytes(buffer)));
    +    }
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Constructs an authorizations object from a serialized form. This is NOT a constructor for a set of authorizations of size one. Warning: This method does
    +   * not verify that the encoded serialized form is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          a serialized authorizations string produced by {@link #getAuthorizationsArray()} or {@link #serialize()}, converted to UTF-8 bytes
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   */
    +  public Authorizations(final byte[] authorizations) {
    +
    +    ArgumentChecker.notNull(authorizations);
    +
    +    String authsString = new String(authorizations, UTF_8);
    +    if (authsString.startsWith(HEADER)) {
    +      // it's the new format
    +      authsString = authsString.substring(HEADER.length());
    +      if (authsString.length() > 0) {
    +        for (final String encAuth : authsString.split(",")) {
    +          final byte[] auth = Base64.decodeBase64(encAuth.getBytes(UTF_8));
    +          auths.add(new ArrayByteSequence(auth));
    +        }
    +        checkAuths();
    +      }
    +    } else {
    +      // it's the old format
    +      if (authorizations.length > 0) {
    +		setAuthorizations(authsString.split(","));
    +	}
    +    }
    +  }
    +
    +  /**
    +   * Constructs an empty set of authorizations.
    +   *
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations() {}
    --- End diff --
    
    same as above


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90065742
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/util/DisjunctiveNormalFormConverter.java ---
    @@ -0,0 +1,270 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.mongodb.document.util;
    +
    +import static java.nio.charset.StandardCharsets.UTF_8;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.LinkedHashSet;
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.log4j.Logger;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.Node;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.NodeType;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.collect.Lists;
    +
    +/**
    + * Utility for converting document visibility boolean expressions into
    + * Disjunctive Normal Form.
    + */
    +public final class DisjunctiveNormalFormConverter {
    +    private static final Logger log = Logger.getLogger(DisjunctiveNormalFormConverter.class);
    +
    +    /**
    +     * Private constructor to prevent instantiation.
    +     */
    +    private DisjunctiveNormalFormConverter() {
    --- End diff --
    
    It's not a singleton.  It's a static-method only class.  So, the private constructor is to enforce that idea and I added final to the class to prevent subclassing.  I usually do both for my utility classes.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89897692
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDbRdfConstants.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.mongodb;
    +
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
    +
    +/**
    + * Interface MongoDbRdfConstants.
    --- End diff --
    
    why do you need a class for just constants?  if its absolutely necessary, doc it better


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89900897
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/util/DocumentVisibilityUtil.java ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.mongodb.document.util;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.log4j.Logger;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.apache.rya.mongodb.document.visibility.ByteSequence;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.Node;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.NodeType;
    +import org.apache.rya.mongodb.document.visibility.VisibilityEvaluator;
    +import org.apache.rya.mongodb.document.visibility.VisibilityParseException;
    +
    +import com.google.common.base.Charsets;
    +import com.google.common.collect.Lists;
    +import com.mongodb.BasicDBList;
    +
    +/**
    + * Utility methods for converting boolean expressions between an Accumulo column
    + * visibility string style and a multidimensional array that can be used
    + * in MongoDB expressions.
    + */
    +public final class DocumentVisibilityUtil {
    +    private static final Logger log = Logger.getLogger(DocumentVisibilityUtil.class);
    +
    +    /**
    +     * Private constructor to prevent instantiation.
    +     */
    +    private DocumentVisibilityUtil() {
    +    }
    +
    +    /**
    +     * Converts a boolean string expression into a multidimensional
    +     * array representation of the boolean expression.
    +     * @param booleanString the boolean string expression.
    +     * @return the multidimensional array representation of the boolean
    +     * expression.
    +     */
    +    public static Object[] toMultidimensionalArray(final String booleanString) {
    +        final DocumentVisibility dv = new DocumentVisibility(booleanString);
    +        return toMultidimensionalArray(dv);
    +    }
    +
    +    /**
    +     * Converts a {@link DocumentVisibility} object into a multidimensional
    +     * array representation of the boolean expression.
    +     * @param dv the {@link DocumentVisibility}. (not {@code null})
    +     * @return the multidimensional array representation of the boolean
    +     * expression.
    +     */
    +    public static Object[] toMultidimensionalArray(final DocumentVisibility dv) {
    +        checkNotNull(dv);
    +        final byte[] expression = dv.flatten();
    +        final DocumentVisibility flattenedDv = DisjunctiveNormalFormConverter.createDnfDocumentVisibility(expression);
    +        final Object[] result = toMultidimensionalArray(flattenedDv.getParseTree(), expression);
    +        return result;
    +    }
    +
    +    /**
    +     * Converts a {@link Node} and its corresponding expression into a
    +     * multidimensional array representation of the boolean expression.
    +     * @param node the {@link Node}. (not {@code null})
    +     * @param expression the expression byte array.
    +     * @return the multidimensional array representation of the boolean
    +     * expression.
    +     */
    +    public static Object[] toMultidimensionalArray(final Node node, final byte[] expression) {
    +        checkNotNull(node);
    +        final List<Object> array = new ArrayList<>();
    +
    +        if (node.getChildren().isEmpty() && node.getType() == NodeType.TERM) {
    +            final String data = getTermNodeData(node);
    +            array.add(data);
    +        }
    +
    +        log.trace("Children size: " + node.getChildren().size() + " Type: " + node.getType());
    +        for (final Node child : node.getChildren()) {
    +            switch (child.getType()) {
    +                case EMPTY:
    +                case TERM:
    +                    String data;
    +                    if (child.getType() == NodeType.TERM) {
    +                        data = getTermNodeData(child);
    +                    } else {
    +                        data = "";
    +                    }
    +                    if (node.getType() == NodeType.OR) {
    +                        array.add(Lists.newArrayList(data).toArray(new Object[0]));
    +                    } else {
    +                        array.add(data);
    +                    }
    +                    break;
    +                case OR:
    +                case AND:
    +                    array.add(toMultidimensionalArray(child, expression));
    +                    break;
    +                default:
    +                    break;
    +            }
    +        }
    +
    +        return array.toArray(new Object[0]);
    +    }
    +
    +    public static String nodeToBooleanString(final Node node) {
    +        boolean isFirst = true;
    +        final StringBuilder sb = new StringBuilder();
    +        if (node.getType() == NodeType.TERM) {
    +            final String data = getTermNodeData(node);
    +            sb.append(data);
    +        }
    +        if (node.getType() == NodeType.AND) {
    +            sb.append("(");
    +        }
    +        for (final Node child : node.getChildren()) {
    +            if (isFirst) {
    +                isFirst = false;
    +            } else {
    +                if (node.getType() == NodeType.OR) {
    +                    sb.append("|");
    +                } else if (node.getType() == NodeType.AND) {
    +                    sb.append("&");
    +                }
    +            }
    +            switch (child.getType()) {
    +                case EMPTY:
    +                    sb.append("");
    +                    break;
    +                case TERM:
    +                    final String data = getTermNodeData(child);
    +                    sb.append(data);
    +                    break;
    +                case OR:
    +                    sb.append("(");
    +                    sb.append(nodeToBooleanString(child));
    +                    sb.append(")");
    +                    break;
    +                case AND:
    +                    sb.append(nodeToBooleanString(child));
    +                    break;
    +                default:
    +                    break;
    --- End diff --
    
    is this case possible?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89896570
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRyaDAO.java ---
    @@ -164,26 +174,31 @@ public void add(final Iterator<RyaStatement> statement) throws RyaDAOException {
             final List<DBObject> dbInserts = new ArrayList<DBObject>();
             while (statement.hasNext()){
                 final RyaStatement ryaStatement = statement.next();
    --- End diff --
    
    I know this has been here, but can you rename this to statements?  or statementIter?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90055207
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDBRyaDAO.java ---
    @@ -164,26 +174,31 @@ public void add(final Iterator<RyaStatement> statement) throws RyaDAOException {
             final List<DBObject> dbInserts = new ArrayList<DBObject>();
             while (statement.hasNext()){
                 final RyaStatement ryaStatement = statement.next();
    -            final DBObject insert = storageStrategy.serialize(ryaStatement);
    -            dbInserts.add(insert);
    -
    -            try {
    -                for (final RyaSecondaryIndexer index : secondaryIndexers) {
    -                    index.storeStatement(ryaStatement);
    +            final boolean canAdd = DocumentVisibilityUtil.doesUserHaveDocumentAccess(auths, ryaStatement.getColumnVisibility());
    +            if (canAdd) {
    +                final DBObject insert = storageStrategy.serialize(ryaStatement);
    +                dbInserts.add(insert);
    +
    +                try {
    +                    for (final RyaSecondaryIndexer index : secondaryIndexers) {
    +                        index.storeStatement(ryaStatement);
    +                    }
    +                } catch (final IOException e) {
    +                    log.error("Failed to add: " + ryaStatement.toString() + " to the indexer");
                     }
    -            } catch (final IOException e) {
    -                log.error("Failed to add: " + ryaStatement.toString() + " to the indexer");
                 }
    -
             }
             coll.insert(dbInserts, new InsertOptions().continueOnError(true));
         }
     
         @Override
         public void delete(final RyaStatement statement, final MongoDBRdfConfiguration conf)
                 throws RyaDAOException {
    -        final DBObject obj = storageStrategy.getQuery(statement);
    -        coll.remove(obj);
    +        final boolean canDelete = DocumentVisibilityUtil.doesUserHaveDocumentAccess(auths, statement.getColumnVisibility());
    +        if (canDelete) {
    +            final DBObject obj = storageStrategy.getQuery(statement);
    +            coll.remove(obj);
    --- End diff --
    
    This PR was rebased off master before posting so I don't think I took out anybody's code.  I'll rebase again if I make any more updates.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89903542
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    + */
    +public class Authorizations implements Iterable<byte[]>, Serializable, AuthorizationContainer {
    +
    +  private static final long serialVersionUID = 1L;
    +
    +  private final Set<ByteSequence> auths = new HashSet<ByteSequence>();
    +  private final List<byte[]> authsList = new ArrayList<byte[]>(); // sorted order
    +
    +  /**
    +   * An empty set of authorizations.
    +   */
    +  public static final Authorizations EMPTY = new Authorizations();
    --- End diff --
    
    didn't you already make an empty auth in the constants class?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89906785
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/DocumentVisibility.java ---
    @@ -0,0 +1,584 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static java.nio.charset.StandardCharsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.List;
    +import java.util.TreeSet;
    +
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.io.WritableComparator;
    +
    +/**
    + * Validate the document visibility is a valid expression and set the visibility for a Mutation. See {@link DocumentVisibility#DocumentVisibility(byte[])} for the
    + * definition of an expression.
    + *
    + * <p>
    + * The expression is a sequence of characters from the set [A-Za-z0-9_-.] along with the binary operators "&amp;" and "|" indicating that both operands are
    + * necessary, or the either is necessary. The following are valid expressions for visibility:
    + *
    + * <pre>
    + * A
    + * A|B
    + * (A|B)&amp;(C|D)
    + * orange|(red&amp;yellow)
    + * </pre>
    + *
    + * <p>
    + * The following are not valid expressions for visibility:
    + *
    + * <pre>
    + * A|B&amp;C
    + * A=B
    + * A|B|
    + * A&amp;|B
    + * ()
    + * )
    + * dog|!cat
    + * </pre>
    + *
    + * <p>
    + * In addition to the base set of visibilities, any character can be used in the expression if it is quoted. If the quoted term contains '&quot;' or '\', then
    + * escape the character with '\'. The {@link #quote(String)} method can be used to properly quote and escape terms automatically. The following is an example of
    + * a quoted term:
    + *
    + * <pre>
    + * &quot;A#C&quot; &amp; B
    + * </pre>
    + */
    +public class DocumentVisibility {
    +
    +  Node node = null;
    +  private byte[] expression;
    +
    +  /**
    +   * Accessor for the underlying byte string.
    +   *
    +   * @return byte array representation of a visibility expression
    +   */
    +  public byte[] getExpression() {
    +    return expression;
    +  }
    +
    +  /**
    +   * The node types in a parse tree for a visibility expression.
    +   */
    +  public static enum NodeType {
    +    EMPTY, TERM, OR, AND,
    +  }
    +
    +  /**
    +   * All empty nodes are equal and represent the same value.
    +   */
    +  private static final Node EMPTY_NODE = new Node("".getBytes(), NodeType.EMPTY, 0);
    +
    +  /**
    +   * A node in the parse tree for a visibility expression.
    +   */
    +  public static class Node {
    +    /**
    +     * An empty list of nodes.
    +     */
    +    public final static List<Node> EMPTY = Collections.emptyList();
    +    NodeType type;
    +    int start;
    +    int end;
    +    List<Node> children = EMPTY;
    +    byte[] expression;
    +
    +    public Node(final byte[] expression, final NodeType type, final int start) {
    +      this.type = type;
    +      this.start = start;
    +      this.end = start + 1;
    +      this.expression = expression;
    +    }
    +
    +    public Node(final byte[] expression, final int start, final int end) {
    +      this.type = NodeType.TERM;
    +      this.start = start;
    +      this.end = end;
    +      this.expression = expression;
    +    }
    +
    +    public void add(final Node child) {
    +      if (children == EMPTY) {
    +		children = new ArrayList<>();
    +	}
    +
    +      children.add(child);
    +    }
    +
    +    public NodeType getType() {
    +      return type;
    +    }
    +
    +    public List<Node> getChildren() {
    +      return children;
    +    }
    +
    +    public int getTermStart() {
    +      return start;
    +    }
    +
    +    public int getTermEnd() {
    +      return end;
    +    }
    +
    +    public byte[] getExpression() {
    +      return expression;
    +    }
    +
    +    public ByteSequence getTerm(final byte expression[]) {
    +      if (type != NodeType.TERM) {
    +		throw new RuntimeException();
    +	}
    +
    +      if (expression[start] == '"') {
    +        // its a quoted term
    +        final int qStart = start + 1;
    +        final int qEnd = end - 1;
    +
    +        return new ArrayByteSequence(expression, qStart, qEnd - qStart);
    +      }
    +      return new ArrayByteSequence(expression, start, end - start);
    +    }
    +  }
    +
    +  /**
    +   * A node comparator. Nodes sort according to node type, terms sort lexicographically. AND and OR nodes sort by number of children, or if the same by
    +   * corresponding children.
    +   */
    +  public static class NodeComparator implements Comparator<Node>, Serializable {
    +
    +    private static final long serialVersionUID = 1L;
    +    byte[] text;
    +
    +    /**
    +     * Creates a new comparator.
    +     *
    +     * @param text
    +     *          expression string, encoded in UTF-8
    +     */
    +    public NodeComparator(final byte[] text) {
    +      this.text = text;
    +    }
    +
    +    @Override
    +    public int compare(final Node a, final Node b) {
    +      int diff = a.type.ordinal() - b.type.ordinal();
    +      if (diff != 0) {
    +		return diff;
    +	}
    +      switch (a.type) {
    +        case EMPTY:
    +          return 0; // All empty nodes are the same
    +        case TERM:
    +          return WritableComparator.compareBytes(text, a.start, a.end - a.start, text, b.start, b.end - b.start);
    +        case OR:
    +        case AND:
    +          diff = a.children.size() - b.children.size();
    +          if (diff != 0) {
    +			return diff;
    +		}
    +          for (int i = 0; i < a.children.size(); i++) {
    +            diff = compare(a.children.get(i), b.children.get(i));
    +            if (diff != 0) {
    +				return diff;
    +			}
    +          }
    +      }
    +      return 0;
    +    }
    +  }
    +
    +  /*
    +   * Convience method that delegates to normalize with a new NodeComparator constructed using the supplied expression.
    +   */
    +  public static Node normalize(final Node root, final byte[] expression) {
    +    return normalize(root, expression, new NodeComparator(expression));
    +  }
    +
    +  // @formatter:off
    +  /*
    +   * Walks an expression's AST in order to:
    +   *  1) roll up expressions with the same operant (`a&(b&c) becomes a&b&c`)
    +   *  2) sorts labels lexicographically (permutations of `a&b&c` are re-ordered to appear as `a&b&c`)
    +   *  3) dedupes labels (`a&b&a` becomes `a&b`)
    +   */
    +  // @formatter:on
    +  public static Node normalize(final Node root, final byte[] expression, final NodeComparator comparator) {
    +    if (root.type != NodeType.TERM) {
    +      final TreeSet<Node> rolledUp = new TreeSet<>(comparator);
    +      final java.util.Iterator<Node> itr = root.children.iterator();
    +      while (itr.hasNext()) {
    +        final Node c = normalize(itr.next(), expression, comparator);
    +        if (c.type == root.type) {
    +          rolledUp.addAll(c.children);
    +          itr.remove();
    +        }
    +      }
    +      rolledUp.addAll(root.children);
    +      root.children.clear();
    +      root.children.addAll(rolledUp);
    +
    +      // need to promote a child if it's an only child
    +      if (root.children.size() == 1) {
    +        return root.children.get(0);
    +      }
    +    }
    +
    +    return root;
    +  }
    +
    +  /*
    +   * Walks an expression's AST and appends a string representation to a supplied StringBuilder. This method adds parens where necessary.
    +   */
    +  public static void stringify(final Node root, final byte[] expression, final StringBuilder out) {
    +    if (root.type == NodeType.TERM) {
    +      out.append(new String(expression, root.start, root.end - root.start, UTF_8));
    +    } else {
    +      String sep = "";
    +      for (final Node c : root.children) {
    +        out.append(sep);
    +        final boolean parens = (c.type != NodeType.TERM && root.type != c.type);
    +        if (parens) {
    +			out.append("(");
    +		}
    +        stringify(c, expression, out);
    +        if (parens) {
    +			out.append(")");
    +		}
    +        sep = root.type == NodeType.AND ? "&" : "|";
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Generates a byte[] that represents a normalized, but logically equivalent, form of this evaluator's expression.
    +   *
    +   * @return normalized expression in byte[] form
    +   */
    +  public byte[] flatten() {
    +    final Node normRoot = normalize(node, expression);
    +    final StringBuilder builder = new StringBuilder(expression.length);
    +    stringify(normRoot, expression, builder);
    +    return builder.toString().getBytes(UTF_8);
    +  }
    +
    +  private static class DocumentVisibilityParser {
    +    private int index = 0;
    +    private int parens = 0;
    +
    +    public DocumentVisibilityParser() {}
    +
    +    Node parse(final byte[] expression) {
    +      if (expression.length > 0) {
    +        final Node node = parse_(expression);
    +        if (node == null) {
    +          throw new BadArgumentException("operator or missing parens", new String(expression, UTF_8), index - 1);
    +        }
    +        if (parens != 0) {
    +          throw new BadArgumentException("parenthesis mis-match", new String(expression, UTF_8), index - 1);
    +        }
    +        return node;
    +      }
    +      return null;
    +    }
    +
    +    Node processTerm(final int start, final int end, final Node expr, final byte[] expression) {
    +      if (start != end) {
    +        if (expr != null) {
    +			throw new BadArgumentException("expression needs | or &", new String(expression, UTF_8), start);
    +		}
    +        return new Node(expression, start, end);
    +      }
    +      if (expr == null) {
    +		throw new BadArgumentException("empty term", new String(expression, UTF_8), start);
    +	}
    +      return expr;
    +    }
    +
    +    Node parse_(final byte[] expression) {
    +      Node result = null;
    +      Node expr = null;
    +      final int wholeTermStart = index;
    +      int subtermStart = index;
    +      boolean subtermComplete = false;
    +
    +      while (index < expression.length) {
    +        switch (expression[index++]) {
    +          case '&': {
    +            expr = processTerm(subtermStart, index - 1, expr, expression);
    +            if (result != null) {
    +              if (!result.type.equals(NodeType.AND)) {
    +				throw new BadArgumentException("cannot mix & and |", new String(expression, UTF_8), index - 1);
    +			}
    +            } else {
    +              result = new Node(expression, NodeType.AND, wholeTermStart);
    +            }
    +            result.add(expr);
    +            expr = null;
    +            subtermStart = index;
    +            subtermComplete = false;
    +            break;
    +          }
    +          case '|': {
    +            expr = processTerm(subtermStart, index - 1, expr, expression);
    +            if (result != null) {
    +              if (!result.type.equals(NodeType.OR)) {
    +				throw new BadArgumentException("cannot mix | and &", new String(expression, UTF_8), index - 1);
    +			}
    +            } else {
    +              result = new Node(expression, NodeType.OR, wholeTermStart);
    +            }
    +            result.add(expr);
    +            expr = null;
    +            subtermStart = index;
    +            subtermComplete = false;
    +            break;
    +          }
    +          case '(': {
    +            parens++;
    +            if (subtermStart != index - 1 || expr != null) {
    +				throw new BadArgumentException("expression needs & or |", new String(expression, UTF_8), index - 1);
    +			}
    +            expr = parse_(expression);
    +            subtermStart = index;
    +            subtermComplete = false;
    +            break;
    +          }
    +          case ')': {
    +            parens--;
    +            final Node child = processTerm(subtermStart, index - 1, expr, expression);
    +            if (child == null && result == null) {
    +				throw new BadArgumentException("empty expression not allowed", new String(expression, UTF_8), index);
    +			}
    +            if (result == null) {
    +				return child;
    +			}
    +            if (result.type == child.type) {
    +				for (final Node c : child.children) {
    +					result.add(c);
    +				}
    +			} else {
    +				result.add(child);
    +			}
    +            result.end = index - 1;
    +            return result;
    +          }
    +          case '"': {
    +            if (subtermStart != index - 1) {
    +				throw new BadArgumentException("expression needs & or |", new String(expression, UTF_8), index - 1);
    +			}
    +
    +            while (index < expression.length && expression[index] != '"') {
    +              if (expression[index] == '\\') {
    +                index++;
    +                if (expression[index] != '\\' && expression[index] != '"') {
    +					throw new BadArgumentException("invalid escaping within quotes", new String(expression, UTF_8), index - 1);
    +				}
    +              }
    +              index++;
    +            }
    +
    +            if (index == expression.length) {
    +				throw new BadArgumentException("unclosed quote", new String(expression, UTF_8), subtermStart);
    +			}
    +
    +            if (subtermStart + 1 == index) {
    +				throw new BadArgumentException("empty term", new String(expression, UTF_8), subtermStart);
    +			}
    +
    +            index++;
    +
    +            subtermComplete = true;
    +
    +            break;
    +          }
    +          default: {
    +            if (subtermComplete) {
    +				throw new BadArgumentException("expression needs & or |", new String(expression, UTF_8), index - 1);
    +			}
    +
    +            final byte c = expression[index - 1];
    +            if (!Authorizations.isValidAuthChar(c)) {
    +				throw new BadArgumentException("bad character (" + c + ")", new String(expression, UTF_8), index - 1);
    +			}
    +          }
    +        }
    +      }
    +      final Node child = processTerm(subtermStart, index, expr, expression);
    +      if (result != null) {
    +        result.add(child);
    +        result.end = index;
    +      } else {
    +		result = child;
    +	}
    +      if (result.type != NodeType.TERM) {
    +		if (result.children.size() < 2) {
    +			throw new BadArgumentException("missing term", new String(expression, UTF_8), index);
    +		}
    +	}
    +      return result;
    +    }
    +  }
    +
    +  private void validate(final byte[] expression) {
    +    if (expression != null && expression.length > 0) {
    +      final DocumentVisibilityParser p = new DocumentVisibilityParser();
    +      node = p.parse(expression);
    +    } else {
    +      node = EMPTY_NODE;
    +    }
    +    this.expression = expression;
    +  }
    +
    +  /**
    +   * Creates an empty visibility. Normally, elements with empty visibility can be seen by everyone. Though, one could change this behavior with filters.
    +   *
    +   * @see #DocumentVisibility(String)
    +   */
    +  public DocumentVisibility() {
    +    this(new byte[] {});
    +  }
    +
    +  /**
    +   * Creates a document visibility for a Mutation.
    +   *
    +   * @param expression
    +   *          An expression of the rights needed to see this mutation. The expression syntax is defined at the class-level documentation
    +   */
    +  public DocumentVisibility(final String expression) {
    +    this(expression.getBytes(UTF_8));
    +  }
    +
    +  /**
    +   * Creates a document visibility for a Mutation.
    +   *
    +   * @param expression
    +   *          visibility expression
    +   * @see #DocumentVisibility(String)
    +   */
    +  public DocumentVisibility(final Text expression) {
    +    this(TextUtil.getBytes(expression));
    +  }
    +
    +  /**
    +   * Creates a document visibility for a Mutation from a string already encoded in UTF-8 bytes.
    +   *
    +   * @param expression
    +   *          visibility expression, encoded as UTF-8 bytes
    +   * @see #DocumentVisibility(String)
    +   */
    +  public DocumentVisibility(final byte[] expression) {
    +    validate(expression);
    +  }
    +
    +  @Override
    +  public String toString() {
    +    return "[" + new String(expression, UTF_8) + "]";
    +  }
    +
    +  /**
    +   * See {@link #equals(DocumentVisibility)}
    +   */
    +  @Override
    +  public boolean equals(final Object obj) {
    +    if (obj instanceof DocumentVisibility) {
    +		return equals(obj);
    +	}
    +    return false;
    +  }
    +
    +  /**
    +   * Compares two DocumentVisibilities for string equivalence, not as a meaningful comparison of terms and conditions.
    +   *
    +   * @param otherLe
    +   *          other document visibility
    +   * @return true if this visibility equals the other via string comparison
    +   */
    +  public boolean equals(final DocumentVisibility otherLe) {
    +    return Arrays.equals(expression, otherLe.expression);
    +  }
    +
    +  @Override
    +  public int hashCode() {
    +    return Arrays.hashCode(expression);
    +  }
    +
    +  /**
    +   * Gets the parse tree for this document visibility.
    +   *
    +   * @return parse tree node
    +   */
    +  public Node getParseTree() {
    +    return node;
    +  }
    +
    +  /**
    +   * Properly quotes terms in a document visibility expression. If no quoting is needed, then nothing is done.
    +   *
    +   * <p>
    +   * Examples of using quote :
    +   *
    +   * <pre>
    +   * import static org.apache.rya.mongodb.document.visibility.DocumentVisibility.quote;
    +   *   .
    +   *   .
    +   *   .
    +   * DocumentVisibility dv = new DocumentVisibility(quote(&quot;A#C&quot;) + &quot;&amp;&quot; + quote(&quot;FOO&quot;));
    +   * </pre>
    +   *
    +   * @param term
    +   *          term to quote
    +   * @return quoted term (unquoted if unnecessary)
    +   */
    +  public static String quote(final String term) {
    +    return new String(quote(term.getBytes(UTF_8)), UTF_8);
    +  }
    +
    +  /**
    +   * Properly quotes terms in a document visibility expression. If no quoting is needed, then nothing is done.
    +   *
    +   * @param term
    +   *          term to quote, encoded as UTF-8 bytes
    +   * @return quoted term (unquoted if unnecessary), encoded as UTF-8 bytes
    +   * @see #quote(String)
    +   */
    +  public static byte[] quote(final byte[] term) {
    +    boolean needsQuote = false;
    +
    +    for (int i = 0; i < term.length; i++) {
    +      if (!Authorizations.isValidAuthChar(term[i])) {
    +        needsQuote = true;
    +        break;
    +      }
    +    }
    +
    +    if (!needsQuote) {
    +		return term;
    --- End diff --
    
    odd whitespace


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89908376
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/iter/AggregationUtil.java ---
    @@ -0,0 +1,322 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.mongodb.iter;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.LinkedList;
    +import java.util.List;
    +
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.dao.SimpleMongoDBStorageStrategy;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +
    +import com.google.common.collect.Lists;
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.DBObject;
    +
    +/**
    + * Utility methods for MongoDB aggregation.
    + */
    +public final class AggregationUtil {
    +    /**
    +     * Private constructor to prevent instantiation.
    +     */
    +    private AggregationUtil() {
    +    }
    +
    +    /**
    +     * Creates a MongoDB $redact aggregation pipeline that only include
    +     * documents whose document visibility match the provided authorizations.
    +     * All other documents are excluded.
    +     * @param authorizations the {@link Authorization}s to include in the
    +     * $redact. Only documents that match the authorizations will be returned.
    +     * @return the {@link List} of {@link DBObject}s that represents the $redact
    +     * aggregation pipeline.
    +     */
    +    public static List<DBObject> createRedactPipeline(final Authorizations authorizations) {
    +        if (MongoDbRdfConstants.ALL_AUTHORIZATIONS.equals(authorizations)) {
    +            return Lists.newArrayList();
    +        }
    +        final List<String> authAndList = authorizations.getAuthorizationsStrings();
    +
    +        // Generate all combinations of the authorization strings without repetition.
    +        final List<List<String>> authOrList = createCombinations(authorizations.getAuthorizationsStrings());
    +
    +        final String documentVisibilityField = "$" + SimpleMongoDBStorageStrategy.DOCUMENT_VISIBILITY;
    +
    +        final BasicDBObject setIsSubset =
    +            setIsSubsetNullSafe(
    +                documentVisibilityField,
    +                authAndList.toArray()
    +            );
    +
    +        final BasicDBObject setIntersectionExists =
    +            gt(
    +                size(
    +                    setIntersection(
    +                        documentVisibilityField,
    +                        authOrList.toArray()
    +                    )
    +                ),
    +                0
    +            );
    +
    +        final BasicDBObject orExpression = or(setIsSubset, setIntersectionExists);
    +
    +        final List<DBObject> pipeline = new ArrayList<>();
    +        pipeline.add(
    +            redact(
    +               orExpression,
    +               "$$DESCEND",
    +               "$$PRUNE"
    +            )
    +        );
    +
    +        return pipeline;
    +    }
    +
    +    /**
    +     * Creates all combinations of the values that are of the size of value
    +     * array or smaller without repetition.
    +     * @param values the {@link List} of values to create combinations from.
    +     * @return the {@link List} of combinations.
    +     */
    +    public static <T> List<List<T>> createCombinations(final List<T> values) {
    +        final List<List<T>> allCombinations = new ArrayList<>();
    +        for (int i = 1; i <= values.size(); i++) {
    +            allCombinations.addAll(createCombinations(values, i));
    +        }
    +        return allCombinations;
    +    }
    +
    +    /**
    +     * Creates all combinations of the values that are of the specified size
    +     * without repetition.
    +     * @param values the {@link List} of values to create combinations from.
    +     * @param size the size of the combinations.
    +     * @return the {@link List} of combinations.
    +     */
    +    public static <T> List<List<T>> createCombinations(final List<T> values, final int size) {
    --- End diff --
    
    why not do an Auth -> bit, then do bit comparison?  This seems potentially expensive, there's gotta be a better way to do ordering comparisons


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89902691
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/ArgumentChecker.java ---
    @@ -0,0 +1,112 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +/**
    + * This class provides methods to check arguments of a variable number for null values, or anything else that might be required on a routine basis. These
    + * methods should be used for early failures as close to the end user as possible, so things do not fail later on the server side, when they are harder to
    + * debug.
    + *
    + * Methods are created for a specific number of arguments, due to the poor performance of array allocation for varargs methods.
    + */
    +public class ArgumentChecker {
    --- End diff --
    
    what is this?  Why not use preconditions?


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

[GitHub] incubator-rya issue #124: RYA-119 Added MongoDB Column Visibility (called Do...

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

    https://github.com/apache/incubator-rya/pull/124
  
    oh one other thing, was anything done to integrate document visibility into the indexers?  if not, there should be a bug to add that in the future


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89898179
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBStorageStrategy.java ---
    @@ -96,12 +103,19 @@ public DBObject getQuery(final RyaStatement stmt) {
     
         @Override
         public RyaStatement deserializeDBObject(final DBObject queryResult) {
    -        final Map result = queryResult.toMap();
    +        final Map<?, ?> result = queryResult.toMap();
             final String subject = (String) result.get(SUBJECT);
             final String object = (String) result.get(OBJECT);
             final String objectType = (String) result.get(OBJECT_TYPE);
             final String predicate = (String) result.get(PREDICATE);
             final String context = (String) result.get(CONTEXT);
    +        final Object documentVisibilityObject = result.get(DOCUMENT_VISIBILITY);
    --- End diff --
    
    create an adapter for converting to/from mongo object to the Authorization type and use that here.
    You'll find examples on adapters I've made in temporal indexing


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90067593
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    --- End diff --
    
    same as above


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90057717
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/dao/SimpleMongoDBStorageStrategy.java ---
    @@ -96,12 +103,19 @@ public DBObject getQuery(final RyaStatement stmt) {
     
         @Override
         public RyaStatement deserializeDBObject(final DBObject queryResult) {
    -        final Map result = queryResult.toMap();
    +        final Map<?, ?> result = queryResult.toMap();
    --- End diff --
    
    A Map<String, Object> is returned here but I would need to add @SuppressWarnings("unchecked") here which I'd prefer not to do.  Leaving it as just Map generates a raw type warning.  Map<?, ?> gets rid of the warning and is safe.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r119368692
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/iter/RyaStatementCursorIterator.java ---
    @@ -22,83 +40,83 @@
     
     import info.aduna.iteration.CloseableIteration;
     
    -import java.util.Iterator;
    -import java.util.Map.Entry;
    -import java.util.Set;
    -
    -import org.apache.rya.api.RdfCloudTripleStoreUtils;
    -import org.apache.rya.api.domain.RyaStatement;
    -import org.apache.rya.api.persist.RyaDAOException;
    -import org.apache.rya.mongodb.dao.MongoDBStorageStrategy;
    -
    -import org.calrissian.mango.collect.CloseableIterable;
    -import org.openrdf.query.BindingSet;
    -
    -import com.mongodb.DBCollection;
    -import com.mongodb.DBCursor;
    -import com.mongodb.DBObject;
    -
     public class RyaStatementCursorIterator implements CloseableIteration<RyaStatement, RyaDAOException> {
    -
    -	private DBCollection coll;
    -	private Iterator<DBObject> queryIterator;
    -	private DBCursor currentCursor;
    -	private MongoDBStorageStrategy strategy;
    -	private Long maxResults;
    -
    -	public RyaStatementCursorIterator(DBCollection coll, Set<DBObject> queries, MongoDBStorageStrategy strategy) {
    -		this.coll = coll;
    -		this.queryIterator = queries.iterator();
    -		this.strategy = strategy;
    -	}
    -
    -	@Override
    -	public boolean hasNext() {
    -		if (!currentCursorIsValid()) {
    -			findNextValidCursor();
    -		}
    -		return currentCursorIsValid();
    -	}
    -
    -	@Override
    -	public RyaStatement next() {
    -		if (!currentCursorIsValid()) {
    -			findNextValidCursor();
    -		}
    -		if (currentCursorIsValid()) {
    -			// convert to Rya Statement
    -			DBObject queryResult = currentCursor.next();
    -			RyaStatement statement = strategy.deserializeDBObject(queryResult);
    -			return statement;
    -		}
    -		return null;
    -	}
    -	
    -	private void findNextValidCursor() {
    -		while (queryIterator.hasNext()){
    -			DBObject currentQuery = queryIterator.next();
    -			currentCursor = coll.find(currentQuery);
    -			if (currentCursor.hasNext()) break;
    -		}
    -	}
    -	
    -	private boolean currentCursorIsValid() {
    -		return (currentCursor != null) && currentCursor.hasNext();
    -	}
    -
    -
    -	public void setMaxResults(Long maxResults) {
    -		this.maxResults = maxResults;
    -	}
    -
    -	@Override
    -	public void close() throws RyaDAOException {
    -		// TODO don't know what to do here
    -	}
    -
    -	@Override
    -	public void remove() throws RyaDAOException {
    -		next();
    -	}
    -
    +    private static final Logger log = Logger.getLogger(RyaStatementCursorIterator.class);
    +
    +    private final DBCollection coll;
    +    private final Iterator<DBObject> queryIterator;
    +    private Iterator<DBObject> resultsIterator;
    +    private final MongoDBStorageStrategy<RyaStatement> strategy;
    +    private Long maxResults;
    +    private final Authorizations auths;
    +
    +    public RyaStatementCursorIterator(final DBCollection coll, final Set<DBObject> queries, final MongoDBStorageStrategy<RyaStatement> strategy, final MongoDBRdfConfiguration conf) {
    +        this.coll = coll;
    +        this.queryIterator = queries.iterator();
    +        this.strategy = strategy;
    +        if (conf != null) {
    +            this.auths = conf.getAuthorizations();
    +        } else {
    +            auths = MongoDbRdfConstants.ALL_AUTHORIZATIONS;
    +        }
    +    }
    +
    +    @Override
    +    public boolean hasNext() {
    +        if (!currentCursorIsValid()) {
    +            findNextValidCursor();
    +        }
    +        return currentCursorIsValid();
    +    }
    +
    +    @Override
    +    public RyaStatement next() {
    +        if (!currentCursorIsValid()) {
    +            findNextValidCursor();
    +        }
    +        if (currentCursorIsValid()) {
    +            // convert to Rya Statement
    +            final DBObject queryResult = resultsIterator.next();
    +            final RyaStatement statement = strategy.deserializeDBObject(queryResult);
    +            return statement;
    +        }
    +        return null;
    +    }
    +
    +    private void findNextValidCursor() {
    +        while (queryIterator.hasNext()){
    +            final DBObject currentQuery = queryIterator.next();
    +
    +            // Executing redact aggregation to only return documents the user
    +            // has access to.
    +            final List<DBObject> pipeline = new ArrayList<>();
    +            pipeline.add(new BasicDBObject("$match", currentQuery));
    +            pipeline.addAll(AggregationUtil.createRedactPipeline(auths));
    +            log.debug(pipeline);
    +            final AggregationOutput output = coll.aggregate(pipeline);
    +            resultsIterator = output.results().iterator();
    +            if (resultsIterator.hasNext()) {
    +                break;
    +            }
    +        }
    +    }
    +
    +    private boolean currentCursorIsValid() {
    +        return (resultsIterator != null) && resultsIterator.hasNext();
    +    }
    +
    +
    +    public void setMaxResults(final Long maxResults) {
    +        this.maxResults = maxResults;
    +    }
    +
    +    @Override
    +    public void close() throws RyaDAOException {
    +        // TODO don't know what to do here
    --- End diff --
    
    This PR was prematurely closed.


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

[GitHub] incubator-rya issue #124: RYA-119 Added MongoDB Column Visibility (called Do...

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

    https://github.com/apache/incubator-rya/pull/124
  
    a few questions, but otherwise looks good to me :ship: :it: 


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89905111
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    + */
    +public class Authorizations implements Iterable<byte[]>, Serializable, AuthorizationContainer {
    +
    +  private static final long serialVersionUID = 1L;
    +
    +  private final Set<ByteSequence> auths = new HashSet<ByteSequence>();
    +  private final List<byte[]> authsList = new ArrayList<byte[]>(); // sorted order
    +
    +  /**
    +   * An empty set of authorizations.
    +   */
    +  public static final Authorizations EMPTY = new Authorizations();
    +
    +  private static final boolean[] validAuthChars = new boolean[256];
    +
    +  /**
    +   * A special header string used when serializing instances of this class.
    +   *
    +   * @see #serialize()
    +   */
    +  public static final String HEADER = "!AUTH1:";
    +
    +  static {
    +    for (int i = 0; i < 256; i++) {
    +      validAuthChars[i] = false;
    +    }
    +
    +    for (int i = 'a'; i <= 'z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = 'A'; i <= 'Z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = '0'; i <= '9'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    validAuthChars['_'] = true;
    +    validAuthChars['-'] = true;
    +    validAuthChars[':'] = true;
    +    validAuthChars['.'] = true;
    +    validAuthChars['/'] = true;
    +  }
    +
    +  static final boolean isValidAuthChar(final byte b) {
    +    return validAuthChars[0xff & b];
    +  }
    +
    +  private void checkAuths() {
    +    final Set<ByteSequence> sortedAuths = new TreeSet<ByteSequence>(auths);
    +
    +    for (final ByteSequence bs : sortedAuths) {
    +      if (bs.length() == 0) {
    +        throw new IllegalArgumentException("Empty authorization");
    +      }
    +
    +      authsList.add(bs.toArray());
    +    }
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a collection of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does
    +   * not verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          collection of authorizations, as strings encoded in UTF-8
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final Collection<byte[]> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    for (final byte[] auth : authorizations) {
    +		auths.add(new ArrayByteSequence(auth));
    +	}
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a list of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does not
    +   * verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          list of authorizations, as strings encoded in UTF-8 and placed in buffers
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final List<ByteBuffer> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    for (final ByteBuffer buffer : authorizations) {
    +      auths.add(new ArrayByteSequence(ByteBufferUtil.toBytes(buffer)));
    +    }
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Constructs an authorizations object from a serialized form. This is NOT a constructor for a set of authorizations of size one. Warning: This method does
    +   * not verify that the encoded serialized form is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          a serialized authorizations string produced by {@link #getAuthorizationsArray()} or {@link #serialize()}, converted to UTF-8 bytes
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   */
    +  public Authorizations(final byte[] authorizations) {
    +
    +    ArgumentChecker.notNull(authorizations);
    +
    +    String authsString = new String(authorizations, UTF_8);
    +    if (authsString.startsWith(HEADER)) {
    +      // it's the new format
    +      authsString = authsString.substring(HEADER.length());
    +      if (authsString.length() > 0) {
    +        for (final String encAuth : authsString.split(",")) {
    +          final byte[] auth = Base64.decodeBase64(encAuth.getBytes(UTF_8));
    +          auths.add(new ArrayByteSequence(auth));
    +        }
    +        checkAuths();
    +      }
    +    } else {
    +      // it's the old format
    +      if (authorizations.length > 0) {
    +		setAuthorizations(authsString.split(","));
    +	}
    +    }
    +  }
    +
    +  /**
    +   * Constructs an empty set of authorizations.
    +   *
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations() {}
    +
    +  /**
    +   * Constructs an authorizations object from a set of human-readable authorizations.
    +   *
    +   * @param authorizations
    +   *          array of authorizations
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   */
    +  public Authorizations(final String... authorizations) {
    +    setAuthorizations(authorizations);
    +  }
    +
    +  private void setAuthorizations(final String... authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    auths.clear();
    +    for (String str : authorizations) {
    +      str = str.trim();
    +      auths.add(new ArrayByteSequence(str.getBytes(UTF_8)));
    +    }
    +
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Returns a serialized form of these authorizations.
    +   *
    +   * @return serialized form of these authorizations, as a string encoded in UTF-8
    +   * @see #serialize()
    +   */
    +  public byte[] getAuthorizationsArray() {
    +    return serialize().getBytes(UTF_8);
    +  }
    +
    +  /**
    +   * Gets the authorizations in sorted order. The returned list is not modifiable.
    +   *
    +   * @return authorizations, each as a string encoded in UTF-8
    +   * @see #Authorizations(Collection)
    +   */
    +  public List<byte[]> getAuthorizations() {
    +    final ArrayList<byte[]> copy = new ArrayList<byte[]>(authsList.size());
    +    for (final byte[] auth : authsList) {
    +      final byte[] bytes = new byte[auth.length];
    +      System.arraycopy(auth, 0, bytes, 0, auth.length);
    +      copy.add(bytes);
    +    }
    +    return Collections.unmodifiableList(copy);
    +  }
    +
    +  /**
    +   * Gets the authorizations in sorted order. The returned list is not modifiable.
    +   *
    +   * @return authorizations, each as a string encoded in UTF-8 and within a buffer
    +   */
    +  public List<ByteBuffer> getAuthorizationsBB() {
    +    final ArrayList<ByteBuffer> copy = new ArrayList<ByteBuffer>(authsList.size());
    +    for (final byte[] auth : authsList) {
    +      final byte[] bytes = new byte[auth.length];
    +      System.arraycopy(auth, 0, bytes, 0, auth.length);
    +      copy.add(ByteBuffer.wrap(bytes));
    +    }
    +    return Collections.unmodifiableList(copy);
    +  }
    +
    +  /**
    +   * Gets the authorizations in sorted order. The returned list is not modifiable.
    +   *
    +   * @return authorizations, each as a string encoded in UTF-8
    +   */
    +  public List<String> getAuthorizationsStrings() {
    --- End diff --
    
    why do we have so many different representations?  why not allow it to build an Authorization from a bunch of different types, but get 1?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89905731
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Base64.java ---
    @@ -0,0 +1,73 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import org.apache.commons.codec.binary.StringUtils;
    +
    +/**
    + * A wrapper around commons-codec's Base64 to make sure we get the non-chunked behavior that became the default in commons-codec version 1.5+ while relying on
    + * the commons-codec version 1.4 that Hadoop Client provides.
    + */
    +public final class Base64 {
    --- End diff --
    
    why not just use 1.4?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89903230
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/ArrayByteSequence.java ---
    @@ -0,0 +1,126 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +
    +public class ArrayByteSequence extends ByteSequence implements Serializable {
    --- End diff --
    
    This looks like Java's ByteBuffer could replace the whole class?


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90343779
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/Authorizations.java ---
    @@ -0,0 +1,369 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import static com.google.common.base.Charsets.UTF_8;
    +
    +import java.io.Serializable;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.TreeSet;
    +
    +import com.google.common.base.Charsets;
    +
    +/**
    + * A collection of authorization strings.
    + */
    +public class Authorizations implements Iterable<byte[]>, Serializable, AuthorizationContainer {
    +
    +  private static final long serialVersionUID = 1L;
    +
    +  private final Set<ByteSequence> auths = new HashSet<ByteSequence>();
    +  private final List<byte[]> authsList = new ArrayList<byte[]>(); // sorted order
    +
    +  /**
    +   * An empty set of authorizations.
    +   */
    +  public static final Authorizations EMPTY = new Authorizations();
    +
    +  private static final boolean[] validAuthChars = new boolean[256];
    +
    +  /**
    +   * A special header string used when serializing instances of this class.
    +   *
    +   * @see #serialize()
    +   */
    +  public static final String HEADER = "!AUTH1:";
    +
    +  static {
    +    for (int i = 0; i < 256; i++) {
    +      validAuthChars[i] = false;
    +    }
    +
    +    for (int i = 'a'; i <= 'z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = 'A'; i <= 'Z'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    for (int i = '0'; i <= '9'; i++) {
    +      validAuthChars[i] = true;
    +    }
    +
    +    validAuthChars['_'] = true;
    +    validAuthChars['-'] = true;
    +    validAuthChars[':'] = true;
    +    validAuthChars['.'] = true;
    +    validAuthChars['/'] = true;
    +  }
    +
    +  static final boolean isValidAuthChar(final byte b) {
    +    return validAuthChars[0xff & b];
    +  }
    +
    +  private void checkAuths() {
    +    final Set<ByteSequence> sortedAuths = new TreeSet<ByteSequence>(auths);
    +
    +    for (final ByteSequence bs : sortedAuths) {
    +      if (bs.length() == 0) {
    +        throw new IllegalArgumentException("Empty authorization");
    +      }
    +
    +      authsList.add(bs.toArray());
    +    }
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a collection of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does
    +   * not verify that each encoded string is valid UTF-8.
    +   *
    +   * @param authorizations
    +   *          collection of authorizations, as strings encoded in UTF-8
    +   * @throws IllegalArgumentException
    +   *           if authorizations is null
    +   * @see #Authorizations(String...)
    +   */
    +  public Authorizations(final Collection<byte[]> authorizations) {
    +    ArgumentChecker.notNull(authorizations);
    +    for (final byte[] auth : authorizations) {
    +		auths.add(new ArrayByteSequence(auth));
    +	}
    +    checkAuths();
    +  }
    +
    +  /**
    +   * Constructs an authorization object from a list of string authorizations that have each already been encoded as UTF-8 bytes. Warning: This method does not
    +   * verify that each encoded string is valid UTF-8.
    --- End diff --
    
    same as above


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89907399
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/visibility/VisibilityEvaluator.java ---
    @@ -0,0 +1,152 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.rya.mongodb.document.visibility;
    +
    +import java.util.ArrayList;
    +
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.Node;
    +
    +/**
    + * A class which evaluates visibility expressions against a set of authorizations.
    + */
    +public class VisibilityEvaluator {
    +  private final AuthorizationContainer auths;
    +
    +  /**
    +   * Creates a new {@link Authorizations} object with escaped forms of the authorizations in the given object.
    +   *
    --- End diff --
    
    give an example of an escaped form


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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

    https://github.com/apache/incubator-rya/pull/124#discussion_r90055421
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/MongoDbRdfConstants.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.mongodb;
    +
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
    +
    +/**
    + * Interface MongoDbRdfConstants.
    --- End diff --
    
    I was mimicking how the Rya Accumulo side did things.


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

[GitHub] incubator-rya pull request #124: RYA-119 Added MongoDB Column Visibility (ca...

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/124#discussion_r89902538
  
    --- Diff: dao/mongodb.rya/src/main/java/org/apache/rya/mongodb/document/util/DocumentVisibilityUtil.java ---
    @@ -0,0 +1,316 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +package org.apache.rya.mongodb.document.util;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.log4j.Logger;
    +import org.apache.rya.mongodb.MongoDbRdfConstants;
    +import org.apache.rya.mongodb.document.visibility.Authorizations;
    +import org.apache.rya.mongodb.document.visibility.ByteSequence;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.Node;
    +import org.apache.rya.mongodb.document.visibility.DocumentVisibility.NodeType;
    +import org.apache.rya.mongodb.document.visibility.VisibilityEvaluator;
    +import org.apache.rya.mongodb.document.visibility.VisibilityParseException;
    +
    +import com.google.common.base.Charsets;
    +import com.google.common.collect.Lists;
    +import com.mongodb.BasicDBList;
    +
    +/**
    + * Utility methods for converting boolean expressions between an Accumulo column
    + * visibility string style and a multidimensional array that can be used
    + * in MongoDB expressions.
    + */
    +public final class DocumentVisibilityUtil {
    +    private static final Logger log = Logger.getLogger(DocumentVisibilityUtil.class);
    +
    +    /**
    +     * Private constructor to prevent instantiation.
    +     */
    +    private DocumentVisibilityUtil() {
    +    }
    +
    +    /**
    +     * Converts a boolean string expression into a multidimensional
    +     * array representation of the boolean expression.
    +     * @param booleanString the boolean string expression.
    +     * @return the multidimensional array representation of the boolean
    +     * expression.
    +     */
    +    public static Object[] toMultidimensionalArray(final String booleanString) {
    +        final DocumentVisibility dv = new DocumentVisibility(booleanString);
    +        return toMultidimensionalArray(dv);
    +    }
    +
    +    /**
    +     * Converts a {@link DocumentVisibility} object into a multidimensional
    +     * array representation of the boolean expression.
    +     * @param dv the {@link DocumentVisibility}. (not {@code null})
    +     * @return the multidimensional array representation of the boolean
    +     * expression.
    +     */
    +    public static Object[] toMultidimensionalArray(final DocumentVisibility dv) {
    +        checkNotNull(dv);
    +        final byte[] expression = dv.flatten();
    +        final DocumentVisibility flattenedDv = DisjunctiveNormalFormConverter.createDnfDocumentVisibility(expression);
    +        final Object[] result = toMultidimensionalArray(flattenedDv.getParseTree(), expression);
    +        return result;
    +    }
    +
    +    /**
    +     * Converts a {@link Node} and its corresponding expression into a
    +     * multidimensional array representation of the boolean expression.
    +     * @param node the {@link Node}. (not {@code null})
    +     * @param expression the expression byte array.
    +     * @return the multidimensional array representation of the boolean
    +     * expression.
    +     */
    +    public static Object[] toMultidimensionalArray(final Node node, final byte[] expression) {
    +        checkNotNull(node);
    +        final List<Object> array = new ArrayList<>();
    +
    +        if (node.getChildren().isEmpty() && node.getType() == NodeType.TERM) {
    +            final String data = getTermNodeData(node);
    +            array.add(data);
    +        }
    +
    +        log.trace("Children size: " + node.getChildren().size() + " Type: " + node.getType());
    +        for (final Node child : node.getChildren()) {
    +            switch (child.getType()) {
    +                case EMPTY:
    +                case TERM:
    +                    String data;
    +                    if (child.getType() == NodeType.TERM) {
    +                        data = getTermNodeData(child);
    +                    } else {
    +                        data = "";
    +                    }
    +                    if (node.getType() == NodeType.OR) {
    +                        array.add(Lists.newArrayList(data).toArray(new Object[0]));
    +                    } else {
    +                        array.add(data);
    +                    }
    +                    break;
    +                case OR:
    +                case AND:
    +                    array.add(toMultidimensionalArray(child, expression));
    +                    break;
    +                default:
    +                    break;
    +            }
    +        }
    +
    +        return array.toArray(new Object[0]);
    +    }
    +
    +    public static String nodeToBooleanString(final Node node) {
    +        boolean isFirst = true;
    +        final StringBuilder sb = new StringBuilder();
    +        if (node.getType() == NodeType.TERM) {
    +            final String data = getTermNodeData(node);
    +            sb.append(data);
    +        }
    +        if (node.getType() == NodeType.AND) {
    +            sb.append("(");
    +        }
    +        for (final Node child : node.getChildren()) {
    +            if (isFirst) {
    +                isFirst = false;
    +            } else {
    +                if (node.getType() == NodeType.OR) {
    +                    sb.append("|");
    +                } else if (node.getType() == NodeType.AND) {
    +                    sb.append("&");
    +                }
    +            }
    +            switch (child.getType()) {
    +                case EMPTY:
    +                    sb.append("");
    +                    break;
    +                case TERM:
    +                    final String data = getTermNodeData(child);
    +                    sb.append(data);
    +                    break;
    +                case OR:
    +                    sb.append("(");
    +                    sb.append(nodeToBooleanString(child));
    +                    sb.append(")");
    +                    break;
    +                case AND:
    +                    sb.append(nodeToBooleanString(child));
    +                    break;
    +                default:
    +                    break;
    +            }
    +        }
    +        if (node.getType() == NodeType.AND) {
    +            sb.append(")");
    +        }
    +
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Converts a multidimensional array object representation of the document
    +     * visibility boolean expression into a string.
    +     * @param object the multidimensional array object representing the
    +     * document visibility boolean expression.
    +     * @return the boolean string expression.
    +     */
    +    public static String multidimensionalArrayToBooleanString(final Object[] object) {
    +        final String booleanString = multidimensionalArrayToBooleanStringInternal(object);
    +
    +        // Simplify and clean up the formatting.
    +        final DocumentVisibility dv = DisjunctiveNormalFormConverter.createDnfDocumentVisibility(booleanString);
    +        final byte[] bytes = dv.flatten();
    +        final String result = new String(bytes, Charsets.UTF_8);
    +
    +        return result;
    +    }
    +
    +    private static String multidimensionalArrayToBooleanStringInternal(final Object[] object) {
    +        final StringBuilder sb = new StringBuilder();
    +
    +        int count = 0;
    +        boolean isAnd = false;
    +        for (final Object child : object) {
    +            if (child instanceof String) {
    +                isAnd = true;
    +                if (count > 0) {
    +                    sb.append("&");
    +                }
    +                sb.append(child);
    +            } else if (child instanceof Object[]) {
    +                if (count > 0 && isAnd) {
    +                    sb.append("&");
    +                }
    +                final Object[] obj = (Object[]) child;
    +                sb.append("(");
    +                sb.append(multidimensionalArrayToBooleanStringInternal(obj));
    +                sb.append(")");
    +            }
    +
    +            if (object.length > 1 && count + 1 < object.length && !isAnd) {
    +                sb.append("|");
    +            }
    +            count++;
    +        }
    +
    +        return sb.toString();
    +    }
    +
    +    /**
    +     * Conditionally adds quotes around a string.
    +     * @param data the string to add quotes to.
    +     * @param addQuotes {@code true} to add quotes. {@code false} to leave the
    +     * string as is.
    +     * @return the quoted string if {@code addQuotes} is {@code true}.
    +     * Otherwise, returns the string as is.
    +     */
    +    public static String addQuotes(final String data, final boolean addQuotes) {
    +        if (addQuotes) {
    +            return "\"" + data + "\"";
    +        } else {
    +            return data;
    +        }
    +    }
    +
    +    /**
    +     * Returns the term node's data.
    +     * @param node the {@link Node}.
    +     * @return the term node's data.
    +     */
    +    public static String getTermNodeData(final Node node) {
    +        final boolean isQuotedTerm = node.getExpression()[node.getTermStart()] == '"';
    +        final ByteSequence bs = node.getTerm(node.getExpression());
    +        final String data = addQuotes(new String(bs.toArray(), Charsets.UTF_8), isQuotedTerm);
    +        return data;
    +    }
    +
    +    /**
    +     * Checks if the user's authorizations allows them to have access to the
    +     * provided document based on its document visibility.
    +     * @param authorizations the {@link Authorizations}.
    +     * @param documentVisibility the document visibility byte expression.
    +     * @return {@code true} if the user has access to the document.
    +     * {@code false} otherwise.
    +     */
    +    public static boolean doesUserHaveDocumentAccess(final Authorizations authorizations, final byte[] documentVisibilityExpression) {
    +        final byte[] expression = documentVisibilityExpression != null ? documentVisibilityExpression : MongoDbRdfConstants.EMPTY_DV.getExpression();
    +        final DocumentVisibility documentVisibility = new DocumentVisibility(expression);
    +        return doesUserHaveDocumentAccess(authorizations, documentVisibility);
    +    }
    +
    +    /**
    +     * Checks if the user's authorizations allows them to have access to the
    +     * provided document based on its document visibility.
    +     * @param authorizations the {@link Authorizations}.
    +     * @param documentVisibility the {@link DocumentVisibility}.
    +     * @return {@code true} if the user has access to the document.
    +     * {@code false} otherwise.
    +     */
    +    public static boolean doesUserHaveDocumentAccess(final Authorizations authorizations, final DocumentVisibility documentVisibility) {
    +        return doesUserHaveDocumentAccess(authorizations, documentVisibility, true);
    +    }
    +
    +    /**
    +     * Checks if the user's authorizations allows them to have access to the
    +     * provided document based on its document visibility.
    +     * @param authorizations the {@link Authorizations}.
    +     * @param documentVisibility the {@link DocumentVisibility}.
    +     * @param doesEmptyAccessPass {@code true} if an empty authorization pass
    +     * allows access to everything. {@code false} otherwise.
    +     * @return {@code true} if the user has access to the document.
    +     * {@code false} otherwise.
    +     */
    +    public static boolean doesUserHaveDocumentAccess(final Authorizations authorizations, final DocumentVisibility documentVisibility, final boolean doesEmptyAccessPass) {
    +        final Authorizations userAuths = authorizations != null ? authorizations : MongoDbRdfConstants.ALL_AUTHORIZATIONS;
    +        final VisibilityEvaluator visibilityEvaluator = new VisibilityEvaluator(userAuths);
    +        boolean accept = false;
    +        if (doesEmptyAccessPass && MongoDbRdfConstants.ALL_AUTHORIZATIONS.equals(userAuths)) {
    +            accept = true;
    +        } else {
    +            try {
    +                accept = visibilityEvaluator.evaluate(documentVisibility);
    +            } catch (final VisibilityParseException e) {
    +                log.error("Could not parse document visibility.");
    +            }
    +        }
    +
    +        return accept;
    +    }
    +
    +    public static Object[] convertBasicDBListToObjectArray(final BasicDBList basicDbList) {
    --- End diff --
    
    doc


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