You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by "Brian Loss (JIRA)" <ji...@apache.org> on 2013/01/11 17:10:14 UTC

[jira] [Updated] (ACCUMULO-956) Iterator to transform key parts

     [ https://issues.apache.org/jira/browse/ACCUMULO-956?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Brian Loss updated ACCUMULO-956:
--------------------------------

    Status: Patch Available  (was: Open)

Index: core/src/test/java/org/apache/accumulo/core/iterators/user/KeyTransformingIteratorTest.java
===================================================================
--- core/src/test/java/org/apache/accumulo/core/iterators/user/KeyTransformingIteratorTest.java	(revision 0)
+++ core/src/test/java/org/apache/accumulo/core/iterators/user/KeyTransformingIteratorTest.java	(revision 0)
@@ -0,0 +1,332 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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 static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map.Entry;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+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.WrappingIterator;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Test;
+
+public class KeyTransformingIteratorTest {
+  private static final String TABLE_NAME = "test_table";
+  private static Authorizations authorizations = new Authorizations("vis0", "vis1", "vis2", "vis3", "vis4");
+  private Connector connector;
+  private Scanner scanner;
+  
+  @Before
+  public void setUpMockAccumulo() throws Exception {
+    MockInstance instance = new MockInstance("test");
+    connector = instance.getConnector("user", "password");
+    connector.securityOperations().changeUserAuthorizations("user", authorizations);
+    
+    if (connector.tableOperations().exists(TABLE_NAME))
+      connector.tableOperations().delete(TABLE_NAME);
+    connector.tableOperations().create(TABLE_NAME);
+    BatchWriterConfig bwCfg = new BatchWriterConfig();
+    bwCfg.setMaxWriteThreads(1);
+    
+    BatchWriter bw = connector.createBatchWriter(TABLE_NAME, bwCfg);
+    bw.addMutation(createDefaultMutation("row1"));
+    bw.addMutation(createDefaultMutation("row2"));
+    bw.addMutation(createDefaultMutation("row3"));
+    
+    bw.flush();
+    bw.close();
+    
+    scanner = connector.createScanner(TABLE_NAME, authorizations);
+    scanner.addScanIterator(new IteratorSetting(20, ReuseIterator.class));
+    IteratorSetting cfg = new IteratorSetting(21, TestKeyTransformingIterator.class);
+    cfg.addOption(KeyTransformingIterator.AUTH_OPT, "vis0, vis1, vis2, vis3");
+    scanner.addScanIterator(cfg);
+  }
+  
+  @Test
+  public void testIdentityScan() throws Exception {
+    TestKeyTransformingIterator.keyTransformer = new IdentityTransformer(); 
+    
+    // This is just an identity scan, but with the "ruse" iterator that reuses
+    // the same key/value pair for every getTopKey/getTopValue call.  The code
+    // will always return the final key/alue if we didn't copy the original key
+    // in the iterator.
+    TreeMap<Key,Value> expected = new TreeMap<Key,Value>();
+    for (int row = 1; row <= 3; ++row) {
+      for (int cf = 1; cf <= 3; ++cf) {
+        for (int cq = 1; cq <= 3; ++cq) {
+          for (int cv = 1; cv <= 3; ++cv) {
+            putExpected(expected, row, cf, cq, cv, null);
+          }
+        }
+      }
+    }
+    
+    checkExpected(expected);
+  }
+  
+  @Test
+  public void testNoRangeScan() throws Exception {
+    // Test transforming col fam, col qual, col vis
+    PartialKey[] parts = { PartialKey.ROW, PartialKey.ROW_COLFAM, PartialKey.ROW_COLFAM_COLQUAL };
+    for (PartialKey part : parts) {
+      TestKeyTransformingIterator.keyTransformer = new KeyPartReversingTransformer(part); 
+      
+      // All rows with visibilities reversed
+      TreeMap<Key,Value> expected = new TreeMap<Key,Value>();
+      for (int row = 1; row <= 3; ++row) {
+        for (int cf = 1; cf <= 3; ++cf) {
+          for (int cq = 1; cq <= 3; ++cq) {
+            for (int cv = 1; cv <= 3; ++cv) {
+              putExpected(expected, row, cf, cq, cv, part);
+            }
+          }
+        }
+      }
+      
+      checkExpected(expected);    
+    }
+  }
+  
+  @Test
+  public void testVisbilityFiltering() throws Exception {
+    TestKeyTransformingIterator.keyTransformer = new KeyTransformer() {            
+      @Override
+      public PartialKey getKeyPrefix() {
+        return PartialKey.ROW_COLFAM_COLQUAL;
+      }
+      
+      @Override
+      public Key transformKey(Key ok) {
+        return new Key(ok.getRow(), ok.getColumnFamily(), ok.getColumnQualifier(), new Text("badvis"), ok.getTimestamp());
+      }
+    };
+    
+    // Should return nothing since we produced visibilities that can't be seen
+    checkExpected(new TreeMap<Key,Value>());
+  }
+  
+  @Test
+  public void testRangeStart() throws Exception {
+    PartialKey part = PartialKey.ROW_COLFAM_COLQUAL;
+    TestKeyTransformingIterator.keyTransformer = new KeyPartReversingTransformer(part);
+    scanner.setRange(new Range(new Key("row1", "cf2", "cq2", "vis1"), true, new Key("row1", "cf2", "cq3"), false));
+    
+    TreeMap<Key,Value> expected = new TreeMap<Key,Value>();
+    putExpected(expected, 1, 2, 2, 1, part); // before the range start, but transforms in the range
+    putExpected(expected, 1, 2, 2, 2, part);
+    
+    checkExpected(expected);
+  }
+  
+  @Test
+  public void testRangeEnd() throws Exception {
+    PartialKey part = PartialKey.ROW_COLFAM_COLQUAL;
+    TestKeyTransformingIterator.keyTransformer = new KeyPartReversingTransformer(part);
+    scanner.setRange(new Range(new Key("row1", "cf2", "cq2"), true, new Key("row1", "cf2", "cq2", "vis2"), false));
+    
+    TreeMap<Key,Value> expected = new TreeMap<Key,Value>();
+    //putExpected(expected, 1, 2, 2, 1, part); // transforms vis outside range end
+    putExpected(expected, 1, 2, 2, 2, part);
+    putExpected(expected, 1, 2, 2, 3, part);
+    
+    checkExpected(expected);
+  }
+  
+  private void checkExpected(TreeMap<Key,Value> expectedEntries) {
+    for (Entry<Key,Value> entry : scanner) {
+      Entry<Key,Value> expected = expectedEntries.pollFirstEntry();
+      Key actualKey = entry.getKey();
+      Value actualValue = entry.getValue();
+      
+      assertNotNull("Ran out of expected entries on: " + entry, expected);
+      assertEquals("Key mismatch", expected.getKey(), actualKey);
+      assertEquals("Value mismatch", expected.getValue(), actualValue);
+    }
+    
+    assertTrue("Scanner did not return all expected entries: " + expectedEntries, expectedEntries.isEmpty());
+  }
+  
+  private static void putExpected(SortedMap<Key,Value> expected, int rowID, int cfID, int cqID, int cvID, PartialKey part) {
+    String row = "row" + rowID;
+    String cf = "cf" + cfID;
+    String cq = "cq" + cqID;
+    String cv = "vis" + cvID;
+    long ts = 100*cfID + 10*cqID + cvID;
+    String val = "val" + ts;
+    
+    if (part != null) {
+      switch (part) {
+        case ROW:
+          cf = transform(new Text(cf)).toString(); break;
+        case ROW_COLFAM:
+          cq = transform(new Text(cq)).toString(); break;
+        case ROW_COLFAM_COLQUAL:
+          cv = transform(new Text(cv)).toString(); break;
+        default:
+          break;
+      }
+    }
+    
+    expected.put(new Key(row, cf, cq, cv, ts), new Value(val.getBytes()));
+  }
+  
+  private static Text transform(Text val) {
+    String s = val.toString();
+    // Reverse the order of the number at the end, and subtract one
+    int i = 3 - Integer.parseInt(s.substring(s.length()-1));
+    StringBuilder sb = new StringBuilder();
+    sb.append(s.substring(0, s.length() - 1));
+    sb.append(i);
+    return new Text(sb.toString());
+  }
+  
+  private static Mutation createDefaultMutation(String row) {
+    Mutation m = new Mutation(row);
+    for (int cfID = 1; cfID <= 3; ++cfID) {
+      for (int cqID = 1; cqID <= 3; ++cqID) {
+        for (int cvID = 1; cvID <= 3; ++cvID) {
+          String cf = "cf" + cfID;
+          String cq = "cq" + cqID;
+          String cv = "vis" + cvID;
+          long ts = 100*cfID + 10*cqID + cvID;
+          String val = "val" + ts;
+          
+          m.put(cf, cq, new ColumnVisibility(cv), ts, val);
+        }
+      }
+    }
+    return m;
+  }
+  
+  private static interface KeyTransformer {
+    Key transformKey(Key originalKey);
+    PartialKey getKeyPrefix();
+  }
+  
+  private static class IdentityTransformer implements KeyTransformer {
+    @Override
+    public PartialKey getKeyPrefix() {
+      return PartialKey.ROW;
+    }
+    
+    @Override
+    public Key transformKey(Key ok) {
+      return ok;
+    }
+  }
+  
+  private static class KeyPartReversingTransformer implements KeyTransformer {
+    private PartialKey part;
+    
+    public KeyPartReversingTransformer(PartialKey part) {
+      this.part = part;
+    }
+    
+    @Override
+    public PartialKey getKeyPrefix() {
+      return part;
+    }
+    
+    @Override
+    public Key transformKey(Key ok) {
+      Text row = ok.getRow();
+      Text cf = ok.getColumnFamily();
+      Text cq = ok.getColumnQualifier();
+      Text cv = ok.getColumnVisibility();
+      long ts = ok.getTimestamp();
+      switch (part) {
+        case ROW:
+          cf = transform(cf); break;
+        case ROW_COLFAM:
+          cq = transform(cq); break;
+        case ROW_COLFAM_COLQUAL:
+          cv = transform(cv); break;
+        default:
+          break;
+      }
+      return new Key(row, cf, cq, cv, ts);
+    }
+  }
+  
+  public static class TestKeyTransformingIterator extends KeyTransformingIterator {
+    static KeyTransformer keyTransformer;
+    
+    @Override
+    protected Key transformKey(Key originalKey) {
+      return keyTransformer.transformKey(originalKey);
+    }
+    
+    @Override
+    protected PartialKey getKeyPrefix() {
+      return keyTransformer.getKeyPrefix();
+    }
+  }
+  
+  public static class ReuseIterator extends WrappingIterator {
+    private Key topKey = new Key();
+    private Value topValue = new Value();
+    
+    @Override
+    public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
+      super.seek(range, columnFamilies, inclusive);
+      loadTop();
+    }
+    
+    @Override
+    public void next() throws IOException {
+      super.next();
+      loadTop();
+    }
+    
+    @Override
+    public Key getTopKey() {
+      return topKey;
+    }
+    
+    @Override
+    public Value getTopValue() {
+      return topValue;
+    }
+    
+    private void loadTop() {
+      if (hasTop()) {
+        topKey.set(super.getTopKey());
+        topValue.set(super.getTopValue().get());
+      }
+    }
+  }
+}
Index: core/src/main/java/org/apache/accumulo/core/iterators/user/KeyTransformingIterator.java
===================================================================
--- core/src/main/java/org/apache/accumulo/core/iterators/user/KeyTransformingIterator.java	(revision 0)
+++ core/src/main/java/org/apache/accumulo/core/iterators/user/KeyTransformingIterator.java	(revision 0)
@@ -0,0 +1,392 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.Map;
+
+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.map.LRUMap;
+import org.apache.log4j.Logger;
+
+/**
+ * The KeyTransformingIterator 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.
+ */
+abstract public class KeyTransformingIterator extends WrappingIterator implements OptionDescriber {
+  public static final String AUTH_OPT = "authorizations";
+  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;
+
+  private VisibilityEvaluator ve = null;
+  private LRUMap cache = null;
+  
+  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 KeyTransformingIterator() {
+  }
+  
+  @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.split(",")));
+        cache = new LRUMap(100);
+      }
+    }
+  }
+  
+  @Override
+  public IteratorOptions describeOptions() {
+    String desc = "This iterator allows keys to be transformed.";
+    String authDesc = "Comma-separated list of user's scan authorizations.  " +
+    		"If excluded or empty, then no visibility check is performed on transformed keys.";
+    return new IteratorOptions(getClass().getSimpleName(), desc, Collections.singletonMap(AUTH_OPT, authDesc), null);
+  }
+  
+  @Override
+  public boolean validateOptions(Map<String,String> options) {
+    return true;
+  }
+  
+  @Override
+  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+    KeyTransformingIterator 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.ve = ve;
+    if (cache != null) {
+      copy.cache = new LRUMap(cache.maxSize());
+      copy.cache.putAll(cache);
+    }
+    
+    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;
+
+    // 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), 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();
+    }
+  }
+  
+  /**
+   * 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();
+    Key prefixKey = super.hasTop() ? new Key(super.getTopKey()) : null;
+
+    while (super.hasTop()) {
+      Key sourceTopKey = super.getTopKey();
+
+      // If the source key equals our prefix key (up to the prefix), then
+      // we have a key that needs transformed.  Otherwise, we're done.
+      if (sourceTopKey.equals(prefixKey, getKeyPrefix())) {
+        Key transformedKey = transformKey(sourceTopKey);
+        
+        // If the transformed key didn't actually change, then we need
+        // to make a copy since we're caching it in our keys list.
+        if (transformedKey == sourceTopKey)
+          transformedKey = new Key(sourceTopKey);
+
+        // 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(transformedKey))
+          keys.add(new Pair<Key,Value>(transformedKey, new Value(super.getTopValue())));
+      } else {
+        break;
+      }
+
+      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 is outside of the seek range.  We only do any work if we're
+   * scanning since seek ranges for major/minor compaction 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 boundary 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 = true;
+    if (scanning) {
+      include = canSee(transformedKey);
+      if (seekRange != null)
+        include = include && seekRange.contains(transformedKey);
+    }
+    return include;
+  }
+  
+  /**
+   * Indicates whether or not the user is able to see {@code key}.  If the
+   * iterator is in scan mode, then this method simply returns {@code true}.
+   * 
+   * @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) {
+    if (!scanning || ve == null || cache == null)
+      return true;
+
+    ByteSequence visibility = key.getColumnVisibilityData();
+    Boolean visible = (Boolean) cache.get(visibility);
+    if (visible == null) {
+      try {
+        visible = ve.evaluate(new ColumnVisibility(visibility.getBackingArray()));
+        cache.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;
+  }
+
+  /**
+   * 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;
+  }
+  
+  /**
+   * 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 #transformKey(Key)} 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();
+  
+  /**
+   * Transforms {@code originalKey}.  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.
+   * 
+   * @param originalKey the key to be transformed
+   * @return the modified key
+   */
+  abstract protected Key transformKey(Key originalKey);
+}

                
> Iterator to transform key parts
> -------------------------------
>
>                 Key: ACCUMULO-956
>                 URL: https://issues.apache.org/jira/browse/ACCUMULO-956
>             Project: Accumulo
>          Issue Type: Improvement
>    Affects Versions: 1.5.0
>            Reporter: Brian Loss
>             Fix For: 1.5.0
>
>
> Iterators that transform parts of the key can be tricky if any transformation affects sort ordering.  Implement an iterator that takes care of the tricky details that come with modifying sort order (e.g., handling scan-time iterator reconstruction and the associated seek).

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira