You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jclouds.apache.org by ad...@apache.org on 2014/11/22 17:09:49 UTC

[3/3] jclouds git commit: JCLOUDS-480 support version 4 signatures for aws-ec2.

JCLOUDS-480 support version 4 signatures for aws-ec2.


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

Branch: refs/heads/master
Commit: a449b24e7dd9d4872b3d729a3c63add5b6033318
Parents: ba868af
Author: Adrian Cole <ac...@twitter.com>
Authored: Wed Nov 19 10:52:36 2014 -0800
Committer: Adrian Cole <ad...@gmail.com>
Committed: Sat Nov 22 08:08:37 2014 -0800

----------------------------------------------------------------------
 .../binders/BindS3UploadPolicyAndSignature.java |  10 +-
 .../jclouds/ec2/features/BaseEC2ApiTest.java    |   8 +-
 .../aws/config/FormSigningHttpApiModule.java    |  14 +-
 .../org/jclouds/aws/filters/FormSigner.java     | 258 +++++-----
 .../org/jclouds/aws/filters/FormSignerV4.java   | 219 ++++++++
 .../org/jclouds/aws/filters/FormSignerTest.java | 102 ----
 .../jclouds/aws/filters/FormSignerV2Test.java   |  98 ++++
 .../aws/filters/FormSignerV4LiveTest.java       |  91 ++++
 .../jclouds/aws/filters/FormSignerV4Test.java   | 102 ++++
 .../java/org/jclouds/aws/util/AWSUtilsTest.java |   4 +-
 .../aws/xml/SessionCredentialsHandlerTest.java  |   1 -
 .../java/org/jclouds/rest/RequestSigner.java    |   5 +
 .../org/jclouds/rest/config/RestModule.java     |   5 -
 providers/aws-ec2/pom.xml                       |  12 +
 .../aws/ec2/config/AWSEC2HttpApiModule.java     |  17 +-
 .../AWSEC2ComputeServiceApiMockTest.java        | 237 +++++++++
 .../compute/AWSEC2ComputeServiceExpectTest.java | 278 -----------
 .../compute/AWSEC2TemplateBuilderLiveTest.java  |   6 +
 ...AWSEC2SecurityGroupExtensionApiMockTest.java | 167 +++++++
 .../AWSEC2SecurityGroupExtensionExpectTest.java | 414 ----------------
 .../BaseAWSEC2ComputeServiceExpectTest.java     | 151 ------
 .../aws/ec2/features/AWSAMIApiMockTest.java     | 254 ++++++++++
 .../jclouds/aws/ec2/features/AWSAMIApiTest.java | 496 -------------------
 .../ec2/features/AWSSecurityGroupApiTest.java   |  10 +-
 .../aws/ec2/features/BaseAWSEC2ApiTest.java     |  54 +-
 .../aws/ec2/features/MonitoringApiMockTest.java |  58 +++
 .../aws/ec2/features/MonitoringApiTest.java     |  71 ---
 .../features/PlacementGroupApiExpectTest.java   |  75 ---
 .../ec2/features/PlacementGroupApiMockTest.java | 100 ++++
 .../aws/ec2/features/PlacementGroupApiTest.java | 134 -----
 .../ec2/features/SpotInstanceApiExpectTest.java |  75 ---
 .../ec2/features/SpotInstanceApiMockTest.java   | 127 +++++
 .../aws/ec2/features/SpotInstanceApiTest.java   |  98 ----
 .../aws/ec2/internal/BaseAWSEC2ApiMockTest.java | 179 +++++++
 34 files changed, 1863 insertions(+), 2067 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jclouds/blob/a449b24e/apis/ec2/src/main/java/org/jclouds/ec2/binders/BindS3UploadPolicyAndSignature.java
----------------------------------------------------------------------
diff --git a/apis/ec2/src/main/java/org/jclouds/ec2/binders/BindS3UploadPolicyAndSignature.java b/apis/ec2/src/main/java/org/jclouds/ec2/binders/BindS3UploadPolicyAndSignature.java
index bc5b9fd..b49c9c4 100644
--- a/apis/ec2/src/main/java/org/jclouds/ec2/binders/BindS3UploadPolicyAndSignature.java
+++ b/apis/ec2/src/main/java/org/jclouds/ec2/binders/BindS3UploadPolicyAndSignature.java
@@ -21,21 +21,20 @@ import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.io.BaseEncoding.base64;
 
 import javax.inject.Inject;
-import javax.inject.Singleton;
 
-import org.jclouds.aws.filters.FormSigner;
+import org.jclouds.aws.filters.FormSigner.FormSignerV2;
 import org.jclouds.http.HttpRequest;
 import org.jclouds.rest.Binder;
 
 import com.google.common.collect.ImmutableMultimap;
 import com.google.common.collect.ImmutableMultimap.Builder;
 
-@Singleton
+// TODO: see if this still exists in V4
 public class BindS3UploadPolicyAndSignature implements Binder {
-   private final FormSigner signer;
+   private final FormSignerV2 signer;
 
    @Inject
-   BindS3UploadPolicyAndSignature(FormSigner signer) {
+   BindS3UploadPolicyAndSignature(FormSignerV2 signer) {
       this.signer = signer;
    }
 
@@ -49,5 +48,4 @@ public class BindS3UploadPolicyAndSignature implements Binder {
       builder.put("Storage.S3.UploadPolicySignature", signature);
       return (R) request.toBuilder().replaceFormParams(builder.build()).build();
    }
-
 }

http://git-wip-us.apache.org/repos/asf/jclouds/blob/a449b24e/apis/ec2/src/test/java/org/jclouds/ec2/features/BaseEC2ApiTest.java
----------------------------------------------------------------------
diff --git a/apis/ec2/src/test/java/org/jclouds/ec2/features/BaseEC2ApiTest.java b/apis/ec2/src/test/java/org/jclouds/ec2/features/BaseEC2ApiTest.java
index e70f760..be2a732 100644
--- a/apis/ec2/src/test/java/org/jclouds/ec2/features/BaseEC2ApiTest.java
+++ b/apis/ec2/src/test/java/org/jclouds/ec2/features/BaseEC2ApiTest.java
@@ -28,7 +28,7 @@ import javax.inject.Singleton;
 
 import org.jclouds.apis.ApiMetadata;
 import org.jclouds.aws.domain.Region;
-import org.jclouds.aws.filters.FormSigner;
+import org.jclouds.aws.filters.FormSigner.FormSignerV2;
 import org.jclouds.compute.domain.Image;
 import org.jclouds.date.DateService;
 import org.jclouds.ec2.EC2ApiMetadata;
@@ -115,19 +115,19 @@ public abstract class BaseEC2ApiTest<T> extends BaseRestAnnotationProcessingTest
       }
    }
 
-   protected FormSigner filter;
+   protected FormSignerV2 filter;
 
    @Override
    protected void checkFilters(HttpRequest request) {
       assertEquals(request.getFilters().size(), 1);
-      assertEquals(request.getFilters().get(0).getClass(), FormSigner.class);
+      assertEquals(request.getFilters().get(0).getClass(), FormSignerV2.class);
    }
 
    @Override
    @BeforeTest
    protected void setupFactory() throws IOException {
       super.setupFactory();
-      this.filter = injector.getInstance(FormSigner.class);
+      this.filter = injector.getInstance(FormSignerV2.class);
    }
 
    @Override

http://git-wip-us.apache.org/repos/asf/jclouds/blob/a449b24e/apis/sts/src/main/java/org/jclouds/aws/config/FormSigningHttpApiModule.java
----------------------------------------------------------------------
diff --git a/apis/sts/src/main/java/org/jclouds/aws/config/FormSigningHttpApiModule.java b/apis/sts/src/main/java/org/jclouds/aws/config/FormSigningHttpApiModule.java
index 57d695a..5989a29 100644
--- a/apis/sts/src/main/java/org/jclouds/aws/config/FormSigningHttpApiModule.java
+++ b/apis/sts/src/main/java/org/jclouds/aws/config/FormSigningHttpApiModule.java
@@ -23,6 +23,7 @@ import javax.inject.Singleton;
 import org.jclouds.aws.filters.FormSigner;
 import org.jclouds.date.DateService;
 import org.jclouds.date.TimeStamp;
+import org.jclouds.http.HttpRequest;
 import org.jclouds.rest.ConfiguresHttpApi;
 import org.jclouds.rest.RequestSigner;
 
@@ -51,7 +52,18 @@ public abstract class FormSigningHttpApiModule<A> extends AWSHttpApiModule<A> {
    @Provides
    @Singleton
    RequestSigner provideRequestSigner(FormSigner in) {
-      return in;
+      if (in instanceof RequestSigner) {
+         return (RequestSigner) in;
+      }
+      return new RequestSigner() {
+         @Override public String createStringToSign(HttpRequest input) {
+            return null;
+         }
+
+         @Override public String sign(String toSign) {
+            return null;
+         }
+      };
    }
 
 }

http://git-wip-us.apache.org/repos/asf/jclouds/blob/a449b24e/apis/sts/src/main/java/org/jclouds/aws/filters/FormSigner.java
----------------------------------------------------------------------
diff --git a/apis/sts/src/main/java/org/jclouds/aws/filters/FormSigner.java b/apis/sts/src/main/java/org/jclouds/aws/filters/FormSigner.java
index f9aee46..64c3842 100644
--- a/apis/sts/src/main/java/org/jclouds/aws/filters/FormSigner.java
+++ b/apis/sts/src/main/java/org/jclouds/aws/filters/FormSigner.java
@@ -42,7 +42,6 @@ import javax.annotation.Resource;
 import javax.inject.Inject;
 import javax.inject.Named;
 import javax.inject.Provider;
-import javax.inject.Singleton;
 
 import org.jclouds.Constants;
 import org.jclouds.aws.domain.SessionCredentials;
@@ -66,154 +65,145 @@ import com.google.common.collect.Multimap;
 import com.google.common.collect.TreeMultimap;
 import com.google.common.io.ByteProcessor;
 import com.google.common.net.HttpHeaders;
