You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2014/09/22 22:45:12 UTC
[14/15] ACCUMULO-652 merged changes from trunk
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java
index 0000000,ead0964..85bd74a
mode 000000,100644..100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java
@@@ -1,0 -1,661 +1,656 @@@
+ /**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.accumulo.core.iterators.user;
+
+ import java.io.IOException;
+ import java.util.ArrayList;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.Comparator;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Map;
+ import java.util.NoSuchElementException;
+
+ import org.apache.accumulo.core.client.IteratorSetting;
+ import org.apache.accumulo.core.conf.AccumuloConfiguration;
+ import org.apache.accumulo.core.data.ByteSequence;
+ import org.apache.accumulo.core.data.Key;
+ import org.apache.accumulo.core.data.PartialKey;
+ import org.apache.accumulo.core.data.Range;
+ import org.apache.accumulo.core.data.Value;
+ import org.apache.accumulo.core.iterators.IteratorEnvironment;
+ import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+ import org.apache.accumulo.core.iterators.OptionDescriber;
+ import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+ import org.apache.accumulo.core.iterators.WrappingIterator;
+ import org.apache.accumulo.core.security.Authorizations;
+ import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.VisibilityEvaluator;
-import org.apache.accumulo.core.security.VisibilityParseException;
+ import org.apache.accumulo.core.util.BadArgumentException;
+ import org.apache.accumulo.core.util.Pair;
+ import org.apache.commons.collections.BufferOverflowException;
+ import org.apache.commons.collections.map.LRUMap;
+ import org.apache.hadoop.io.Text;
+ import org.apache.log4j.Logger;
+
+ /**
+ * The TransformingIterator allows portions of a key (except for the row) to be transformed. This iterator handles the details that come with modifying keys
+ * (i.e., that the sort order could change). In order to do so, however, the iterator must put all keys sharing the same prefix in memory. Prefix is defined as
+ * the parts of the key that are not modified by this iterator. That is, if the iterator modifies column qualifier and timestamp, then the prefix is row and
+ * column family. In that case, the iterator must load all column qualifiers for each row/column family pair into memory. Given this constraint, care must be
+ * taken by users of this iterator to ensure it is not run in such a way that will overrun memory in a tablet server.
+ * <p>
+ * If the implementing iterator is transforming column families, then it must also override {@code untransformColumnFamilies(Collection)} to handle the case
+ * when column families are fetched at scan time. The fetched column families will/must be in the transformed space, and the untransformed column families need
+ * to be passed to this iterator's source. If it is not possible to write a reverse transformation (e.g., the column family transformation depends on the row
+ * value or something like that), then the iterator must not fetch specific column families (or only fetch column families that are known to not transform at
+ * all).
+ * <p>
+ * If the implementing iterator is transforming column visibilities, then users must be careful NOT to fetch column qualifiers from the scanner. The reason for
+ * this is due to ACCUMULO-??? (insert issue number).
+ * <p>
+ * If the implementing iterator is transforming column visibilities, then the user should be sure to supply authorizations via the {@link #AUTH_OPT} iterator
+ * option (note that this is only necessary for scan scope iterators). The supplied authorizations should be in the transformed space, but the authorizations
+ * supplied to the scanner should be in the untransformed space. That is, if the iterator transforms A to 1, B to 2, C to 3, etc, then the auths supplied when
+ * the scanner is constructed should be A,B,C,... and the auths supplied to the iterator should be 1,2,3,... The reason for this is that the scanner performs
+ * security filtering before this iterator is called, so the authorizations need to be in the original untransformed space. Since the iterator can transform
+ * visibilities, it is possible that it could produce visibilities that the user cannot see, so the transformed keys must be tested to ensure the user is
+ * allowed to view them. Note that this test is not necessary when the iterator is not used in the scan scope since no security filtering is performed during
+ * major and minor compactions. It should also be noted that this iterator implements the security filtering rather than relying on a follow-on iterator to do
+ * it so that we ensure the test is performed.
+ */
+ abstract public class TransformingIterator extends WrappingIterator implements OptionDescriber {
+ public static final String AUTH_OPT = "authorizations";
+ public static final String MAX_BUFFER_SIZE_OPT = "maxBufferSize";
+ private static final long DEFAULT_MAX_BUFFER_SIZE = 10000000;
+
+ protected Logger log = Logger.getLogger(getClass());
+
+ protected ArrayList<Pair<Key,Value>> keys = new ArrayList<Pair<Key,Value>>();
+ protected int keyPos = -1;
+ protected boolean scanning;
+ protected Range seekRange;
+ protected Collection<ByteSequence> seekColumnFamilies;
+ protected boolean seekColumnFamiliesInclusive;
+
- private VisibilityEvaluator ve = null;
++ private Authorizations auths = null;
+ private LRUMap visibleCache = null;
+ private LRUMap parsedVisibilitiesCache = null;
+ private long maxBufferSize;
+
+ private static Comparator<Pair<Key,Value>> keyComparator = new Comparator<Pair<Key,Value>>() {
+ @Override
+ public int compare(Pair<Key,Value> o1, Pair<Key,Value> o2) {
+ return o1.getFirst().compareTo(o2.getFirst());
+ }
+ };
+
+ public TransformingIterator() {}
+
+ @Override
+ public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
+ super.init(source, options, env);
+ scanning = IteratorScope.scan.equals(env.getIteratorScope());
+ if (scanning) {
+ String auths = options.get(AUTH_OPT);
+ if (auths != null && !auths.isEmpty()) {
- ve = new VisibilityEvaluator(new Authorizations(auths.getBytes()));
++ this.auths = new Authorizations(auths.getBytes());
+ visibleCache = new LRUMap(100);
+ }
+ }
+
+ if (options.containsKey(MAX_BUFFER_SIZE_OPT)) {
+ maxBufferSize = AccumuloConfiguration.getMemoryInBytes(options.get(MAX_BUFFER_SIZE_OPT));
+ } else {
+ maxBufferSize = DEFAULT_MAX_BUFFER_SIZE;
+ }
+
+ parsedVisibilitiesCache = new LRUMap(100);
+ }
+
+ @Override
+ public IteratorOptions describeOptions() {
+ String desc = "This iterator allows ranges of key to be transformed (with the exception of row transformations).";
+ String authDesc = "Comma-separated list of user's scan authorizations. "
+ + "If excluded or empty, then no visibility check is performed on transformed keys.";
+ String bufferDesc = "Maximum buffer size (in accumulo memory spec) to use for buffering keys before throwing a BufferOverflowException. " +
+ "Users should keep this limit in mind when deciding what to transform. That is, if transforming the column family for example, then all " +
+ "keys sharing the same row and column family must fit within this limit (along with their associated values)";
+ HashMap<String,String> namedOptions = new HashMap<String,String>();
+ namedOptions.put(AUTH_OPT, authDesc);
+ namedOptions.put(MAX_BUFFER_SIZE_OPT, bufferDesc);
+ return new IteratorOptions(getClass().getSimpleName(), desc, namedOptions, null);
+ }
+
+ @Override
+ public boolean validateOptions(Map<String,String> options) {
+ return true;
+ }
+
+ @Override
+ public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+ TransformingIterator copy;
+
+ try {
+ copy = getClass().newInstance();
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+
+ copy.setSource(getSource().deepCopy(env));
+
+ copy.scanning = scanning;
+ copy.keyPos = keyPos;
+ copy.keys.addAll(keys);
+ copy.seekRange = (seekRange == null) ? null : new Range(seekRange);
+ copy.seekColumnFamilies = (seekColumnFamilies == null) ? null : new HashSet<ByteSequence>(seekColumnFamilies);
+ copy.seekColumnFamiliesInclusive = seekColumnFamiliesInclusive;
+
- copy.ve = ve;
++ copy.auths = auths;
+ if (visibleCache != null) {
+ copy.visibleCache = new LRUMap(visibleCache.maxSize());
+ copy.visibleCache.putAll(visibleCache);
+ }
+
+ if (parsedVisibilitiesCache != null) {
+ copy.parsedVisibilitiesCache = new LRUMap(parsedVisibilitiesCache.maxSize());
+ copy.parsedVisibilitiesCache.putAll(parsedVisibilitiesCache);
+ }
+
+ copy.maxBufferSize = maxBufferSize;
+
+ return copy;
+ }
+
+ @Override
+ public boolean hasTop() {
+ return keyPos >= 0 && keyPos < keys.size();
+ }
+
+ @Override
+ public Key getTopKey() {
+ return hasTop() ? keys.get(keyPos).getFirst() : null;
+ }
+
+ @Override
+ public Value getTopValue() {
+ return hasTop() ? keys.get(keyPos).getSecond() : null;
+ }
+
+ @Override
+ public void next() throws IOException {
+ // Move on to the next entry since we returned the entry at keyPos before
+ if (keyPos >= 0)
+ keyPos++;
+
+ // If we emptied out the transformed key map then transform the next key
+ // set from the source. It’s possible that transformation could produce keys
+ // that are outside of our range or are not visible to the end user, so after the
+ // call below we might not have added any keys to the map. Keep going until
+ // we either get some keys in the map or exhaust the source iterator.
+ while (!hasTop() && super.hasTop())
+ transformKeys();
+ }
+
+ @Override
+ public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+ seekRange = (range != null) ? new Range(range) : null;
+ seekColumnFamilies = columnFamilies;
+ seekColumnFamiliesInclusive = inclusive;
+
+ // Seek the source iterator, but use a recalculated range that ensures
+ // we see all keys with the same "prefix." We need to do this since
+ // transforming could change the sort order and transformed keys that
+ // are before the range start could be inside the range after transformation.
+ super.seek(computeReseekRange(range), untransformColumnFamilies(columnFamilies), inclusive);
+
+ // Range clipping could cause us to trim out all the keys we transformed.
+ // Keep looping until we either have some keys in the output range, or have
+ // exhausted the source iterator.
+ keyPos = -1; // “Clear” list so hasTop returns false to get us into the loop (transformKeys actually clears)
+ while (!hasTop() && super.hasTop()) {
+ // Build up a sorted list of all keys for the same prefix. When
+ // people ask for keys, return from this list first until it is empty
+ // before incrementing the source iterator.
+ transformKeys();
+ }
+ }
+
+ private static class RangeIterator implements SortedKeyValueIterator<Key,Value> {
+
+ private SortedKeyValueIterator<Key,Value> source;
+ private Key prefixKey;
+ private PartialKey keyPrefix;
+ private boolean hasTop = false;
+
+ RangeIterator(SortedKeyValueIterator<Key,Value> source, Key prefixKey, PartialKey keyPrefix) {
+ this.source = source;
+ this.prefixKey = prefixKey;
+ this.keyPrefix = keyPrefix;
+ }
+
+ @Override
+ public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean hasTop() {
+ // only have a top if the prefix matches
+ return hasTop = source.hasTop() && source.getTopKey().equals(prefixKey, keyPrefix);
+ }
+
+ @Override
+ public void next() throws IOException {
+ // do not let user advance too far and try to avoid reexecuting hasTop()
+ if (!hasTop && !hasTop())
+ throw new NoSuchElementException();
+ hasTop = false;
+ source.next();
+ }
+
+ @Override
+ public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Key getTopKey() {
+ return source.getTopKey();
+ }
+
+ @Override
+ public Value getTopValue() {
+ return source.getTopValue();
+ }
+
+ @Override
+ public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+ throw new UnsupportedOperationException();
+ }
+
+ }
+
+ /**
+ * Reads all keys matching the first key's prefix from the source iterator, transforms them, and sorts the resulting keys. Transformed keys that fall outside
+ * of our seek range or can't be seen by the user are excluded.
+ */
+ protected void transformKeys() throws IOException {
+ keyPos = -1;
+ keys.clear();
+ final Key prefixKey = super.hasTop() ? new Key(super.getTopKey()) : null;
+
+ transformRange(new RangeIterator(getSource(), prefixKey, getKeyPrefix()), new KVBuffer() {
+
+ long appened = 0;
+
+ @Override
+ public void append(Key key, Value val) {
+ // ensure the key provided by the user has the correct prefix
+ if (!key.equals(prefixKey, getKeyPrefix()))
+ throw new IllegalArgumentException("Key prefixes are not equal " + key + " " + prefixKey);
+
+ // Transformation could have produced a key that falls outside
+ // of the seek range, or one that the user cannot see. Check
+ // these before adding it to the output list.
+ if (includeTransformedKey(key)) {
+
+ // try to defend against a scan or compaction using all memory in a tablet server
+ if (appened > maxBufferSize)
+ throw new BufferOverflowException("Exceeded buffer size of " + maxBufferSize + ", prefixKey: " + prefixKey);
+
+ if (getSource().hasTop() && key == getSource().getTopKey())
+ key = new Key(key);
+ keys.add(new Pair<Key,Value>(key, new Value(val)));
+ appened += (key.getSize() + val.getSize() + 128);
+ }
+ }
+ });
+
+ // consume any key in range that user did not consume
+ while (super.hasTop() && super.getTopKey().equals(prefixKey, getKeyPrefix())) {
+ super.next();
+ }
+
+ if (!keys.isEmpty()) {
+ Collections.sort(keys, keyComparator);
+ keyPos = 0;
+ }
+ }
+
+ /**
+ * Determines whether or not to include {@code transformedKey} in the output. It is possible that transformation could have produced a key that falls outside
+ * of the seek range, a key with a visibility the user can't see, a key with a visibility that doesn't parse, or a key with a column family that wasn't
+ * fetched. We only do some checks (outside the range, user can see) if we're scanning. The range check is not done for major/minor compaction since seek
+ * ranges won't be in our transformed key space and we will never change the row so we can't produce keys that would fall outside the tablet anyway.
+ *
+ * @param transformedKey
+ * the key to check
+ * @return {@code true} if the key should be included and {@code false} if not
+ */
+ protected boolean includeTransformedKey(Key transformedKey) {
+ boolean include = canSee(transformedKey);
+ if (scanning && seekRange != null) {
+ include = include && seekRange.contains(transformedKey);
+ }
+ return include;
+ }
+
+ /**
+ * Indicates whether or not the user is able to see {@code key}. If the user has not supplied authorizations, or the iterator is not in the scan scope, then
+ * this method simply returns {@code true}. Otherwise, {@code key}'s column visibility is tested against the user-supplied authorizations, and the test result
+ * is returned. For performance, the test results are cached so that the same visibility is not tested multiple times.
+ *
+ * @param key
+ * the key to test
+ * @return {@code true} if the key is visible or iterator is not scanning, and {@code false} if not
+ */
+ protected boolean canSee(Key key) {
+ // Ensure that the visibility (which could have been transformed) parses. Must always do this check, even if visibility is not evaluated.
+ ByteSequence visibility = key.getColumnVisibilityData();
+ ColumnVisibility colVis = null;
+ Boolean parsed = (Boolean) parsedVisibilitiesCache.get(visibility);
+ if (parsed == null) {
+ try {
+ colVis = new ColumnVisibility(visibility.toArray());
+ parsedVisibilitiesCache.put(visibility, Boolean.TRUE);
+ } catch (BadArgumentException e) {
+ log.error("Parse error after transformation : " + visibility);
+ parsedVisibilitiesCache.put(visibility, Boolean.FALSE);
+ if (scanning) {
+ return false;
+ } else {
+ throw e;
+ }
+ }
+ } else if (!parsed) {
+ if (scanning)
+ return false;
+ else
+ throw new IllegalStateException();
+ }
+
+ Boolean visible = canSeeColumnFamily(key);
+
- if (!scanning || !visible || ve == null || visibleCache == null || visibility.length() == 0)
++ if (!scanning || !visible || auths == null || visibleCache == null || visibility.length() == 0)
+ return visible;
+
+ visible = (Boolean) visibleCache.get(visibility);
+ if (visible == null) {
+ try {
+ if (colVis == null)
+ colVis = new ColumnVisibility(visibility.toArray());
- visible = ve.evaluate(colVis);
++ visible = colVis.evaluate(auths);
+ visibleCache.put(visibility, visible);
- } catch (VisibilityParseException e) {
- log.error("Parse Error", e);
- visible = Boolean.FALSE;
+ } catch (BadArgumentException e) {
+ log.error("Parse Error", e);
+ visible = Boolean.FALSE;
+ }
+ }
+
+ return visible;
+ }
+
+ /**
+ * Indicates whether or not {@code key} can be seen, according to the fetched column families for this iterator.
+ *
+ * @param key
+ * the key whose column family is to be tested
+ * @return {@code true} if {@code key}'s column family is one of those fetched in the set passed to our {@link #seek(Range, Collection, boolean)} method
+ */
+ protected boolean canSeeColumnFamily(Key key) {
+ boolean visible = true;
+ if (seekColumnFamilies != null) {
+ ByteSequence columnFamily = key.getColumnFamilyData();
+ if (seekColumnFamiliesInclusive)
+ visible = seekColumnFamilies.contains(columnFamily);
+ else
+ visible = !seekColumnFamilies.contains(columnFamily);
+ }
+ return visible;
+ }
+
+ /**
+ * Possibly expand {@code range} to include everything for the key prefix we are working with. That is, if our prefix is ROW_COLFAM, then we need to expand
+ * the range so we're sure to include all entries having the same row and column family as the start/end of the range.
+ *
+ * @param range
+ * the range to expand
+ * @return the modified range
+ */
+ protected Range computeReseekRange(Range range) {
+ Key startKey = range.getStartKey();
+ boolean startKeyInclusive = range.isStartKeyInclusive();
+ // If anything after the prefix is set, then clip the key so we include
+ // everything for the prefix.
+ if (isSetAfterPart(startKey, getKeyPrefix())) {
+ startKey = copyPartialKey(startKey, getKeyPrefix());
+ startKeyInclusive = true;
+ }
+ Key endKey = range.getEndKey();
+ boolean endKeyInclusive = range.isEndKeyInclusive();
+ if (isSetAfterPart(endKey, getKeyPrefix())) {
+ endKey = endKey.followingKey(getKeyPrefix());
+ endKeyInclusive = true;
+ }
+ return new Range(startKey, startKeyInclusive, endKey, endKeyInclusive);
+ }
+
+ /**
+ * Indicates whether or not any part of {@code key} excluding {@code part} is set. For example, if part is ROW_COLFAM_COLQUAL, then this method determines
+ * whether or not the column visibility, timestamp, or delete flag is set on {@code key}.
+ *
+ * @param key
+ * the key to check
+ * @param part
+ * the part of the key that doesn't need to be checked (everything after does)
+ * @return {@code true} if anything after {@code part} is set on {@code key}, and {@code false} if not
+ */
+ protected boolean isSetAfterPart(Key key, PartialKey part) {
+ boolean isSet = false;
+ if (key != null) {
+ // Breaks excluded on purpose.
+ switch (part) {
+ case ROW:
+ isSet = isSet || key.getColumnFamilyData().length() > 0;
+ case ROW_COLFAM:
+ isSet = isSet || key.getColumnQualifierData().length() > 0;
+ case ROW_COLFAM_COLQUAL:
+ isSet = isSet || key.getColumnVisibilityData().length() > 0;
+ case ROW_COLFAM_COLQUAL_COLVIS:
+ isSet = isSet || key.getTimestamp() < Long.MAX_VALUE;
+ case ROW_COLFAM_COLQUAL_COLVIS_TIME:
+ isSet = isSet || key.isDeleted();
+ case ROW_COLFAM_COLQUAL_COLVIS_TIME_DEL:
+ break;
+ }
+ }
+ return isSet;
+ }
+
+ /**
+ * Creates a copy of {@code key}, copying only the parts of the key specified in {@code part}. For example, if {@code part} is ROW_COLFAM_COLQUAL, then this
+ * method would copy the row, column family, and column qualifier from {@code key} into a new key.
+ *
+ * @param key
+ * the key to copy
+ * @param part
+ * the parts of {@code key} to copy
+ * @return the new key containing {@code part} of {@code key}
+ */
+ protected Key copyPartialKey(Key key, PartialKey part) {
+ Key keyCopy;
+ switch (part) {
+ case ROW:
+ keyCopy = new Key(key.getRow());
+ break;
+ case ROW_COLFAM:
+ keyCopy = new Key(key.getRow(), key.getColumnFamily());
+ break;
+ case ROW_COLFAM_COLQUAL:
+ keyCopy = new Key(key.getRow(), key.getColumnFamily(), key.getColumnQualifier());
+ break;
+ case ROW_COLFAM_COLQUAL_COLVIS:
+ keyCopy = new Key(key.getRow(), key.getColumnFamily(), key.getColumnQualifier(), key.getColumnVisibility());
+ break;
+ case ROW_COLFAM_COLQUAL_COLVIS_TIME:
+ keyCopy = new Key(key.getRow(), key.getColumnFamily(), key.getColumnQualifier(), key.getColumnVisibility(), key.getTimestamp());
+ break;
+ default:
+ throw new IllegalArgumentException("Unsupported key part: " + part);
+ }
+ return keyCopy;
+ }
+
+ /**
+ * Make a new key with all parts (including delete flag) coming from {@code originalKey} but use {@code newColFam} as the column family.
+ */
+ protected Key replaceColumnFamily(Key originalKey, Text newColFam) {
+ byte[] row = originalKey.getRowData().toArray();
+ byte[] cf = newColFam.getBytes();
+ byte[] cq = originalKey.getColumnQualifierData().toArray();
+ byte[] cv = originalKey.getColumnVisibilityData().toArray();
+ long timestamp = originalKey.getTimestamp();
+ Key newKey = new Key(row, 0, row.length, cf, 0, newColFam.getLength(), cq, 0, cq.length, cv, 0, cv.length, timestamp);
+ newKey.setDeleted(originalKey.isDeleted());
+ return newKey;
+ }
+
+ /**
+ * Make a new key with all parts (including delete flag) coming from {@code originalKey} but use {@code newColQual} as the column qualifier.
+ */
+ protected Key replaceColumnQualifier(Key originalKey, Text newColQual) {
+ byte[] row = originalKey.getRowData().toArray();
+ byte[] cf = originalKey.getColumnFamilyData().toArray();
+ byte[] cq = newColQual.getBytes();
+ byte[] cv = originalKey.getColumnVisibilityData().toArray();
+ long timestamp = originalKey.getTimestamp();
+ Key newKey = new Key(row, 0, row.length, cf, 0, cf.length, cq, 0, newColQual.getLength(), cv, 0, cv.length, timestamp);
+ newKey.setDeleted(originalKey.isDeleted());
+ return newKey;
+ }
+
+ /**
+ * Make a new key with all parts (including delete flag) coming from {@code originalKey} but use {@code newColVis} as the column visibility.
+ */
+ protected Key replaceColumnVisibility(Key originalKey, Text newColVis) {
+ byte[] row = originalKey.getRowData().toArray();
+ byte[] cf = originalKey.getColumnFamilyData().toArray();
+ byte[] cq = originalKey.getColumnQualifierData().toArray();
+ byte[] cv = newColVis.getBytes();
+ long timestamp = originalKey.getTimestamp();
+ Key newKey = new Key(row, 0, row.length, cf, 0, cf.length, cq, 0, cq.length, cv, 0, newColVis.getLength(), timestamp);
+ newKey.setDeleted(originalKey.isDeleted());
+ return newKey;
+ }
+
+ /**
+ * Make a new key with a column family, column qualifier, and column visibility. Copy the rest of the parts of the key (including delete flag) from
+ * {@code originalKey}.
+ */
+ protected Key replaceKeyParts(Key originalKey, Text newColFam, Text newColQual, Text newColVis) {
+ byte[] row = originalKey.getRowData().toArray();
+ byte[] cf = newColFam.getBytes();
+ byte[] cq = newColQual.getBytes();
+ byte[] cv = newColVis.getBytes();
+ long timestamp = originalKey.getTimestamp();
+ Key newKey = new Key(row, 0, row.length, cf, 0, newColFam.getLength(), cq, 0, newColQual.getLength(), cv, 0, newColVis.getLength(), timestamp);
+ newKey.setDeleted(originalKey.isDeleted());
+ return newKey;
+ }
+
+ /**
+ * Make a new key with a column qualifier, and column visibility. Copy the rest of the parts of the key (including delete flag) from {@code originalKey}.
+ */
+ protected Key replaceKeyParts(Key originalKey, Text newColQual, Text newColVis) {
+ byte[] row = originalKey.getRowData().toArray();
+ byte[] cf = originalKey.getColumnFamilyData().toArray();
+ byte[] cq = newColQual.getBytes();
+ byte[] cv = newColVis.getBytes();
+ long timestamp = originalKey.getTimestamp();
+ Key newKey = new Key(row, 0, row.length, cf, 0, cf.length, cq, 0, newColQual.getLength(), cv, 0, newColVis.getLength(), timestamp);
+ newKey.setDeleted(originalKey.isDeleted());
+ return newKey;
+ }
+
+ /**
+ * Reverses the transformation applied to column families that are fetched at seek time. If this iterator is transforming column families, then this method
+ * should be overridden to reverse the transformation on the supplied collection of column families. This is necessary since the fetch/seek will be performed
+ * in the transformed space, but when passing the column family set on to the source, the column families need to be in the untransformed space.
+ *
+ * @param columnFamilies
+ * the column families that have been fetched at seek time
+ * @return the untransformed column families that would transform info {@code columnFamilies}
+ */
+ protected Collection<ByteSequence> untransformColumnFamilies(Collection<ByteSequence> columnFamilies) {
+ return columnFamilies;
+ }
+
+ /**
+ * Indicates the prefix of keys that will be transformed by this iterator. In other words, this is the part of the key that will <i>not</i> be transformed by
+ * this iterator. For example, if this method returns ROW_COLFAM, then {@link #transformKeys()} may be changing the column qualifier, column visibility, or
+ * timestamp, but it won't be changing the row or column family.
+ *
+ * @return the part of the key this iterator is not transforming
+ */
+ abstract protected PartialKey getKeyPrefix();
+
+ public static interface KVBuffer {
+ void append(Key key, Value val);
+ }
+
+ /**
+ * Transforms {@code input}. This method must not change the row part of the key, and must only change the parts of the key after the return value of
+ * {@link #getKeyPrefix()}. Implementors must also remember to copy the delete flag from {@code originalKey} onto the new key. Or, implementors should use one
+ * of the helper methods to produce the new key. See any of the replaceKeyParts methods.
+ *
+ * @param input
+ * An iterator over a group of keys with the same prefix. This iterator provides an efficient view, bounded by the prefix, of the underlying iterator
+ * and can not be seeked.
+ * @param output
+ * An output buffer that holds transformed key values. All key values added to the buffer must have the same prefix as the input keys.
+ * @throws IOException
+ * @see #replaceColumnFamily(Key, Text)
+ * @see #replaceColumnQualifier(Key, Text)
+ * @see #replaceColumnVisibility(Key, Text)
+ * @see #replaceKeyParts(Key, Text, Text)
+ * @see #replaceKeyParts(Key, Text, Text, Text)
+ */
+ abstract protected void transformRange(SortedKeyValueIterator<Key,Value> input, KVBuffer output) throws IOException;
+
+ /**
+ * Configure authoriations used for post transformation filtering.
+ *
+ * @param config
+ * @param auths
+ */
+ public static void setAutorizations(IteratorSetting config, Authorizations auths) {
+ config.addOption(AUTH_OPT, auths.serialize());
+ }
+
+ /**
+ * Configure the maximum amount of memory that can be used for transformation. If this memory is exceeded an exception will be thrown.
+ *
+ * @param config
+ * @param maxBufferSize
+ * size in bytes
+ */
+ public static void setMaxBufferSize(IteratorSetting config, long maxBufferSize) {
+ config.addOption(MAX_BUFFER_SIZE_OPT, maxBufferSize + "");
+ }
+
+ }
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java
index 1b72b33,ad7fdb2..d629cef
--- a/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java
+++ b/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java
@@@ -16,9 -16,12 +16,10 @@@
*/
package org.apache.accumulo.core.security;
+import java.io.ByteArrayOutputStream;
+ import java.io.UnsupportedEncodingException;
-import java.util.ArrayList;
import java.util.Arrays;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
+import java.util.Iterator;
import java.util.TreeSet;
import org.apache.accumulo.core.data.ArrayByteSequence;
@@@ -38,62 -52,59 +39,63 @@@ public class ColumnVisibility
public static enum NodeType {
TERM, OR, AND,
}
-
- public static class Node {
- public final static List<Node> EMPTY = Collections.emptyList();
- NodeType type;
- int start = 0;
- int end = 0;
- List<Node> children = EMPTY;
+
+ private static abstract class Node implements Comparable<Node> {
+ protected final NodeType type;
- public Node(NodeType type) {
+ public Node(NodeType type)
+ {
this.type = type;
}
-
- public Node(int start, int end) {
- this.type = NodeType.TERM;
- this.start = start;
- this.end = end;
+
+ public byte[] generate() {
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ generate(baos,false);
+ return baos.toByteArray();
}
- public void add(Node child) {
- if (children == EMPTY)
- children = new ArrayList<Node>();
-
- children.add(child);
- }
+ public abstract boolean evaluate(Authorizations auths);
- public NodeType getType() {
- return type;
- }
+ protected abstract void generate(ByteArrayOutputStream baos, boolean parens);
+ }
+
+ private static class TermNode extends Node {
- public List<Node> getChildren() {
- return children;
+ final ByteSequence bs;
+
+ public TermNode(final ByteSequence bs) {
+ super(NodeType.TERM);
+ this.bs = bs;
}
- public int getTermStart() {
- return start;
+ public boolean evaluate(Authorizations auths)
+ {
+ return auths.contains(bs);
+ }
+
+
+ protected void generate(ByteArrayOutputStream baos, boolean parens)
+ {
- baos.write(bs.getBackingArray(), bs.offset(), bs.length());
++ byte [] quoted = quote(bs.toArray());
++ baos.write(quoted, 0, quoted.length);
}
- public int getTermEnd() {
- return end;
+ @Override
+ public boolean equals(Object other) {
+ if(other instanceof TermNode)
+ {
+ return bs.compareTo(((TermNode)other).bs) == 0;
+ }
+ return false;
}
- public ByteSequence getTerm(byte expression[]) {
- if (type != NodeType.TERM)
- throw new RuntimeException();
-
- if (expression[start] == '"') {
- // its a quoted term
- int qStart = start + 1;
- int qEnd = end - 1;
-
- return new ArrayByteSequence(expression, qStart, qEnd - qStart);
+ @Override
+ public int compareTo(Node o) {
+ if(o.type == NodeType.TERM)
+ {
+ return bs.compareTo(((TermNode)o).bs);
}
- return new ArrayByteSequence(expression, start, end - start);
+ return type.ordinal() - o.type.ordinal();
}
}
@@@ -144,81 -137,79 +146,87 @@@
}
return 0;
}
+
}
-
+
- /* Convience method that delegates to normalize with a new
- * NodeComparator constructed using the supplied expression.
- */
- private static Node normalize(Node root, byte[] expression) {
- return normalize(root, expression, new NodeComparator(expression));
- }
+ private static class OrNode extends AggregateNode {
- /* 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`)
- */
- private static Node normalize(Node root, byte[] expression, NodeComparator comparator) {
- if(root.type != NodeType.TERM) {
- TreeSet<Node> rolledUp = new TreeSet<Node>(comparator);
- java.util.Iterator<Node> itr = root.children.iterator();
- while(itr.hasNext()) {
- 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);
- }
+ public OrNode() {
+ super(NodeType.OR);
}
- return root;
- }
-
- /* Walks an expression's AST and appends a string representation to a supplied
- * StringBuilder. This method adds parens where necessary.
- */
- private static void stringify(Node root, byte[] expression, StringBuilder out) {
- if (root.type == NodeType.TERM) {
- out.append(new String(expression, root.start, root.end - root.start));
+ @Override
+ public boolean evaluate(Authorizations auths) {
+ for(Node child:children)
+ if(child.evaluate(auths))
+ return true;
+ return false;
}
- else {
- String sep = "";
- for (Node c : root.children) {
- out.append(sep);
- 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 ? "&" : "|";
- }
+
+ @Override
+ protected byte getOperator() {
+ return '|';
}
+
}
-
+
+ /**
+ * Generates a byte[] that represents a normalized, but logically equivalent,
+ * form of the supplied expression.
+ *
+ * @return normalized expression in byte[] form
+ */
+ private static class AndNode extends AggregateNode {
+
+ public AndNode()
+ {
+ super(NodeType.AND);
+ }
+
+ @Override
+ public boolean evaluate(Authorizations auths) {
+ for(Node child:children)
+ if(!child.evaluate(auths))
+ return false;
+ return true;
+ }
+
+ @Override
+ protected byte getOperator() {
+ return '&';
+ }
+
+ }
+
+ private byte[] expression = null;
+
+ /**
+ * @deprecated
+ * @see org.apache.accumulo.security.ColumnVisibility#getExpression()
+ */
public byte[] flatten() {
- Node normRoot = normalize(node, expression);
- StringBuilder builder = new StringBuilder(expression.length);
- stringify(normRoot, expression, builder);
- return builder.toString().getBytes();
+ return getExpression();
- }
+ }
+ /**
+ * Generate the byte[] that represents this ColumnVisibility.
+ * @return a byte[] representation of this visibility
+ */
+ public byte[] getExpression(){
+ if(expression != null)
+ return expression;
+ expression = _flatten();
+ return expression;
+ }
+
+ private static final byte[] emptyExpression = new byte[0];
+
+ private byte[] _flatten() {
+ if(node == null)
+ return emptyExpression;
+ return node.generate();
+ }
+
private static class ColumnVisibilityParser {
private int index = 0;
private int parens = 0;
@@@ -239,11 -230,11 +247,13 @@@
return null;
}
-- Node processTerm(int start, int end, Node expr, byte[] expression) {
++ Node processTerm(int start, int end, Node expr, byte[] expression, boolean quoted) {
if (start != end) {
if (expr != null)
throw new BadArgumentException("expression needs | or &", new String(expression), start);
- return new Node(start, end);
++ if(quoted)
++ return new TermNode(unquote(expression, start, end - start));
+ return new TermNode(new ArrayByteSequence(expression, start, end - start));
}
if (expr == null)
throw new BadArgumentException("empty term", new String(expression), start);
@@@ -254,32 -245,36 +264,45 @@@
Node result = null;
Node expr = null;
int termStart = index;
++ boolean quoted = false;
+ boolean termComplete = false;
+
while (index < expression.length) {
switch (expression[index++]) {
case '&': {
-- expr = processTerm(termStart, index - 1, expr, expression);
++ expr = processTerm(termStart, index - 1, expr, expression, quoted);
if (result != null) {
if (!result.type.equals(NodeType.AND))
throw new BadArgumentException("cannot mix & and |", new String(expression), index - 1);
} else {
- result = new Node(NodeType.AND);
+ result = new AndNode();
}
- ((AggregateNode)result).children.add(expr);
- result.add(expr);
++ if(expr.type == NodeType.AND)
++ ((AggregateNode)result).children.addAll(((AggregateNode)expr).children);
++ else
++ ((AggregateNode)result).children.add(expr);
expr = null;
termStart = index;
+ termComplete = false;
++ quoted = false;
break;
}
case '|': {
-- expr = processTerm(termStart, index - 1, expr, expression);
++ expr = processTerm(termStart, index - 1, expr, expression, quoted);
if (result != null) {
if (!result.type.equals(NodeType.OR))
throw new BadArgumentException("cannot mix | and &", new String(expression), index - 1);
} else {
- result = new Node(NodeType.OR);
+ result = new OrNode();
}
- ((AggregateNode)result).children.add(expr);
- result.add(expr);
++ if(expr.type == NodeType.OR)
++ ((AggregateNode)result).children.addAll(((AggregateNode)expr).children);
++ else
++ ((AggregateNode)result).children.add(expr);
expr = null;
termStart = index;
+ termComplete = false;
++ quoted = false;
break;
}
case '(': {
@@@ -288,50 -283,62 +311,81 @@@
throw new BadArgumentException("expression needs & or |", new String(expression), index - 1);
expr = parse_(expression);
termStart = index;
+ termComplete = false;
++ quoted = false;
break;
}
case ')': {
parens--;
-- Node child = processTerm(termStart, index - 1, expr, expression);
++ Node child = processTerm(termStart, index - 1, expr, expression, quoted);
if (child == null && result == null)
throw new BadArgumentException("empty expression not allowed", new String(expression), index);
if (result == null)
return child;
if (result.type == child.type)
- for (Node c : child.children)
- result.add(c);
+ {
+ AggregateNode parenNode = (AggregateNode)child;
+ for (Node c : parenNode.children)
+ ((AggregateNode)result).children.add(c);
+ }
else
- result.add(child);
- result.end = index - 1;
+ ((AggregateNode)result).children.add(child);
+ if (result.type != NodeType.TERM)
+ {
+ AggregateNode resultNode = (AggregateNode)result;
+ if (resultNode.children.size() == 1)
+ return resultNode.children.first();
+ if (resultNode.children.size() < 2)
+ throw new BadArgumentException("missing term", new String(expression), index);
+ }
return result;
}
+ case '"': {
+ if (termStart != index - 1)
+ throw new BadArgumentException("expression needs & or |", new String(expression), 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), index - 1);
+ }
+ index++;
+ }
+
+ if (index == expression.length)
+ throw new BadArgumentException("unclosed quote", new String(expression), termStart);
+
+ if (termStart + 1 == index)
+ throw new BadArgumentException("empty term", new String(expression), termStart);
+
+ index++;
+
++ quoted = true;
+ termComplete = true;
+
+ break;
+ }
default: {
+ if (termComplete)
+ throw new BadArgumentException("expression needs & or |", new String(expression), index - 1);
+
byte c = expression[index - 1];
if (!Authorizations.isValidAuthChar(c))
throw new BadArgumentException("bad character (" + c + ")", new String(expression), index - 1);
}
}
}
-- Node child = processTerm(termStart, index, expr, expression);
++ Node child = processTerm(termStart, index, expr, expression, quoted);
if (result != null)
- result.add(child);
+ {
+ if(result.type == child.type)
+ {
+ ((AggregateNode)result).children.addAll(((AggregateNode)child).children);
+ }
+ else
+ ((AggregateNode)result).children.add(child);
+ }
else
result = child;
if (result.type != NodeType.TERM)
@@@ -432,49 -454,62 +508,137 @@@
@Override
public int hashCode() {
- return Arrays.hashCode(expression);
+ return Arrays.hashCode(getExpression());
}
- public Node getParseTree() {
- return node;
+ public boolean evaluate(Authorizations auths) {
+ if(node == null)
+ return true;
+ return node.evaluate(auths);
}
+ public ColumnVisibility or(ColumnVisibility other)
+ {
+ if(node == null)
+ return this;
+ if(other.node == null)
+ return other;
+ OrNode orNode = new OrNode();
+ if(other.node instanceof OrNode)
+ orNode.children.addAll(((OrNode)other.node).children);
+ else
+ orNode.children.add(other.node);
+ if(node instanceof OrNode)
+ orNode.children.addAll(((OrNode)node).children);
+ else
+ orNode.children.add(node);
+ return new ColumnVisibility(orNode);
+ }
+
+ public ColumnVisibility and(ColumnVisibility other)
+ {
+ if(node == null)
+ return other;
+ if(other.node == null)
+ return this;
+ AndNode andNode = new AndNode();
+ if(other.node instanceof AndNode)
+ andNode.children.addAll(((AndNode)other.node).children);
+ else
+ andNode.children.add(other.node);
+ if(node instanceof AndNode)
+ andNode.children.addAll(((AndNode)node).children);
+ else
+ andNode.children.add(node);
+ return new ColumnVisibility(andNode);
+ }
+
+ /**
+ * see {@link #quote(byte[])}
+ *
+ */
+ public static String quote(String term) {
+ return quote(term, "UTF-8");
+ }
+
+ /**
+ * see {@link #quote(byte[])}
+ *
+ */
+ public static String quote(String term, String encoding) {
+ try {
+ return new String(quote(term.getBytes(encoding)), encoding);
+ } catch (UnsupportedEncodingException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ /**
+ * Use to properly quote terms in a column visibility expression. If no quoting is needed, then nothing is done.
+ *
+ * <p>
+ * Examples of using quote :
+ *
+ * <pre>
+ * import static org.apache.accumulo.core.security.ColumnVisibility.quote;
+ * .
+ * .
+ * .
+ * ColumnVisibility cv = new ColumnVisibility(quote("A#C") + "&" + quote("FOO"));
+ * </pre>
+ *
+ */
+
+ public static byte[] quote(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;
+
- return VisibilityEvaluator.escape(term, true);
++ return escape(term, true);
+ }
++
++ private static byte[] escape(byte[] auth, boolean quote) {
++ int escapeCount = 0;
++
++ for (int i = 0; i < auth.length; i++)
++ if (auth[i] == '"' || auth[i] == '\\')
++ escapeCount++;
++
++ if (escapeCount > 0 || quote) {
++ byte[] escapedAuth = new byte[auth.length + escapeCount + (quote ? 2 : 0)];
++ int index = quote ? 1 : 0;
++ for (int i = 0; i < auth.length; i++) {
++ if (auth[i] == '"' || auth[i] == '\\')
++ escapedAuth[index++] = '\\';
++ escapedAuth[index++] = auth[i];
++ }
++
++ if (quote) {
++ escapedAuth[0] = '"';
++ escapedAuth[escapedAuth.length - 1] = '"';
++ }
++
++ auth = escapedAuth;
++ }
++ return auth;
++ }
++
++ private static ByteSequence unquote(byte[] expression, int start, int length) {
++ ByteArrayOutputStream baos = new ByteArrayOutputStream();
++ for(int i = start+1; i < start+length-1; i++) {
++ if(expression[i] == '\\')
++ i++;
++ baos.write(expression[i]);
++ }
++ return new ArrayByteSequence(baos.toByteArray());
++ }
++
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/core/src/test/java/org/apache/accumulo/core/file/rfile/AuthorizationFilterTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/file/rfile/AuthorizationFilterTest.java
index 080b6fb,0000000..d99cfa6
mode 100644,000000..100644
--- a/core/src/test/java/org/apache/accumulo/core/file/rfile/AuthorizationFilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/AuthorizationFilterTest.java
@@@ -1,110 -1,0 +1,108 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.file.rfile;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.util.Collections;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.TreeMap;
+
- import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
+import org.apache.accumulo.core.file.rfile.RFileTest.SeekableByteArrayInputStream;
+import org.apache.accumulo.core.iterators.Predicate;
+import org.apache.accumulo.core.iterators.predicates.ColumnVisibilityPredicate;
- import org.apache.accumulo.core.iterators.predicates.TimestampRangePredicate;
- import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator;
+import org.apache.accumulo.core.iterators.system.VisibilityFilter;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.junit.Test;
+
+public class AuthorizationFilterTest {
+
++ @SuppressWarnings("unchecked")
+ @Test
+ public void testRFileAuthorizationFiltering() throws Exception {
+ Authorizations auths = new Authorizations("a", "b", "c");
+ Predicate<Key,Value> columnVisibilityPredicate = new ColumnVisibilityPredicate(auths);
+ int expected = 0;
+ Random r = new Random();
+ Configuration conf = new Configuration();
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ FSDataOutputStream dos = new FSDataOutputStream(baos, new FileSystem.Statistics("a"));
+ CachableBlockFile.Writer _cbw = new CachableBlockFile.Writer(dos, "gz", conf);
+ RFile.Writer writer = new RFile.Writer(_cbw, 1000, 1000);
+ writer.startDefaultLocalityGroup();
+ byte[] row = new byte[10];
+ byte[] colFam = new byte[10];
+ byte[] colQual = new byte[10];
+ Value value = new Value(new byte[0]);
+ TreeMap<Key,Value> inputBuffer = new TreeMap<Key,Value>();
+ ColumnVisibility[] goodColVises = {new ColumnVisibility("a&b"), new ColumnVisibility("b&c"), new ColumnVisibility("a&c")};
+ ColumnVisibility[] badColVises = {new ColumnVisibility("x"), new ColumnVisibility("y"), new ColumnVisibility("a&z")};
+ for (ColumnVisibility colVis : goodColVises)
+ for (int i = 0; i < 10; i++) {
+ r.nextBytes(row);
+ r.nextBytes(colFam);
+ r.nextBytes(colQual);
+ Key k = new Key(row, colFam, colQual, colVis.getExpression(), (long) i);
+ if (columnVisibilityPredicate.evaluate(k, value))
+ expected++;
+ inputBuffer.put(k, value);
+ }
+ for (ColumnVisibility colVis : badColVises)
+ for (int i = 0; i < 10000; i++) {
+ r.nextBytes(row);
+ r.nextBytes(colFam);
+ r.nextBytes(colQual);
+ Key k = new Key(row, colFam, colQual, colVis.getExpression(), (long) i);
+ if (columnVisibilityPredicate.evaluate(k, value))
+ expected++;
+ inputBuffer.put(k, value);
+ }
+ for (Entry<Key,Value> e : inputBuffer.entrySet()) {
+ writer.append(e.getKey(), e.getValue());
+ }
+ writer.close();
+
+ // scan the RFile to bring back keys in a given timestamp range
+ byte[] data = baos.toByteArray();
+
+ ByteArrayInputStream bais = new SeekableByteArrayInputStream(data);
+ FSDataInputStream in = new FSDataInputStream(bais);
+ CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader(in, data.length, conf);
+ RFile.Reader reader = new RFile.Reader(_cbr);
+ int count = 0;
+ VisibilityFilter vf = new VisibilityFilter(reader, auths, new byte[0]);
+ vf.seek(new Range(), Collections.EMPTY_SET, false);
+ while (vf.hasTop()) {
+ count++;
+ assertTrue(columnVisibilityPredicate.evaluate(vf.getTopKey(), vf.getTopValue()));
+ vf.next();
+ }
+ assertEquals(expected, count);
+ }
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/core/src/test/java/org/apache/accumulo/core/file/rfile/BlockIndexTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/file/rfile/BlockIndexTest.java
index 0000000,1684feb..de970fd
mode 000000,100644..100644
--- a/core/src/test/java/org/apache/accumulo/core/file/rfile/BlockIndexTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/BlockIndexTest.java
@@@ -1,0 -1,175 +1,176 @@@
+ /**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.accumulo.core.file.rfile;
+
+ import java.io.ByteArrayOutputStream;
+ import java.io.DataOutputStream;
+ import java.io.IOException;
+
+ import org.apache.accumulo.core.data.Key;
+ import org.apache.accumulo.core.data.Value;
+ import org.apache.accumulo.core.file.blockfile.ABlockReader;
+ import org.apache.accumulo.core.file.blockfile.cache.CacheEntry;
+ import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
+ import org.apache.accumulo.core.file.rfile.BlockIndex.BlockIndexEntry;
+ import org.apache.accumulo.core.file.rfile.MultiLevelIndex.IndexEntry;
++import org.apache.accumulo.core.security.ColumnVisibility;
+ import org.junit.Assert;
+ import org.junit.Test;
+
+ /**
+ *
+ */
+ public class BlockIndexTest {
+
+ private static class MyCacheEntry implements CacheEntry {
+ Object idx;
+ byte[] data;
+
+ MyCacheEntry(byte[] d) {
+ this.data = d;
+ }
+
+ @Override
+ public void setIndex(Object idx) {
+ this.idx = idx;
+ }
+
+ @Override
+ public Object getIndex() {
+ return idx;
+ }
+
+ @Override
+ public byte[] getBuffer() {
+ return data;
+ }
+ }
+
+ @Test
+ public void test1() throws IOException {
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ DataOutputStream out = new DataOutputStream(baos);
+
+ Key prevKey = null;
+
+ int num = 1000;
+
+ for (int i = 0; i < num; i++) {
+ Key key = new Key(RFileTest.nf("", i), "cf1", "cq1");
+ new RelativeKey(prevKey, key).write(out);
+ new Value(new byte[0]).write(out);
+ prevKey = key;
+ }
+
+ out.close();
+ final byte[] data = baos.toByteArray();
+
+ CacheEntry ce = new MyCacheEntry(data);
+
+ ABlockReader cacheBlock = new CachableBlockFile.CachedBlockRead(ce, data);
+ BlockIndex blockIndex = null;
+
+ for (int i = 0; i < 129; i++)
- blockIndex = BlockIndex.getIndex(cacheBlock, new IndexEntry(prevKey, num, 0, 0, 0));
++ blockIndex = BlockIndex.getIndex(cacheBlock, new IndexEntry(prevKey, new BlockStats(Long.MAX_VALUE, Long.MAX_VALUE, new ColumnVisibility(), num), 0, 0, 0, RFile.RINDEX_VER_7));
+
+ BlockIndexEntry[] indexEntries = blockIndex.getIndexEntries();
+
+ for (int i = 0; i < indexEntries.length; i++) {
+ int row = Integer.parseInt(indexEntries[i].getPrevKey().getRowData().toString());
+
+ BlockIndexEntry bie;
+
+
+ bie = blockIndex.seekBlock(new Key(RFileTest.nf("", row), "cf1", "cq1"), cacheBlock);
+ if (i == 0)
+ Assert.assertSame(null, bie);
+ else
+ Assert.assertSame(indexEntries[i - 1], bie);
+
+ Assert.assertSame(bie, blockIndex.seekBlock(new Key(RFileTest.nf("", row - 1), "cf1", "cq1"), cacheBlock));
+
+ bie = blockIndex.seekBlock(new Key(RFileTest.nf("", row + 1), "cf1", "cq1"), cacheBlock);
+ Assert.assertSame(indexEntries[i], bie);
+
+ RelativeKey rk = new RelativeKey();
+ rk.setPrevKey(bie.getPrevKey());
+ rk.readFields(cacheBlock);
+
+ Assert.assertEquals(rk.getKey(), new Key(RFileTest.nf("", row + 1), "cf1", "cq1"));
+
+ }
+ }
+
+ @Test
+ public void testSame() throws IOException {
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ DataOutputStream out = new DataOutputStream(baos);
+
+ Key prevKey = null;
+
+ int num = 1000;
+
+ for (int i = 0; i < num; i++) {
+ Key key = new Key(RFileTest.nf("", 1), "cf1", "cq1");
+ new RelativeKey(prevKey, key).write(out);
+ new Value(new byte[0]).write(out);
+ prevKey = key;
+ }
+
+ for (int i = 0; i < num; i++) {
+ Key key = new Key(RFileTest.nf("", 3), "cf1", "cq1");
+ new RelativeKey(prevKey, key).write(out);
+ new Value(new byte[0]).write(out);
+ prevKey = key;
+ }
+
+ for (int i = 0; i < num; i++) {
+ Key key = new Key(RFileTest.nf("", 5), "cf1", "cq1");
+ new RelativeKey(prevKey, key).write(out);
+ new Value(new byte[0]).write(out);
+ prevKey = key;
+ }
+
+ out.close();
+ final byte[] data = baos.toByteArray();
+
+ CacheEntry ce = new MyCacheEntry(data);
+
+ ABlockReader cacheBlock = new CachableBlockFile.CachedBlockRead(ce, data);
+ BlockIndex blockIndex = null;
+
+ for (int i = 0; i < 257; i++)
- blockIndex = BlockIndex.getIndex(cacheBlock, new IndexEntry(prevKey, num, 0, 0, 0));
++ blockIndex = BlockIndex.getIndex(cacheBlock, new IndexEntry(prevKey, new BlockStats(Long.MAX_VALUE, Long.MAX_VALUE, new ColumnVisibility(), num), 0, 0, 0, RFile.RINDEX_VER_7));
+
+ Assert.assertSame(null, blockIndex.seekBlock(new Key(RFileTest.nf("", 0), "cf1", "cq1"), cacheBlock));
+ Assert.assertSame(null, blockIndex.seekBlock(new Key(RFileTest.nf("", 1), "cf1", "cq1"), cacheBlock));
+
+ for (int i = 2; i < 6; i++) {
+ Key seekKey = new Key(RFileTest.nf("", i), "cf1", "cq1");
+ BlockIndexEntry bie = blockIndex.seekBlock(seekKey, cacheBlock);
+
+ Assert.assertTrue(bie.getPrevKey().compareTo(seekKey) < 0);
+
+ RelativeKey rk = new RelativeKey();
+ rk.setPrevKey(bie.getPrevKey());
+ rk.readFields(cacheBlock);
+
+ Assert.assertTrue(rk.getKey().compareTo(seekKey) <= 0);
+ }
+
+ }
+ }
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java
index 25bea00,00405d1..a248bd7
--- a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java
@@@ -37,13 -36,9 +37,13 @@@ import org.apache.accumulo.core.util.Ca
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
+import org.junit.Test;
+
- import static junit.framework.Assert.*;
++import static org.junit.Assert.*;
-public class MultiLevelIndexTest extends TestCase {
+public class MultiLevelIndexTest {
+ @Test
public void test1() throws Exception {
runTest(500, 1);
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
index 71f5c6c,a23b6cc..6620649
--- a/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java
@@@ -16,6 -16,10 +16,8 @@@
*/
package org.apache.accumulo.core.file.rfile;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
++import static org.junit.Assert.*;
+
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/core/src/test/java/org/apache/accumulo/core/file/rfile/TimestampFilterTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/file/rfile/TimestampFilterTest.java
index f72a42b,0000000..731b225
mode 100644,000000..100644
--- a/core/src/test/java/org/apache/accumulo/core/file/rfile/TimestampFilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/TimestampFilterTest.java
@@@ -1,99 -1,0 +1,97 @@@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.file.rfile;
+
+import static org.junit.Assert.*;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.util.Collections;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.TreeMap;
+
- import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile;
+import org.apache.accumulo.core.file.rfile.RFileTest.SeekableByteArrayInputStream;
+import org.apache.accumulo.core.iterators.Predicate;
+import org.apache.accumulo.core.iterators.predicates.TimestampRangePredicate;
- import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator;
+import org.apache.accumulo.core.iterators.system.GenericFilterer;
- import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.junit.Test;
+
+public class TimestampFilterTest {
+
++ @SuppressWarnings("unchecked")
+ @Test
+ public void testRFileTimestampFiltering() throws Exception {
+ Predicate<Key,Value> timeRange = new TimestampRangePredicate(73, 117);
+ int expected = 0;
+ Random r = new Random();
+ Configuration conf = new Configuration();
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ FSDataOutputStream dos = new FSDataOutputStream(baos, new FileSystem.Statistics("a"));
+ CachableBlockFile.Writer _cbw = new CachableBlockFile.Writer(dos, "gz", conf);
+ RFile.Writer writer = new RFile.Writer(_cbw, 1000, 1000);
+ writer.startDefaultLocalityGroup();
+ byte [] row = new byte[10];
+ byte [] colFam = new byte[10];
+ byte [] colQual = new byte[10];
+ Value value = new Value(new byte[0]);
+ byte [] colVis = new byte[0];
+ TreeMap<Key,Value> inputBuffer = new TreeMap<Key,Value>();
+ for(int i = 0; i < 100000; i++)
+ {
+ r.nextBytes(row);
+ r.nextBytes(colFam);
+ r.nextBytes(colQual);
+ Key k = new Key(row,colFam,colQual,colVis,(long)i);
+ if(timeRange.evaluate(k, value))
+ expected++;
+ inputBuffer.put(k, value);
+ }
+ for(Entry<Key,Value> e:inputBuffer.entrySet())
+ {
+ writer.append(e.getKey(), e.getValue());
+ }
+ writer.close();
+
+ // scan the RFile to bring back keys in a given timestamp range
+ byte[] data = baos.toByteArray();
+ ByteArrayInputStream bais = new SeekableByteArrayInputStream(data);
+ FSDataInputStream in = new FSDataInputStream(bais);
+ CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader(in, data.length, conf);
+ RFile.Reader reader = new RFile.Reader(_cbr);
+ GenericFilterer filterer = new GenericFilterer(reader);
+ int count = 0;
+ filterer.applyFilter(timeRange,true);
+ filterer.seek(new Range(), Collections.EMPTY_SET, false);
+ while(filterer.hasTop())
+ {
+ count++;
+ assertTrue(timeRange.evaluate(filterer.getTopKey(),filterer.getTopValue()));
+ filterer.next();
+ }
+ assertEquals(expected, count);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/core/src/test/java/org/apache/accumulo/core/iterators/user/FilterTest.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/core/src/test/java/org/apache/accumulo/core/iterators/user/IndexedDocIteratorTest.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/core/src/test/java/org/apache/accumulo/core/security/ColumnVisibilityTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/security/ColumnVisibilityTest.java
index d463f42,6dfc8e5..3a90009
--- a/core/src/test/java/org/apache/accumulo/core/security/ColumnVisibilityTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/security/ColumnVisibilityTest.java
@@@ -81,51 -82,13 +81,64 @@@ public class ColumnVisibilityTest
@Test
public void testNormalization() {
normalized("a", "a", "(a)", "a", "b|a", "a|b", "(b)|a", "a|b", "(b|(a|c))&x", "x&(a|b|c)", "(((a)))", "a");
+ normalized("a|a", "a", "a|(a&a)", "a", "(a&b)|(b&a)", "a&b");
+ normalized("a|(a|(a|b))","a|b");
+ normalized("a|(a|(a|a))","a");
+ final String normForm = "a&b&c";
- normalized("b&c&a", normForm, "c&b&a", normForm, "a&(b&c)", normForm, "(a&c)&b", normForm);
++ normalized("b&c&a", normForm);
++ normalized("c&b&a", normForm);
++ normalized("a&(b&c)", normForm);
++ normalized("(a&c)&b", normForm);
++ final String normForm2 = "a|b|c";
++ normalized("b|c|a", normForm2);
++ normalized("c|b|a", normForm2);
++ normalized("a|(b|c)", normForm2);
++ normalized("(a|c)|b", normForm2);
+
+ // this an expression that's basically `expr | expr`
+ normalized("(d&c&b&a)|(b&c&a&d)", "a&b&c&d");
}
+ public void aOrBEqualC(String a, String b, String c)
+ {
+ ColumnVisibility cvA = new ColumnVisibility(a.getBytes());
+ ColumnVisibility cvB = new ColumnVisibility(b.getBytes());
+ ColumnVisibility cvC = cvA.or(cvB);
+ assertArrayEquals(cvC.getExpression(), c.getBytes());
+ // check that we didn't disturb the original ColumnVisibilities
+ assertArrayEquals(cvA.getExpression(), a.getBytes());
+ assertArrayEquals(cvB.getExpression(), b.getBytes());
+ }
+
+ @Test
+ public void testDisjunction() {
+ aOrBEqualC("a", "b", "a|b");
+ aOrBEqualC("c|(a&b)", "b", "b|c|(a&b)");
+ aOrBEqualC("c|(a&b)", "a|c","a|c|(a&b)");
+ aOrBEqualC("a&b","c&d","(a&b)|(c&d)");
+ aOrBEqualC("a","","");
+ }
+
+ public void aAndBEqualC(String a, String b, String c)
+ {
+ ColumnVisibility cvA = new ColumnVisibility(a.getBytes());
+ ColumnVisibility cvB = new ColumnVisibility(b.getBytes());
+ ColumnVisibility cvC = cvA.and(cvB);
+ assertArrayEquals(cvC.getExpression(), c.getBytes());
+ // check that we didn't disturb the original ColumnVisibilities
+ assertArrayEquals(cvA.getExpression(), a.getBytes());
+ assertArrayEquals(cvB.getExpression(), b.getBytes());
+ }
+
+ @Test
+ public void testConjunction() {
+ aAndBEqualC("a", "b", "a&b");
+ aAndBEqualC("a&b", "c", "a&b&c");
+ aAndBEqualC("a&(b|(c&d))", "e&(b|(c&d))","a&e&(b|(c&d))");
+ aAndBEqualC("a|b","c|d","(a|b)&(c|d)");
+ aAndBEqualC("a","","a");
+ }
+
@Test
public void testDanglingOperators() {
shouldThrow("a|b&");
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/core/src/test/java/org/apache/accumulo/core/security/VisibilityEvaluatorTest.java
----------------------------------------------------------------------
diff --cc core/src/test/java/org/apache/accumulo/core/security/VisibilityEvaluatorTest.java
index 7612e15,64aeeeb..d9b2ff0
--- a/core/src/test/java/org/apache/accumulo/core/security/VisibilityEvaluatorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/security/VisibilityEvaluatorTest.java
@@@ -52,9 -58,9 +58,9 @@@ public class VisibilityEvaluatorTest
// test missing separators; these should throw an exception
for (String marking : new String[] {"one(five)", "(five)one", "(one)(two)", "a|(b(c))"}) {
try {
- ct.evaluate(new ColumnVisibility(marking));
+ new ColumnVisibility(marking).evaluate(auths);
fail(marking + " failed to throw");
- } catch (Throwable e) {
+ } catch (BadArgumentException e) {
// all is good
}
}
@@@ -62,9 -68,9 +68,9 @@@
// test unexpected separator
for (String marking : new String[] {"&(five)", "|(five)", "(five)&", "five|", "a|(b)&", "(&five)", "(five|)"}) {
try {
- ct.evaluate(new ColumnVisibility(marking));
+ new ColumnVisibility(marking).evaluate(auths);
fail(marking + " failed to throw");
- } catch (Throwable e) {
+ } catch (BadArgumentException e) {
// all is good
}
}
@@@ -72,11 -78,49 +78,47 @@@
// test mismatched parentheses
for (String marking : new String[] {"(", ")", "(a&b", "b|a)"}) {
try {
- ct.evaluate(new ColumnVisibility(marking));
+ new ColumnVisibility(marking).evaluate(auths);
fail(marking + " failed to throw");
- } catch (Throwable e) {
+ } catch (BadArgumentException e) {
// all is good
}
}
}
+
+ @Test
- public void testQuotedExpressions() throws VisibilityParseException {
- VisibilityEvaluator ct = new VisibilityEvaluator(new Authorizations("A#C", "A\"C", "A\\C", "AC"));
++ public void testQuotedExpressions() {
++ Authorizations auths = new Authorizations("A#C", "A\"C", "A\\C", "AC");
+
- assertTrue(ct.evaluate(new ColumnVisibility(quote("A#C") + "|" + quote("A?C"))));
- assertTrue(ct.evaluate(new ColumnVisibility(new ColumnVisibility(quote("A#C") + "|" + quote("A?C")).flatten())));
- assertTrue(ct.evaluate(new ColumnVisibility(quote("A\"C") + "&" + quote("A\\C"))));
- assertTrue(ct.evaluate(new ColumnVisibility(new ColumnVisibility(quote("A\"C") + "&" + quote("A\\C")).flatten())));
- assertTrue(ct.evaluate(new ColumnVisibility("(" + quote("A\"C") + "|B)&(" + quote("A#C") + "|D)")));
-
- assertFalse(ct.evaluate(new ColumnVisibility(quote("A#C") + "&B")));
-
- assertTrue(ct.evaluate(new ColumnVisibility(quote("A#C"))));
- assertTrue(ct.evaluate(new ColumnVisibility("(" + quote("A#C") + ")")));
++ assertTrue((new ColumnVisibility(quote("A#C") + "|" + quote("A?C"))).evaluate(auths));
++ assertTrue((new ColumnVisibility(new ColumnVisibility(quote("A#C") + "|" + quote("A?C")).getExpression())).evaluate(auths));
++ assertTrue((new ColumnVisibility(quote("A\"C") + "&" + quote("A\\C"))).evaluate(auths));
++ assertTrue((new ColumnVisibility(new ColumnVisibility(quote("A\"C") + "&" + quote("A\\C")).getExpression())).evaluate(auths));
++ assertTrue((new ColumnVisibility("(" + quote("A\"C") + "|B)&(" + quote("A#C") + "|D)")).evaluate(auths));
++ assertFalse((new ColumnVisibility(quote("A#C") + "&B")).evaluate(auths));
++ assertTrue((new ColumnVisibility(quote("A#C"))).evaluate(auths));
++ assertTrue((new ColumnVisibility("(" + quote("A#C") + ")")).evaluate(auths));
+ }
+
+ @Test
+ public void testQuote() {
+ assertEquals("\"A#C\"", quote("A#C"));
+ assertEquals("\"A\\\"C\"", quote("A\"C"));
+ assertEquals("\"A\\\"\\\\C\"", quote("A\"\\C"));
+ assertEquals("ACS", quote("ACS"));
+ assertEquals("\"九\"", quote("九"));
+ assertEquals("\"五十\"", quote("五十"));
+ }
+
+ @Test
- public void testNonAscii() throws VisibilityParseException, UnsupportedEncodingException {
- VisibilityEvaluator ct = new VisibilityEvaluator(new Authorizations(Charset.forName("UTF-8"), "五", "六", "八", "九", "五十"));
++ public void testNonAscii() throws UnsupportedEncodingException {
++ Authorizations auths = new Authorizations(Charset.forName("UTF-8"), "五", "六", "八", "九", "五十");
+
- assertTrue(ct.evaluate(new ColumnVisibility(quote("五") + "|" + quote("四"), "UTF-8")));
- assertFalse(ct.evaluate(new ColumnVisibility(quote("五") + "&" + quote("四"), "UTF-8")));
- assertTrue(ct.evaluate(new ColumnVisibility(quote("五") + "&(" + quote("四") + "|" + quote("九") + ")", "UTF-8")));
- assertTrue(ct.evaluate(new ColumnVisibility("\"五\"&(\"四\"|\"五十\")", "UTF-8")));
- assertFalse(ct.evaluate(new ColumnVisibility(quote("五") + "&(" + quote("四") + "|" + quote("三") + ")", "UTF-8")));
- assertFalse(ct.evaluate(new ColumnVisibility("\"五\"&(\"四\"|\"三\")", "UTF-8")));
++ assertTrue((new ColumnVisibility(quote("五") + "|" + quote("四"), "UTF-8")).evaluate(auths));
++ assertFalse((new ColumnVisibility(quote("五") + "&" + quote("四"), "UTF-8")).evaluate(auths));
++ assertTrue((new ColumnVisibility(quote("五") + "&(" + quote("四") + "|" + quote("九") + ")", "UTF-8")).evaluate(auths));
++ assertTrue((new ColumnVisibility("\"五\"&(\"四\"|\"五十\")", "UTF-8")).evaluate(auths));
++ assertFalse((new ColumnVisibility(quote("五") + "&(" + quote("四") + "|" + quote("三") + ")", "UTF-8")).evaluate(auths));
++ assertFalse((new ColumnVisibility("\"五\"&(\"四\"|\"三\")", "UTF-8")).evaluate(auths));
+ }
}
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/docs/examples/README.bloom
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/docs/examples/README.mapred
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/docs/examples/README.maxmutation
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/examples/instamo/pom.xml
----------------------------------------------------------------------
diff --cc examples/instamo/pom.xml
index 0000000,6ba714c..b50cb79
mode 000000,100644..100644
--- a/examples/instamo/pom.xml
+++ b/examples/instamo/pom.xml
@@@ -1,0 -1,117 +1,117 @@@
+ <?xml version="1.0"?>
+ <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <groupId>org.apache.accumulo</groupId>
+ <artifactId>instamo</artifactId>
+ <packaging>jar</packaging>
- <version>1.5.0-SNAPSHOT</version>
++ <version>ACCUMULO-652-SNAPSHOT</version>
+ <name>instamo</name>
+ <url>http://maven.apache.org</url>
+ <properties>
- <accumulo.version>1.5.0-SNAPSHOT</accumulo.version>
++ <accumulo.version>ACCUMULO-652-SNAPSHOT</accumulo.version>
+ <hadoop-one.version>1.0.4</hadoop-one.version>
+ <hadoop-two.version>2.0.2-alpha</hadoop-two.version>
+ <maclass>org.apache.accumulo.instamo.MapReduceExample</maclass>
+ </properties>
+ <profiles>
+ <!-- profile for building against Hadoop 1.0.x
+ Activate by not specifying hadoop.profile -->
+ <profile>
+ <id>hadoop-1.0</id>
+ <activation>
+ <property>
+ <name>!hadoop.profile</name>
+ </property>
+ </activation>
+ <properties>
+ <hadoop.version>${hadoop-one.version}</hadoop.version>
+ </properties>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-core</artifactId>
+ <version>${hadoop.version}</version>
+ </dependency>
+ </dependencies>
+ </profile>
+ <!-- profile for building against Hadoop 2.0.x
+ Activate using: mvn -Dhadoop.profile=2.0 -->
+ <profile>
+ <id>hadoop-2.0</id>
+ <activation>
+ <property>
+ <name>hadoop.profile</name>
+ <value>2.0</value>
+ </property>
+ </activation>
+ <properties>
+ <hadoop.version>${hadoop-two.version}</hadoop.version>
+ </properties>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-client</artifactId>
+ <version>${hadoop.version}</version>
+ </dependency>
+ </dependencies>
+ </profile>
+ </profiles>
+ <dependencies>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <version>4.11</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.accumulo</groupId>
+ <artifactId>accumulo-core</artifactId>
+ <version>${accumulo.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.accumulo</groupId>
+ <artifactId>accumulo-server</artifactId>
+ <version>${accumulo.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.zookeeper</groupId>
+ <artifactId>zookeeper</artifactId>
+ <version>3.3.4</version>
+ </dependency>
+ <dependency>
+ <groupId>commons-io</groupId>
+ <artifactId>commons-io</artifactId>
+ <version>2.4</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.accumulo</groupId>
+ <artifactId>accumulo-test</artifactId>
+ <version>${accumulo.version}</version>
+ </dependency>
+ </dependencies>
+ <build>
+ <plugins>
+ <plugin>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.6</source>
+ <target>1.6</target>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>exec-maven-plugin</artifactId>
+ <version>1.2.1</version>
+ <configuration>
+ <executable>java</executable>
+ <arguments>
+ <argument>-classpath</argument>
+ <classpath/>
+ <argument>${maclass}</argument>
+ </arguments>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+ </project>
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/examples/pom.xml
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/examples/simple/pom.xml
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/fate/pom.xml
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/accumulo/blob/58fcad6e/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index a8fe066,54c1fbb..c76e612
--- a/pom.xml
+++ b/pom.xml
@@@ -549,8 -618,13 +618,13 @@@
</dependency>
<dependency>
<groupId>org.apache.accumulo</groupId>
+ <artifactId>accumulo-test</artifactId>
- <version>1.5.0-SNAPSHOT</version>
++ <version>ACCUMULO-652-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.accumulo</groupId>
<artifactId>examples-simple</artifactId>
- <version>1.5.0-SNAPSHOT</version>
+ <version>ACCUMULO-652-SNAPSHOT</version>
</dependency>
<dependency>
<groupId>org.apache.accumulo</groupId>