You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by cj...@apache.org on 2015/01/10 06:07:49 UTC

accumulo git commit: ACCUMULO-3458 Propagating scan-time authorizations through the IteratorEnvironment so that scan-time iterators can use them.

Repository: accumulo
Updated Branches:
  refs/heads/master 9edeca84d -> 9952957c8


ACCUMULO-3458 Propagating scan-time authorizations through the IteratorEnvironment so that scan-time iterators can use them.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/9952957c
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/9952957c
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/9952957c

Branch: refs/heads/master
Commit: 9952957c8c0280364b5c85a26552317da2589034
Parents: 9edeca8
Author: Corey J. Nolet <cj...@gmail.com>
Authored: Tue Jan 6 10:53:42 2015 -0500
Committer: Corey J. Nolet <cj...@gmail.com>
Committed: Sat Jan 10 00:07:34 2015 -0500

----------------------------------------------------------------------
 .../core/client/ClientSideIteratorScanner.java  |  11 ++
 .../accumulo/core/client/ScannerBase.java       |   9 ++
 .../core/client/impl/OfflineScanner.java        |  19 ++-
 .../accumulo/core/client/impl/ScannerImpl.java  |   4 +
 .../core/client/impl/ScannerOptions.java        |   6 +
 .../client/impl/TabletServerBatchReader.java    |   6 +-
 .../core/client/mock/MockScannerBase.java       |  18 ++-
 .../core/iterators/IteratorEnvironment.java     |   2 +
 .../core/client/impl/ScannerImplTest.java       |  28 +++-
 .../impl/TabletServerBatchReaderTest.java       |  52 +++++++
 .../iterators/DefaultIteratorEnvironment.java   |   6 +
 .../iterators/FirstEntryInRowIteratorTest.java  |  14 +-
 .../iterators/user/RowDeletingIteratorTest.java |   5 +
 .../user/TransformingIteratorTest.java          |   6 +-
 .../iterators/MetadataBulkLoadFilterTest.java   |   6 +
 .../server/replication/StatusCombinerTest.java  |   6 +
 .../monitor/servlets/trace/NullScanner.java     |   5 +
 .../tserver/TabletIteratorEnvironment.java      |  16 +-
 .../accumulo/tserver/tablet/Compactor.java      |   1 +
 .../accumulo/tserver/tablet/ScanDataSource.java |   3 +-
 .../accumulo/test/functional/AuthsIterator.java |  51 +++++++
 .../apache/accumulo/test/ScanIteratorIT.java    | 147 +++++++++++++++++++
 22 files changed, 410 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
index d6261fb..ec13b1f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
@@ -41,6 +41,7 @@ import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -190,6 +191,11 @@ public class ClientSideIteratorScanner extends ScannerOptions implements Scanner
 
         @Override
         public void registerSideChannel(final SortedKeyValueIterator<Key,Value> iter) {}
+
+        @Override
+        public Authorizations getAuthorizations() {
+            return smi.scanner.getAuthorizations();
+        }
       }, false, null);
     } catch (IOException e) {
       throw new RuntimeException(e);
@@ -209,6 +215,11 @@ public class ClientSideIteratorScanner extends ScannerOptions implements Scanner
     return new IteratorAdapter(skvi);
   }
 
+  @Override
+  public Authorizations getAuthorizations() {
+    return smi.scanner.getAuthorizations();
+  }
+
   @Deprecated
   @Override
   public void setTimeOut(int timeOut) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java b/core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java
index 82a3299..e49a502 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ScannerBase.java
@@ -22,6 +22,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
 
 /**
@@ -127,4 +128,12 @@ public interface ScannerBase extends Iterable<Entry<Key,Value>> {
    * @since 1.5.0
    */
   void close();