+import com.google.inject.ImplementedBy;
+
+@ImplementedBy(FormSigner.FormSignerV2.class)
+public interface FormSigner extends HttpRequestFilter {
+
+   static final class FormSignerV2 implements FormSigner, RequestSigner {
+
+      public static final Set<String> mandatoryParametersForSignature = ImmutableSet
+            .of(ACTION, SIGNATURE_METHOD, SIGNATURE_VERSION, VERSION);
+
+      private final SignatureWire signatureWire;
+      private final String apiVersion;
+      private final Supplier<Credentials> creds;
+      private final Provider<String> dateService;
+      private final Crypto crypto;
+      private final HttpUtils utils;
+
+      @Resource @Named(Constants.LOGGER_SIGNATURE)
+      private Logger signatureLog = Logger.NULL;
+
+      @Inject FormSignerV2(SignatureWire signatureWire, @ApiVersion String apiVersion,
+            @org.jclouds.location.Provider Supplier<Credentials> creds, @TimeStamp Provider<String> dateService,
+            Crypto crypto, HttpUtils utils) {
+         this.signatureWire = signatureWire;
+         this.apiVersion = apiVersion;
+         this.creds = creds;
+         this.dateService = dateService;
+         this.crypto = crypto;
+         this.utils = utils;
+      }
 
-/**
- * 
- * @see <a href=
- *      "http://docs.amazonwebservices.com/AWSEC2/latest/APIReference/Query-Common-Parameters.html"
- *      />
- */
-@Singleton
-public class FormSigner implements HttpRequestFilter, RequestSigner {
-
-   public static final Set<String> mandatoryParametersForSignature = ImmutableSet.of(ACTION, SIGNATURE_METHOD,
-         SIGNATURE_VERSION, VERSION);
-
-   private final SignatureWire signatureWire;
-   private final String apiVersion;
-   private final Supplier<Credentials> creds;
-   private final Provider<String> dateService;
-   private final Crypto crypto;
-   private final HttpUtils utils;
-
-   @Resource
-   @Named(Constants.LOGGER_SIGNATURE)
-   private Logger signatureLog = Logger.NULL;
-
-   @Inject
-   public FormSigner(SignatureWire signatureWire, @ApiVersion String apiVersion,
-         @org.jclouds.location.Provider Supplier<Credentials> creds, @TimeStamp Provider<String> dateService,
-         Crypto crypto, HttpUtils utils) {
-      this.signatureWire = signatureWire;
-      this.apiVersion = apiVersion;
-      this.creds = creds;
-      this.dateService = dateService;
-      this.crypto = crypto;
-      this.utils = utils;
-   }
-
-   public HttpRequest filter(HttpRequest request) throws HttpException {
-      checkNotNull(request.getFirstHeaderOrNull(HttpHeaders.HOST), "request is not ready to sign; host not present");
-      Multimap<String, String> decodedParams = queryParser().apply(request.getPayload().getRawContent().toString()); 
-      decodedParams.replaceValues(VERSION, ImmutableSet.of(apiVersion));
-      addSigningParams(decodedParams);
-      validateParams(decodedParams);
-      String stringToSign = createStringToSign(request, decodedParams);
-      String signature = sign(stringToSign);
-      addSignature(decodedParams, signature);
-      request = setPayload(request, decodedParams);
-      utils.logRequest(signatureLog, request, "<<");
-      return request;
-   }
-   
-   HttpRequest setPayload(HttpRequest request, Multimap<String, String> decodedParams) {
-      String queryLine = buildQueryLine(decodedParams);
-      request.setPayload(queryLine);
-      request.getPayload().getContentMetadata().setContentType("application/x-www-form-urlencoded");
-      return request;
-   }
+      public HttpRequest filter(HttpRequest request) throws HttpException {
+         checkNotNull(request.getFirstHeaderOrNull(HttpHeaders.HOST), "request is not ready to sign; host not present");
+         Multimap<String, String> decodedParams = queryParser().apply(request.getPayload().getRawContent().toString());
+         decodedParams.replaceValues(VERSION, ImmutableSet.of(apiVersion));
+         addSigningParams(decodedParams);
+         validateParams(decodedParams);
+         String stringToSign = createStringToSign(request, decodedParams);
+         String signature = sign(stringToSign);
+         addSignature(decodedParams, signature);
+         request = setPayload(request, decodedParams);
+         utils.logRequest(signatureLog, request, "<<");
+         return request;
+      }
 
-   private static final Comparator<String> actionFirstAccessKeyLast = new Comparator<String>() {
-      static final int LEFT_IS_GREATER = 1;
-      static final int RIGHT_IS_GREATER = -1;
+      HttpRequest setPayload(HttpRequest request, Multimap<String, String> decodedParams) {
+         String queryLine = buildQueryLine(decodedParams);
+         request.setPayload(queryLine);
+         request.getPayload().getContentMetadata().setContentType("application/x-www-form-urlencoded");
+         return request;
+      }
 
-      @Override
-      public int compare(String left, String right) {
-         if (left == right) {
-            return 0;
-         }
-         if ("Action".equals(right) || "AWSAccessKeyId".equals(left)) {
-            return LEFT_IS_GREATER;
+      private static final Comparator<String> actionFirstAccessKeyLast = new Comparator<String>() {
+         static final int LEFT_IS_GREATER = 1;
+         static final int RIGHT_IS_GREATER = -1;
+
+         @Override
+         public int compare(String left, String right) {
+            if (left == right) {
+               return 0;
+            }
+            if ("Action".equals(right) || "AWSAccessKeyId".equals(left)) {
+               return LEFT_IS_GREATER;
+            }
+            if ("Action".equals(left) || "AWSAccessKeyId".equals(right)) {
+               return RIGHT_IS_GREATER;
+            }
+            return natural().compare(left, right);
          }
-         if ("Action".equals(left) || "AWSAccessKeyId".equals(right)) {
-            return RIGHT_IS_GREATER;
-         }
-         return natural().compare(left, right);
+      };
+
+      private static String buildQueryLine(Multimap<String, String> decodedParams) {
+         Multimap<String, String> sortedParams = TreeMultimap.create(actionFirstAccessKeyLast, natural());
+         sortedParams.putAll(decodedParams);
+         return encodeQueryLine(sortedParams);
       }
-   };
 
-   private static String buildQueryLine(Multimap<String, String> decodedParams) {
-      Multimap<String, String> sortedParams = TreeMultimap.create(actionFirstAccessKeyLast, natural());
-      sortedParams.putAll(decodedParams);
-      return encodeQueryLine(sortedParams);
-   }
+      @VisibleForTesting void validateParams(Multimap<String, String> params) {
+         for (String parameter : mandatoryParametersForSignature) {
+            checkState(params.containsKey(parameter), "parameter " + parameter + " is required for signature");
+         }
+      }
 
-   @VisibleForTesting
-   void validateParams(Multimap<String, String> params) {
-      for (String parameter : mandatoryParametersForSignature) {
-         checkState(params.containsKey(parameter), "parameter " + parameter + " is required for signature");
+      @VisibleForTesting void addSignature(Multimap<String, String> params, String signature) {
+         params.replaceValues(SIGNATURE, ImmutableList.of(signature));
       }
-   }
 
-   @VisibleForTesting
-   void addSignature(Multimap<String, String> params, String signature) {
-      params.replaceValues(SIGNATURE, ImmutableList.of(signature));
-   }
+      @VisibleForTesting
+      public String sign(String toSign) {
+         String signature;
+         try {
+            ByteProcessor<byte[]> hmacSHA256 = asByteProcessor(
+                  crypto.hmacSHA256(creds.get().credential.getBytes(UTF_8)));
+            signature = base64().encode(readBytes(toInputStream(toSign), hmacSHA256));
+            if (signatureWire.enabled())
+               signatureWire.input(toInputStream(signature));
+         } catch (Exception e) {
+            throw new HttpException("error signing request", e);
+         }
+         return signature;
+      }
 
-   @VisibleForTesting
-   public String sign(String toSign) {
-      String signature;
-      try {
-         ByteProcessor<byte[]> hmacSHA256 = asByteProcessor(crypto.hmacSHA256(creds.get().credential.getBytes(UTF_8)));
-         signature = base64().encode(readBytes(toInputStream(toSign), hmacSHA256));
+      @VisibleForTesting
+      public String createStringToSign(HttpRequest request, Multimap<String, String> decodedParams) {
+         utils.logRequest(signatureLog, request, ">>");
+         StringBuilder stringToSign = new StringBuilder();
+         // StringToSign = HTTPVerb + "\n" +
+         stringToSign.append(request.getMethod()).append("\n");
+         // ValueOfHostHeaderInLowercase + "\n" +
+         stringToSign.append(request.getFirstHeaderOrNull(HttpHeaders.HOST).toLowerCase()).append("\n");
+         // HTTPRequestURI + "\n" +
+         stringToSign.append(request.getEndpoint().getPath()).append("\n");
+         // CanonicalizedFormString <from the preceding step>
+         stringToSign.append(buildCanonicalizedString(decodedParams));
          if (signatureWire.enabled())
-            signatureWire.input(toInputStream(signature));
-      } catch (Exception e) {
-         throw new HttpException("error signing request", e);
+            signatureWire.output(stringToSign.toString());
+         return stringToSign.toString();
       }
-      return signature;
-   }
-
-   @VisibleForTesting
-   public String createStringToSign(HttpRequest request, Multimap<String, String> decodedParams) {
-      utils.logRequest(signatureLog, request, ">>");
-      StringBuilder stringToSign = new StringBuilder();
-      // StringToSign = HTTPVerb + "\n" +
-      stringToSign.append(request.getMethod()).append("\n");
-      // ValueOfHostHeaderInLowercase + "\n" +
-      stringToSign.append(request.getFirstHeaderOrNull(HttpHeaders.HOST).toLowerCase()).append("\n");
-      // HTTPRequestURI + "\n" +
-      stringToSign.append(request.getEndpoint().getPath()).append("\n");
-      // CanonicalizedFormString <from the preceding step>
-      stringToSign.append(buildCanonicalizedString(decodedParams));
-      if (signatureWire.enabled())
-         signatureWire.output(stringToSign.toString());
-      return stringToSign.toString();
-   }
-
-   @VisibleForTesting
-   String buildCanonicalizedString(Multimap<String, String> decodedParams) {
-      // note that aws wants to percent encode the canonicalized string without skipping '/' and '?'
-      return encodeQueryLine(TreeMultimap.create(decodedParams), ImmutableList.<Character> of());
-   }
 
+      @VisibleForTesting String buildCanonicalizedString(Multimap<String, String> decodedParams) {
+         // note that aws wants to percent encode the canonicalized string without skipping '/' and '?'
+         return encodeQueryLine(TreeMultimap.create(decodedParams), ImmutableList.<Character>of());
+      }
 
-   @VisibleForTesting
-   void addSigningParams(Multimap<String, String> params) {
-      params.removeAll(SIGNATURE);
-      params.removeAll(SECURITY_TOKEN);
-      Credentials current = creds.get();
-      if (current instanceof SessionCredentials) {
-         params.put(SECURITY_TOKEN, SessionCredentials.class.cast(current).getSessionToken());
+      @VisibleForTesting void addSigningParams(Multimap<String, String> params) {
+         params.removeAll(SIGNATURE);
+         params.removeAll(SECURITY_TOKEN);
+         Credentials current = creds.get();
+         if (current instanceof SessionCredentials) {
+            params.put(SECURITY_TOKEN, SessionCredentials.class.cast(current).getSessionToken());
+         }
+         params.replaceValues(SIGNATURE_METHOD, ImmutableList.of("HmacSHA256"));
+         params.replaceValues(SIGNATURE_VERSION, ImmutableList.of("2"));
+         params.replaceValues(TIMESTAMP, ImmutableList.of(dateService.get()));
+         params.replaceValues(AWS_ACCESS_KEY_ID, ImmutableList.of(creds.get().identity));
       }
-      params.replaceValues(SIGNATURE_METHOD, ImmutableList.of("HmacSHA256"));
-      params.replaceValues(SIGNATURE_VERSION, ImmutableList.of("2"));
-      params.replaceValues(TIMESTAMP, ImmutableList.of(dateService.get()));
-      params.replaceValues(AWS_ACCESS_KEY_ID, ImmutableList.of(creds.get().identity));
-   }
 
-   public String createStringToSign(HttpRequest input) {
-      return createStringToSign(input, queryParser().apply(input.getPayload().getRawContent().toString()));
+      public String createStringToSign(HttpRequest input) {
+         return createStringToSign(input, queryParser().apply(input.getPayload().getRawContent().toString()));
+      }
    }
-
 }

http://git-wip-us.apache.org/repos/asf/jclouds/blob/a449b24e/apis/sts/src/main/java/org/jclouds/aws/filters/FormSignerV4.java
----------------------------------------------------------------------
diff --git a/apis/sts/src/main/java/org/jclouds/aws/filters/FormSignerV4.java b/apis/sts/src/main/java/org/jclouds/aws/filters/FormSignerV4.java
new file mode 100644
index 0000000..2743c3d
--- /dev/null
+++ b/apis/sts/src/main/java/org/jclouds/aws/filters/FormSignerV4.java
@@ -0,0 +1,219 @@
+/*
+ * 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.aws.filters;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import static com.google.common.hash.Hashing.sha256;
+import static com.google.common.io.BaseEncoding.base16;
+import static com.google.common.net.HttpHeaders.AUTHORIZATION;
+import static com.google.common.net.HttpHeaders.HOST;
+import static javax.ws.rs.core.MediaType.APPLICATION_FORM_URLENCODED;
+import static org.jclouds.aws.reference.FormParameters.ACTION;
+import static org.jclouds.aws.reference.FormParameters.VERSION;
+
+import java.net.URI;
+import java.security.GeneralSecurityException;
+import java.util.List;
+import java.util.Map;
+
+import javax.crypto.Mac;
+import javax.crypto.spec.SecretKeySpec;
+import javax.inject.Inject;
+
+import org.jclouds.aws.domain.SessionCredentials;
+import org.jclouds.date.TimeStamp;
+import org.jclouds.domain.Credentials;
+import org.jclouds.http.HttpException;
+import org.jclouds.http.HttpRequest;
+import org.jclouds.io.Payload;
+import org.jclouds.io.Payloads;
+import org.jclouds.location.Provider;
+import org.jclouds.providers.ProviderMetadata;
+import org.jclouds.rest.annotations.ApiVersion;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Splitter;
+import com.google.common.base.Supplier;
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.ImplementedBy;
+
+public final class FormSignerV4 implements FormSigner {
+
+   // Specifying a default for how to parse the service and region in this way allows
+   // tests or other downstream services to not have to use guice overrides.
+   @ImplementedBy(ServiceAndRegion.AWSServiceAndRegion.class)
+   public interface ServiceAndRegion {
+      String service();
+      String region(String host);
+
+      static final class AWSServiceAndRegion implements ServiceAndRegion {
+         private final String service;
+
+         @Inject AWSServiceAndRegion(ProviderMetadata provider) {
+            this(provider.getEndpoint());
+         }
+
+         AWSServiceAndRegion(String endpoint) {
+            this.service = parseServiceAndRegion(URI.create(checkNotNull(endpoint, "endpoint")).getHost()).get(0);
+         }
+
+         @Override public String service() {
+            return service;
+         }
+
+         @Override public String region(String host) {
+            return parseServiceAndRegion(host).get(1);
+         }
+
+         /** This will only work for amazon deployments, and perhaps not all of them. */
+         private static List<String> parseServiceAndRegion(String host) {
+            checkArgument(host.endsWith(".amazonaws.com"), "Only AWS endpoints currently supported %s", host);
+            return Splitter.on('.').splitToList(host);
+         }
+      }
+   }
+
+   private final String apiVersion;
+   private final Supplier<Credentials> creds;
+   private final javax.inject.Provider<String> iso8601Timestamp;
+   private final ServiceAndRegion serviceAndRegion;
+
+   @Inject FormSignerV4(@ApiVersion String apiVersion, @Provider Supplier<Credentials> creds,
+         @TimeStamp javax.inject.Provider<String> iso8601Timestamp, ServiceAndRegion serviceAndRegion) {
+      this.apiVersion = apiVersion;
+      this.creds = creds;
+      this.iso8601Timestamp = iso8601Timestamp;
+      this.serviceAndRegion = serviceAndRegion;
+   }
+
+   @Override public HttpRequest filter(HttpRequest request) throws HttpException {
+      checkArgument(request.getHeaders().containsKey(HOST), "request is not ready to sign; host not present");
+      String host = request.getFirstHeaderOrNull(HOST);
+      String form = request.getPayload().getRawContent().toString();
+      checkArgument(form.indexOf(ACTION) != -1, "request is not ready to sign; Action not present %s", form);
+
+      String timestamp = iso8601Timestamp.get();
+      String datestamp = timestamp.substring(0, 8);
+
+      String service = serviceAndRegion.service();
+      String region = serviceAndRegion.region(host);
+      String credentialScope = Joiner.on('/').join(datestamp, region, service, "aws4_request");
+
+      // content-type is not a required signing param. However, examples use this, so we include it to ease testing.
+      ImmutableMap.Builder<String, String> signedHeadersBuilder = ImmutableMap.<String, String> builder() //
+            .put("content-type", request.getPayload().getContentMetadata().getContentType()) //
+            .put("host", host) //
+            .put("x-amz-date", timestamp);
+
+      HttpRequest.Builder<?> requestBuilder = request.toBuilder() //
+            .removeHeader(AUTHORIZATION) //
+            .replaceHeader("X-Amz-Date", timestamp);
+
+      if (form.indexOf(VERSION) == -1) {
+         requestBuilder.addFormParam("Version", apiVersion);
+      }
+
+      Credentials credentials = creds.get();
+
+      if (credentials instanceof SessionCredentials) {
+         String token = SessionCredentials.class.cast(credentials).getSessionToken();
+         requestBuilder.replaceHeader("X-Amz-Security-Token", token);
+         signedHeadersBuilder.put("x-amz-security-token", token);
+      }
+
+      ImmutableMap<String, String> signedHeaders = signedHeadersBuilder.build();
+
+      String stringToSign = createStringToSign(requestBuilder.build(), signedHeaders, credentialScope);
+      byte[] signatureKey = signatureKey(credentials.credential, datestamp, region, service);
+      String signature = base16().lowerCase().encode(hmacSHA256(stringToSign, signatureKey));
+
+      StringBuilder authorization = new StringBuilder("AWS4-HMAC-SHA256 ");
+      authorization.append("Credential=").append(credentials.identity).append('/').append(credentialScope).append(", ");
+      authorization.append("SignedHeaders=").append(Joiner.on(';').join(signedHeaders.keySet())).append(", ");
+      authorization.append("Signature=").append(signature);
+
+      return requestBuilder.addHeader(AUTHORIZATION, authorization.toString()).build();
+   }
+
+   // TODO: change EC2 apis to add this themselves with @FormParams
+   private Payload addVersionIfNecessary(Payload payload, String form) {
+      if (form.indexOf(VERSION) == -1) {
+         form += "&Version=" + apiVersion;
+         payload = Payloads.newStringPayload(form);
+         payload.getContentMetadata().setContentType(APPLICATION_FORM_URLENCODED);
+         payload.getContentMetadata().setContentLength((long) form.length());
+      }
+      return payload;
+   }
+
+   static byte[] signatureKey(String secretKey, String datestamp, String region, String service) {
+      byte[] kSecret = ("AWS4" + secretKey).getBytes(UTF_8);
+      byte[] kDate = hmacSHA256(datestamp, kSecret);
+      byte[] kRegion = hmacSHA256(region, kDate);
+      byte[] kService = hmacSHA256(service, kRegion);
+      byte[] kSigning = hmacSHA256("aws4_request", kService);
+      return kSigning;
+   }
+
+   static byte[] hmacSHA256(String data, byte[] key) {
+      try {
+         String algorithm = "HmacSHA256";
+         Mac mac = Mac.getInstance(algorithm);
+         mac.init(new SecretKeySpec(key, algorithm));
+         return mac.doFinal(data.getBytes(UTF_8));
+      } catch (GeneralSecurityException e) {
+         throw new HttpException(e);
+      }
+   }
+
+   static String createStringToSign(HttpRequest request, Map<String, String> signedHeaders, String credentialScope) {
+      StringBuilder canonicalRequest = new StringBuilder();
+      // HTTPRequestMethod + '\n' +
+      canonicalRequest.append(request.getMethod()).append("\n");
+      // CanonicalURI + '\n' +
+      canonicalRequest.append(request.getEndpoint().getPath()).append("\n");
+      // CanonicalQueryString + '\n' +
+      checkArgument(request.getEndpoint().getQuery() == null, "Query parameters not yet supported %s", request);
+      canonicalRequest.append("\n");
+      // CanonicalHeaders + '\n' +
+      for (Map.Entry<String, String> entry : signedHeaders.entrySet()) {
+         canonicalRequest.append(entry.getKey()).append(':').append(entry.getValue()).append('\n');
+      }
+      canonicalRequest.append("\n");
+
+      // SignedHeaders + '\n' +
+      canonicalRequest.append(Joiner.on(';').join(signedHeaders.keySet())).append('\n');
+
+      // HexEncode(Hash(Payload))
+      String payload = request.getPayload().getRawContent().toString();
+      canonicalRequest.append(base16().lowerCase().encode(sha256().hashString(payload, UTF_8).asBytes()));
+
+      StringBuilder toSign = new StringBuilder();
+      // Algorithm + '\n' +
+      toSign.append("AWS4-HMAC-SHA256").append('\n');
+      // RequestDate + '\n' +
+      toSign.append(signedHeaders.get("x-amz-date")).append('\n');
+      // CredentialScope + '\n' +
+      toSign.append(credentialScope).append('\n');
+      // HexEncode(Hash(CanonicalRequest))
+      toSign.append(base16().lowerCase().encode(sha256().hashString(canonicalRequest.toString(), UTF_8).asBytes()));
+
+      return toSign.toString();
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds/blob/a449b24e/apis/sts/src/test/java/org/jclouds/aws/filters/FormSignerTest.java
----------------------------------------------------------------------
diff --git a/apis/sts/src/test/java/org/jclouds/aws/filters/FormSignerTest.java b/apis/sts/src/test/java/org/jclouds/aws/filters/FormSignerTest.java
deleted file mode 100644
index 2325751..0000000
--- a/apis/sts/src/test/java/org/jclouds/aws/filters/FormSignerTest.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * 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.aws.filters;
-
-import static javax.ws.rs.HttpMethod.GET;
-import static org.jclouds.aws.reference.AWSConstants.PROPERTY_HEADER_TAG;
-import static org.jclouds.providers.AnonymousProviderMetadata.forApiOnEndpoint;
-import static org.testng.Assert.assertEquals;
-
-import org.jclouds.ContextBuilder;
-import org.jclouds.aws.xml.SessionCredentialsHandlerTest;
-import org.jclouds.date.TimeStamp;
-import org.jclouds.domain.Credentials;
-import org.jclouds.http.HttpRequest;
-import org.jclouds.http.IntegrationTestClient;
-import org.jclouds.logging.config.NullLoggingModule;
-import org.jclouds.rest.RequestSigner;
-import org.jclouds.rest.internal.BaseRestApiTest.MockModule;
-import org.testng.annotations.Test;
-
-import com.google.common.base.Suppliers;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMultimap;
-import com.google.common.net.HttpHeaders;
-import com.google.inject.AbstractModule;
-import com.google.inject.Injector;
-import com.google.inject.Module;
-import com.google.inject.name.Names;
-/**
- * Tests behavior of {@code FormSigner}
- */
-// NOTE:without testName, this will not call @Before* and fail w/NPE during
-// surefire
-@Test(groups = "unit", singleThreaded = true, testName = "FormSignerTest")
-public class FormSignerTest {
-   public static Injector injector(Credentials creds) {
-      return ContextBuilder
-            .newBuilder(forApiOnEndpoint(IntegrationTestClient.class, "http://localhost"))
-            .credentialsSupplier(Suppliers.<Credentials> ofInstance(creds)).apiVersion("apiVersion")
-            .modules(ImmutableList.<Module> of(new MockModule(), new NullLoggingModule(), new AbstractModule() {
-               @Override
-               protected void configure() {
-                  bind(RequestSigner.class).to(FormSigner.class);
-                  bind(String.class).annotatedWith(Names.named(PROPERTY_HEADER_TAG)).toInstance("amz");
-                  bind(String.class).annotatedWith(TimeStamp.class).toInstance("2009-11-08T15:54:08.897Z");
-               }
-
-            })).buildInjector();
-   }
-
-   public static FormSigner filter(Credentials creds) {
-      return injector(creds).getInstance(FormSigner.class);
-   }
-
-   public static FormSigner staticCredentialsFilter = filter(new Credentials("identity", "credential"));
-
-   HttpRequest request = HttpRequest.builder().method(GET)
-         .endpoint("http://localhost")
-         .addHeader(HttpHeaders.HOST, "localhost")
-         .addFormParam("Action", "DescribeImages")
-         .addFormParam("ImageId.1", "ami-2bb65342").build();
-
-   @Test
-   void testAddsSecurityToken() {
-      HttpRequest filtered = filter(new SessionCredentialsHandlerTest().expected()).filter(request);
-      assertEquals(
-            filtered.getPayload().getRawContent(),
-            "Action=DescribeImages&ImageId.1=ami-2bb65342&SecurityToken=AQoEXAMPLEH4aoAH0gNCAPyJxz4BlCFFxWNE1OPTgk5TthT&Signature=/8ReFVH1tvyNORsJb%2BSBieT9zvdqREQQr/olwmxC7VY%3D&SignatureMethod=HmacSHA256&SignatureVersion=2&Timestamp=2009-11-08T15%3A54%3A08.897Z&Version=apiVersion&AWSAccessKeyId=AKIAIOSFODNN7EXAMPLE");
-   }
-
-   @Test
-   void testBuildCanonicalizedStringSetsVersion() {
-      HttpRequest filtered = staticCredentialsFilter.filter(request);
-      assertEquals(filtered.getPayload().getRawContent(),
-            "Action=DescribeImages&ImageId.1=ami-2bb65342&Signature=ugnt4m2eHE7Ka/vXTr9EhKZq7bhxOfvW0y4pAEqF97w%3D&SignatureMethod=HmacSHA256&SignatureVersion=2&Timestamp=2009-11-08T15%3A54%3A08.897Z&Version=apiVersion&AWSAccessKeyId=identity");
-   }
-
-   @Test
-   void testBuildCanonicalizedString() {
-      assertEquals(
-            staticCredentialsFilter.buildCanonicalizedString(new ImmutableMultimap.Builder<String, String>()
-                  .put("AWSAccessKeyId", "foo").put("Action", "DescribeImages").put("Expires", "2008-02-10T12:00:00Z")
-                  .put("ImageId.1", "ami-2bb65342").put("SignatureMethod", "HmacSHA256").put("SignatureVersion", "2")
-                  .put("Version", "2010-06-15").build()),
-            "AWSAccessKeyId=foo&Action=DescribeImages&Expires=2008-02-10T12%3A00%3A00Z&ImageId.1=ami-2bb65342&SignatureMethod=HmacSHA256&SignatureVersion=2&Version=2010-06-15");
-   }
-
-}

http://git-wip-us.apache.org/repos/asf/jclouds/blob/a449b24e/apis/sts/src/test/java/org/jclouds/aws/filters/FormSignerV2Test.java
----------------------------------------------------------------------
diff --git a/apis/sts/src/test/java/org/jclouds/aws/filters/FormSignerV2Test.java b/apis/sts/src/test/java/org/jclouds/aws/filters/FormSignerV2Test.java
new file mode 100644
index 0000000..71fa8f9
--- /dev/null
+++ b/apis/sts/src/test/java/org/jclouds/aws/filters/FormSignerV2Test.java
@@ -0,0 +1,98 @@
+/*
+ * 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.aws.filters;
+
+import static javax.ws.rs.HttpMethod.GET;
+import static org.jclouds.aws.reference.AWSConstants.PROPERTY_HEADER_TAG;
+import static org.jclouds.providers.AnonymousProviderMetadata.forApiOnEndpoint;
+import static org.testng.Assert.assertEquals;
+
+import org.jclouds.ContextBuilder;
+import org.jclouds.aws.filters.FormSigner.FormSignerV2;
+import org.jclouds.aws.xml.SessionCredentialsHandlerTest;
+import org.jclouds.date.TimeStamp;
+import org.jclouds.domain.Credentials;
+import org.jclouds.http.HttpRequest;
+import org.jclouds.http.IntegrationTestClient;
+import org.jclouds.logging.config.NullLoggingModule;
+import org.jclouds.rest.RequestSigner;
+import org.jclouds.rest.internal.BaseRestApiTest.MockModule;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Suppliers;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMultimap;
+import com.google.common.net.HttpHeaders;
+import com.google.inject.AbstractModule;
+import com.google.inject.Injector;
+import com.google.inject.Module;
+import com.google.inject.name.Names;
+
+@Test(groups = "unit", singleThreaded = true, testName = "FormSignerV2Test")
+public class FormSignerV2Test {
+   public static Injector injector(Credentials creds) {
+      return ContextBuilder
+            .newBuilder(forApiOnEndpoint(IntegrationTestClient.class, "http://localhost"))
+            .credentialsSupplier(Suppliers.<Credentials> ofInstance(creds)).apiVersion("apiVersion")
+            .modules(ImmutableList.<Module> of(new MockModule(), new NullLoggingModule(), new AbstractModule() {
+               @Override
+               protected void configure() {
+                  bind(RequestSigner.class).to(FormSignerV2.class);
+                  bind(String.class).annotatedWith(Names.named(PROPERTY_HEADER_TAG)).toInstance("amz");
+                  bind(String.class).annotatedWith(TimeStamp.class).toInstance("2009-11-08T15:54:08.897Z");
+               }
+
+            })).buildInjector();
+   }
+
+   public static FormSignerV2 filter(Credentials creds) {
+      return injector(creds).getInstance(FormSignerV2.class);
+   }
+
+   public static FormSignerV2 staticCredentialsFilter = filter(new Credentials("identity", "credential"));
+
+   HttpRequest request = HttpRequest.builder().method(GET)
+         .endpoint("http://localhost")
+         .addHeader(HttpHeaders.HOST, "localhost")
+         .addFormParam("Action", "DescribeImages")
+         .addFormParam("ImageId.1", "ami-2bb65342").build();
+
+   @Test
+   void testAddsSecurityToken() {
+      HttpRequest filtered = filter(new SessionCredentialsHandlerTest().expected()).filter(request);
+      assertEquals(
+            filtered.getPayload().getRawContent(),
+            "Action=DescribeImages&ImageId.1=ami-2bb65342&SecurityToken=AQoEXAMPLEH4aoAH0gNCAPyJxz4BlCFFxWNE1OPTgk5TthT&Signature=/8ReFVH1tvyNORsJb%2BSBieT9zvdqREQQr/olwmxC7VY%3D&SignatureMethod=HmacSHA256&SignatureVersion=2&Timestamp=2009-11-08T15%3A54%3A08.897Z&Version=apiVersion&AWSAccessKeyId=AKIAIOSFODNN7EXAMPLE");
+   }
+
+   @Test
+   void testBuildCanonicalizedStringSetsVersion() {
+      HttpRequest filtered = staticCredentialsFilter.filter(request);
+      assertEquals(filtered.getPayload().getRawContent(),
+            "Action=DescribeImages&ImageId.1=ami-2bb65342&Signature=ugnt4m2eHE7Ka/vXTr9EhKZq7bhxOfvW0y4pAEqF97w%3D&SignatureMethod=HmacSHA256&SignatureVersion=2&Timestamp=2009-11-08T15%3A54%3A08.897Z&Version=apiVersion&AWSAccessKeyId=identity");
+   }
+
+   @Test
+   void testBuildCanonicalizedString() {
+      assertEquals(
+            staticCredentialsFilter.buildCanonicalizedString(new ImmutableMultimap.Builder<String, String>()
+                  .put("AWSAccessKeyId", "foo").put("Action", "DescribeImages").put("Expires", "2008-02-10T12:00:00Z")
+                  .put("ImageId.1", "ami-2bb65342").put("SignatureMethod", "HmacSHA256").put("SignatureVersion", "2")
+                  .put("Version", "2010-06-15").build()),
+            "AWSAccessKeyId=foo&Action=DescribeImages&Expires=2008-02-10T12%3A00%3A00Z&ImageId.1=ami-2bb65342&SignatureMethod=HmacSHA256&SignatureVersion=2&Version=2010-06-15");
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds/blob/a449b24e/apis/sts/src/test/java/org/jclouds/aws/filters/FormSignerV4LiveTest.java
----------------------------------------------------------------------
diff --git a/apis/sts/src/test/java/org/jclouds/aws/filters/FormSignerV4LiveTest.java b/apis/sts/src/test/java/org/jclouds/aws/filters/FormSignerV4LiveTest.java
new file mode 100644
index 0000000..bd697bd
--- /dev/null
+++ b/apis/sts/src/test/java/org/jclouds/aws/filters/FormSignerV4LiveTest.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.aws.filters;
+
+import static java.util.concurrent.TimeUnit.MINUTES;
+import static org.jclouds.aws.filters.FormSignerV4.ServiceAndRegion;
+import static org.jclouds.aws.filters.FormSignerV4.ServiceAndRegion.AWSServiceAndRegion;
+import static org.jclouds.sts.options.SessionCredentialsOptions.Builder.durationSeconds;
+import static org.testng.Assert.assertEquals;
+
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.TimeZone;
+
+import javax.inject.Provider;
+
+import org.jclouds.apis.BaseApiLiveTest;
+import org.jclouds.aws.domain.SessionCredentials;
+import org.jclouds.domain.Credentials;
+import org.jclouds.http.HttpRequest;
+import org.jclouds.rest.ApiContext;
+import org.jclouds.sts.STSApi;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
+
+@Test(groups = "live", singleThreaded = true, testName = "FormSignerV4LiveTest")
+public class FormSignerV4LiveTest extends BaseApiLiveTest<ApiContext<STSApi>> {
+
+   /** Example request, which hopefully the test user's account has access to! */
+   private final HttpRequest sampleRequest = HttpRequest.builder() //
+         .method("POST") //
+         .endpoint("https://ec2.us-east-1.amazonaws.com/") //
+         .addHeader("Host", "ec2.us-east-1.amazonaws.com") //
+         .addFormParam("Action", "DescribeRegions") //
+         .addFormParam("Version", "2010-08-31") //
+         .build();
+
+   /** Provides the expected iso8601 timestamp format for signature v4. */
+   private final Provider<String> timestamp = new Provider<String>() {
+      SimpleDateFormat iso8601 = new SimpleDateFormat("yyyyMMdd'T'HHmmss'Z'");
+
+      @Override public String get() {
+         iso8601.setTimeZone(TimeZone.getTimeZone("GMT"));
+         return iso8601.format(new Date());
+      }
+   };
+
+   /** Test how we parse the AWS service and region based on an endpoint. */
+   private final ServiceAndRegion serviceAndRegion = new AWSServiceAndRegion(sampleRequest.getEndpoint().toString());
+
+   public FormSignerV4LiveTest() {
+      provider = "sts";
+   }
+
+   public void signatureV4() {
+      Supplier<Credentials> accessAndSecretKey = Suppliers.ofInstance(new Credentials(identity, credential));
+
+      FormSignerV4 filter = new FormSignerV4(apiVersion, accessAndSecretKey, timestamp, serviceAndRegion);
+
+      HttpRequest request = filter.filter(sampleRequest);
+
+      assertEquals(api.utils().http().invoke(request).getStatusCode(), 200);
+   }
+
+   public void signatureV4_session() {
+      SessionCredentials creds = api.getApi().createTemporaryCredentials(durationSeconds(MINUTES.toSeconds(15)));
+      Supplier<Credentials> sessionToken = Suppliers.<Credentials>ofInstance(creds);
+
+      FormSignerV4 filter = new FormSignerV4(apiVersion, sessionToken, timestamp, serviceAndRegion);
+
+      HttpRequest request = filter.filter(sampleRequest);
+
+      assertEquals(api.utils().http().invoke(request).getStatusCode(), 200);
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds/blob/a449b24e/apis/sts/src/test/java/org/jclouds/aws/filters/FormSignerV4Test.java
----------------------------------------------------------------------
diff --git a/apis/sts/src/test/java/org/jclouds/aws/filters/FormSignerV4Test.java b/apis/sts/src/test/java/org/jclouds/aws/filters/FormSignerV4Test.java
new file mode 100644
index 0000000..2f94c26
--- /dev/null
+++ b/apis/sts/src/test/java/org/jclouds/aws/filters/FormSignerV4Test.java
@@ -0,0 +1,102 @@
+/*
+ * 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.aws.filters;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.testng.Assert.assertEquals;
+
+import javax.inject.Provider;
+
+import org.jclouds.aws.domain.SessionCredentials;
+import org.jclouds.aws.filters.FormSignerV4.ServiceAndRegion;
+import org.jclouds.aws.xml.SessionCredentialsHandlerTest;
+import org.jclouds.domain.Credentials;
+import org.jclouds.http.HttpRequest;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
+
+/**
+ * Using samples from <a href="http://docs.aws.amazon.com/general/latest/gr/sigv4-create-canonical-request.html">Amazon
+ * docs</a>
+ */
+@Test
+public class FormSignerV4Test {
+
+   String apiVersion = "2010-05-08";
+
+   Supplier<Credentials> accessAndSecretKey = Suppliers
+         .ofInstance(new Credentials("AKIAIOSFODNN7EXAMPLE", "wJalrXUtnFEMI/K7MDENG+bPxRfiCYEXAMPLEKEY"));
+
+   Provider<String> timestamp = new Provider<String>() {
+      @Override public String get() {
+         return "20110909T233600Z";
+      }
+   };
+
+   ServiceAndRegion serviceAndRegion = new ServiceAndRegion() {
+      @Override public String service() {
+         return "iam";
+      }
+
+      @Override public String region(String host) {
+         return "us-east-1";
+      }
+   };
+
+   public void signSampleRequest() {
+      HttpRequest request = HttpRequest.builder() //
+            .method("POST") //
+            .endpoint("https://iam.amazonaws.com/") //
+            .addHeader("Host", "iam.amazonaws.com") //
+            .payload("Action=ListUsers&Version=2010-05-08")
+            .build();
+
+      request.getPayload().getContentMetadata().setContentType("application/x-www-form-urlencoded; charset=utf-8");
+
+      FormSignerV4 filter = new FormSignerV4(apiVersion, accessAndSecretKey, timestamp, serviceAndRegion);
+
+      HttpRequest filtered = filter.filter(request);
+
+      assertEquals(filtered.getFirstHeaderOrNull("X-Amz-Date"), timestamp.get());
+
+      String sampleSignature = "ced6826de92d2bdeed8f846f0bf508e8559e98e4b0199114b84c54174deb456c";
+
+      assertThat(filtered.getFirstHeaderOrNull("Authorization")).endsWith("Signature=" + sampleSignature);
+   }
+
+   public void sessionTokenRequest() {
+      HttpRequest request = HttpRequest.builder() //
+            .method("POST") //
+            .endpoint("https://iam.amazonaws.com/") //
+            .addHeader("Host", "iam.amazonaws.com") //
+            .payload("Action=ListUsers&Version=2010-05-08").build();
+
+      request.getPayload().getContentMetadata().setContentType("application/x-www-form-urlencoded; charset=utf-8");
+
+      SessionCredentials sessionCredentials = new SessionCredentialsHandlerTest().expected();
+
+      FormSignerV4 filter = new FormSignerV4(apiVersion, Suppliers.<Credentials>ofInstance(sessionCredentials),
+            timestamp, serviceAndRegion);
+
+      HttpRequest filtered = filter.filter(request);
+
+      assertEquals(filtered.getFirstHeaderOrNull("X-Amz-Date"), timestamp.get());
+      assertEquals(filtered.getFirstHeaderOrNull("X-Amz-Security-Token"), sessionCredentials.getSessionToken());
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds/blob/a449b24e/apis/sts/src/test/java/org/jclouds/aws/util/AWSUtilsTest.java
----------------------------------------------------------------------
diff --git a/apis/sts/src/test/java/org/jclouds/aws/util/AWSUtilsTest.java b/apis/sts/src/test/java/org/jclouds/aws/util/AWSUtilsTest.java
index 8c80c32..33014d4 100644
--- a/apis/sts/src/test/java/org/jclouds/aws/util/AWSUtilsTest.java
+++ b/apis/sts/src/test/java/org/jclouds/aws/util/AWSUtilsTest.java
@@ -27,7 +27,7 @@ import java.io.IOException;
 import java.io.InputStream;
 
 import org.jclouds.aws.domain.AWSError;
-import org.jclouds.aws.filters.FormSignerTest;
+import org.jclouds.aws.filters.FormSignerV2Test;
 import org.jclouds.domain.Credentials;
 import org.jclouds.http.HttpCommand;
 import org.jclouds.http.HttpRequest;
@@ -47,7 +47,7 @@ public class AWSUtilsTest {
    @BeforeTest
    protected void setUpInjector() throws IOException {
 
-      utils = FormSignerTest.injector(new Credentials("identity", "credential")).getInstance(AWSUtils.class);
+      utils = FormSignerV2Test.injector(new Credentials("identity", "credential")).getInstance(AWSUtils.class);
 
       command = createMock(HttpCommand.class);
       expect(command.getCurrentRequest()).andReturn(createMock(HttpRequest.class)).atLeastOnce();

http://git-wip-us.apache.org/repos/asf/jclouds/blob/a449b24e/apis/sts/src/test/java/org/jclouds/aws/xml/SessionCredentialsHandlerTest.java
----------------------------------------------------------------------
diff --git a/apis/sts/src/test/java/org/jclouds/aws/xml/SessionCredentialsHandlerTest.java b/apis/sts/src/test/java/org/jclouds/aws/xml/SessionCredentialsHandlerTest.java
index a615046..cac60a4 100644
--- a/apis/sts/src/test/java/org/jclouds/aws/xml/SessionCredentialsHandlerTest.java
+++ b/apis/sts/src/test/java/org/jclouds/aws/xml/SessionCredentialsHandlerTest.java
@@ -51,5 +51,4 @@ public class SessionCredentialsHandlerTest extends BaseHandlerTest {
             .sessionToken("AQoEXAMPLEH4aoAH0gNCAPyJxz4BlCFFxWNE1OPTgk5TthT")
             .expiration(new SimpleDateFormatDateService().iso8601DateParse("2011-07-11T19:55:29.611Z")).build();
    }
-
 }

http://git-wip-us.apache.org/repos/asf/jclouds/blob/a449b24e/core/src/main/java/org/jclouds/rest/RequestSigner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/jclouds/rest/RequestSigner.java b/core/src/main/java/org/jclouds/rest/RequestSigner.java
index 5673684..7bff9e0 100644
--- a/core/src/main/java/org/jclouds/rest/RequestSigner.java
+++ b/core/src/main/java/org/jclouds/rest/RequestSigner.java
@@ -18,6 +18,11 @@ package org.jclouds.rest;
 
 import org.jclouds.http.HttpRequest;
 
+/**
+ * @deprecated This is an internal interface historically used to debug signature logic. It currently is a broken
+ * abstraction as AWS Signature v4 requires multiple parameters to sign a request.
+ */
+@Deprecated
 public interface RequestSigner {
 
    String createStringToSign(HttpRequest input);

http://git-wip-us.apache.org/repos/asf/jclouds/blob/a449b24e/core/src/main/java/org/jclouds/rest/config/RestModule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/jclouds/rest/config/RestModule.java b/core/src/main/java/org/jclouds/rest/config/RestModule.java
index 3544300..f2b428e 100644
--- a/core/src/main/java/org/jclouds/rest/config/RestModule.java
+++ b/core/src/main/java/org/jclouds/rest/config/RestModule.java
@@ -40,16 +40,11 @@ import org.jclouds.rest.internal.TransformerForRequest;
 
 import com.google.common.base.Function;
 import com.google.common.base.Predicate;
-import com.google.common.base.Supplier;
 import com.google.inject.AbstractModule;
 import com.google.inject.TypeLiteral;
 import com.google.inject.assistedinject.FactoryModuleBuilder;
 
 public class RestModule extends AbstractModule {
-
-   public static final TypeLiteral<Supplier<URI>> URI_SUPPLIER_TYPE = new TypeLiteral<Supplier<URI>>() {
-   };
-
    protected final AtomicReference<AuthorizationException> authException = newReference();
 
    protected void installLocations() {

http://git-wip-us.apache.org/repos/asf/jclouds/blob/a449b24e/providers/aws-ec2/pom.xml
----------------------------------------------------------------------
diff --git a/providers/aws-ec2/pom.xml b/providers/aws-ec2/pom.xml
index 780c193..c15b190 100644
--- a/providers/aws-ec2/pom.xml
+++ b/providers/aws-ec2/pom.xml
@@ -101,6 +101,18 @@
       <version>${project.version}</version>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>com.squareup.okhttp</groupId>
+      <artifactId>mockwebserver</artifactId>
+      <scope>test</scope>
+      <exclusions>
+        <!-- Already provided by jclouds-sshj -->
+        <exclusion>
+          <groupId>org.bouncycastle</groupId>
+          <artifactId>bcprov-jdk15on</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/jclouds/blob/a449b24e/providers/aws-ec2/src/main/java/org/jclouds/aws/ec2/config/AWSEC2HttpApiModule.java
----------------------------------------------------------------------
diff --git a/providers/aws-ec2/src/main/java/org/jclouds/aws/ec2/config/AWSEC2HttpApiModule.java b/providers/aws-ec2/src/main/java/org/jclouds/aws/ec2/config/AWSEC2HttpApiModule.java
index 9ba5af8..d025fc7 100644
--- a/providers/aws-ec2/src/main/java/org/jclouds/aws/ec2/config/AWSEC2HttpApiModule.java
+++ b/providers/aws-ec2/src/main/java/org/jclouds/aws/ec2/config/AWSEC2HttpApiModule.java
@@ -16,6 +16,9 @@
  */
 package org.jclouds.aws.ec2.config;
 
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.TimeZone;
 
 import javax.inject.Singleton;
 
@@ -24,12 +27,15 @@ import org.jclouds.aws.ec2.domain.AWSRunningInstance;
 import org.jclouds.aws.ec2.domain.SpotInstanceRequest;
 import org.jclouds.aws.ec2.functions.SpotInstanceRequestToAWSRunningInstance;
 import org.jclouds.aws.ec2.options.AWSRunInstancesOptions;
+import org.jclouds.aws.filters.FormSigner;
+import org.jclouds.aws.filters.FormSignerV4;
+import org.jclouds.date.DateService;
 import org.jclouds.ec2.EC2Api;
 import org.jclouds.ec2.config.BaseEC2HttpApiModule;
-import org.jclouds.ec2.options.RunInstancesOptions;
 import org.jclouds.ec2.features.AMIApi;
 import org.jclouds.ec2.features.InstanceApi;
 import org.jclouds.ec2.features.SecurityGroupApi;
+import org.jclouds.ec2.options.RunInstancesOptions;
 import org.jclouds.rest.ConfiguresHttpApi;
 
 import com.google.common.base.Function;
@@ -42,8 +48,11 @@ import com.google.inject.TypeLiteral;
 @ConfiguresHttpApi
 public class AWSEC2HttpApiModule extends BaseEC2HttpApiModule<AWSEC2Api> {
 
+   private final SimpleDateFormat iso8601 = new SimpleDateFormat("yyyyMMdd'T'HHmmss'Z'");
+
    public AWSEC2HttpApiModule() {
       super(AWSEC2Api.class);
+      iso8601.setTimeZone(TimeZone.getTimeZone("GMT"));
    }
 
    @Singleton
@@ -72,9 +81,15 @@ public class AWSEC2HttpApiModule extends BaseEC2HttpApiModule<AWSEC2Api> {
 
    @Override
    protected void configure() {
+      bind(FormSigner.class).to(FormSignerV4.class);
       bind(RunInstancesOptions.class).to(AWSRunInstancesOptions.class);
       bind(new TypeLiteral<Function<SpotInstanceRequest, AWSRunningInstance>>() {
       }).to(SpotInstanceRequestToAWSRunningInstance.class);
       super.configure();
    }
+
+   @Override protected String provideTimeStamp(DateService dateService) {
+      // 20120416T155408Z not 2012-04-16T15:54:08Z
+      return iso8601.format(new Date());
+   }
 }

http://git-wip-us.apache.org/repos/asf/jclouds/blob/a449b24e/providers/aws-ec2/src/test/java/org/jclouds/aws/ec2/compute/AWSEC2ComputeServiceApiMockTest.java
----------------------------------------------------------------------
diff --git a/providers/aws-ec2/src/test/java/org/jclouds/aws/ec2/compute/AWSEC2ComputeServiceApiMockTest.java b/providers/aws-ec2/src/test/java/org/jclouds/aws/ec2/compute/AWSEC2ComputeServiceApiMockTest.java
new file mode 100644
index 0000000..6782373
--- /dev/null
+++ b/providers/aws-ec2/src/test/java/org/jclouds/aws/ec2/compute/AWSEC2ComputeServiceApiMockTest.java
@@ -0,0 +1,237 @@
+/*
+ * 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.aws.ec2.compute;
+
+import static org.jclouds.aws.ec2.compute.AWSEC2TemplateOptions.Builder.blockUntilRunning;
+import static org.testng.Assert.assertEquals;
+
+import org.jclouds.aws.ec2.internal.BaseAWSEC2ApiMockTest;
+import org.jclouds.compute.ComputeService;
+import org.jclouds.compute.domain.NodeMetadata;
+import org.jclouds.compute.domain.Template;
+import org.jclouds.compute.predicates.NodePredicates;
+import org.testng.annotations.Test;
+
+import com.google.common.collect.Iterables;
+import com.squareup.okhttp.mockwebserver.MockResponse;
+
+@Test(groups = "unit", testName = "AWSEC2ComputeServiceMockTest", singleThreaded = true)
+public class AWSEC2ComputeServiceApiMockTest extends BaseAWSEC2ApiMockTest {
+
+   public void launchVPCSpotInstanceSubnetId() throws Exception {
+      enqueueRegions(DEFAULT_REGION);
+      enqueueXml(DEFAULT_REGION, "/availabilityZones.xml");
+      enqueueXml(DEFAULT_REGION, "/describe_images.xml");
+      enqueueXml(DEFAULT_REGION, "/describe_images_cc.xml");
+      enqueueXml(DEFAULT_REGION, "/request_spot_instances-ebs.xml");
+      enqueueXml(DEFAULT_REGION, "/request_spot_instances-ebs.xml");
+      enqueueXml(DEFAULT_REGION, "/describe_images_ebs.xml");
+      enqueue(DEFAULT_REGION, new MockResponse()); // create tags
+
+      ComputeService computeService = computeService();
+
+      Template template = computeService.templateBuilder().locationId("us-east-1a").build();
+
+      template.getOptions().as(AWSEC2TemplateOptions.class)
+            .spotPrice(1f).subnetId("subnet-xyz").keyPair("Demo").blockUntilRunning(false);
+
+      NodeMetadata node = Iterables.getOnlyElement(computeService.createNodesInGroup("test", 1, template));
+      assertEquals(node.getId(), "us-east-1/sir-228e6406");
+
+      assertPosted(DEFAULT_REGION, "Action=DescribeRegions");
+      assertPosted(DEFAULT_REGION, "Action=DescribeAvailabilityZones");
+      assertPosted(DEFAULT_REGION, "Action=DescribeImages&Filter.1.Name=owner-id&Filter.1.Value.1=137112412989&Filter.1.Value.2=801119661308&Filter.1.Value.3=063491364108&Filter.1.Value.4=099720109477&Filter.1.Value.5=411009282317&Filter.2.Name=state&Filter.2.Value.1=available&Filter.3.Name=image-type&Filter.3.Value.1=machine");
+      assertPosted(DEFAULT_REGION, "Action=DescribeImages&Filter.1.Name=virtualization-type&Filter.1.Value.1=hvm&Filter.2.Name=architecture&Filter.2.Value.1=x86_64&Filter.3.Name=owner-id&Filter.3.Value.1=137112412989&Filter.3.Value.2=099720109477&Filter.4.Name=hypervisor&Filter.4.Value.1=xen&Filter.5.Name=state&Filter.5.Value.1=available&Filter.6.Name=image-type&Filter.6.Value.1=machine&Filter.7.Name=root-device-type&Filter.7.Value.1=ebs");
+      assertPosted(DEFAULT_REGION, "Action=RequestSpotInstances&SpotPrice=1.0&InstanceCount=1&LaunchSpecification.ImageId=ami-be3adfd7&LaunchSpecification.Placement.AvailabilityZone=us-east-1a&LaunchSpecification.InstanceType=m1.small&LaunchSpecification.SubnetId=subnet-xyz&LaunchSpecification.KeyName=Demo&LaunchSpecification.UserData=I2Nsb3VkLWNvbmZpZwpyZXBvX3VwZ3JhZGU6IG5vbmUK");
+      assertPosted(DEFAULT_REGION, "Action=DescribeSpotInstanceRequests&SpotInstanceRequestId.1=sir-228e6406");
+      assertPosted(DEFAULT_REGION, "Action=DescribeImages&ImageId.1=ami-595a0a1c");
+      assertPosted(DEFAULT_REGION, "Action=CreateTags&Tag.1.Key=Name&Tag.1.Value=test-228e6406&ResourceId.1=sir-228e6406");
+   }
+
+   String iamInstanceProfileArn = "arn:aws:iam::123456789012:instance-profile/application_abc/component_xyz/Webserver";
+
+   public void launchSpotInstanceIAMInstanceProfileArn() throws Exception {
+      enqueueRegions(DEFAULT_REGION);
+      enqueueXml(DEFAULT_REGION, "/availabilityZones.xml");
+      enqueueXml(DEFAULT_REGION, "/describe_images.xml");
+      enqueueXml(DEFAULT_REGION, "/describe_images_cc.xml");
+      enqueueXml(DEFAULT_REGION, "/created_securitygroup.xml");
+      enqueueXml(DEFAULT_REGION, "/new_securitygroup.xml");
+      enqueueXml(DEFAULT_REGION, "/new_securitygroup.xml");
+      enqueueXml(DEFAULT_REGION, "/new_securitygroup.xml");
+      enqueueXml(DEFAULT_REGION, "/authorize_securitygroup_ingress_response.xml");
+      enqueueXml(DEFAULT_REGION, "/request_spot_instances-ebs.xml");
+      enqueueXml(DEFAULT_REGION, "/request_spot_instances-ebs.xml");
+      enqueueXml(DEFAULT_REGION, "/describe_images_ebs.xml");
+      enqueue(DEFAULT_REGION, new MockResponse()); // create tags
+
+      ComputeService computeService = computeService();
+
+      Template template = computeService.templateBuilder().locationId("us-east-1a").build();
+
+      template.getOptions().as(AWSEC2TemplateOptions.class).spotPrice(1f).iamInstanceProfileArn(iamInstanceProfileArn)
+            .noKeyPair().blockUntilRunning(false);
+
+      NodeMetadata node = Iterables.getOnlyElement(computeService.createNodesInGroup("test", 1, template));
+      assertEquals(node.getId(), "us-east-1/sir-228e6406");
+
+      assertPosted(DEFAULT_REGION, "Action=DescribeRegions");
+      assertPosted(DEFAULT_REGION, "Action=DescribeAvailabilityZones");
+      assertPosted(DEFAULT_REGION, "Action=DescribeImages&Filter.1.Name=owner-id&Filter.1.Value.1=137112412989&Filter.1.Value.2=801119661308&Filter.1.Value.3=063491364108&Filter.1.Value.4=099720109477&Filter.1.Value.5=411009282317&Filter.2.Name=state&Filter.2.Value.1=available&Filter.3.Name=image-type&Filter.3.Value.1=machine");
+      assertPosted(DEFAULT_REGION, "Action=DescribeImages&Filter.1.Name=virtualization-type&Filter.1.Value.1=hvm&Filter.2.Name=architecture&Filter.2.Value.1=x86_64&Filter.3.Name=owner-id&Filter.3.Value.1=137112412989&Filter.3.Value.2=099720109477&Filter.4.Name=hypervisor&Filter.4.Value.1=xen&Filter.5.Name=state&Filter.5.Value.1=available&Filter.6.Name=image-type&Filter.6.Value.1=machine&Filter.7.Name=root-device-type&Filter.7.Value.1=ebs");
+      assertPosted(DEFAULT_REGION, "Action=CreateSecurityGroup&GroupName=jclouds%23test&GroupDescription=jclouds%23test");
+      assertPosted(DEFAULT_REGION, "Action=DescribeSecurityGroups&GroupName.1=jclouds%23test");
+      assertPosted(DEFAULT_REGION, "Action=DescribeSecurityGroups&GroupName.1=jclouds%23test");
+      assertPosted(DEFAULT_REGION, "Action=DescribeSecurityGroups&GroupName.1=jclouds%23test");
+      assertPosted(DEFAULT_REGION, "Action=AuthorizeSecurityGroupIngress&GroupId=sg-3c6ef654&IpPermissions.0.IpProtocol=tcp&IpPermissions.0.FromPort=22&IpPermissions.0.ToPort=22&IpPermissions.0.IpRanges.0.CidrIp=0.0.0.0/0&IpPermissions.1.IpProtocol=tcp&IpPermissions.1.FromPort=0&IpPermissions.1.ToPort=65535&IpPermissions.1.Groups.0.UserId=993194456877&IpPermissions.1.Groups.0.GroupId=sg-3c6ef654&IpPermissions.2.IpProtocol=udp&IpPermissions.2.FromPort=0&IpPermissions.2.ToPort=65535&IpPermissions.2.Groups.0.UserId=993194456877&IpPermissions.2.Groups.0.GroupId=sg-3c6ef654");
+      assertPosted(DEFAULT_REGION, "Action=RequestSpotInstances&SpotPrice=1.0&InstanceCount=1&LaunchSpecification.ImageId=ami-be3adfd7&LaunchSpecification.Placement.AvailabilityZone=us-east-1a&LaunchSpecification.SecurityGroup.1=jclouds%23test&LaunchSpecification.InstanceType=m1.small&LaunchSpecification.UserData=I2Nsb3VkLWNvbmZpZwpyZXBvX3VwZ3JhZGU6IG5vbmUK&LaunchSpecification.IamInstanceProfile.Arn=arn%3Aaws%3Aiam%3A%3A123456789012%3Ainstance-profile/application_abc/component_xyz/Webserver");
+      assertPosted(DEFAULT_REGION, "Action=DescribeSpotInstanceRequests&SpotInstanceRequestId.1=sir-228e6406");
+      assertPosted(DEFAULT_REGION, "Action=DescribeImages&ImageId.1=ami-595a0a1c");
+      assertPosted(DEFAULT_REGION, "Action=CreateTags&Tag.1.Key=Name&Tag.1.Value=test-228e6406&ResourceId.1=sir-228e6406");
+   }
+
+   public void launchSpotInstanceIAMInstanceProfileName() throws Exception {
+      enqueueRegions(DEFAULT_REGION);
+      enqueueXml(DEFAULT_REGION, "/availabilityZones.xml");
+      enqueueXml(DEFAULT_REGION, "/describe_images.xml");
+      enqueueXml(DEFAULT_REGION, "/describe_images_cc.xml");
+      enqueueXml(DEFAULT_REGION, "/created_securitygroup.xml");
+      enqueueXml(DEFAULT_REGION, "/new_securitygroup.xml");
+      enqueueXml(DEFAULT_REGION, "/new_securitygroup.xml");
+      enqueueXml(DEFAULT_REGION, "/new_securitygroup.xml");
+      enqueueXml(DEFAULT_REGION, "/authorize_securitygroup_ingress_response.xml");
+      enqueueXml(DEFAULT_REGION, "/request_spot_instances-ebs.xml");
+      enqueueXml(DEFAULT_REGION, "/request_spot_instances-ebs.xml");
+      enqueueXml(DEFAULT_REGION, "/describe_images_ebs.xml");
+      enqueue(DEFAULT_REGION, new MockResponse()); // create tags
+
+      ComputeService computeService = computeService();
+
+      Template template = computeService.templateBuilder().locationId("us-east-1a").build();
+
+      template.getOptions().as(AWSEC2TemplateOptions.class).spotPrice(1f).iamInstanceProfileName("Webserver")
+            .noKeyPair().blockUntilRunning(false);
+
+      NodeMetadata node = Iterables.getOnlyElement(computeService.createNodesInGroup("test", 1, template));
+      assertEquals(node.getId(), "us-east-1/sir-228e6406");
+
+      assertPosted(DEFAULT_REGION, "Action=DescribeRegions");
+      assertPosted(DEFAULT_REGION, "Action=DescribeAvailabilityZones");
+      assertPosted(DEFAULT_REGION, "Action=DescribeImages&Filter.1.Name=owner-id&Filter.1.Value.1=137112412989&Filter.1.Value.2=801119661308&Filter.1.Value.3=063491364108&Filter.1.Value.4=099720109477&Filter.1.Value.5=411009282317&Filter.2.Name=state&Filter.2.Value.1=available&Filter.3.Name=image-type&Filter.3.Value.1=machine");
+      assertPosted(DEFAULT_REGION, "Action=DescribeImages&Filter.1.Name=virtualization-type&Filter.1.Value.1=hvm&Filter.2.Name=architecture&Filter.2.Value.1=x86_64&Filter.3.Name=owner-id&Filter.3.Value.1=137112412989&Filter.3.Value.2=099720109477&Filter.4.Name=hypervisor&Filter.4.Value.1=xen&Filter.5.Name=state&Filter.5.Value.1=available&Filter.6.Name=image-type&Filter.6.Value.1=machine&Filter.7.Name=root-device-type&Filter.7.Value.1=ebs");
+      assertPosted(DEFAULT_REGION, "Action=CreateSecurityGroup&GroupName=jclouds%23test&GroupDescription=jclouds%23test");
+      assertPosted(DEFAULT_REGION, "Action=DescribeSecurityGroups&GroupName.1=jclouds%23test");
+      assertPosted(DEFAULT_REGION, "Action=DescribeSecurityGroups&GroupName.1=jclouds%23test");
+      assertPosted(DEFAULT_REGION, "Action=DescribeSecurityGroups&GroupName.1=jclouds%23test");
+      assertPosted(DEFAULT_REGION, "Action=AuthorizeSecurityGroupIngress&GroupId=sg-3c6ef654&IpPermissions.0.IpProtocol=tcp&IpPermissions.0.FromPort=22&IpPermissions.0.ToPort=22&IpPermissions.0.IpRanges.0.CidrIp=0.0.0.0/0&IpPermissions.1.IpProtocol=tcp&IpPermissions.1.FromPort=0&IpPermissions.1.ToPort=65535&IpPermissions.1.Groups.0.UserId=993194456877&IpPermissions.1.Groups.0.GroupId=sg-3c6ef654&IpPermissions.2.IpProtocol=udp&IpPermissions.2.FromPort=0&IpPermissions.2.ToPort=65535&IpPermissions.2.Groups.0.UserId=993194456877&IpPermissions.2.Groups.0.GroupId=sg-3c6ef654");
+      assertPosted(DEFAULT_REGION, "Action=RequestSpotInstances&SpotPrice=1.0&InstanceCount=1&LaunchSpecification.ImageId=ami-be3adfd7&LaunchSpecification.Placement.AvailabilityZone=us-east-1a&LaunchSpecification.SecurityGroup.1=jclouds%23test&LaunchSpecification.InstanceType=m1.small&LaunchSpecification.UserData=I2Nsb3VkLWNvbmZpZwpyZXBvX3VwZ3JhZGU6IG5vbmUK&LaunchSpecification.IamInstanceProfile.Name=Webserver");
+      assertPosted(DEFAULT_REGION, "Action=DescribeSpotInstanceRequests&SpotInstanceRequestId.1=sir-228e6406");
+      assertPosted(DEFAULT_REGION, "Action=DescribeImages&ImageId.1=ami-595a0a1c");
+      assertPosted(DEFAULT_REGION, "Action=CreateTags&Tag.1.Key=Name&Tag.1.Value=test-228e6406&ResourceId.1=sir-228e6406");
+   }
+
+   public void createNodeWithIAMInstanceProfileArn() throws Exception {
+      enqueueRegions(DEFAULT_REGION);
+      enqueueXml(DEFAULT_REGION, "/amzn_images.xml");
+      enqueueXml(DEFAULT_REGION, "/describe_images_cc.xml");
+      enqueueXml(DEFAULT_REGION, "/availabilityZones.xml");
+      enqueueXml(DEFAULT_REGION, "/created_securitygroup.xml");
+      enqueueXml(DEFAULT_REGION, "/new_securitygroup.xml");
+      enqueueXml(DEFAULT_REGION, "/new_securitygroup.xml");
+      enqueueXml(DEFAULT_REGION, "/new_securitygroup.xml");
+      enqueueXml(DEFAULT_REGION, "/authorize_securitygroup_ingress_response.xml");
+      enqueueXml(DEFAULT_REGION, "/new_instance.xml");
+      enqueueXml(DEFAULT_REGION, "/describe_instances_running-1.xml");
+      enqueueXml(DEFAULT_REGION, "/describe_images.xml");
+      enqueue(DEFAULT_REGION, new MockResponse()); // create tags
+
+      ComputeService computeService = computeService();
+
+      NodeMetadata node = Iterables.getOnlyElement(computeService.createNodesInGroup("test", 1,
+            blockUntilRunning(false).iamInstanceProfileArn(iamInstanceProfileArn).noKeyPair()));
+      assertEquals(node.getId(), "us-east-1/i-2baa5550");
+
+      assertPosted(DEFAULT_REGION, "Action=DescribeRegions");
+      assertPosted(DEFAULT_REGION, "Action=DescribeImages&Filter.1.Name=owner-id&Filter.1.Value.1=137112412989&Filter.1.Value.2=801119661308&Filter.1.Value.3=063491364108&Filter.1.Value.4=099720109477&Filter.1.Value.5=411009282317&Filter.2.Name=state&Filter.2.Value.1=available&Filter.3.Name=image-type&Filter.3.Value.1=machine");
+      assertPosted(DEFAULT_REGION, "Action=DescribeImages&Filter.1.Name=virtualization-type&Filter.1.Value.1=hvm&Filter.2.Name=architecture&Filter.2.Value.1=x86_64&Filter.3.Name=owner-id&Filter.3.Value.1=137112412989&Filter.3.Value.2=099720109477&Filter.4.Name=hypervisor&Filter.4.Value.1=xen&Filter.5.Name=state&Filter.5.Value.1=available&Filter.6.Name=image-type&Filter.6.Value.1=machine&Filter.7.Name=root-device-type&Filter.7.Value.1=ebs");
+      assertPosted(DEFAULT_REGION, "Action=DescribeAvailabilityZones");
+      assertPosted(DEFAULT_REGION, "Action=CreateSecurityGroup&GroupName=jclouds%23test&GroupDescription=jclouds%23test");
+      assertPosted(DEFAULT_REGION, "Action=DescribeSecurityGroups&GroupName.1=jclouds%23test");
+      assertPosted(DEFAULT_REGION, "Action=DescribeSecurityGroups&GroupName.1=jclouds%23test");
+      assertPosted(DEFAULT_REGION, "Action=DescribeSecurityGroups&GroupName.1=jclouds%23test");
+      assertPosted(DEFAULT_REGION, "Action=AuthorizeSecurityGroupIngress&GroupId=sg-3c6ef654&IpPermissions.0.IpProtocol=tcp&IpPermissions.0.FromPort=22&IpPermissions.0.ToPort=22&IpPermissions.0.IpRanges.0.CidrIp=0.0.0.0/0&IpPermissions.1.IpProtocol=tcp&IpPermissions.1.FromPort=0&IpPermissions.1.ToPort=65535&IpPermissions.1.Groups.0.UserId=993194456877&IpPermissions.1.Groups.0.GroupId=sg-3c6ef654&IpPermissions.2.IpProtocol=udp&IpPermissions.2.FromPort=0&IpPermissions.2.ToPort=65535&IpPermissions.2.Groups.0.UserId=993194456877&IpPermissions.2.Groups.0.GroupId=sg-3c6ef654");
+      assertPosted(DEFAULT_REGION, "Action=RunInstances&ImageId=ami-8ce4b5c9&MinCount=1&MaxCount=1&InstanceType=t1.micro&SecurityGroup.1=jclouds%23test&UserData=I2Nsb3VkLWNvbmZpZwpyZXBvX3VwZ3JhZGU6IG5vbmUK&IamInstanceProfile.Arn=arn%3Aaws%3Aiam%3A%3A123456789012%3Ainstance-profile/application_abc/component_xyz/Webserver");
+      assertPosted(DEFAULT_REGION, "Action=DescribeInstances&InstanceId.1=i-2baa5550");
+      assertPosted(DEFAULT_REGION, "Action=DescribeImages&ImageId.1=ami-aecd60c7");
+      assertPosted(DEFAULT_REGION, "Action=CreateTags&Tag.1.Key=Name&Tag.1.Value=test-2baa5550&ResourceId.1=i-2baa5550");
+   }
+
+   public void createNodeWithIAMInstanceProfileName() throws Exception {
+      enqueueRegions(DEFAULT_REGION);
+      enqueueXml(DEFAULT_REGION, "/amzn_images.xml");
+      enqueueXml(DEFAULT_REGION, "/describe_images_cc.xml");
+      enqueueXml(DEFAULT_REGION, "/availabilityZones.xml");
+      enqueueXml(DEFAULT_REGION, "/created_securitygroup.xml");
+      enqueueXml(DEFAULT_REGION, "/new_securitygroup.xml");
+      enqueueXml(DEFAULT_REGION, "/new_securitygroup.xml");
+      enqueueXml(DEFAULT_REGION, "/new_securitygroup.xml");
+      enqueueXml(DEFAULT_REGION, "/authorize_securitygroup_ingress_response.xml");
+      enqueueXml(DEFAULT_REGION, "/new_instance.xml");
+      enqueueXml(DEFAULT_REGION, "/describe_instances_running-1.xml");
+      enqueueXml(DEFAULT_REGION, "/describe_images.xml");
+      enqueue(DEFAULT_REGION, new MockResponse()); // create tags
+
+      ComputeService computeService = computeService();
+
+      NodeMetadata node = Iterables.getOnlyElement(computeService.createNodesInGroup("test", 1,
+            blockUntilRunning(false).iamInstanceProfileName("Webserver").noKeyPair()));
+      assertEquals(node.getId(), "us-east-1/i-2baa5550");
+
+      assertPosted(DEFAULT_REGION, "Action=DescribeRegions");
+      assertPosted(DEFAULT_REGION, "Action=DescribeImages&Filter.1.Name=owner-id&Filter.1.Value.1=137112412989&Filter.1.Value.2=801119661308&Filter.1.Value.3=063491364108&Filter.1.Value.4=099720109477&Filter.1.Value.5=411009282317&Filter.2.Name=state&Filter.2.Value.1=available&Filter.3.Name=image-type&Filter.3.Value.1=machine");
+      assertPosted(DEFAULT_REGION, "Action=DescribeImages&Filter.1.Name=virtualization-type&Filter.1.Value.1=hvm&Filter.2.Name=architecture&Filter.2.Value.1=x86_64&Filter.3.Name=owner-id&Filter.3.Value.1=137112412989&Filter.3.Value.2=099720109477&Filter.4.Name=hypervisor&Filter.4.Value.1=xen&Filter.5.Name=state&Filter.5.Value.1=available&Filter.6.Name=image-type&Filter.6.Value.1=machine&Filter.7.Name=root-device-type&Filter.7.Value.1=ebs");
+      assertPosted(DEFAULT_REGION, "Action=DescribeAvailabilityZones");
+      assertPosted(DEFAULT_REGION, "Action=CreateSecurityGroup&GroupName=jclouds%23test&GroupDescription=jclouds%23test");
+      assertPosted(DEFAULT_REGION, "Action=DescribeSecurityGroups&GroupName.1=jclouds%23test");
+      assertPosted(DEFAULT_REGION, "Action=DescribeSecurityGroups&GroupName.1=jclouds%23test");
+      assertPosted(DEFAULT_REGION, "Action=DescribeSecurityGroups&GroupName.1=jclouds%23test");
+      assertPosted(DEFAULT_REGION, "Action=AuthorizeSecurityGroupIngress&GroupId=sg-3c6ef654&IpPermissions.0.IpProtocol=tcp&IpPermissions.0.FromPort=22&IpPermissions.0.ToPort=22&IpPermissions.0.IpRanges.0.CidrIp=0.0.0.0/0&IpPermissions.1.IpProtocol=tcp&IpPermissions.1.FromPort=0&IpPermissions.1.ToPort=65535&IpPermissions.1.Groups.0.UserId=993194456877&IpPermissions.1.Groups.0.GroupId=sg-3c6ef654&IpPermissions.2.IpProtocol=udp&IpPermissions.2.FromPort=0&IpPermissions.2.ToPort=65535&IpPermissions.2.Groups.0.UserId=993194456877&IpPermissions.2.Groups.0.GroupId=sg-3c6ef654");
+      assertPosted(DEFAULT_REGION, "Action=RunInstances&ImageId=ami-8ce4b5c9&MinCount=1&MaxCount=1&InstanceType=t1.micro&SecurityGroup.1=jclouds%23test&UserData=I2Nsb3VkLWNvbmZpZwpyZXBvX3VwZ3JhZGU6IG5vbmUK&IamInstanceProfile.Name=Webserver");
+      assertPosted(DEFAULT_REGION, "Action=DescribeInstances&InstanceId.1=i-2baa5550");
+      assertPosted(DEFAULT_REGION, "Action=DescribeImages&ImageId.1=ami-aecd60c7");
+      assertPosted(DEFAULT_REGION, "Action=CreateTags&Tag.1.Key=Name&Tag.1.Value=test-2baa5550&ResourceId.1=i-2baa5550");
+   }
+
+   public void listNodesWhereImageDoesntExist() throws Exception {
+      enqueueRegions(DEFAULT_REGION);
+      enqueueXml(DEFAULT_REGION, "/describe_instances_running-1.xml");
+      enqueueXml(DEFAULT_REGION, "/availabilityZones.xml");
+      enqueue(DEFAULT_REGION, new MockResponse().setBody("<DescribeImagesResponse><imagesSet></imagesSet></DescribeImagesResponse>"));
+      enqueue(DEFAULT_REGION, new MockResponse().setBody("<DescribeSpotInstanceRequestsResponse><spotInstanceRequestSet></spotInstanceRequestSet></DescribeSpotInstanceRequestsResponse>"));
+
+      ComputeService computeService = computeService();
+
+      NodeMetadata node = Iterables.getOnlyElement(computeService.listNodesDetailsMatching(NodePredicates.all()));
+      assertEquals(node.getId(), "us-east-1/i-2baa5550");
+
+      assertPosted(DEFAULT_REGION, "Action=DescribeRegions");
+      assertPosted(DEFAULT_REGION, "Action=DescribeInstances");
+      assertPosted(DEFAULT_REGION, "Action=DescribeAvailabilityZones");
+      assertPosted(DEFAULT_REGION, "Action=DescribeImages&ImageId.1=ami-aecd60c7");
+      assertPosted(DEFAULT_REGION, "Action=DescribeSpotInstanceRequests");
+   }
+}