You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jclouds.apache.org by na...@apache.org on 2013/07/25 10:45:08 UTC

git commit: Generalized the Arg0ToPagedIterable to propagate all args

Updated Branches:
  refs/heads/master c64c7423c -> 462117651


Generalized the Arg0ToPagedIterable to propagate all args

Generalized the Arg0ToPagedIterable to allow to propagating
all arguments. This will help building PagedIterables for
api methods that require more than one argument to be invoked.


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

Branch: refs/heads/master
Commit: 462117651662b91048fb4ea0a62db490f3e9708d
Parents: c64c742
Author: Ignasi Barrera <ig...@gmail.com>
Authored: Mon Jul 22 16:05:02 2013 +0200
Committer: Ignasi Barrera <ig...@gmail.com>
Committed: Thu Jul 25 10:31:09 2013 +0200

----------------------------------------------------------------------
 .../collect/internal/Arg0ToPagedIterable.java   |  28 +---
 .../collect/internal/ArgsToPagedIterable.java   |  91 ++++++++++
 .../internal/Arg0ToPagedIterableTest.java       |   2 +-
 .../internal/ArgsToPagedIterableTest.java       | 168 +++++++++++++++++++
 4 files changed, 263 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-jclouds/blob/46211765/core/src/main/java/org/jclouds/collect/internal/Arg0ToPagedIterable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/jclouds/collect/internal/Arg0ToPagedIterable.java b/core/src/main/java/org/jclouds/collect/internal/Arg0ToPagedIterable.java
index c80b90c..431eefb 100644
--- a/core/src/main/java/org/jclouds/collect/internal/Arg0ToPagedIterable.java
+++ b/core/src/main/java/org/jclouds/collect/internal/Arg0ToPagedIterable.java
@@ -16,15 +16,10 @@
  */
 package org.jclouds.collect.internal;
 
-import static org.jclouds.collect.PagedIterables.advance;
-import static org.jclouds.collect.PagedIterables.onlyPage;
-
 import java.util.List;
 
 import org.jclouds.collect.IterableWithMarker;
 import org.jclouds.collect.PagedIterable;
-import org.jclouds.http.HttpRequest;
-import org.jclouds.rest.InvocationContext;
 import org.jclouds.rest.internal.GeneratedHttpRequest;
 
 import com.google.common.annotations.Beta;
@@ -38,22 +33,12 @@ import com.google.common.base.Optional;
  * @author Adrian Cole
  */
 @Beta
