You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by ma...@apache.org on 2016/01/22 00:06:25 UTC

aurora git commit: Allowing dual authorizing params to account for thrift API deprecations.

Repository: aurora
Updated Branches:
  refs/heads/master b2cc604a6 -> a2c7ccc17


Allowing dual authorizing params to account for thrift API deprecations.

Also, added missing test coverage.

Reviewed at https://reviews.apache.org/r/42614/


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

Branch: refs/heads/master
Commit: a2c7ccc17e8bc2e0a2aaef6c90db949d5885d60b
Parents: b2cc604
Author: Maxim Khutornenko <ma...@apache.org>
Authored: Thu Jan 21 15:06:07 2016 -0800
Committer: Maxim Khutornenko <ma...@apache.org>
Committed: Thu Jan 21 15:06:07 2016 -0800

----------------------------------------------------------------------
 config/legacy_untested_classes.txt              |  1 -
 .../http/api/security/AuthorizingParam.java     | 11 ++-
 .../ShiroAuthorizingParamInterceptor.java       | 94 +++++++++++---------
 .../ShiroAuthorizingParamInterceptorTest.java   | 56 ++++++++++++
 4 files changed, 115 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/a2c7ccc1/config/legacy_untested_classes.txt
----------------------------------------------------------------------
diff --git a/config/legacy_untested_classes.txt b/config/legacy_untested_classes.txt
index 6b71fd2..144b258 100644
--- a/config/legacy_untested_classes.txt
+++ b/config/legacy_untested_classes.txt
@@ -54,7 +54,6 @@ org/apache/aurora/scheduler/http/Utilization$4
 org/apache/aurora/scheduler/http/Utilization$5
 org/apache/aurora/scheduler/http/Utilization$Display
 org/apache/aurora/scheduler/http/Utilization$DisplayMetric
-org/apache/aurora/scheduler/http/api/security/FieldGetter$IdentityFieldGetter
 org/apache/aurora/scheduler/http/api/security/Kerberos5Realm
 org/apache/aurora/scheduler/log/mesos/MesosLogStreamModule
 org/apache/aurora/scheduler/log/mesos/MesosLogStreamModule$3

http://git-wip-us.apache.org/repos/asf/aurora/blob/a2c7ccc1/src/main/java/org/apache/aurora/scheduler/http/api/security/AuthorizingParam.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/api/security/AuthorizingParam.java b/src/main/java/org/apache/aurora/scheduler/http/api/security/AuthorizingParam.java
index 11d7e46..73c539b 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/api/security/AuthorizingParam.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/api/security/AuthorizingParam.java
@@ -26,9 +26,14 @@ import java.lang.annotation.Target;
  * parameter, otherwise a privilege escalation vulnerability exists.
  *
  * <p>
- * A method intercepted by this interceptor that does not contain an AuthorizingParam or with
- * multiple AuthorizingParams will result in an {@link java.lang.IllegalStateException} from the
- * interceptor.
+ * A method intercepted by this interceptor that does not contain an AuthorizingParam will result
+ * in an {@link java.lang.IllegalStateException} from the interceptor.
+ *
+ * <p>
+ * It is possible to have more than one AuthorizingParam annotations applied to a method. This may
+ * be needed to ensure graceful deprecation cycle of the old annotated parameter for backwards
+ * compatibility reasons. In such cases, only one of the annotated arguments must be non-null or
+ * an {@link java.lang.IllegalStateException} is thrown.
  *
  * <p>
  * If the parameter type is not known to the interceptor an {@link java.lang.IllegalStateException}

http://git-wip-us.apache.org/repos/asf/aurora/blob/a2c7ccc1/src/main/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptor.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptor.java b/src/main/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptor.java
index 6905662..21e565e 100644
--- a/src/main/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptor.java
+++ b/src/main/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptor.java
@@ -14,7 +14,7 @@
 package org.apache.aurora.scheduler.http.api.security;
 
 import java.lang.reflect.Method;
-import java.util.List;
+import java.lang.reflect.Parameter;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
@@ -33,10 +33,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.google.common.reflect.Invokable;
-import com.google.common.reflect.Parameter;
 
 import org.aopalliance.intercept.MethodInterceptor;
 import org.aopalliance.intercept.MethodInvocation;
