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

jclouds-labs git commit: JCLOUDS-1005: Backblaze B2 portable abstraction

Repository: jclouds-labs
Updated Branches:
  refs/heads/master d135fcde4 -> cc8c8a9a2


JCLOUDS-1005: Backblaze B2 portable abstraction


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

Branch: refs/heads/master
Commit: cc8c8a9a238472dbda4769f7a52a3ece3659825a
Parents: d135fcd
Author: Andrew Gaul <ga...@apache.org>
Authored: Wed May 25 18:17:17 2016 -0700
Committer: Andrew Gaul <ga...@apache.org>
Committed: Fri Jun 24 17:14:36 2016 -0700

----------------------------------------------------------------------
 b2/pom.xml                                      |   1 -
 .../main/java/org/jclouds/b2/B2ApiMetadata.java |   6 +-
 .../org/jclouds/b2/blobstore/B2BlobStore.java   | 502 +++++++++++++++++++
 .../config/B2BlobStoreContextModule.java        |  32 ++
 .../java/org/jclouds/b2/domain/B2Object.java    |   7 +-
 .../b2/functions/ParseB2ObjectFromResponse.java |   4 +-
 .../integration/B2BlobIntegrationLiveTest.java  | 264 ++++++++++
 .../blobstore/integration/B2BlobLiveTest.java   |  42 ++
 .../integration/B2BlobSignerLiveTest.java       | 112 +++++
 .../B2ContainerIntegrationLiveTest.java         |  84 ++++
 .../integration/B2ContainerLiveTest.java        |  27 +
 11 files changed, 1075 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cc8c8a9a/b2/pom.xml
----------------------------------------------------------------------
diff --git a/b2/pom.xml b/b2/pom.xml
index 6283223..1dbc48f 100644
--- a/b2/pom.xml
+++ b/b2/pom.xml
@@ -126,7 +126,6 @@
                 </goals>
                 <configuration>
                   <groups>live</groups>
-                  <excludedGroups>livelong</excludedGroups>
                   <systemPropertyVariables>
                     <jclouds.blobstore.httpstream.url>${jclouds.blobstore.httpstream.url}</jclouds.blobstore.httpstream.url>
                     <jclouds.blobstore.httpstream.md5>${jclouds.blobstore.httpstream.md5}</jclouds.blobstore.httpstream.md5>

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cc8c8a9a/b2/src/main/java/org/jclouds/b2/B2ApiMetadata.java
----------------------------------------------------------------------
diff --git a/b2/src/main/java/org/jclouds/b2/B2ApiMetadata.java b/b2/src/main/java/org/jclouds/b2/B2ApiMetadata.java
index b5aecaa..5a0aa68 100644
--- a/b2/src/main/java/org/jclouds/b2/B2ApiMetadata.java
+++ b/b2/src/main/java/org/jclouds/b2/B2ApiMetadata.java
@@ -25,6 +25,8 @@ import java.util.concurrent.TimeUnit;
 import org.jclouds.Constants;
 import org.jclouds.apis.ApiMetadata;
 import org.jclouds.blobstore.BlobStoreContext;
+import org.jclouds.blobstore.reference.BlobStoreConstants;
+import org.jclouds.b2.blobstore.config.B2BlobStoreContextModule;
 import org.jclouds.b2.config.B2HttpApiModule;
 import org.jclouds.rest.internal.BaseHttpApiMetadata;
 