-public abstract class Arg0ToPagedIterable<T, I extends Arg0ToPagedIterable<T, I>> implements
-      Function<IterableWithMarker<T>, PagedIterable<T>>, InvocationContext<I> {
-
-   private GeneratedHttpRequest request;
+public abstract class Arg0ToPagedIterable<T, I extends Arg0ToPagedIterable<T, I>> extends ArgsToPagedIterable<T, I> {
 
    @Override
-   public PagedIterable<T> apply(IterableWithMarker<T> input) {
-      if (!input.nextMarker().isPresent())
-         return onlyPage(input);
-      List<Object> args = getArgs(request);
+   protected Function<Object, IterableWithMarker<T>> markerToNextForArgs(List<Object> args) {
       Optional<Object> arg0 = Optional.fromNullable(args.size() > 0 ? args.get(0) : null);
-      return advance(input, markerToNextForArg0(arg0));
-   }
-
-   protected List<Object> getArgs(GeneratedHttpRequest request) {
-      return request.getInvocation().getArgs();
+      return markerToNextForArg0(arg0);
    }
 
    /**
@@ -62,13 +47,6 @@ public abstract class Arg0ToPagedIterable<T, I extends Arg0ToPagedIterable<T, I>
     */
    protected abstract Function<Object, IterableWithMarker<T>> markerToNextForArg0(Optional<Object> arg0);
 
-   @SuppressWarnings("unchecked")
-   @Override
-   public I setContext(HttpRequest request) {
-      this.request = GeneratedHttpRequest.class.cast(request);
-      return (I) this;
-   }
-
    /**
     * Used to propagate caller {@code arg0} to a callee during an advance in a {@link PagedIterable}. For example, in
     * the call {@code api.getUserApiForZone(zone).list()}, the caller arg0 is the value of {@code zone}, and the callee

http://git-wip-us.apache.org/repos/asf/incubator-jclouds/blob/46211765/core/src/main/java/org/jclouds/collect/internal/ArgsToPagedIterable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/jclouds/collect/internal/ArgsToPagedIterable.java b/core/src/main/java/org/jclouds/collect/internal/ArgsToPagedIterable.java
new file mode 100644
index 0000000..4ae1204
--- /dev/null
+++ b/core/src/main/java/org/jclouds/collect/internal/ArgsToPagedIterable.java
@@ -0,0 +1,91 @@
+/*
+ * 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.jclouds.collect.internal;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.jclouds.collect.PagedIterables.advance;
+import static org.jclouds.collect.PagedIterables.onlyPage;
+
+import java.util.List;
+
+import org.jclouds.collect.IterableWithMarker;
+import org.jclouds.collect.PagedIterable;
+import org.jclouds.http.HttpRequest;
+import org.jclouds.rest.InvocationContext;
+import org.jclouds.rest.internal.GeneratedHttpRequest;
+
+import com.google.common.annotations.Beta;
+import com.google.common.base.Function;
+
+/**
+ * Used to propagate the invoked method arguments during an advance in a
+ * {@link PagedIterable}.
+ * <p>
+ * In order to fetch the next page in the result set, subclasses may need to
+ * have the context of the initial request. This class propagates the arguments
+ * used in the original request, so they can be used to fetch the next page in
+ * the result set.
+ * 
+ * @author Ignasi Barrera
+ */
+@Beta
+public abstract class ArgsToPagedIterable<T, I extends ArgsToPagedIterable<T, I>> implements
+      Function<IterableWithMarker<T>, PagedIterable<T>>, InvocationContext<I> {
+
+   protected GeneratedHttpRequest request;
+
+   @Override
+   public PagedIterable<T> apply(IterableWithMarker<T> input) {
+      return input.nextMarker().isPresent() ? advance(input, markerToNextForArgs(getArgs(request))) : onlyPage(input);
+   }
+
+   protected List<Object> getArgs(GeneratedHttpRequest request) {
+      return request.getInvocation().getArgs();
+   }
+
+   protected abstract Function<Object, IterableWithMarker<T>> markerToNextForArgs(List<Object> args);
+
+   @SuppressWarnings("unchecked")
+   @Override
+   public I setContext(HttpRequest request) {
+      checkArgument(request instanceof GeneratedHttpRequest,
+            "ArgsToPagedIterable only supports a GeneratedHttpRequest");
+      this.request = GeneratedHttpRequest.class.cast(request);
+      return (I) this;
+   }
+
+   /**
+    * Sometimes the arguments in the invoked method do not provide enough
+    * information to fetch the next page of the result set. This, for example,
+    * is common in APIs dealing with several endpoints.
+    * <p>
+    * This class provides a way to propagate the arguments passed to the caller
+    * of the method, so they can be used to fetch the next page of the result
+    * set. For example, in the call {@code api.getUserApiForZone(zone).list()},
+    * the caller arg0 is the value of {@code zone}, and the callee is
+    * {@code UserApi.list()}
+    * 
+    * @author Adrian Cole
+    * @see ParseImages function in openstack-glance for a usage example.
+    */
+   public abstract static class FromCaller<T, I extends FromCaller<T, I>> extends ArgsToPagedIterable<T, I> {
+      @Override
+      protected List<Object> getArgs(GeneratedHttpRequest request) {
+         return request.getCaller().get().getArgs();
+      }
+   }
+}

http://git-wip-us.apache.org/repos/asf/incubator-jclouds/blob/46211765/core/src/test/java/org/jclouds/collect/internal/Arg0ToPagedIterableTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/jclouds/collect/internal/Arg0ToPagedIterableTest.java b/core/src/test/java/org/jclouds/collect/internal/Arg0ToPagedIterableTest.java
index a849c94..a45a37e 100644
--- a/core/src/test/java/org/jclouds/collect/internal/Arg0ToPagedIterableTest.java
+++ b/core/src/test/java/org/jclouds/collect/internal/Arg0ToPagedIterableTest.java
@@ -53,7 +53,7 @@ public class Arg0ToPagedIterableTest {
 
          @Override
          protected Function<Object, IterableWithMarker<String>> markerToNextForArg0(Optional<Object> arg0) {
-            fail();
+            fail("The Iterable should not advance");
             return null;
          }
 

http://git-wip-us.apache.org/repos/asf/incubator-jclouds/blob/46211765/core/src/test/java/org/jclouds/collect/internal/ArgsToPagedIterableTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/jclouds/collect/internal/ArgsToPagedIterableTest.java b/core/src/test/java/org/jclouds/collect/internal/ArgsToPagedIterableTest.java
new file mode 100644
index 0000000..2f764c8
--- /dev/null
+++ b/core/src/test/java/org/jclouds/collect/internal/ArgsToPagedIterableTest.java
@@ -0,0 +1,168 @@
+/*
+ * 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.jclouds.collect.internal;
+
+import static com.google.common.base.Throwables.propagate;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+import static org.testng.Assert.fail;
+
+import java.util.List;
+
+import org.jclouds.collect.IterableWithMarker;
+import org.jclouds.collect.IterableWithMarkers;
+import org.jclouds.reflect.Invocation;
+import org.jclouds.rest.internal.GeneratedHttpRequest;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Function;
+import com.google.common.base.Functions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.reflect.Invokable;
+
+/**
+ * @author Ignasi Barrera
+ */
+@Test(testName = "ArgsToPagedIterableTest")
+public class ArgsToPagedIterableTest {
+
+   private abstract static class TestArgs extends ArgsToPagedIterable<String, TestArgs> {
+      private TestArgs(GeneratedHttpRequest in) {
+         this.setContext(in);
+      }
+   }
+
+   @Test
+   public void testWhenNextMarkerAbsentDoesntAdvance() {
+      GeneratedHttpRequest request = args(ImmutableList.of());
+
+      TestArgs converter = new TestArgs(request) {
+
+         @Override
+         protected Function<Object, IterableWithMarker<String>> markerToNextForArgs(List<Object> args) {
+            fail("The Iterable should not advance");
+            return null;
+         }
+
+      };
+
+      assertEquals(converter.apply(IterableWithMarkers.from(ImmutableSet.of("foo", "bar"))).concat().toSet(),
+            ImmutableSet.of("foo", "bar"));
+   }
+
+   @Test
+   public void testWhenNextMarkerPresentButNoArgsMarkerToNextForArgsParamIsAbsent() {
+      GeneratedHttpRequest request = args(ImmutableList.of());
+      final IterableWithMarker<String> next = IterableWithMarkers.from(ImmutableSet.of("baz"));
+
+      TestArgs converter = new TestArgs(request) {
+
+         @Override
+         protected Function<Object, IterableWithMarker<String>> markerToNextForArgs(List<Object> args) {
+            assertTrue(args.isEmpty());
+            return Functions.constant(next);
+         }
+
+      };
+
+      assertEquals(converter.apply(IterableWithMarkers.from(ImmutableSet.of("foo", "bar"), "marker")).concat().toSet(),
+            ImmutableSet.of("foo", "bar", "baz"));
+   }
+
+   @Test
+   public void testWhenNextMarkerPresentWithArgsMarkerToNextForArgsParamIsPresent() {
+      GeneratedHttpRequest request = args(ImmutableList.<Object> of("path"));
+      final IterableWithMarker<String> next = IterableWithMarkers.from(ImmutableSet.of("baz"));
+
+      TestArgs converter = new TestArgs(request) {
+
+         @Override
+         protected Function<Object, IterableWithMarker<String>> markerToNextForArgs(List<Object> args) {
+            assertEquals(args.get(0), "path");
+            return Functions.constant(next);
+         }
+
+      };
+
+      assertEquals(converter.apply(IterableWithMarkers.from(ImmutableSet.of("foo", "bar"), "marker")).concat().toSet(),
+            ImmutableSet.of("foo", "bar", "baz"));
+   }
+
+   private GeneratedHttpRequest args(ImmutableList<Object> args) {
+      try {
+         return GeneratedHttpRequest.builder().method("GET").endpoint("http://localhost")
+               .invocation(Invocation.create(Invokable.from(String.class.getMethod("toString")), args)).build();
+      } catch (Exception e) {
+         throw propagate(e);
+      }
+   }
+
+   private abstract static class TestCallerArgs extends ArgsToPagedIterable.FromCaller<String, TestCallerArgs> {
+      private TestCallerArgs(GeneratedHttpRequest in) {
+         this.setContext(in);
+      }
+   }
+
+   @Test
+   public void testFromCallerWhenNextMarkerPresentButNoArgsMarkerToNextForArgsParamIsAbsent() {
+      GeneratedHttpRequest request = callerArgs(ImmutableList.of());
+      final IterableWithMarker<String> next = IterableWithMarkers.from(ImmutableSet.of("baz"));
+
+      TestCallerArgs converter = new TestCallerArgs(request) {
+
+         @Override
+         protected Function<Object, IterableWithMarker<String>> markerToNextForArgs(List<Object> args) {
+            assertTrue(args.isEmpty());
+            return Functions.constant(next);
+         }
+
+      };
+
+      assertEquals(converter.apply(IterableWithMarkers.from(ImmutableSet.of("foo", "bar"), "marker")).concat().toSet(),
+            ImmutableSet.of("foo", "bar", "baz"));
+   }
+
+   @Test
+   public void testFromCallerWhenNextMarkerPresentWithArgsMarkerToNextForArgsParamIsPresent() {
+      GeneratedHttpRequest request = callerArgs(ImmutableList.<Object> of("path"));
+      final IterableWithMarker<String> next = IterableWithMarkers.from(ImmutableSet.of("baz"));
+
+      TestCallerArgs converter = new TestCallerArgs(request) {
+
+         @Override
+         protected Function<Object, IterableWithMarker<String>> markerToNextForArgs(List<Object> args) {
+            assertEquals(args.get(0), "path");
+            return Functions.constant(next);
+         }
+
+      };
+
+      assertEquals(converter.apply(IterableWithMarkers.from(ImmutableSet.of("foo", "bar"), "marker")).concat().toSet(),
+            ImmutableSet.of("foo", "bar", "baz"));
+   }
+
+   private GeneratedHttpRequest callerArgs(ImmutableList<Object> args) {
+      try {
+         return GeneratedHttpRequest.builder().method("GET").endpoint("http://localhost")
+               .invocation(Invocation.create(Invokable.from(String.class.getMethod("toString"))))
+               .caller(Invocation.create(Invokable.from(String.class.getMethod("toString")), args)).build();
+      } catch (Exception e) {
+         throw propagate(e);
+      }
+   }
+}