@@ -62,7 +59,6 @@ import org.apache.aurora.scheduler.storage.entities.IJobKey;
 import org.apache.aurora.scheduler.thrift.Responses;
 import org.apache.shiro.authz.Permission;
 import org.apache.shiro.subject.Subject;
-import org.apache.thrift.TBase;
 
 import static java.util.Objects.requireNonNull;
 
@@ -106,6 +102,16 @@ class ShiroAuthorizingParamInterceptor implements MethodInterceptor {
         }
       };
 
+  private static class JobKeyGetter {
+    private final int index;
+    private final Function<Object, Optional<JobKey>> func;
+
+    JobKeyGetter(int index, Function<Object, Optional<JobKey>> func) {
+      this.index = index;
+      this.func = func;
+    }
+  }
+
   private static final FieldGetter<JobUpdateRequest, TaskConfig> UPDATE_REQUEST_GETTER =
       new ThriftFieldGetter<>(
           JobUpdateRequest.class,
@@ -195,26 +201,33 @@ class ShiroAuthorizingParamInterceptor implements MethodInterceptor {
     };
   }
 
-  private static int annotatedParameterIndex(Method method) {
+  private static Iterable<JobKeyGetter> annotatedParameterGetters(Method method) {
     for (Method candidateMethod : getCandidateMethods(method)) {
-      List<Parameter> parameters = Invokable.from(candidateMethod).getParameters();
-      List<Integer> parameterIndicies = Lists.newArrayList();
-      for (int i = 0; i < parameters.size(); i++) {
-        if (parameters.get(i).isAnnotationPresent(AuthorizingParam.class)) {
-          parameterIndicies.add(i);
+      Parameter[] parameters = candidateMethod.getParameters();
+      ImmutableList.Builder<JobKeyGetter> jobKeyGetters = ImmutableList.builder();
+      for (int i = 0; i < parameters.length; i++) {
+        Parameter param = parameters[i];
+        if (param.isAnnotationPresent(AuthorizingParam.class)) {
+          Class<?> parameterType = param.getType();
+          @SuppressWarnings("unchecked")
+          Optional<Function<Object, Optional<JobKey>>> jobKeyGetter =
+              Optional.fromNullable(
+                  (Function<Object, Optional<JobKey>>) FIELD_GETTERS_BY_TYPE.get(parameterType));
+          if (!jobKeyGetter.isPresent()) {
+            throw new UnsupportedOperationException(
+                "No "
+                    + JobKey.class.getName()
+                    + " field getter was supplied for "
+                    + parameterType.getName());
+          }
+
+          jobKeyGetters.add(new JobKeyGetter(i, jobKeyGetter.get()));
         }
       }
 
-      if (parameterIndicies.size() == 1) {
-        return Iterables.getOnlyElement(parameterIndicies);
-      } else if (parameterIndicies.size() > 1) {
-        throw new UnsupportedOperationException(
-            "Too many parameters annotated with "
-                + AuthorizingParam.class.getName()
-                + " found on method "
-                + method.getName()
-                + " of class "
-                + method.getDeclaringClass().getName());
+      ImmutableList<JobKeyGetter> getters = jobKeyGetters.build();
+      if (!Iterables.isEmpty(getters)) {
+        return getters;
       }
     }
 
@@ -242,30 +255,25 @@ class ShiroAuthorizingParamInterceptor implements MethodInterceptor {
                     + Response.class.getName());
           }
 
-          final int index = annotatedParameterIndex(method);
-          ImmutableList<Parameter> parameters = Invokable.from(method).getParameters();
-          Class<?> parameterType = parameters.get(index).getType().getRawType();
-          if (!TBase.class.isAssignableFrom(parameterType)) {
-            throw new UnsupportedOperationException(
-                "Annotated parameter must be a thrift struct.");
-          }
-          @SuppressWarnings("unchecked")
-          final Optional<Function<Object, Optional<JobKey>>> jobKeyGetter =
-              Optional.fromNullable(
-                  (Function<Object, Optional<JobKey>>) FIELD_GETTERS_BY_TYPE.get(parameterType));
-          if (!jobKeyGetter.isPresent()) {
-            throw new UnsupportedOperationException(
-                "No "
-                    + JobKey.class.getName()
-                    + " field getter was supplied for "
-                    + parameterType.getName());
-          }
+          Iterable<JobKeyGetter> getters = annotatedParameterGetters(method);
           return arguments -> {
-            Optional<Object> argument = Optional.fromNullable(arguments[index]);
-            if (argument.isPresent()) {
-              return jobKeyGetter.get().apply(argument.get());
-            } else {
+            Iterable<JobKeyGetter> nonNullArgGetters =
+                Iterables.filter(getters, getter -> arguments[getter.index] != null);
+            if (Iterables.isEmpty(nonNullArgGetters)) {
               return Optional.absent();
+            } else {
+              if (Iterables.size(nonNullArgGetters) > 1) {
+                throw new IllegalStateException(
+                    "Too many non-null arguments annotated with "
+                        + AuthorizingParam.class.getName()
+                        + " passed to "
+                        + method.getName()
+                        + " of "
+                        + method.getDeclaringClass().getName());
+              }
+
+              JobKeyGetter getter = Iterables.getOnlyElement(nonNullArgGetters);
+              return getter.func.apply(arguments[getter.index]);
             }
           };
         }

http://git-wip-us.apache.org/repos/asf/aurora/blob/a2c7ccc1/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptorTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptorTest.java b/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptorTest.java
index 16a3a3b..79e70fd 100644
--- a/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptorTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/http/api/security/ShiroAuthorizingParamInterceptorTest.java
@@ -16,7 +16,10 @@ package org.apache.aurora.scheduler.http.api.security;
 import java.lang.reflect.Method;
 import java.util.concurrent.atomic.AtomicLong;
 
+import com.google.common.base.Function;
+import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.UncheckedExecutionException;
 import com.google.inject.AbstractModule;
 import com.google.inject.Guice;
 import com.google.inject.matcher.Matchers;
@@ -24,6 +27,8 @@ import com.google.inject.matcher.Matchers;
 import org.apache.aurora.common.stats.StatsProvider;
 import org.apache.aurora.common.testing.easymock.EasyMockTest;
 import org.apache.aurora.gen.JobConfiguration;
+import org.apache.aurora.gen.JobKey;
+import org.apache.aurora.gen.JobUpdateRequest;
 import org.apache.aurora.gen.Response;
 import org.apache.aurora.gen.ResponseCode;
 import org.apache.aurora.gen.TaskQuery;
@@ -187,4 +192,55 @@ public class ShiroAuthorizingParamInterceptorTest extends EasyMockTest {
                 ImmutableSet.of(JOB_KEY.newBuilder(), JOB_KEY.newBuilder().setName("other"))))
         .orNull());
   }
+
+  @Test
+  public void testHandlesMultipleAnnotations() {
+    control.replay();
+
+    Function<Object[], Optional<JobKey>> func =
+        interceptor.getAuthorizingParamGetters().getUnchecked(Params.class.getMethods()[0]);
+
+    func.apply(new Object[]{new TaskQuery(), null, null});
+    func.apply(new Object[]{null, new JobKey(), null});
+    func.apply(new Object[]{null, null, new JobUpdateRequest()});
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testThrowsOnMultipleNonNullArguments() {
+    control.replay();
+
+    Function<Object[], Optional<JobKey>> func =
+        interceptor.getAuthorizingParamGetters().getUnchecked(Params.class.getMethods()[0]);
+
+    func.apply(new Object[]{new TaskQuery(), new JobKey(), null});
+  }
+
+  @Test(expected = UncheckedExecutionException.class)
+     public void testThrowsNoAuthParams() {
+    control.replay();
+
+    interceptor.getAuthorizingParamGetters().getUnchecked(NoParams.class.getMethods()[0]);
+  }
+
+  @Test(expected = UncheckedExecutionException.class)
+  public void testThrowsNoResponseReturned() {
+    control.replay();
+
+    interceptor.getAuthorizingParamGetters().getUnchecked(NoResponse.class.getMethods()[0]);
+  }
+
+  private interface NoResponse {
+    void test(@AuthorizingParam TaskQuery query);
+  }
+
+  private interface NoParams {
+    Response test(TaskQuery query);
+  }
+
+  private interface Params {
+    Response test(
+        @AuthorizingParam TaskQuery query,
+        @AuthorizingParam JobKey job,
+        @AuthorizingParam JobUpdateRequest request);
+  }
 }