You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jclouds.apache.org by za...@apache.org on 2015/05/13 02:14:12 UTC

jclouds git commit: JCLOUDS-894 Multipart upload code fixes for swift

Repository: jclouds
Updated Branches:
  refs/heads/master 2c53ef38a -> f397bebeb


JCLOUDS-894 Multipart upload code fixes for swift


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

Branch: refs/heads/master
Commit: f397bebeb2dfd41f01e22b2f2d4aebc2cafa9494
Parents: 2c53ef3
Author: Zack Shoylev <za...@rackspace.com>
Authored: Fri May 8 13:04:16 2015 -0500
Committer: Zack Shoylev <za...@rackspace.com>
Committed: Tue May 12 19:13:15 2015 -0500

----------------------------------------------------------------------
 .../v1/binders/BindManifestToJsonPayload.java   | 65 ++++++++++++++
 .../swift/v1/binders/BindMetadataToHeaders.java |  6 --
 .../swift/v1/binders/BindToHeaders.java         | 81 +++++++++++++++++
 .../blobstore/RegionScopedSwiftBlobStore.java   | 27 +++++-
 .../openstack/swift/v1/features/ObjectApi.java  |  8 +-
 .../swift/v1/features/StaticLargeObjectApi.java | 42 +++++++++
 .../SwiftBlobIntegrationLiveTest.java           | 17 ++--
 .../integration/SwiftBlobSignerLiveTest.java    | 11 +++
 .../SwiftContainerIntegrationLiveTest.java      | 11 +++
 .../swift/v1/features/ObjectApiMockTest.java    |  2 +-
 .../features/StaticLargeObjectApiMockTest.java  | 95 ++++++++++++++++++++
 .../test/resources/manifest_get_response.json   | 17 ++++
 .../internal/BaseBlobIntegrationTest.java       |  1 +
 13 files changed, 361 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jclouds/blob/f397bebe/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/binders/BindManifestToJsonPayload.java
----------------------------------------------------------------------
diff --git a/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/binders/BindManifestToJsonPayload.java b/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/binders/BindManifestToJsonPayload.java
new file mode 100644
index 0000000..76ad60f
--- /dev/null
+++ b/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/binders/BindManifestToJsonPayload.java
@@ -0,0 +1,65 @@
+/*
+ * 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.openstack.swift.v1.binders;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.util.Map;
+
+import javax.inject.Inject;
+
+import org.jclouds.http.HttpRequest;
+import org.jclouds.json.Json;
+import org.jclouds.rest.MapBinder;
+
+/**
+ * Binds the object to the request as a json object.
+ */
+public class BindManifestToJsonPayload implements MapBinder {
+
+   protected final Json jsonBinder;
+
+   @Inject
+   BindManifestToJsonPayload(Json jsonBinder) {
+      this.jsonBinder = jsonBinder;
+   }
+
+   @Override
+   public <R extends HttpRequest> R bindToRequest(R request, Map<String, Object> postParams) {
+      return bindToRequest(request, (Object) postParams);
+   }
+
+   @Override
+   public <R extends HttpRequest> R bindToRequest(R request, Object payload) {
+      String json = jsonBinder.toJson(checkNotNull(payload, "payload"));
+      request.setPayload(json);
+      /**
+       * The Content-Length request header must contain the length of the JSON content, not the length of the segment
+       * objects. However, after the PUT operation is complete, the Content-Length metadata is set to the total length
+       * of all the object segments. A similar situation applies to the ETag header. If it is used in the PUT
+       * operation, it must contain the MD5 checksum of the JSON content. The ETag metadata value is then set to be
+       * the MD5 checksum of the concatenated ETag values of the object segments. You can also set the Content-Type
+       * request header and custom object metadata.
+       * When the PUT operation sees the ?multipart-manifest=put query string, it reads the request body and verifies
+       * that each segment object exists and that the sizes and ETags match. If there is a mismatch, the PUT operation
+       * fails.
+       */
+      request.getPayload().getContentMetadata().setContentLength((long)json.length());
+      return request;
+   }
+
+}