@@ -47,6 +49,7 @@ public final class B2ApiMetadata extends BaseHttpApiMetadata {
 
    public static Properties defaultProperties() {
       Properties properties = BaseHttpApiMetadata.defaultProperties();
+      properties.setProperty(BlobStoreConstants.PROPERTY_USER_METADATA_PREFIX, "X-Bz-Info-");
       properties.setProperty(Constants.PROPERTY_SESSION_INTERVAL, String.valueOf(TimeUnit.HOURS.toSeconds(1)));
       properties.setProperty(Constants.PROPERTY_IDEMPOTENT_METHODS, "DELETE,GET,HEAD,OPTIONS,POST,PUT");
       properties.setProperty(Constants.PROPERTY_RETRY_DELAY_START, String.valueOf(TimeUnit.SECONDS.toMillis(1)));
@@ -66,7 +69,8 @@ public final class B2ApiMetadata extends BaseHttpApiMetadata {
                  .defaultProperties(B2ApiMetadata.defaultProperties())
                  .view(typeToken(BlobStoreContext.class))
                  .defaultModules(ImmutableSet.<Class<? extends Module>>of(
-                         B2HttpApiModule.class));
+                         B2HttpApiModule.class,
+                         B2BlobStoreContextModule.class));
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cc8c8a9a/b2/src/main/java/org/jclouds/b2/blobstore/B2BlobStore.java
----------------------------------------------------------------------
diff --git a/b2/src/main/java/org/jclouds/b2/blobstore/B2BlobStore.java b/b2/src/main/java/org/jclouds/b2/blobstore/B2BlobStore.java
new file mode 100644
index 0000000..5619522
--- /dev/null
+++ b/b2/src/main/java/org/jclouds/b2/blobstore/B2BlobStore.java
@@ -0,0 +1,502 @@
+/*
+ * 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.b2.blobstore;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import javax.inject.Inject;
+
+import org.jclouds.b2.B2Api;
+import org.jclouds.b2.B2ResponseException;
+import org.jclouds.b2.domain.Authorization;
+import org.jclouds.b2.domain.B2Object;
+import org.jclouds.b2.domain.B2ObjectList;
+import org.jclouds.b2.domain.Bucket;
+import org.jclouds.b2.domain.BucketList;
+import org.jclouds.b2.domain.BucketType;
+import org.jclouds.b2.domain.GetUploadPartResponse;
+import org.jclouds.b2.domain.ListPartsResponse;
+import org.jclouds.b2.domain.ListUnfinishedLargeFilesResponse;
+import org.jclouds.b2.domain.MultipartUploadResponse;
+import org.jclouds.b2.domain.UploadFileResponse;
+import org.jclouds.b2.domain.UploadUrlResponse;
+import org.jclouds.b2.domain.UploadPartResponse;
+import org.jclouds.blobstore.BlobStoreContext;
+import org.jclouds.blobstore.ContainerNotFoundException;
+import org.jclouds.blobstore.domain.Blob;
+import org.jclouds.blobstore.domain.BlobAccess;
+import org.jclouds.blobstore.domain.BlobMetadata;
+import org.jclouds.blobstore.domain.ContainerAccess;
+import org.jclouds.blobstore.domain.MultipartPart;
+import org.jclouds.blobstore.domain.MultipartUpload;
+import org.jclouds.blobstore.domain.MutableBlobMetadata;
+import org.jclouds.blobstore.domain.PageSet;
+import org.jclouds.blobstore.domain.StorageMetadata;
+import org.jclouds.blobstore.domain.StorageType;
+import org.jclouds.blobstore.domain.internal.BlobImpl;
+import org.jclouds.blobstore.domain.internal.BlobMetadataImpl;
+import org.jclouds.blobstore.domain.internal.MutableBlobMetadataImpl;
+import org.jclouds.blobstore.domain.internal.PageSetImpl;
+import org.jclouds.blobstore.domain.internal.StorageMetadataImpl;
+import org.jclouds.blobstore.functions.BlobToHttpGetOptions;
+import org.jclouds.blobstore.internal.BaseBlobStore;
+import org.jclouds.blobstore.options.CreateContainerOptions;
+import org.jclouds.blobstore.options.GetOptions;
+import org.jclouds.blobstore.options.ListContainerOptions;
+import org.jclouds.blobstore.options.PutOptions;
+import org.jclouds.blobstore.util.BlobUtils;
+import org.jclouds.collect.Memoized;
+import org.jclouds.domain.Location;
+import org.jclouds.io.ByteStreams2;
+import org.jclouds.io.ContentMetadata;
+import org.jclouds.io.ContentMetadataBuilder;
+import org.jclouds.io.MutableContentMetadata;
+import org.jclouds.io.Payload;
+import org.jclouds.io.PayloadSlicer;
+import org.jclouds.io.payloads.BaseMutableContentMetadata;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.base.Supplier;
+import com.google.common.base.Throwables;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Sets;
+import com.google.common.hash.Hashing;
+import com.google.common.net.HttpHeaders;
+import com.google.common.util.concurrent.UncheckedExecutionException;
+
+public final class B2BlobStore extends BaseBlobStore {
+   private final B2Api api;
+   private final BlobToHttpGetOptions blob2ObjectGetOptions;
+   private final LoadingCache<String, Bucket> bucketNameToBucket;
+   private final Supplier<Authorization> auth;
+
+   @Inject
+   B2BlobStore(BlobStoreContext context, BlobUtils blobUtils, Supplier<Location> defaultLocation,
+            @Memoized Supplier<Set<? extends Location>> locations, PayloadSlicer slicer, final B2Api api,
+            BlobToHttpGetOptions blob2ObjectGetOptions, @Memoized Supplier<Authorization> auth) {
+      super(context, blobUtils, defaultLocation, locations, slicer);
+      this.api = api;
+      this.blob2ObjectGetOptions = blob2ObjectGetOptions;
+      this.auth = auth;
+      this.bucketNameToBucket = CacheBuilder.newBuilder()
+            .expireAfterWrite(5, TimeUnit.MINUTES)
+            .build(new CacheLoader<String, Bucket>() {
+               @Override
+               public Bucket load(String bucketName) {
+                  BucketList list = api.getBucketApi().listBuckets();
+                  for (Bucket bucket : list.buckets()) {
+                     if (bucket.bucketName().equals(bucketName)) {
+                        return bucket;
+                     }
+                  }
+                  throw new ContainerNotFoundException(bucketName, null);
+               }
+            });
+   }
+
+   @Override
+   public PageSet<? extends StorageMetadata> list() {
+      ImmutableList.Builder<StorageMetadata> builder = ImmutableList.builder();
+      BucketList list = api.getBucketApi().listBuckets();
+      for (Bucket bucket : list.buckets()) {
+         builder.add(new StorageMetadataImpl(StorageType.CONTAINER, null, bucket.bucketName(), defaultLocation.get(), null, null, null, null, ImmutableMap.<String, String>of(), null));
+      }
+      return new PageSetImpl<StorageMetadata>(builder.build(), null);
+   }
+
+   @Override
+   public boolean containerExists(String container) {
+      BucketList list = api.getBucketApi().listBuckets();
+      for (Bucket bucket : list.buckets()) {
+         if (bucket.bucketName().equals(container)) {
+            return true;
+         }
+      }
+      return false;
+   }
+
+   @Override
+   public boolean createContainerInLocation(Location location, String container) {
+      return createContainerInLocation(location, container, CreateContainerOptions.NONE);
+   }
+
+   @Override
+   public boolean createContainerInLocation(Location location, String container, CreateContainerOptions options) {
+      BucketType bucketType = options.isPublicRead() ? BucketType.ALL_PUBLIC : BucketType.ALL_PRIVATE;
+      try {
+         Bucket bucket = api.getBucketApi().createBucket(container, bucketType);
+         bucketNameToBucket.put(container, bucket);
+      } catch (B2ResponseException bre) {
+         if (bre.getError().code().equals("duplicate_bucket_name")) {
+            return false;
+         }
+         throw bre;
+      }
+      return true;
+   }
+
+   @Override
+   public ContainerAccess getContainerAccess(String container) {
+      Bucket bucket = getBucket(container);
+      return bucket.bucketType() == BucketType.ALL_PUBLIC ? ContainerAccess.PUBLIC_READ : ContainerAccess.PRIVATE;
+   }
+
+   @Override
+   public void setContainerAccess(String container, ContainerAccess access) {
+      Bucket bucket = getBucket(container);
+
+      BucketType bucketType = access == ContainerAccess.PUBLIC_READ ? BucketType.ALL_PUBLIC : BucketType.ALL_PRIVATE;
+      bucket = api.getBucketApi().updateBucket(bucket.bucketId(), bucketType);
+      bucketNameToBucket.put(container, bucket);
+   }
+
+   @Override
+   public PageSet<? extends StorageMetadata> list(String container) {
+      return list(container, ListContainerOptions.NONE);
+   }
+
+   @Override
+   public PageSet<? extends StorageMetadata> list(String container, ListContainerOptions options) {
+      Preconditions.checkArgument(options.getDir() == null, "B2 does not support directories");
+      String delimiter = null;
+      if (!options.isRecursive()) {
+         delimiter = "/";
+      }
+      if (options.getDelimiter() != null) {
+         delimiter = options.getDelimiter();
+      }
+
+      Bucket bucket = getBucket(container);
+
+      int size = 0;
+      ImmutableList.Builder<StorageMetadata> builder = ImmutableList.builder();
+      Set<String> commonPrefixes = Sets.newHashSet();
+      String marker = options.getMarker();
+      while (true) {
+         B2ObjectList list = api.getObjectApi().listFileNames(bucket.bucketId(), marker, options.getMaxResults());
+         for (B2ObjectList.Entry entry : list.files()) {
+            // B2 does not support server-side filtering via prefix and delimiter so we emulate it on the client.
+            if (options.getPrefix() != null && !entry.fileName().startsWith(options.getPrefix())) {
+               continue;
+            }
+
+            if (delimiter != null) {
+               String fileName = entry.fileName();
+               int index = entry.fileName().indexOf(delimiter, Strings.nullToEmpty(options.getPrefix()).length());
+               if (index != -1) {
+                  String prefix = entry.fileName().substring(0, index + 1);
+                  if (!commonPrefixes.contains(prefix)) {
+                     commonPrefixes.add(prefix);
+                     ++size;
+                     builder.add(new StorageMetadataImpl(StorageType.RELATIVE_PATH, null, prefix, null, null, null, null, null, ImmutableMap.<String, String>of(), null));
+                  }
+                  continue;
+               }
+            }
+
+            if (options.isDetailed()) {
+               BlobMetadata metadata = blobMetadata(container, entry.fileName());
+               if (metadata != null) {
+                  ++size;
+                  builder.add(metadata);
+               }
+            } else {
+               Map<String, String> userMetadata = ImmutableMap.of();
+               ContentMetadata metadata = ContentMetadataBuilder.create()
+                     .contentLength(entry.size())
+                     .build();
+               ++size;
+               builder.add(new BlobMetadataImpl(null, entry.fileName(), null, null, null, null, entry.uploadTimestamp(), userMetadata, null, container, metadata, entry.size()));
+            }
+         }
+         marker = list.nextFileName();
+         if (marker == null || options.getMaxResults() == null || size == options.getMaxResults()) {
+            break;
+         }
+      }
+      return new PageSetImpl<StorageMetadata>(builder.build(), marker);
+   }
+
+   @Override
+   public boolean blobExists(String container, String name) {
+      return blobMetadata(container, name) != null;
+   }
+
+   @Override
+   public String putBlob(String container, Blob blob) {
+      return putBlob(container, blob, PutOptions.NONE);
+   }
+
+   @Override
+   public String putBlob(String container, Blob blob, PutOptions options) {
+      Preconditions.checkArgument(blob.getPayload().isRepeatable(), "B2 requires repeatable payload to calculate SHA1");
+
+      if (options.getBlobAccess() != BlobAccess.PRIVATE) {
+         throw new UnsupportedOperationException("B2 only supports private access blobs");
+      }
+
+      if (options.isMultipart()) {
+         return putMultipartBlob(container, blob, options);
+      } else {
+         String contentSha1;
+         try {
+            contentSha1 = ByteStreams2.hashAndClose(blob.getPayload().openStream(), Hashing.sha1()).toString();
+         } catch (IOException ioe) {
+            throw Throwables.propagate(ioe);
+         }
+
+         String name = blob.getMetadata().getName();
+
+         // B2 versions all files so we store the original fileId to delete it after the upload succeeds
+         String oldFileId = getFileId(container, name);
+
+         Bucket bucket = getBucket(container);
+         UploadUrlResponse uploadUrl = api.getObjectApi().getUploadUrl(bucket.bucketId());
+         UploadFileResponse uploadFile = api.getObjectApi().uploadFile(uploadUrl, name, contentSha1, blob.getMetadata().getUserMetadata(), blob.getPayload());
+
+         if (oldFileId != null) {
+            api.getObjectApi().deleteFileVersion(name, oldFileId);
+         }
+
+         return uploadFile.contentSha1();  // B2 does not support ETag, fake it with SHA-1
+      }
+   }
+
+   @Override
+   public BlobMetadata blobMetadata(String container, String name) {
+      String fileId = getFileId(container, name);
+      if (fileId == null) {
+         return null;
+      }
+
+      B2Object b2Object = api.getObjectApi().getFileInfo(fileId);
+      if (b2Object == null) {
+         return null;
+      }
+
+      return toBlobMetadata(container, b2Object);
+   }
+
+   @Override
+   public Blob getBlob(String container, String name, GetOptions options) {
+      if (options.getIfMatch() != null ||
+            options.getIfNoneMatch() != null ||
+            options.getIfModifiedSince() != null ||
+            options.getIfUnmodifiedSince() != null) {
+         throw new UnsupportedOperationException("B2 does not support conditional get");
+      }
+
+      B2Object b2Object = api.getObjectApi().downloadFileByName(container, name, blob2ObjectGetOptions.apply(options));
+      if (b2Object == null) {
+         return null;
+      }
+
+      MutableBlobMetadata metadata = toBlobMetadata(container, b2Object);
+      Blob blob = new BlobImpl(metadata);
+      blob.setPayload(b2Object.payload());
+      if (b2Object.contentRange() != null) {
+         blob.getAllHeaders().put(HttpHeaders.CONTENT_RANGE, b2Object.contentRange());
+      }
+      return blob;
+   }
+
+   @Override
+   public void removeBlob(String container, String name) {
+      String fileId = getFileId(container, name);
+      if (fileId == null) {
+         return;
+      }
+
+      api.getObjectApi().deleteFileVersion(name, fileId);
+   }
+
+   @Override
+   public BlobAccess getBlobAccess(String container, String name) {
+      return BlobAccess.PRIVATE;
+   }
+
+   @Override
+   public void setBlobAccess(String container, String name, BlobAccess access) {
+      throw new UnsupportedOperationException("B2 does not support object access control");
+   }
+
+   @Override
+   public void deleteContainer(String container) {
+      // Explicitly abort multi-part uploads which B2 requires to delete a bucket but other providers do not.
+      try {
+         for (MultipartUpload upload : listMultipartUploads(container)) {
+            abortMultipartUpload(upload);
+         }
+      } catch (ContainerNotFoundException cnfe) {
+         // ignore
+      }
+
+      super.deleteContainer(container);
+   }
+
+   @Override
+   protected boolean deleteAndVerifyContainerGone(String container) {
+      Bucket bucket = getBucket(container);
+      try {
+         api.getBucketApi().deleteBucket(bucket.bucketId());
+      } catch (B2ResponseException bre) {
+         if (bre.getError().code().equals("cannot_delete_non_empty_bucket")) {
+            return false;
+         }
+         throw bre;
+      }
+      return true;
+   }
+
+   @Override
+   public MultipartUpload initiateMultipartUpload(String container, BlobMetadata blobMetadata, PutOptions options) {
+      Bucket bucket = getBucket(container);
+      MultipartUploadResponse response = api.getMultipartApi().startLargeFile(bucket.bucketId(), blobMetadata.getName(), blobMetadata.getContentMetadata().getContentType(), blobMetadata.getUserMetadata());
+      return MultipartUpload.create(container, blobMetadata.getName(), response.fileId(), blobMetadata, options);
+   }
+
+   @Override
+   public void abortMultipartUpload(MultipartUpload mpu) {
+      api.getMultipartApi().cancelLargeFile(mpu.id());
+   }
+
+   @Override
+   public String completeMultipartUpload(MultipartUpload mpu, List<MultipartPart> parts) {
+      ImmutableList.Builder<String> sha1 = ImmutableList.builder();
+      for (MultipartPart part : parts) {
+         sha1.add(part.partETag());
+      }
+      B2Object b2Object = api.getMultipartApi().finishLargeFile(mpu.id(), sha1.build());
+      return b2Object.contentSha1();  // this is always "none"
+   }
+
+   @Override
+   public MultipartPart uploadMultipartPart(MultipartUpload mpu, int partNumber, Payload payload) {
+      Preconditions.checkArgument(payload.isRepeatable());
+
+      String contentSha1;
+      try {
+         contentSha1 = ByteStreams2.hashAndClose(payload.openStream(), Hashing.sha1()).toString();
+      } catch (IOException ioe) {
+         throw Throwables.propagate(ioe);
+      }
+
+      GetUploadPartResponse getUploadPart = api.getMultipartApi().getUploadPartUrl(mpu.id());
+      UploadPartResponse uploadPart = api.getMultipartApi().uploadPart(getUploadPart, partNumber, contentSha1, payload);
+
+      return MultipartPart.create(uploadPart.partNumber(), uploadPart.contentLength(), uploadPart.contentSha1());
+   }
+
+   @Override
+   public List<MultipartPart> listMultipartUpload(MultipartUpload mpu) {
+      ListPartsResponse response = api.getMultipartApi().listParts(mpu.id(), null, null);
+      ImmutableList.Builder<MultipartPart> parts = ImmutableList.builder();
+      for (ListPartsResponse.Entry entry : response.parts()) {
+         parts.add(MultipartPart.create(entry.partNumber(), entry.contentLength(), entry.contentSha1()));
+      }
+      return parts.build();
+   }
+
+   @Override
+   public List<MultipartUpload> listMultipartUploads(String container) {
+      ImmutableList.Builder<MultipartUpload> builder = ImmutableList.builder();
+      Bucket bucket = getBucket(container);
+
+      String marker = null;
+      while (true) {
+         ListUnfinishedLargeFilesResponse response = api.getMultipartApi().listUnfinishedLargeFiles(bucket.bucketId(), marker, null);
+         for (ListUnfinishedLargeFilesResponse.Entry entry : response.files()) {
+            builder.add(MultipartUpload.create(container, entry.fileName(), entry.fileId(), null, null));
+         }
+
+         if (response.nextFileId() == null || response.files().isEmpty()) {
+            break;
+         }
+      }
+
+      return builder.build();
+   }
+
+   @Override
+   public long getMinimumMultipartPartSize() {
+      return auth.get().minimumPartSize();
+   }
+
+   @Override
+   public long getMaximumMultipartPartSize() {
+      return 5L * 1024L * 1024L * 1024L;
+   }
+
+   @Override
+   public int getMaximumNumberOfParts() {
+      return 10 * 1000;
+   }
+
+   private Bucket getBucket(String container) {
+      Bucket bucket;
+      try {
+         bucket = bucketNameToBucket.getUnchecked(container);
+      } catch (UncheckedExecutionException uee) {
+         if (uee.getCause() instanceof ContainerNotFoundException) {
+            throw (ContainerNotFoundException) uee.getCause();
+         }
+         throw uee;
+      }
+      return bucket;
+   }
+
+   private String getFileId(String container, String name) {
+      Bucket bucket = getBucket(container);
+      B2ObjectList list = api.getObjectApi().listFileNames(bucket.bucketId(), name, 1);
+      if (list.files().isEmpty()) {
+         return null;
+      }
+
+      B2ObjectList.Entry entry = list.files().get(0);
+      if (!entry.fileName().equals(name)) {
+         return null;
+      }
+
+      return entry.fileId();
+   }
+
+   private MutableBlobMetadata toBlobMetadata(String container, B2Object b2Object) {
+      MutableBlobMetadata metadata = new MutableBlobMetadataImpl();
+      metadata.setContainer(container);
+      metadata.setETag(b2Object.contentSha1());  // B2 does not support ETag, fake it with SHA-1
+      metadata.setLastModified(b2Object.uploadTimestamp());
+      metadata.setName(b2Object.fileName());
+      metadata.setSize(b2Object.contentLength());
+      MutableContentMetadata contentMetadata = new BaseMutableContentMetadata();
+      contentMetadata.setContentLength(b2Object.contentLength());
+      contentMetadata.setContentType(b2Object.contentType());
+      metadata.setContentMetadata(contentMetadata);
+      metadata.setUserMetadata(b2Object.fileInfo());
+      metadata.setPublicUri(URI.create(auth.get().downloadUrl() + "/file/" + container + "/" + b2Object.fileName()));
+      return metadata;
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cc8c8a9a/b2/src/main/java/org/jclouds/b2/blobstore/config/B2BlobStoreContextModule.java
----------------------------------------------------------------------
diff --git a/b2/src/main/java/org/jclouds/b2/blobstore/config/B2BlobStoreContextModule.java b/b2/src/main/java/org/jclouds/b2/blobstore/config/B2BlobStoreContextModule.java
new file mode 100644
index 0000000..151390b
--- /dev/null
+++ b/b2/src/main/java/org/jclouds/b2/blobstore/config/B2BlobStoreContextModule.java
@@ -0,0 +1,32 @@
+/*
+ * 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.b2.blobstore.config;
+
+import org.jclouds.blobstore.BlobStore;
+import org.jclouds.blobstore.attr.ConsistencyModel;
+import org.jclouds.b2.blobstore.B2BlobStore;
+
+import com.google.inject.AbstractModule;
+import com.google.inject.Scopes;
+
+public final class B2BlobStoreContextModule extends AbstractModule {
+   @Override
+   protected void configure() {
+      bind(ConsistencyModel.class).toInstance(ConsistencyModel.EVENTUAL);
+      bind(BlobStore.class).to(B2BlobStore.class).in(Scopes.SINGLETON);
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cc8c8a9a/b2/src/main/java/org/jclouds/b2/domain/B2Object.java
----------------------------------------------------------------------
diff --git a/b2/src/main/java/org/jclouds/b2/domain/B2Object.java b/b2/src/main/java/org/jclouds/b2/domain/B2Object.java
index bb49667..b5ac4e3 100644
--- a/b2/src/main/java/org/jclouds/b2/domain/B2Object.java
+++ b/b2/src/main/java/org/jclouds/b2/domain/B2Object.java
@@ -39,13 +39,14 @@ public abstract class B2Object {
    @Nullable public abstract String bucketId();
    @Nullable public abstract Long contentLength();
    @Nullable public abstract String contentType();
+   @Nullable public abstract String contentRange();
 
-   @SerializedNames({"fileId", "fileName", "accountId", "bucketId", "contentLength", "contentSha1", "contentType", "fileInfo", "action", "uploadTimestamp", "payload"})
-   public static B2Object create(String fileId, String fileName, @Nullable String accountId, @Nullable String bucketId, @Nullable Long contentLength, @Nullable String contentSha1, @Nullable String contentType, @Nullable Map<String, String> fileInfo, @Nullable Action action, @Nullable Long uploadTimestamp, @Nullable Payload payload) {
+   @SerializedNames({"fileId", "fileName", "accountId", "bucketId", "contentLength", "contentSha1", "contentType", "fileInfo", "action", "uploadTimestamp", "contentRange", "payload"})
+   public static B2Object create(String fileId, String fileName, @Nullable String accountId, @Nullable String bucketId, @Nullable Long contentLength, @Nullable String contentSha1, @Nullable String contentType, @Nullable Map<String, String> fileInfo, @Nullable Action action, @Nullable Long uploadTimestamp, @Nullable String contentRange, @Nullable Payload payload) {
       if (fileInfo != null) {
          fileInfo = ImmutableMap.copyOf(fileInfo);
       }
       Date date = uploadTimestamp == null ? null : new Date(uploadTimestamp);
-      return new AutoValue_B2Object(fileId, fileName, contentSha1, fileInfo, payload, date, action, accountId, bucketId, contentLength, contentType);
+      return new AutoValue_B2Object(fileId, fileName, contentSha1, fileInfo, payload, date, action, accountId, bucketId, contentLength, contentType, contentRange);
    }
 }

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cc8c8a9a/b2/src/main/java/org/jclouds/b2/functions/ParseB2ObjectFromResponse.java
----------------------------------------------------------------------
diff --git a/b2/src/main/java/org/jclouds/b2/functions/ParseB2ObjectFromResponse.java b/b2/src/main/java/org/jclouds/b2/functions/ParseB2ObjectFromResponse.java
index f98e315..3d122aa 100644
--- a/b2/src/main/java/org/jclouds/b2/functions/ParseB2ObjectFromResponse.java
+++ b/b2/src/main/java/org/jclouds/b2/functions/ParseB2ObjectFromResponse.java
@@ -30,6 +30,7 @@ import org.jclouds.b2.reference.B2Headers;
 import com.google.common.base.Function;
 import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.net.HttpHeaders;
 
 public final class ParseB2ObjectFromResponse implements Function<HttpResponse, B2Object> {
    @Override
@@ -58,7 +59,8 @@ public final class ParseB2ObjectFromResponse implements Function<HttpResponse, B
          }
       }
       Date uploadTimestamp = new Date(Long.parseLong(from.getFirstHeaderOrNull(B2Headers.UPLOAD_TIMESTAMP)));
+      String contentRange = from.getFirstHeaderOrNull(HttpHeaders.CONTENT_RANGE);
 
-      return B2Object.create(fileId, fileName, null, null, contentMeta.getContentLength(), contentSha1, contentMeta.getContentType(), fileInfo.build(), null, uploadTimestamp.getTime(), payload);
+      return B2Object.create(fileId, fileName, null, null, contentMeta.getContentLength(), contentSha1, contentMeta.getContentType(), fileInfo.build(), null, uploadTimestamp.getTime(), contentRange, payload);
    }
 }

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cc8c8a9a/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2BlobIntegrationLiveTest.java
----------------------------------------------------------------------
diff --git a/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2BlobIntegrationLiveTest.java b/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2BlobIntegrationLiveTest.java
new file mode 100644
index 0000000..ae2326e
--- /dev/null
+++ b/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2BlobIntegrationLiveTest.java
@@ -0,0 +1,264 @@
+/*
+ * 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.b2.blobstore.integration;
+
+import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+
+import org.jclouds.blobstore.domain.Blob;
+import org.jclouds.blobstore.domain.BlobMetadata;
+import org.jclouds.blobstore.domain.BlobBuilder.PayloadBlobBuilder;
+import org.jclouds.blobstore.integration.internal.BaseBlobIntegrationTest;
+import org.testng.SkipException;
+import org.testng.annotations.Test;
+
+@Test(groups = { "live", "blobstorelive" })
+public final class B2BlobIntegrationLiveTest extends BaseBlobIntegrationTest {
+   public B2BlobIntegrationLiveTest() throws IOException {
+      provider = "b2";
+   }
+
+   @Override
+   protected long getMinimumMultipartBlobSize() {
+      return view.getBlobStore().getMinimumMultipartPartSize() + 1;
+   }
+
+   @Override
+   protected void addContentMetadata(PayloadBlobBuilder blobBuilder) {
+      blobBuilder.contentType("text/csv");
+      // B2 does not support the following:
+      //blobBuilder.contentDisposition("attachment; filename=photo.jpg");
+      //blobBuilder.contentEncoding("gzip");
+      //blobBuilder.contentLanguage("en");
+   }
+
+   @Override
+   protected void checkContentMetadata(Blob blob) {
+      checkContentType(blob, "text/csv");
+      // B2 does not support the following:
+      //checkContentDisposition(blob, "attachment; filename=photo.jpg");
+      //checkContentEncoding(blob, "gzip");
+      //checkContentLanguage(blob, "en");
+   }
+
+   @Override
+   protected void checkMD5(BlobMetadata metadata) throws IOException {
+      // B2 does not support Content-MD5
+   }
+
+   @Override
+   public void testCopyBlobCopyMetadata() throws Exception {
+      try {
+         super.testCopyBlobCopyMetadata();
+         failBecauseExceptionWasNotThrown(IllegalArgumentException.class);
+      } catch (IllegalArgumentException iae) {
+         throw new SkipException("B2 requires repeatable payloads to calculate SHA1 hash", iae);
+      }
+   }
+
+   @Override
+   public void testCopyBlobReplaceMetadata() throws Exception {
+      try {
+         super.testCopyBlobReplaceMetadata();
+         failBecauseExceptionWasNotThrown(IllegalArgumentException.class);
+      } catch (IllegalArgumentException iae) {
+         throw new SkipException("B2 requires repeatable payloads to calculate SHA1 hash", iae);
+      }
+   }
+
+   @Override
+   public void testCopyIfMatch() throws Exception {
+      try {
+         super.testCopyIfMatch();
+         failBecauseExceptionWasNotThrown(IllegalArgumentException.class);
+      } catch (IllegalArgumentException iae) {
+         throw new SkipException("B2 requires repeatable payloads to calculate SHA1 hash", iae);
+      }
+   }
+
+   @Override
+   public void testCopyIfNoneMatch() throws Exception {
+      try {
+         super.testCopyIfNoneMatch();
+         failBecauseExceptionWasNotThrown(IllegalArgumentException.class);
+      } catch (IllegalArgumentException iae) {
+         throw new SkipException("B2 requires repeatable payloads to calculate SHA1 hash", iae);
+      }
+   }
+
+   @Override
+   public void testCopyIfModifiedSince() throws Exception {
+      try {
+         super.testCopyIfModifiedSince();
+         failBecauseExceptionWasNotThrown(IllegalArgumentException.class);
+      } catch (IllegalArgumentException iae) {
+         throw new SkipException("B2 requires repeatable payloads to calculate SHA1 hash", iae);
+      }
+   }
+
+   @Override
+   public void testCopyIfUnmodifiedSince() throws Exception {
+      try {
+         super.testCopyIfUnmodifiedSince();
+         failBecauseExceptionWasNotThrown(IllegalArgumentException.class);
+      } catch (IllegalArgumentException iae) {
+         throw new SkipException("B2 requires repeatable payloads to calculate SHA1 hash", iae);
+      }
+   }
+
+   @Override
+   public void testPutInputStream() throws Exception {
+      try {
+         super.testPutInputStream();
+         failBecauseExceptionWasNotThrown(IllegalArgumentException.class);
+      } catch (IllegalArgumentException iae) {
+         throw new SkipException("B2 requires repeatable payloads to calculate SHA1 hash", iae);
+      }
+   }
+
+   @Override
+   public void testPutMultipartInputStream() throws Exception {
+      try {
+         super.testPutMultipartInputStream();
+         failBecauseExceptionWasNotThrown(IllegalArgumentException.class);
+      } catch (IllegalArgumentException iae) {
+         throw new SkipException("B2 requires repeatable payloads to calculate SHA1 hash", iae);
+      }
+   }
+
+   @Override
+   public void testPutObjectStream() throws InterruptedException, IOException, ExecutionException {
+      try {
+         super.testPutObjectStream();
+         failBecauseExceptionWasNotThrown(IllegalArgumentException.class);
+      } catch (IllegalArgumentException iae) {
+         throw new SkipException("B2 requires repeatable payloads to calculate SHA1 hash", iae);
+      }
+   }
+
+   @Override
+   public void testPutIncorrectContentMD5() throws InterruptedException, IOException {
+      try {
+         super.testPutIncorrectContentMD5();
+         failBecauseExceptionWasNotThrown(AssertionError.class);
+      } catch (AssertionError ae) {
+         throw new SkipException("B2 does not enforce Content-MD5", ae);
+      }
+   }
+
+   @Override
+   public void testCreateBlobWithExpiry() throws InterruptedException {
+      try {
+         super.testCreateBlobWithExpiry();
+         failBecauseExceptionWasNotThrown(IllegalArgumentException.class);
+      } catch (IllegalArgumentException iae) {
+         throw new SkipException("B2 does not allow Expires header", iae);
+      }
+   }
+
+   @Override
+   public void testSetBlobAccess() throws Exception {
+      try {
+         super.testSetBlobAccess();
+         failBecauseExceptionWasNotThrown(UnsupportedOperationException.class);
+      } catch (UnsupportedOperationException uoe) {
+         throw new SkipException("not supported on B2", uoe);
+      }
+   }
+
+   @Override
+   public void testPutBlobAccess() throws Exception {
+      try {
+         super.testPutBlobAccess();
+         failBecauseExceptionWasNotThrown(UnsupportedOperationException.class);
+      } catch (UnsupportedOperationException uoe) {
+         throw new SkipException("not supported on B2", uoe);
+      }
+   }
+
+   @Override
+   public void testPutBlobAccessMultipart() throws Exception {
+      try {
+         super.testPutBlobAccessMultipart();
+         failBecauseExceptionWasNotThrown(UnsupportedOperationException.class);
+      } catch (UnsupportedOperationException uoe) {
+         throw new SkipException("not supported on B2", uoe);
+      }
+   }
+
+   @Override
+   public void testGetIfModifiedSince() throws InterruptedException {
+      try {
+         super.testGetIfModifiedSince();
+         failBecauseExceptionWasNotThrown(UnsupportedOperationException.class);
+      } catch (UnsupportedOperationException uoe) {
+         throw new SkipException("not supported on B2", uoe);
+      }
+   }
+
+   @Override
+   public void testGetIfUnmodifiedSince() throws InterruptedException {
+      try {
+         super.testGetIfUnmodifiedSince();
+         failBecauseExceptionWasNotThrown(UnsupportedOperationException.class);
+      } catch (UnsupportedOperationException uoe) {
+         throw new SkipException("not supported on B2", uoe);
+      }
+   }
+
+   @Override
+   public void testGetIfMatch() throws InterruptedException {
+      try {
+         super.testGetIfMatch();
+         failBecauseExceptionWasNotThrown(UnsupportedOperationException.class);
+      } catch (UnsupportedOperationException uoe) {
+         throw new SkipException("not supported on B2", uoe);
+      }
+   }
+
+   @Override
+   public void testGetIfNoneMatch() throws InterruptedException {
+      try {
+         super.testGetIfNoneMatch();
+         failBecauseExceptionWasNotThrown(UnsupportedOperationException.class);
+      } catch (UnsupportedOperationException uoe) {
+         throw new SkipException("not supported on B2", uoe);
+      }
+   }
+
+   @Override
+   public void testGetRangeOutOfRange() throws InterruptedException, IOException {
+      try {
+         super.testGetRangeOutOfRange();
+         failBecauseExceptionWasNotThrown(AssertionError.class);
+      } catch (AssertionError ae) {
+         throw new SkipException("B2 does not error on invalid ranges", ae);
+      }
+   }
+
+   @Override
+   public void testMultipartUploadSinglePart() throws Exception {
+      try {
+         super.testMultipartUploadSinglePart();
+         failBecauseExceptionWasNotThrown(IllegalArgumentException.class);
+      } catch (IllegalArgumentException iae) {
+         throw new SkipException("B2 requires at least two parts", iae);
+      }
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cc8c8a9a/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2BlobLiveTest.java
----------------------------------------------------------------------
diff --git a/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2BlobLiveTest.java b/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2BlobLiveTest.java
new file mode 100644
index 0000000..34e4950
--- /dev/null
+++ b/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2BlobLiveTest.java
@@ -0,0 +1,42 @@
+/*
+ * 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.b2.blobstore.integration;
+
+import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown;
+
+import org.jclouds.blobstore.integration.internal.BaseBlobLiveTest;
+
+import org.testng.SkipException;
+import org.testng.annotations.Optional;
+import org.testng.annotations.Test;
+
+@Test(groups = { "live" })
+public final class B2BlobLiveTest extends BaseBlobLiveTest {
+   public B2BlobLiveTest() {
+      provider = "b2";
+   }
+
+   @Override
+   public void testCopyUrl(@Optional String httpStreamUrl, @Optional String httpStreamMD5) throws Exception {
+      try {
+         super.testCopyUrl(httpStreamUrl, httpStreamMD5);
+         failBecauseExceptionWasNotThrown(IllegalArgumentException.class);
+      } catch (IllegalArgumentException iae) {
+         throw new SkipException("B2 requires repeatable payloads to calculate SHA1 hash", iae);
+      }
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cc8c8a9a/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2BlobSignerLiveTest.java
----------------------------------------------------------------------
diff --git a/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2BlobSignerLiveTest.java b/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2BlobSignerLiveTest.java
new file mode 100644
index 0000000..7a027f0
--- /dev/null
+++ b/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2BlobSignerLiveTest.java
@@ -0,0 +1,112 @@
+/*
+ * 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.b2.blobstore.integration;
+
+import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown;
+
+import java.io.IOException;
+
+import org.jclouds.blobstore.integration.internal.BaseBlobSignerLiveTest;
+import org.testng.SkipException;
+import org.testng.annotations.Test;
+
+@Test(groups = { "live" })
+public final class B2BlobSignerLiveTest extends BaseBlobSignerLiveTest {
+   public B2BlobSignerLiveTest() {
+      provider = "b2";
+   }
+
+   @Test
+   public void testSignGetUrl() throws Exception {
+      try {
+         super.testSignGetUrl();
+         failBecauseExceptionWasNotThrown(UnsupportedOperationException.class);
+      } catch (UnsupportedOperationException uoe) {
+         throw new SkipException("unsupported by B2", uoe);
+      }
+   }
+
+   @Test
+   public void testSignGetUrlOptions() throws Exception {
+      try {
+         super.testSignGetUrlOptions();
+         failBecauseExceptionWasNotThrown(UnsupportedOperationException.class);
+      } catch (UnsupportedOperationException uoe) {
+         throw new SkipException("unsupported by B2", uoe);
+      }
+   }
+
+   @Test
+   public void testSignGetUrlWithTime() throws InterruptedException, IOException {
+      try {
+         super.testSignGetUrlWithTime();
+         failBecauseExceptionWasNotThrown(UnsupportedOperationException.class);
+      } catch (UnsupportedOperationException uoe) {
+         throw new SkipException("unsupported by B2", uoe);
+      }
+   }
+
+   @Test
+   public void testSignGetUrlWithTimeExpired() throws InterruptedException, IOException {
+      try {
+         super.testSignGetUrlWithTimeExpired();
+         failBecauseExceptionWasNotThrown(UnsupportedOperationException.class);
+      } catch (UnsupportedOperationException uoe) {
+         throw new SkipException("unsupported by B2", uoe);
+      }
+   }
+
+   @Test
+   public void testSignPutUrl() throws Exception {
+      try {
+         super.testSignPutUrl();
+         failBecauseExceptionWasNotThrown(UnsupportedOperationException.class);
+      } catch (UnsupportedOperationException uoe) {
+         throw new SkipException("unsupported by B2", uoe);
+      }
+   }
+
+   @Test
+   public void testSignPutUrlWithTime() throws Exception {
+      try {
+         super.testSignPutUrlWithTime();
+         failBecauseExceptionWasNotThrown(UnsupportedOperationException.class);
+      } catch (UnsupportedOperationException uoe) {
+         throw new SkipException("unsupported by B2", uoe);
+      }
+   }
+
+   @Test
+   public void testSignPutUrlWithTimeExpired() throws Exception {
+      try {
+         super.testSignPutUrlWithTimeExpired();
+         failBecauseExceptionWasNotThrown(UnsupportedOperationException.class);
+      } catch (UnsupportedOperationException uoe) {
+         throw new SkipException("unsupported by B2", uoe);
+      }
+   }
+
+   @Test
+   public void testSignRemoveUrl() throws Exception {
+      try {
+         super.testSignRemoveUrl();
+         failBecauseExceptionWasNotThrown(UnsupportedOperationException.class);
+      } catch (UnsupportedOperationException uoe) {
+         throw new SkipException("unsupported by B2", uoe);
+      }
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cc8c8a9a/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2ContainerIntegrationLiveTest.java
----------------------------------------------------------------------
diff --git a/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2ContainerIntegrationLiveTest.java b/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2ContainerIntegrationLiveTest.java
new file mode 100644
index 0000000..7d262ac
--- /dev/null
+++ b/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2ContainerIntegrationLiveTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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.b2.blobstore.integration;
+
+import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown;
+
+import java.io.IOException;
+
+import org.jclouds.blobstore.domain.BlobMetadata;
+import org.jclouds.blobstore.integration.internal.BaseContainerIntegrationTest;
+import org.testng.SkipException;
+import org.testng.annotations.DataProvider;
+
+import com.google.common.collect.ImmutableSet;
+
+public final class B2ContainerIntegrationLiveTest extends BaseContainerIntegrationTest {
+   public B2ContainerIntegrationLiveTest() {
+      provider = "b2";
+   }
+
+   @Override
+   public void testListMarkerAfterLastKey() throws Exception {
+      try {
+         super.testListMarkerAfterLastKey();
+         failBecauseExceptionWasNotThrown(AssertionError.class);
+      } catch (AssertionError ae) {
+         throw new SkipException("B2 uses the marker as the current key, not the next key", ae);
+      }
+   }
+
+   @Override
+   public void testListContainerWithZeroMaxResults() throws Exception {
+      try {
+         super.testListContainerWithZeroMaxResults();
+         failBecauseExceptionWasNotThrown(AssertionError.class);
+      } catch (AssertionError ae) {
+         throw new SkipException("B2 does not enforce zero max results", ae);
+      }
+   }
+
+   @Override
+   public void testDirectory() throws InterruptedException {
+      throw new SkipException("B2 does not support directories");
+   }
+
+   @Override
+   public void testSetContainerAccess() throws Exception {
+      try {
+         super.testSetContainerAccess();
+         failBecauseExceptionWasNotThrown(UnsupportedOperationException.class);
+      } catch (UnsupportedOperationException uoe) {
+         throw new SkipException("Test uses blob signer which B2 does not support", uoe);
+      }
+   }
+
+   @Override
+   protected void checkMD5(BlobMetadata metadata) throws IOException {
+      // B2 does not support Content-MD5
+   }
+
+   // B2 does not support " " file name
+   @DataProvider
+   @Override
+   public Object[][] getBlobsToEscape() {
+      ImmutableSet<String> testNames = ImmutableSet.of("%20", "%20 ", " %20");
+      Object[][] result = new Object[1][1];
+      result[0][0] = testNames;
+      return result;
+   }
+}

http://git-wip-us.apache.org/repos/asf/jclouds-labs/blob/cc8c8a9a/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2ContainerLiveTest.java
----------------------------------------------------------------------
diff --git a/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2ContainerLiveTest.java b/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2ContainerLiveTest.java
new file mode 100644
index 0000000..65e61db
--- /dev/null
+++ b/b2/src/test/java/org/jclouds/b2/blobstore/integration/B2ContainerLiveTest.java
@@ -0,0 +1,27 @@
+/*
+ * 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.b2.blobstore.integration;
+
+import org.jclouds.blobstore.integration.internal.BaseContainerLiveTest;
+import org.testng.annotations.Test;
+
+@Test(groups = { "live" })
+public final class B2ContainerLiveTest extends BaseContainerLiveTest {
+   public B2ContainerLiveTest() {
+      provider = "b2";
+   }
+}