+
+  /**
+   * Returns the authorizations that have been set on the scanner
+   * @since 1.7.0
+   * @return
+   *       The authorizations set on the scanner instance
+   */
+  Authorizations getAuthorizations();
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java b/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
index d679ffa..0263bfa 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
@@ -74,6 +74,13 @@ import org.apache.hadoop.io.Text;
 class OfflineIterator implements Iterator<Entry<Key,Value>> {
 
   static class OfflineIteratorEnvironment implements IteratorEnvironment {
+
+    private final Authorizations authorizations;
+
+    public OfflineIteratorEnvironment(Authorizations auths) {
+      this.authorizations = auths;
+    }
+
     @Override
     public SortedKeyValueIterator<Key,Value> reserveMapFileReader(String mapFileName) throws IOException {
       throw new NotImplementedException();
@@ -101,6 +108,11 @@ class OfflineIterator implements Iterator<Entry<Key,Value>> {
       topLevelIterators.add(iter);
     }
 
+    @Override
+    public Authorizations getAuthorizations() {
+      return authorizations;
+    }
+
     SortedKeyValueIterator<Key,Value> getTopLevelIterator(SortedKeyValueIterator<Key,Value> iter) {
       if (topLevelIterators.isEmpty())
         return iter;
@@ -304,7 +316,7 @@ class OfflineIterator implements Iterator<Entry<Key,Value>> {
 
     MultiIterator multiIter = new MultiIterator(readers, extent);
 
-    OfflineIteratorEnvironment iterEnv = new OfflineIteratorEnvironment();
+    OfflineIteratorEnvironment iterEnv = new OfflineIteratorEnvironment(authorizations);
 
     DeletingIterator delIter = new DeletingIterator(multiIter, false);
 
@@ -408,6 +420,11 @@ public class OfflineScanner extends ScannerOptions implements Scanner {
   }
 
   @Override
+  public Authorizations getAuthorizations() {
+    return authorizations;
+  }
+
+  @Override
   public long getReadaheadThreshold() {
     throw new UnsupportedOperationException();
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerImpl.java
index 3f73f04..6084024 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerImpl.java
@@ -100,6 +100,10 @@ public class ScannerImpl extends ScannerOptions implements Scanner {
     return new ScannerIterator(context, table, authorizations, range, size, getTimeOut(), this, isolated, readaheadThreshold);
   }
 
+  @Override public Authorizations getAuthorizations() {
+    return authorizations;
+  }
+
   @Override
   public synchronized void enableIsolation() {
     this.isolated = true;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerOptions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerOptions.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerOptions.java
index d6c50c7..5b6d9ac 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerOptions.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerOptions.java
@@ -36,6 +36,7 @@ import org.apache.accumulo.core.data.Column;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.data.thrift.IterInfo;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.hadoop.io.Text;
 
@@ -210,4 +211,9 @@ public class ScannerOptions implements ScannerBase {
   public void close() {
     // Nothing needs to be closed
   }
+
+  @Override
+  public Authorizations getAuthorizations() {
+    throw new UnsupportedOperationException("No authorizations to return");
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java
index a7422c2..c310d01 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java
@@ -17,7 +17,6 @@
 package org.apache.accumulo.core.client.impl;
 
 import static com.google.common.base.Preconditions.checkArgument;
-
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Iterator;
@@ -73,6 +72,11 @@ public class TabletServerBatchReader extends ScannerOptions implements BatchScan
     queryThreadPool.shutdownNow();
   }
 
+  @Override
+  public Authorizations getAuthorizations() {
+    return authorizations;
+  }
+
   /**
    * Warning: do not rely upon finalize to close this class. Finalize is not guaranteed to be called.
    */

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java
index d88c30a..60f2a2a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java
@@ -62,6 +62,13 @@ public class MockScannerBase extends ScannerOptions implements ScannerBase {
   }
 
   static class MockIteratorEnvironment implements IteratorEnvironment {
+
+    private final Authorizations auths;
+
+    MockIteratorEnvironment(Authorizations auths) {
+        this.auths = auths;
+    }
+
     @Override
     public SortedKeyValueIterator<Key,Value> reserveMapFileReader(String mapFileName) throws IOException {
       throw new NotImplementedException();
@@ -89,6 +96,11 @@ public class MockScannerBase extends ScannerOptions implements ScannerBase {
       topLevelIterators.add(iter);
     }
 
+    @Override
+    public Authorizations getAuthorizations() {
+        return auths;
+    }
+
     SortedKeyValueIterator<Key,Value> getTopLevelIterator(SortedKeyValueIterator<Key,Value> iter) {
       if (topLevelIterators.isEmpty())
         return iter;
@@ -104,7 +116,7 @@ public class MockScannerBase extends ScannerOptions implements ScannerBase {
     ColumnQualifierFilter cqf = new ColumnQualifierFilter(inner, new HashSet<Column>(fetchedColumns));
     VisibilityFilter vf = new VisibilityFilter(cqf, auths, defaultLabels);
     AccumuloConfiguration conf = new MockConfiguration(table.settings);
-    MockIteratorEnvironment iterEnv = new MockIteratorEnvironment();
+    MockIteratorEnvironment iterEnv = new MockIteratorEnvironment(auths);
     SortedKeyValueIterator<Key,Value> result = iterEnv.getTopLevelIterator(IteratorUtil.loadIterators(IteratorScope.scan, vf, null, conf,
         serverSideIteratorList, serverSideIteratorOptions, iterEnv, false));
     return result;
@@ -114,4 +126,8 @@ public class MockScannerBase extends ScannerOptions implements ScannerBase {
   public Iterator<Entry<Key,Value>> iterator() {
     throw new UnsupportedOperationException();
   }
+
+  @Override public Authorizations getAuthorizations() {
+    return auths;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/core/src/main/java/org/apache/accumulo/core/iterators/IteratorEnvironment.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorEnvironment.java b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorEnvironment.java
index f708db7..5a53e93 100644
--- a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorEnvironment.java
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorEnvironment.java
@@ -22,6 +22,7 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.security.Authorizations;
 
 public interface IteratorEnvironment {
 
@@ -35,4 +36,5 @@ public interface IteratorEnvironment {
 
   void registerSideChannel(SortedKeyValueIterator<Key,Value> iter);
 
+  Authorizations getAuthorizations();
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerImplTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerImplTest.java
index dec6748..0c65422 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerImplTest.java
@@ -16,6 +16,8 @@
  */
 package org.apache.accumulo.core.client.impl;
 
+import static org.junit.Assert.assertEquals;
+
 import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.mock.MockInstance;
@@ -23,6 +25,7 @@ import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.Credentials;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.Test;
 
 /**
@@ -30,10 +33,17 @@ import org.junit.Test;
  */
 public class ScannerImplTest {
 
+  MockInstance instance;
+  ClientContext context;
+
+  @Before
+  public void setup() {
+    instance = new MockInstance();
+    context = new ClientContext(instance, new Credentials("root", new PasswordToken("")), new ClientConfiguration());
+  }
+
   @Test
   public void testValidReadaheadValues() {
-    MockInstance instance = new MockInstance();
-    ClientContext context = new ClientContext(instance, new Credentials("root", new PasswordToken("")), new ClientConfiguration());
     Scanner s = new ScannerImpl(context, "foo", Authorizations.EMPTY);
     s.setReadaheadThreshold(0);
     s.setReadaheadThreshold(10);
@@ -44,10 +54,20 @@ public class ScannerImplTest {
 
   @Test(expected = IllegalArgumentException.class)
   public void testInValidReadaheadValues() {
-    MockInstance instance = new MockInstance();
-    ClientContext context = new ClientContext(instance, new Credentials("root", new PasswordToken("")), new ClientConfiguration());
     Scanner s = new ScannerImpl(context, "foo", Authorizations.EMPTY);
     s.setReadaheadThreshold(-1);
   }
 
+  @Test
+  public void testGetAuthorizations() {
+    Authorizations expected = new Authorizations("a,b");
+    Scanner s = new ScannerImpl(context, "foo", expected);
+    assertEquals(expected, s.getAuthorizations());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testNullAuthorizationsFails() {
+    new ScannerImpl(context, "foo", null);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/core/src/test/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderTest.java
new file mode 100644
index 0000000..5c7eb1b
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderTest.java
@@ -0,0 +1,52 @@
+/*
+ * 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.client.impl;
+
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.ClientConfiguration;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.Credentials;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TabletServerBatchReaderTest {
+
+  MockInstance instance;
+  ClientContext context;
+
+  @Before
+  public void setup() {
+    instance = new MockInstance();
+    context = new ClientContext(instance, new Credentials("root", new PasswordToken("")), new ClientConfiguration());
+  }
+
+  @Test
+  public void testGetAuthorizations() {
+    Authorizations expected = new Authorizations("a,b");
+    BatchScanner s = new TabletServerBatchReader(context, "foo", expected, 1);
+    assertEquals(expected, s.getAuthorizations());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testNullAuthorizationsFails() {
+    new TabletServerBatchReader(context, "foo", null, 1);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/core/src/test/java/org/apache/accumulo/core/iterators/DefaultIteratorEnvironment.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/DefaultIteratorEnvironment.java b/core/src/test/java/org/apache/accumulo/core/iterators/DefaultIteratorEnvironment.java
index c864fcc..316823c 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/DefaultIteratorEnvironment.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/DefaultIteratorEnvironment.java
@@ -23,6 +23,7 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.system.MapFileIterator;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -66,4 +67,9 @@ public class DefaultIteratorEnvironment implements IteratorEnvironment {
     throw new UnsupportedOperationException();
   }
 
+  @Override
+  public Authorizations getAuthorizations() {
+    throw new UnsupportedOperationException();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/core/src/test/java/org/apache/accumulo/core/iterators/FirstEntryInRowIteratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/FirstEntryInRowIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/FirstEntryInRowIteratorTest.java
index 21c31e2..88be983 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/FirstEntryInRowIteratorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/FirstEntryInRowIteratorTest.java
@@ -29,6 +29,7 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.system.CountingIterator;
+import org.apache.accumulo.core.security.Authorizations;
 import org.junit.Test;
 
 public class FirstEntryInRowIteratorTest {
@@ -40,23 +41,34 @@ public class FirstEntryInRowIteratorTest {
     FirstEntryInRowIterator feiri = new FirstEntryInRowIterator();
     IteratorEnvironment env = new IteratorEnvironment() {
 
+      @Override
       public AccumuloConfiguration getConfig() {
         return null;
       }
 
+      @Override
       public IteratorScope getIteratorScope() {
         return null;
       }
 
+      @Override
       public boolean isFullMajorCompaction() {
         return false;
       }
 
+      @Override
       public void registerSideChannel(SortedKeyValueIterator<Key,Value> arg0) {
 
       }
 
-      public SortedKeyValueIterator<Key,Value> reserveMapFileReader(String arg0) throws IOException {
+      @Override
+      public Authorizations getAuthorizations() {
+        return null;
+      }
+
+      @Override
+      public SortedKeyValueIterator<Key,Value> reserveMapFileReader(
+          String arg0) throws IOException {
         return null;
       }
     };

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/core/src/test/java/org/apache/accumulo/core/iterators/user/RowDeletingIteratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowDeletingIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowDeletingIteratorTest.java
index d2dd6da..6bd92ee 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowDeletingIteratorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowDeletingIteratorTest.java
@@ -34,6 +34,7 @@ import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
 
 public class RowDeletingIteratorTest extends TestCase {
@@ -71,6 +72,10 @@ public class RowDeletingIteratorTest extends TestCase {
     @Override
     public void registerSideChannel(SortedKeyValueIterator<Key,Value> iter) {}
 
+    @Override
+    public Authorizations getAuthorizations() {
+        return null;
+    }
   }
 
   Key nk(String row, String cf, String cq, long time) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
index 4cebab7..758f718 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
@@ -20,7 +20,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -724,5 +723,10 @@ public class TransformingIteratorTest {
     public void registerSideChannel(SortedKeyValueIterator<Key,Value> iter) {
       delegate.registerSideChannel(iter);
     }
+
+    @Override
+    public Authorizations getAuthorizations() {
+      return null;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java b/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java
index cfdd5e9..e7c7161 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java
@@ -32,6 +32,7 @@ import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.ColumnFQ;
 import org.apache.accumulo.fate.zookeeper.TransactionWatcher.Arbitrator;
 import org.apache.hadoop.io.Text;
@@ -114,6 +115,11 @@ public class MetadataBulkLoadFilterTest {
       public void registerSideChannel(SortedKeyValueIterator<Key,Value> iter) {}
 
       @Override
+      public Authorizations getAuthorizations() {
+          return null;
+      }
+
+      @Override
       public boolean isFullMajorCompaction() {
         return false;
       }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java b/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
index 74e5455..7428a19 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
@@ -35,6 +35,7 @@ import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 import org.apache.accumulo.core.replication.StatusUtil;
 import org.apache.accumulo.core.replication.proto.Replication.Status;
+import org.apache.accumulo.core.security.Authorizations;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -78,6 +79,11 @@ public class StatusCombinerTest {
       }
 
       @Override
+      public Authorizations getAuthorizations() {
+        return null;
+      }
+
+      @Override
       public SortedKeyValueIterator<Key,Value> reserveMapFileReader(String arg0) throws IOException {
         return null;
       }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/server/monitor/src/main/java/org/apache/accumulo/monitor/servlets/trace/NullScanner.java
----------------------------------------------------------------------
diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/servlets/trace/NullScanner.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/servlets/trace/NullScanner.java
index 750cdcd..5dae2f7 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/servlets/trace/NullScanner.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/servlets/trace/NullScanner.java
@@ -25,6 +25,7 @@ import org.apache.accumulo.core.client.Scanner;
 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.security.Authorizations;
 import org.apache.hadoop.io.Text;
 
 public class NullScanner implements Scanner {
@@ -104,6 +105,10 @@ public class NullScanner implements Scanner {
   @Override
   public void close() {}
 
+  @Override public Authorizations getAuthorizations() {
+    return null;
+  }
+
   @Override
   public long getReadaheadThreshold() {
     return 0l;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletIteratorEnvironment.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletIteratorEnvironment.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletIteratorEnvironment.java
index e7477b9..532405d 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletIteratorEnvironment.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletIteratorEnvironment.java
@@ -29,6 +29,7 @@ import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.system.MultiIterator;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.tserver.FileManager.ScanFileManager;
 import org.apache.hadoop.fs.Path;
@@ -42,6 +43,8 @@ public class TabletIteratorEnvironment implements IteratorEnvironment {
   private final ArrayList<SortedKeyValueIterator<Key,Value>> topLevelIterators = new ArrayList<SortedKeyValueIterator<Key,Value>>();
   private Map<FileRef,DataFileValue> files;
 
+  private final Authorizations authorizations;  // these will only be supplied during scan scope
+
   public TabletIteratorEnvironment(IteratorScope scope, AccumuloConfiguration config) {
     if (scope == IteratorScope.majc)
       throw new IllegalArgumentException("must set if compaction is full");
@@ -50,9 +53,11 @@ public class TabletIteratorEnvironment implements IteratorEnvironment {
     this.trm = null;
     this.config = config;
     this.fullMajorCompaction = false;
+    this.authorizations = Authorizations.EMPTY;
   }
 
-  public TabletIteratorEnvironment(IteratorScope scope, AccumuloConfiguration config, ScanFileManager trm, Map<FileRef,DataFileValue> files) {
+  public TabletIteratorEnvironment(IteratorScope scope, AccumuloConfiguration config, ScanFileManager trm, Map<FileRef,DataFileValue> files,
+      Authorizations authorizations) {
     if (scope == IteratorScope.majc)
       throw new IllegalArgumentException("must set if compaction is full");
 
@@ -61,6 +66,7 @@ public class TabletIteratorEnvironment implements IteratorEnvironment {
     this.config = config;
     this.fullMajorCompaction = false;
     this.files = files;
+    this.authorizations = authorizations;
   }
 
   public TabletIteratorEnvironment(IteratorScope scope, boolean fullMajC, AccumuloConfiguration config) {
@@ -71,6 +77,7 @@ public class TabletIteratorEnvironment implements IteratorEnvironment {
     this.trm = null;
     this.config = config;
     this.fullMajorCompaction = fullMajC;
+    this.authorizations = Authorizations.EMPTY;
   }
 
   @Override
@@ -101,6 +108,13 @@ public class TabletIteratorEnvironment implements IteratorEnvironment {
     topLevelIterators.add(iter);
   }
 
+  @Override
+  public Authorizations getAuthorizations() {
+    if (scope != IteratorScope.scan)
+      throw new UnsupportedOperationException("Authorizations may only be supplied when scope is scan but scope is " + scope);
+    return authorizations;
+  }
+
   public SortedKeyValueIterator<Key,Value> getTopLevelIterator(SortedKeyValueIterator<Key,Value> iter) {
     if (topLevelIterators.isEmpty())
       return iter;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
index 1c23293..99464f4 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
@@ -48,6 +48,7 @@ import org.apache.accumulo.core.iterators.system.DeletingIterator;
 import org.apache.accumulo.core.iterators.system.MultiIterator;
 import org.apache.accumulo.core.iterators.system.TimeSettingIterator;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.trace.Span;
 import org.apache.accumulo.core.trace.Trace;
 import org.apache.accumulo.core.util.LocalityGroupUtil;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
index 00333c6..33277bd 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java
@@ -159,7 +159,8 @@ class ScanDataSource implements DataSource {
 
     MultiIterator multiIter = new MultiIterator(iters, tablet.getExtent());
 
-    TabletIteratorEnvironment iterEnv = new TabletIteratorEnvironment(IteratorScope.scan, tablet.getTableConfiguration(), fileManager, files);
+    TabletIteratorEnvironment iterEnv = new TabletIteratorEnvironment(IteratorScope.scan, tablet.getTableConfiguration(), fileManager, files,
+        options.getAuthorizations());
 
     statsIterator = new StatsIterator(multiIter, TabletServer.seekCount, tablet.getScannedCounter());
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/test/src/main/java/org/apache/accumulo/test/functional/AuthsIterator.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/AuthsIterator.java b/test/src/main/java/org/apache/accumulo/test/functional/AuthsIterator.java
new file mode 100644
index 0000000..1bb82b6
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/AuthsIterator.java
@@ -0,0 +1,51 @@
+/*
+ * 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.test.functional;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.IteratorEnvironment;
+import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.WrappingIterator;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.hadoop.io.Text;
+
+public class AuthsIterator extends WrappingIterator {
+
+  public static final Authorizations AUTHS = new Authorizations("A,B".getBytes());
+  public static final String SUCCESS = "SUCCESS";
+  public static final String FAIL = "FAIL";
+
+  private IteratorEnvironment env;
+
+  @Override
+  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
+    super.init(source, options, env);
+    this.env = env;
+  }
+
+  @Override
+  public Key getTopKey() {
+    if(env.getAuthorizations().equals(AUTHS))
+      return new Key(new Text(SUCCESS));
+    else
+      return new Key(new Text(FAIL));
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9952957c/test/src/test/java/org/apache/accumulo/test/ScanIteratorIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ScanIteratorIT.java b/test/src/test/java/org/apache/accumulo/test/ScanIteratorIT.java
new file mode 100644
index 0000000..7f22ca9
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/ScanIteratorIT.java
@@ -0,0 +1,147 @@
+/*
+ * 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.test;
+
+import static org.junit.Assert.assertEquals;
+import java.util.Collections;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.BatchScanner;
+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.MutationsRejectedException;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase;
+import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.harness.AccumuloClusterIT;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.AuthsIterator;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class ScanIteratorIT extends AccumuloClusterIT {
+
+  private static final String USER = "authsItrUser";
+
+  @Override
+  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setNumTservers(1);
+  }
+
+  @Override
+  protected int defaultTimeoutSeconds() {
+    return 60;
+  }
+
+  private Connector connector;
+  private String tableName;
+
+  @Before
+  public void setup() throws Exception {
+    connector = getConnector();
+    tableName = getUniqueNames(1)[0];
+
+    connector.tableOperations().create(tableName);
+    connector.securityOperations().createLocalUser(USER, new PasswordToken(""));
+    connector.securityOperations().grantTablePermission(USER, tableName, TablePermission.READ);
+    connector.securityOperations().grantTablePermission(USER, tableName, TablePermission.WRITE);
+    connector.securityOperations().changeUserAuthorizations(USER, AuthsIterator.AUTHS);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    connector.securityOperations().dropLocalUser(USER);
+  }
+
+  @Test
+  public void testAuthsPresentInIteratorEnvironment()
+      throws TableNotFoundException, AccumuloException, AccumuloSecurityException, TableExistsException, InterruptedException {
+
+    runTest(AuthsIterator.AUTHS, false);
+  }
+
+  @Test
+  public void testAuthsNotPresentInIteratorEnvironment()
+      throws TableNotFoundException, AccumuloException, AccumuloSecurityException, TableExistsException, InterruptedException {
+
+    runTest(new Authorizations("B"), true);
+  }
+
+  @Test
+  public void testEmptyAuthsInIteratorEnvironment()
+      throws TableNotFoundException, AccumuloException, AccumuloSecurityException, TableExistsException, InterruptedException {
+
+    runTest(Authorizations.EMPTY, true);
+  }
+
+  private void runTest(ScannerBase scanner, Authorizations auths, boolean shouldFail)
+      throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    int count = 0;
+    for (Map.Entry<Key,Value> entry : scanner) {
+      assertEquals(shouldFail ? AuthsIterator.FAIL : AuthsIterator.SUCCESS, entry.getKey().getRow().toString());
+      count++;
+    }
+
+    assertEquals(1, count);
+  }
+
+  private void runTest(Authorizations auths, boolean shouldFail) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    Connector userC = getCluster().getConnector(USER, new PasswordToken(("")));
+    writeTestMutation(userC);
+
+    IteratorSetting setting = new IteratorSetting(10, AuthsIterator.class);
+
+    Scanner scanner = userC.createScanner(tableName, auths);
+    scanner.addScanIterator(setting);
+
+    BatchScanner batchScanner = userC.createBatchScanner(tableName, auths, 1);
+    batchScanner.setRanges(Collections.singleton(new Range("1")));
+    batchScanner.addScanIterator(setting);
+
+    runTest(scanner, auths, shouldFail);
+    runTest(batchScanner, auths, shouldFail);
+
+    scanner.close();
+    batchScanner.close();
+  }
+
+  private void writeTestMutation(Connector userC) throws TableNotFoundException, MutationsRejectedException {
+    BatchWriter batchWriter = userC.createBatchWriter(tableName, new BatchWriterConfig());
+    Mutation m = new Mutation("1");
+    m.put(new Text("2"), new Text("3"), new Value("".getBytes()));
+    batchWriter.addMutation(m);
+    batchWriter.flush();
+    batchWriter.close();
+
+  }
+
+}