http://git-wip-us.apache.org/repos/asf/jclouds/blob/f397bebe/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/binders/BindMetadataToHeaders.java
----------------------------------------------------------------------
diff --git a/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/binders/BindMetadataToHeaders.java b/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/binders/BindMetadataToHeaders.java
index 59ff29a..9451934 100644
--- a/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/binders/BindMetadataToHeaders.java
+++ b/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/binders/BindMetadataToHeaders.java
@@ -85,12 +85,6 @@ public class BindMetadataToHeaders implements Binder {
       }
    }
 
-   public static class BindRawMetadataToHeaders extends BindMetadataToHeaders {
-      BindRawMetadataToHeaders() {
-         super("");
-      }
-   }
-
    public static class BindRemoveObjectMetadataToHeaders extends BindMetadataToHeaders.ForRemoval {
       BindRemoveObjectMetadataToHeaders() {
          super(OBJECT_METADATA_PREFIX);

http://git-wip-us.apache.org/repos/asf/jclouds/blob/f397bebe/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/binders/BindToHeaders.java
----------------------------------------------------------------------
diff --git a/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/binders/BindToHeaders.java b/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/binders/BindToHeaders.java
new file mode 100644
index 0000000..7dc0665
--- /dev/null
+++ b/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/binders/BindToHeaders.java
@@ -0,0 +1,81 @@
+/*
+ * 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.openstack.swift.v1.binders;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.jclouds.http.HttpRequest;
+import org.jclouds.io.MutableContentMetadata;
+import org.jclouds.rest.Binder;
+
+import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.ImmutableMultimap.Builder;
+
+/**
+ * Will bind to headers, as needed, and will process content-* headers in a jclouds-compatible fashion.
+ */
+public class BindToHeaders implements Binder {
+
+   @SuppressWarnings("unchecked")
+   @Override
+   public <R extends HttpRequest> R bindToRequest(R request, Object input) {
+      checkNotNull(request, "request");
+      checkArgument(input instanceof Map<?, ?>, "input must be a non-null java.util.Map!");
+      // Input map
+      Map<String, String> headers = Map.class.cast(input);
+
+      // Content map
+      if (request.getPayload() == null) {
+         request.setPayload("");
+      }
+      MutableContentMetadata contentMetadata = request.getPayload().getContentMetadata();
+
+      // Regular headers map
+      Builder<String, String> builder = ImmutableMultimap.<String, String> builder();
+
+      for (Entry<String, String> keyVal : headers.entrySet()) {
+         String keyInLowercase = keyVal.getKey().toLowerCase();
+
+         if (keyInLowercase.equals("content-type")) {
+            contentMetadata.setContentType(keyVal.getValue());
+            continue;
+         }
+         if (keyInLowercase.equals("content-disposition")) {
+            contentMetadata.setContentDisposition(keyVal.getValue());
+            continue;
+         }
+         if (keyInLowercase.equals("content-encoding")) {
+            contentMetadata.setContentEncoding(keyVal.getValue());
+            continue;
+         }
+         if (keyInLowercase.equals("content-language")) {
+            contentMetadata.setContentLanguage(keyVal.getValue());
+            continue;
+         }
+         if (keyInLowercase.equals("content-length")) {
+            contentMetadata.setContentLength(Long.parseLong(keyVal.getValue()));
+            continue;
+         }
+         builder.put(keyInLowercase, keyVal.getValue());
+      }
+      request.getPayload().setContentMetadata(contentMetadata);
+      return (R) request.toBuilder().replaceHeaders(builder.build()).build();
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds/blob/f397bebe/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/blobstore/RegionScopedSwiftBlobStore.java
----------------------------------------------------------------------
diff --git a/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/blobstore/RegionScopedSwiftBlobStore.java b/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/blobstore/RegionScopedSwiftBlobStore.java
index 62177f1..7d4294a 100644
--- a/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/blobstore/RegionScopedSwiftBlobStore.java
+++ b/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/blobstore/RegionScopedSwiftBlobStore.java
@@ -81,6 +81,7 @@ import com.google.common.cache.LoadingCache;
 import com.google.common.collect.FluentIterable;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMap.Builder;
 import com.google.common.collect.ImmutableMultimap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
@@ -415,6 +416,26 @@ public class RegionScopedSwiftBlobStore implements BlobStore {
       removeBlobs(mpu.containerName(), names.build());
    }
 
+   private ImmutableMap<String, String> getContentMetadataForManifest(ContentMetadata contentMetadata) {
+      Builder<String, String> mapBuilder = ImmutableMap.builder();
+      if (contentMetadata.getContentType() != null) {
+         mapBuilder.put("content-type", contentMetadata.getContentType());
+      }
+      /**
+       * Do not set content-length. Set automatically to manifest json string length by BindManifestToJsonPayload
+       */
+      if (contentMetadata.getContentDisposition() != null) {
+         mapBuilder.put("content-disposition", contentMetadata.getContentDisposition());
+      }
+      if (contentMetadata.getContentEncoding() != null) {
+         mapBuilder.put("content-encoding", contentMetadata.getContentEncoding());
+      }
+      if (contentMetadata.getContentLanguage() != null) {
+         mapBuilder.put("content-language", contentMetadata.getContentLanguage());
+      }
+      return mapBuilder.build();
+   }
+
    @Override
    public String completeMultipartUpload(MultipartUpload mpu, List<MultipartPart> parts) {
       ImmutableList.Builder<Segment> builder = ImmutableList.builder();
@@ -422,9 +443,9 @@ public class RegionScopedSwiftBlobStore implements BlobStore {
          String path = mpu.containerName() + "/" + mpu.blobName() + "-" + part.partNumber();
          builder.add(Segment.builder().path(path).etag(part.partETag()).sizeBytes(part.partSize()).build());
       }
-      Map<String, String> metadata = ImmutableMap.of();  // TODO: how to populate this from mpu.blobMetadata()?
+
       return api.getStaticLargeObjectApi(regionId, mpu.containerName()).replaceManifest(mpu.blobName(),
-            builder.build(), metadata);
+            builder.build(), mpu.blobMetadata().getUserMetadata(), getContentMetadataForManifest(mpu.blobMetadata().getContentMetadata()));
    }
 
    @Override
@@ -453,7 +474,7 @@ public class RegionScopedSwiftBlobStore implements BlobStore {
 
    @Override
    public long getMinimumMultipartPartSize() {
-      return 1024 * 1024;
+      return 1024 * 1024 + 1;
    }
 
    @Override

http://git-wip-us.apache.org/repos/asf/jclouds/blob/f397bebe/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/features/ObjectApi.java
----------------------------------------------------------------------
diff --git a/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/features/ObjectApi.java b/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/features/ObjectApi.java
index 841fd9d..2d523ca 100644
--- a/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/features/ObjectApi.java
+++ b/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/features/ObjectApi.java
@@ -42,8 +42,8 @@ import org.jclouds.io.Payload;
 import org.jclouds.javax.annotation.Nullable;
 import org.jclouds.openstack.keystone.v2_0.filters.AuthenticateRequest;
 import org.jclouds.openstack.swift.v1.binders.BindMetadataToHeaders.BindObjectMetadataToHeaders;
-import org.jclouds.openstack.swift.v1.binders.BindMetadataToHeaders.BindRawMetadataToHeaders;
 import org.jclouds.openstack.swift.v1.binders.BindMetadataToHeaders.BindRemoveObjectMetadataToHeaders;
+import org.jclouds.openstack.swift.v1.binders.BindToHeaders;
 import org.jclouds.openstack.swift.v1.binders.SetPayload;
 import org.jclouds.openstack.swift.v1.domain.ObjectList;
 import org.jclouds.openstack.swift.v1.domain.SwiftObject;
@@ -224,8 +224,8 @@ public interface ObjectApi {
    @Path("/{objectName}")
    @Produces("")
    @Fallback(FalseOnNotFoundOr404.class)
-   boolean updateRawMetadata(@PathParam("objectName") String objectName,
-         @BinderParam(BindRawMetadataToHeaders.class) Map<String, String> metadata);
+   boolean updateHeaders(@PathParam("objectName") String objectName,
+         @BinderParam(BindToHeaders.class) Map<String, String> metadata);
 
    /**
     * Deletes the metadata from a {@link SwiftObject}.
@@ -313,6 +313,6 @@ public interface ObjectApi {
          @PathParam("sourceContainer") String sourceContainer,
          @PathParam("sourceObject") String sourceObject,
          @BinderParam(BindObjectMetadataToHeaders.class) Map<String, String> userMetadata,
-         @BinderParam(BindRawMetadataToHeaders.class) Map<String, String> objectMetadata);
+         @BinderParam(BindToHeaders.class) Map<String, String> objectMetadata);
 
 }

http://git-wip-us.apache.org/repos/asf/jclouds/blob/f397bebe/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/features/StaticLargeObjectApi.java
----------------------------------------------------------------------
diff --git a/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/features/StaticLargeObjectApi.java b/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/features/StaticLargeObjectApi.java
index 51ba30d..b81df13 100644
--- a/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/features/StaticLargeObjectApi.java
+++ b/apis/openstack-swift/src/main/java/org/jclouds/openstack/swift/v1/features/StaticLargeObjectApi.java
@@ -24,13 +24,17 @@ import java.util.Map;
 import javax.inject.Named;
 import javax.ws.rs.Consumes;
 import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
 
+import org.jclouds.Fallbacks.EmptyListOnNotFoundOr404;
 import org.jclouds.Fallbacks.VoidOnNotFoundOr404;
 import org.jclouds.openstack.keystone.v2_0.filters.AuthenticateRequest;
+import org.jclouds.openstack.swift.v1.binders.BindManifestToJsonPayload;
 import org.jclouds.openstack.swift.v1.binders.BindMetadataToHeaders.BindObjectMetadataToHeaders;
+import org.jclouds.openstack.swift.v1.binders.BindToHeaders;
 import org.jclouds.openstack.swift.v1.domain.Segment;
 import org.jclouds.openstack.swift.v1.functions.ETagHeader;
 import org.jclouds.rest.annotations.BinderParam;
@@ -77,6 +81,30 @@ public interface StaticLargeObjectApi {
          @BinderParam(BindObjectMetadataToHeaders.class) Map<String, String> metadata);
 
    /**
+    * Creates or updates a static large object's manifest.
+    *
+    * @param objectName
+    *           corresponds to {@link SwiftObject#getName()}.
+    * @param segments
+    *           ordered parts which will be concatenated upon download.
+    * @param metadata
+    *           corresponds to {@link SwiftObject#getMetadata()}.
+    * @param headers
+    *           Binds the map to headers, without prefixing/escaping the header name/key.
+    *
+    * @return {@link SwiftObject#getEtag()} of the object, which is the MD5
+    *         checksum of the concatenated ETag values of the {@code segments}.
+    */
+   @Named("staticLargeObject:replaceManifest")
+   @PUT
+   @ResponseParser(ETagHeader.class)
+   @QueryParams(keys = "multipart-manifest", values = "put")
+   String replaceManifest(@PathParam("objectName") String objectName,
+         @BinderParam(BindManifestToJsonPayload.class) List<Segment> segments,
+         @BinderParam(BindObjectMetadataToHeaders.class) Map<String, String> metadata,
+         @BinderParam(BindToHeaders.class) Map<String, String> headers);
+
+   /**
     * Deletes a static large object, if present, including all of its segments.
     *
     * @param objectName
@@ -87,4 +115,18 @@ public interface StaticLargeObjectApi {
    @Fallback(VoidOnNotFoundOr404.class)
    @QueryParams(keys = "multipart-manifest", values = "delete")
    void delete(@PathParam("objectName") String objectName);
+
+   /**
+    * Get a static large object's manifest.
+    *
+    * @param objectName
+    *           corresponds to {@link SwiftObject#getName()}.
+    *
+    * @return A list of the multipart segments
+    */
+   @Named("staticLargeObject:getManifest")
+   @GET
+   @Fallback(EmptyListOnNotFoundOr404.class)
+   @QueryParams(keys = "multipart-manifest", values = "get")
+   List<Segment> getManifest(@PathParam("objectName") String objectName);
 }

http://git-wip-us.apache.org/repos/asf/jclouds/blob/f397bebe/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/blobstore/integration/SwiftBlobIntegrationLiveTest.java
----------------------------------------------------------------------
diff --git a/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/blobstore/integration/SwiftBlobIntegrationLiveTest.java b/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/blobstore/integration/SwiftBlobIntegrationLiveTest.java
index 55e966a..c424b1c 100644
--- a/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/blobstore/integration/SwiftBlobIntegrationLiveTest.java
+++ b/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/blobstore/integration/SwiftBlobIntegrationLiveTest.java
@@ -19,16 +19,20 @@ package org.jclouds.openstack.swift.v1.blobstore.integration;
 import static org.jclouds.openstack.keystone.v2_0.config.KeystoneProperties.CREDENTIAL_TYPE;
 
 import java.util.Properties;
+import java.util.concurrent.TimeUnit;
 
+import org.jclouds.blobstore.attr.ConsistencyModel;
 import org.jclouds.blobstore.domain.Blob;
 import org.jclouds.blobstore.integration.internal.BaseBlobIntegrationTest;
 import org.testng.SkipException;
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
 
+import com.google.common.util.concurrent.Uninterruptibles;
+
 @Test(groups = "live", testName = "SwiftBlobIntegrationLiveTest")
 public class SwiftBlobIntegrationLiveTest extends BaseBlobIntegrationTest {
-   
+
    public SwiftBlobIntegrationLiveTest() {
       provider = "openstack-swift";
    }
@@ -85,12 +89,9 @@ public class SwiftBlobIntegrationLiveTest extends BaseBlobIntegrationTest {
    }
 
    @Override
-   public void testMultipartUploadSinglePart() throws Exception {
-      throw new SkipException("openstack-swift does not support setting blob metadata during multipart upload");
-   }
-
-   @Override
-   public void testMultipartUploadMultipleParts() throws Exception {
-      throw new SkipException("openstack-swift does not support setting blob metadata during multipart upload");
+   protected void awaitConsistency() {
+      if (view.getConsistencyModel() == ConsistencyModel.EVENTUAL) {
+         Uninterruptibles.sleepUninterruptibly(30, TimeUnit.SECONDS);
+      }
    }
 }

http://git-wip-us.apache.org/repos/asf/jclouds/blob/f397bebe/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/blobstore/integration/SwiftBlobSignerLiveTest.java
----------------------------------------------------------------------
diff --git a/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/blobstore/integration/SwiftBlobSignerLiveTest.java b/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/blobstore/integration/SwiftBlobSignerLiveTest.java
index 9dd603f..fda284c 100644
--- a/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/blobstore/integration/SwiftBlobSignerLiveTest.java
+++ b/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/blobstore/integration/SwiftBlobSignerLiveTest.java
@@ -19,10 +19,14 @@ package org.jclouds.openstack.swift.v1.blobstore.integration;
 import static org.jclouds.openstack.keystone.v2_0.config.KeystoneProperties.CREDENTIAL_TYPE;
 
 import java.util.Properties;
+import java.util.concurrent.TimeUnit;
 
+import org.jclouds.blobstore.attr.ConsistencyModel;
 import org.jclouds.blobstore.integration.internal.BaseBlobSignerLiveTest;
 import org.testng.annotations.Test;
 
+import com.google.common.util.concurrent.Uninterruptibles;
+
 @Test(groups = "live", testName = "SwiftBlobSignerLiveTest")
 public class SwiftBlobSignerLiveTest extends BaseBlobSignerLiveTest {
 
@@ -36,4 +40,11 @@ public class SwiftBlobSignerLiveTest extends BaseBlobSignerLiveTest {
       setIfTestSystemPropertyPresent(props, CREDENTIAL_TYPE);
       return props;
    }
+
+   @Override
+   protected void awaitConsistency() {
+      if (view.getConsistencyModel() == ConsistencyModel.EVENTUAL) {
+         Uninterruptibles.sleepUninterruptibly(30, TimeUnit.SECONDS);
+      }
+   }
 }

http://git-wip-us.apache.org/repos/asf/jclouds/blob/f397bebe/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/blobstore/integration/SwiftContainerIntegrationLiveTest.java
----------------------------------------------------------------------
diff --git a/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/blobstore/integration/SwiftContainerIntegrationLiveTest.java b/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/blobstore/integration/SwiftContainerIntegrationLiveTest.java
index fcabb0a..3f7b564 100644
--- a/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/blobstore/integration/SwiftContainerIntegrationLiveTest.java
+++ b/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/blobstore/integration/SwiftContainerIntegrationLiveTest.java
@@ -20,11 +20,15 @@ import static org.jclouds.openstack.keystone.v2_0.config.KeystoneProperties.CRED
 import static org.testng.Assert.assertTrue;
 
 import java.util.Properties;
+import java.util.concurrent.TimeUnit;
 
+import org.jclouds.blobstore.attr.ConsistencyModel;
 import org.jclouds.blobstore.integration.internal.BaseContainerIntegrationTest;
 import org.testng.annotations.Test;
 import org.testng.SkipException;
 
+import com.google.common.util.concurrent.Uninterruptibles;
+
 @Test(groups = "live", testName = "SwiftContainerIntegrationLiveTest")
 public class SwiftContainerIntegrationLiveTest extends BaseContainerIntegrationTest {
 
@@ -57,4 +61,11 @@ public class SwiftContainerIntegrationLiveTest extends BaseContainerIntegrationT
    public void testDelimiter() throws Exception {
       throw new SkipException("openstack-swift does not implement pseudo-directories");
    }
+
+   @Override
+   protected void awaitConsistency() {
+      if (view.getConsistencyModel() == ConsistencyModel.EVENTUAL) {
+         Uninterruptibles.sleepUninterruptibly(30, TimeUnit.SECONDS);
+      }
+   }
 }

http://git-wip-us.apache.org/repos/asf/jclouds/blob/f397bebe/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/features/ObjectApiMockTest.java
----------------------------------------------------------------------
diff --git a/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/features/ObjectApiMockTest.java b/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/features/ObjectApiMockTest.java
index 81a856e..6091c2a 100644
--- a/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/features/ObjectApiMockTest.java
+++ b/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/features/ObjectApiMockTest.java
@@ -522,7 +522,7 @@ public class ObjectApiMockTest extends BaseOpenStackMockTest<SwiftApi> {
 
          List<String> requestHeaders = copyRequest.getHeaders();
          assertTrue(requestHeaders.contains("X-Object-Meta-someuserheader: someUserMetadataValue"));
-         assertTrue(requestHeaders.contains("content-disposition: attachment; filename=\"fname.ext\""));
+         assertTrue(requestHeaders.contains("Content-Disposition: attachment; filename=\"fname.ext\""));
          assertTrue(requestHeaders.contains(SwiftHeaders.OBJECT_COPY_FROM + ": /bar/foo.txt"));
       } finally {
          server.shutdown();

http://git-wip-us.apache.org/repos/asf/jclouds/blob/f397bebe/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/features/StaticLargeObjectApiMockTest.java
----------------------------------------------------------------------
diff --git a/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/features/StaticLargeObjectApiMockTest.java b/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/features/StaticLargeObjectApiMockTest.java
index c5627a6..ec135c1 100644
--- a/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/features/StaticLargeObjectApiMockTest.java
+++ b/apis/openstack-swift/src/test/java/org/jclouds/openstack/swift/v1/features/StaticLargeObjectApiMockTest.java
@@ -19,6 +19,8 @@ package org.jclouds.openstack.swift.v1.features;
 import static org.jclouds.openstack.swift.v1.reference.SwiftHeaders.OBJECT_METADATA_PREFIX;
 import static org.testng.Assert.assertEquals;
 
+import java.util.List;
+
 import org.jclouds.openstack.swift.v1.SwiftApi;
 import org.jclouds.openstack.swift.v1.domain.Segment;
 import org.jclouds.openstack.v2_0.internal.BaseOpenStackMockTest;
@@ -70,6 +72,99 @@ public class StaticLargeObjectApiMockTest extends BaseOpenStackMockTest<SwiftApi
       }
    }
 
+   public void testReplaceManifestWithHeaders() throws Exception {
+      MockWebServer server = mockOpenStackServer();
+      server.enqueue(addCommonHeaders(new MockResponse().setBody(stringFromResource("/access.json"))));
+      server.enqueue(addCommonHeaders(new MockResponse().addHeader(HttpHeaders.ETAG, "\"abcd\"")));
+
+      try {
+         SwiftApi api = api(server.getUrl("/").toString(), "openstack-swift");
+         assertEquals(
+               api.getStaticLargeObjectApi("DFW", "myContainer").replaceManifest(
+                     "myObject",
+                     ImmutableList
+                           .<Segment>builder()
+                           .add(Segment.builder().path("/mycontainer/objseg1").etag("0228c7926b8b642dfb29554cd1f00963")
+                                 .sizeBytes(1468006).build())
+                           .add(Segment.builder().path("/mycontainer/pseudodir/seg-obj2")
+                                 .etag("5bfc9ea51a00b790717eeb934fb77b9b").sizeBytes(1572864).build())
+                           .add(Segment.builder().path("/other-container/seg-final")
+                                 .etag("b9c3da507d2557c1ddc51f27c54bae51").sizeBytes(256).build()).build(),
+                     ImmutableMap.of("MyFoo", "Bar"),
+                     ImmutableMap.of(
+                           "content-language", "en",
+                           "some-header1", "some-header-value")), "abcd");
+
+         assertEquals(server.getRequestCount(), 2);
+         assertAuthentication(server);
+
+         RecordedRequest replaceRequest = server.takeRequest();
+         assertRequest(replaceRequest, "PUT", "/v1/MossoCloudFS_5bcf396e-39dd-45ff-93a1-712b9aba90a9/myContainer/myObject?multipart-manifest=put");
+         assertEquals(replaceRequest.getHeader(OBJECT_METADATA_PREFIX + "myfoo"), "Bar");
+
+         // Content-length is automatically determined based on manifest size
+         // Setting it will result in an error
+         assertEquals(replaceRequest.getHeader("content-language"), "en");
+         assertEquals(replaceRequest.getHeader("some-header1"), "some-header-value");
+
+         assertEquals(
+               new String(replaceRequest.getBody()),
+               "[{\"path\":\"/mycontainer/objseg1\",\"etag\":\"0228c7926b8b642dfb29554cd1f00963\",\"size_bytes\":1468006}," +
+                     "{\"path\":\"/mycontainer/pseudodir/seg-obj2\",\"etag\":\"5bfc9ea51a00b790717eeb934fb77b9b\",\"size_bytes\":1572864}," +
+                     "{\"path\":\"/other-container/seg-final\",\"etag\":\"b9c3da507d2557c1ddc51f27c54bae51\",\"size_bytes\":256}]");
+      } finally {
+         server.shutdown();
+      }
+   }
+
+   public void testGetManifest() throws Exception {
+      MockWebServer server = mockOpenStackServer();
+      server.enqueue(addCommonHeaders(new MockResponse().setBody(stringFromResource("/access.json"))));
+      server.enqueue(addCommonHeaders(new MockResponse().setResponseCode(200).setBody(
+            stringFromResource("/manifest_get_response.json")) ));
+
+      try {
+         SwiftApi api = api(server.getUrl("/").toString(), "openstack-swift");
+         List<Segment> manifest = api.getStaticLargeObjectApi("DFW", "myContainer").getManifest("myObject");
+
+         // Check response
+         assertEquals(manifest.size(), 3);
+         assertEquals(manifest.get(1).getSizeBytes(), 1572864);
+         assertEquals(manifest.get(1).getETag(), "5bfc9ea51a00b790717eeb934fb77b9b");
+         assertEquals(manifest.get(1).getPath(), "/mycontainer/pseudodir/seg-obj2");
+
+         // Check request
+         assertAuthentication(server);
+         RecordedRequest getRequest = server.takeRequest();
+         assertRequest(getRequest, "GET",
+               "/v1/MossoCloudFS_5bcf396e-39dd-45ff-93a1-712b9aba90a9/myContainer/myObject?multipart-manifest=get");
+      } finally {
+         server.shutdown();
+      }
+   }
+
+   public void testGetManifestFail() throws Exception {
+      MockWebServer server = mockOpenStackServer();
+      server.enqueue(addCommonHeaders(new MockResponse().setBody(stringFromResource("/access.json"))));
+      server.enqueue(addCommonHeaders(new MockResponse().setResponseCode(404).setBody(stringFromResource("/manifest_get_response.json")) ));
+
+      try {
+         SwiftApi api = api(server.getUrl("/").toString(), "openstack-swift");
+         List<Segment> manifest = api.getStaticLargeObjectApi("DFW", "myContainer").getManifest("myObject");
+
+         // Check response
+         assertEquals(manifest.size(), 0);
+
+         // Check request
+         assertAuthentication(server);
+         RecordedRequest getRequest = server.takeRequest();
+         assertRequest(getRequest, "GET",
+               "/v1/MossoCloudFS_5bcf396e-39dd-45ff-93a1-712b9aba90a9/myContainer/myObject?multipart-manifest=get");
+      } finally {
+         server.shutdown();
+      }
+   }
+
    public void testDelete() throws Exception {
       MockWebServer server = mockOpenStackServer();
       server.enqueue(addCommonHeaders(new MockResponse().setBody(stringFromResource("/access.json"))));

http://git-wip-us.apache.org/repos/asf/jclouds/blob/f397bebe/apis/openstack-swift/src/test/resources/manifest_get_response.json
----------------------------------------------------------------------
diff --git a/apis/openstack-swift/src/test/resources/manifest_get_response.json b/apis/openstack-swift/src/test/resources/manifest_get_response.json
new file mode 100644
index 0000000..11b926d
--- /dev/null
+++ b/apis/openstack-swift/src/test/resources/manifest_get_response.json
@@ -0,0 +1,17 @@
+[
+  {
+    "path": "/mycontainer/objseg1",
+    "etag": "0228c7926b8b642dfb29554cd1f00963",
+    "size_bytes": 1468006
+  },
+  {
+    "path": "/mycontainer/pseudodir/seg-obj2",
+    "etag": "5bfc9ea51a00b790717eeb934fb77b9b",
+    "size_bytes": 1572864
+  },
+  {
+    "path": "/other-container/seg-final",
+    "etag": "b9c3da507d2557c1ddc51f27c54bae51",
+    "size_bytes": 256
+  }
+]

http://git-wip-us.apache.org/repos/asf/jclouds/blob/f397bebe/blobstore/src/test/java/org/jclouds/blobstore/integration/internal/BaseBlobIntegrationTest.java
----------------------------------------------------------------------
diff --git a/blobstore/src/test/java/org/jclouds/blobstore/integration/internal/BaseBlobIntegrationTest.java b/blobstore/src/test/java/org/jclouds/blobstore/integration/internal/BaseBlobIntegrationTest.java
index 1dc5d81..deaee57 100644
--- a/blobstore/src/test/java/org/jclouds/blobstore/integration/internal/BaseBlobIntegrationTest.java
+++ b/blobstore/src/test/java/org/jclouds/blobstore/integration/internal/BaseBlobIntegrationTest.java
@@ -488,6 +488,7 @@ public class BaseBlobIntegrationTest extends BaseBlobStoreIntegrationTest {
          addBlobToContainer(container, name2, name2, MediaType.TEXT_PLAIN);
          awaitConsistency();
          view.getBlobStore().removeBlobs(container, ImmutableSet.of(name, name2));
+         awaitConsistency();
          assertContainerEmptyDeleting(container, name);
       } finally {
          returnContainer(container);