JCLOUDS-40 remove all implementations of AsyncBlobStore except Submission in preparation for complete removal.

This commit is contained in:
Adrian Cole 2014-10-04 22:35:32 -07:00 committed by Adrian Cole
parent 7e04f6236d
commit 7047874ad9
12 changed files with 566 additions and 904 deletions

View File

@ -23,11 +23,9 @@ import javax.inject.Singleton;
import org.jclouds.atmos.AtmosClient;
import org.jclouds.atmos.blobstore.AtmosBlobRequestSigner;
import org.jclouds.atmos.blobstore.AtmosBlobStore;
import org.jclouds.blobstore.AsyncBlobStore;
import org.jclouds.blobstore.BlobRequestSigner;
import org.jclouds.blobstore.BlobStore;
import org.jclouds.blobstore.attr.ConsistencyModel;
import org.jclouds.blobstore.internal.SubmissionAsyncBlobStore;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
@ -41,7 +39,6 @@ public class AtmosBlobStoreContextModule extends AbstractModule {
@Override
protected void configure() {
bind(ConsistencyModel.class).toInstance(ConsistencyModel.EVENTUAL);
bind(AsyncBlobStore.class).to(SubmissionAsyncBlobStore.class).in(Scopes.SINGLETON);
bind(BlobStore.class).to(AtmosBlobStore.class).in(Scopes.SINGLETON);
bind(BlobRequestSigner.class).to(AtmosBlobRequestSigner.class);
}

View File

@ -16,12 +16,8 @@
*/
package org.jclouds.filesystem.config;
import static org.jclouds.rest.config.BinderUtils.bindSyncToAsyncApi;
import org.jclouds.blobstore.AsyncBlobStore;
import org.jclouds.blobstore.BlobRequestSigner;
import org.jclouds.blobstore.BlobStore;
import org.jclouds.blobstore.LocalAsyncBlobStore;
import org.jclouds.blobstore.LocalBlobRequestSigner;
import org.jclouds.blobstore.LocalStorageStrategy;
import org.jclouds.blobstore.attr.ConsistencyModel;
@ -41,11 +37,7 @@ public class FilesystemBlobStoreContextModule extends AbstractModule {
@Override
protected void configure() {
bind(AsyncBlobStore.class).to(LocalAsyncBlobStore.class).asEagerSingleton();
// forward all requests from TransientBlobStore to TransientAsyncBlobStore. needs above binding as cannot proxy a class
bindSyncToAsyncApi(binder(), LocalBlobStore.class, AsyncBlobStore.class);
bind(BlobStore.class).to(LocalBlobStore.class);
install(new BlobStoreObjectModule());
bind(ConsistencyModel.class).toInstance(ConsistencyModel.STRICT);
bind(LocalStorageStrategy.class).to(FilesystemStorageStrategyImpl.class);

View File

@ -22,11 +22,9 @@ import java.util.concurrent.TimeUnit;
import javax.inject.Singleton;
import org.jclouds.blobstore.AsyncBlobStore;
import org.jclouds.blobstore.BlobRequestSigner;
import org.jclouds.blobstore.BlobStore;
import org.jclouds.blobstore.attr.ConsistencyModel;
import org.jclouds.blobstore.internal.SubmissionAsyncBlobStore;
import org.jclouds.domain.Location;
import org.jclouds.s3.S3Client;
import org.jclouds.s3.blobstore.S3BlobRequestSigner;
@ -47,7 +45,6 @@ public class S3BlobStoreContextModule extends AbstractModule {
@Override
protected void configure() {
bind(ConsistencyModel.class).toInstance(ConsistencyModel.EVENTUAL);
bind(AsyncBlobStore.class).to(SubmissionAsyncBlobStore.class).in(SINGLETON);
bind(BlobStore.class).to(S3BlobStore.class).in(SINGLETON);
bind(new TypeLiteral<Function<String, Location>>() {
}).to(LocationFromBucketName.class);

View File

@ -16,10 +16,8 @@
*/
package org.jclouds.openstack.swift.blobstore.config;
import org.jclouds.blobstore.AsyncBlobStore;
import org.jclouds.blobstore.BlobStore;
import org.jclouds.blobstore.attr.ConsistencyModel;
import org.jclouds.blobstore.internal.SubmissionAsyncBlobStore;
import org.jclouds.openstack.swift.blobstore.SwiftBlobStore;
import com.google.inject.AbstractModule;
@ -30,7 +28,6 @@ public class SwiftBlobStoreContextModule extends AbstractModule {
@Override
protected void configure() {
bind(ConsistencyModel.class).toInstance(ConsistencyModel.EVENTUAL);
bind(AsyncBlobStore.class).to(SubmissionAsyncBlobStore.class).in(Scopes.SINGLETON);
bind(BlobStore.class).to(SwiftBlobStore.class).in(Scopes.SINGLETON);
}
}

View File

@ -23,6 +23,7 @@ import org.jclouds.blobstore.domain.BlobBuilder;
import org.jclouds.blobstore.domain.BlobMetadata;
import org.jclouds.blobstore.domain.PageSet;
import org.jclouds.blobstore.domain.StorageMetadata;
import org.jclouds.blobstore.internal.SubmissionAsyncBlobStore;
import org.jclouds.blobstore.options.CreateContainerOptions;
import org.jclouds.blobstore.options.GetOptions;
import org.jclouds.blobstore.options.ListContainerOptions;
@ -31,6 +32,7 @@ import org.jclouds.domain.Location;
import org.jclouds.javax.annotation.Nullable;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.inject.ImplementedBy;
/**
* Provides hooks needed to run a blob store asynchronously
@ -40,6 +42,7 @@ import com.google.common.util.concurrent.ListenableFuture;
* supported. Please use {@link org.jclouds.blobstore.BlobStore}
*/
@Deprecated
@ImplementedBy(SubmissionAsyncBlobStore.class)
public interface AsyncBlobStore {
/**
* @see BlobStore#getContext

View File

@ -1,558 +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.blobstore;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.base.Throwables.getCausalChain;
import static com.google.common.collect.Iterables.filter;
import static com.google.common.collect.Iterables.find;
import static com.google.common.collect.Iterables.size;
import static com.google.common.collect.Iterables.transform;
import static com.google.common.collect.Sets.filter;
import static com.google.common.collect.Sets.newTreeSet;
import static com.google.common.util.concurrent.Futures.immediateFailedFuture;
import static com.google.common.util.concurrent.Futures.immediateFuture;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.Date;
import java.util.Set;
import java.util.SortedSet;
import java.util.regex.Pattern;
import javax.annotation.Resource;
import javax.inject.Inject;
import javax.inject.Named;
import org.jclouds.Constants;
import org.jclouds.blobstore.domain.Blob;
import org.jclouds.blobstore.domain.Blob.Factory;
import org.jclouds.blobstore.domain.BlobMetadata;
import org.jclouds.blobstore.domain.MutableBlobMetadata;
import org.jclouds.blobstore.domain.MutableStorageMetadata;
import org.jclouds.blobstore.domain.PageSet;
import org.jclouds.blobstore.domain.StorageMetadata;
import org.jclouds.blobstore.domain.StorageType;
import org.jclouds.blobstore.domain.internal.MutableStorageMetadataImpl;
import org.jclouds.blobstore.domain.internal.PageSetImpl;
import org.jclouds.blobstore.domain.internal.StorageMetadataImpl;
import org.jclouds.blobstore.internal.BaseAsyncBlobStore;
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.strategy.IfDirectoryReturnNameStrategy;
import org.jclouds.blobstore.util.BlobStoreUtils;
import org.jclouds.blobstore.util.BlobUtils;
import org.jclouds.collect.Memoized;
import org.jclouds.domain.Location;
import org.jclouds.http.HttpCommand;
import org.jclouds.http.HttpRequest;
import org.jclouds.http.HttpResponse;
import org.jclouds.http.HttpResponseException;
import org.jclouds.http.HttpUtils;
import org.jclouds.io.ByteStreams2;
import org.jclouds.io.ContentMetadata;
import org.jclouds.io.ContentMetadataCodec;
import org.jclouds.io.Payload;
import org.jclouds.logging.Logger;
import com.google.common.base.Function;
import com.google.common.base.Predicate;
import com.google.common.base.Supplier;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
/**
* Implementation of {@link BaseAsyncBlobStore} which uses a pluggable
* LocalStorageStrategy.
*
* @deprecated will be removed in jclouds 1.7, as async interfaces are no longer
* supported. Please create and use {@link LocalBlobStore}
*/
@Deprecated
public class LocalAsyncBlobStore extends BaseAsyncBlobStore {
@Resource
protected Logger logger = Logger.NULL;
protected final ContentMetadataCodec contentMetadataCodec;
protected final IfDirectoryReturnNameStrategy ifDirectoryReturnName;
protected final Factory blobFactory;
protected final LocalStorageStrategy storageStrategy;
@Inject
protected LocalAsyncBlobStore(BlobStoreContext context,
BlobUtils blobUtils,
@Named(Constants.PROPERTY_USER_THREADS) ListeningExecutorService userExecutor,
Supplier<Location> defaultLocation,
@Memoized Supplier<Set<? extends Location>> locations,
ContentMetadataCodec contentMetadataCodec,
IfDirectoryReturnNameStrategy ifDirectoryReturnName,
Factory blobFactory, LocalStorageStrategy storageStrategy) {
super(context, blobUtils, userExecutor, defaultLocation, locations);
this.blobFactory = blobFactory;
this.contentMetadataCodec = contentMetadataCodec;
this.ifDirectoryReturnName = ifDirectoryReturnName;
this.storageStrategy = storageStrategy;
}
/**
* default maxResults is 1000
*/
@Override
public ListenableFuture<PageSet<? extends StorageMetadata>> list(final String container, ListContainerOptions options) {
// Check if the container exists
if (!storageStrategy.containerExists(container))
return immediateFailedFuture(cnfe(container));
// Loading blobs from container
Iterable<String> blobBelongingToContainer = null;
try {
blobBelongingToContainer = storageStrategy.getBlobKeysInsideContainer(container);
} catch (IOException e) {
logger.error(e, "An error occurred loading blobs contained into container %s", container);
Throwables.propagate(e);
}
SortedSet<StorageMetadata> contents = newTreeSet(transform(blobBelongingToContainer,
new Function<String, StorageMetadata>() {
public StorageMetadata apply(String key) {
if (!storageStrategy.blobExists(container, key)) {
// handle directory
return new StorageMetadataImpl(StorageType.FOLDER, /*id=*/ null, key,
/*location=*/ null, /*uri=*/ null, /*eTag=*/ null, /*creationDate=*/ null,
/*lastModified=*/ null, ImmutableMap.<String, String>of());
}
Blob oldBlob = loadBlob(container, key);
checkState(oldBlob != null, "blob " + key + " is not present although it was in the list of "
+ container);
checkState(oldBlob.getMetadata() != null, "blob " + container + "/" + key + " has no metadata");
MutableBlobMetadata md = BlobStoreUtils.copy(oldBlob.getMetadata());
String directoryName = ifDirectoryReturnName.execute(md);
if (directoryName != null) {
md.setName(directoryName);
md.setType(StorageType.RELATIVE_PATH);
}
return md;
}
}));
String marker = null;
if (options != null) {
if (options.getMarker() != null) {
final String finalMarker = options.getMarker();
StorageMetadata lastMarkerMetadata = find(contents, new Predicate<StorageMetadata>() {
public boolean apply(StorageMetadata metadata) {
return metadata.getName().compareTo(finalMarker) > 0;
}
});
contents = contents.tailSet(lastMarkerMetadata);
}
final String prefix = options.getDir();
if (prefix != null) {
contents = newTreeSet(filter(contents, new Predicate<StorageMetadata>() {
public boolean apply(StorageMetadata o) {
return o != null && o.getName().startsWith(prefix) && !o.getName().equals(prefix);
}
}));
}
int maxResults = options.getMaxResults() != null ? options.getMaxResults() : 1000;
if (!contents.isEmpty()) {
StorageMetadata lastElement = contents.last();
contents = newTreeSet(Iterables.limit(contents, maxResults));
if (!contents.contains(lastElement)) {
// Partial listing
marker = contents.last().getName();
}
}
if (!options.isRecursive()) {
String delimiter = storageStrategy.getSeparator();
SortedSet<String> commonPrefixes = newTreeSet(
transform(contents, new CommonPrefixes(prefix, delimiter)));
commonPrefixes.remove(CommonPrefixes.NO_PREFIX);
contents = newTreeSet(filter(contents, new DelimiterFilter(prefix, delimiter)));
for (String o : commonPrefixes) {
MutableStorageMetadata md = new MutableStorageMetadataImpl();
md.setType(StorageType.RELATIVE_PATH);
md.setName(o);
contents.add(md);
}
}
// trim metadata, if the response isn't supposed to be detailed.
if (!options.isDetailed()) {
for (StorageMetadata md : contents) {
md.getUserMetadata().clear();
}
}
}
return Futures.<PageSet<? extends StorageMetadata>> immediateFuture(new PageSetImpl<StorageMetadata>(contents,
marker));
}
private ContainerNotFoundException cnfe(final String name) {
return new ContainerNotFoundException(name, String.format(
"container %s not in %s", name,
storageStrategy.getAllContainerNames()));
}
/**
* {@inheritDoc}
*/
@Override
public ListenableFuture<Void> removeBlob(final String container, final String key) {
if (!storageStrategy.containerExists(container)) {
return Futures.immediateFailedFuture(cnfe(container));
}
storageStrategy.removeBlob(container, key);
return immediateFuture(null);
}
/**
* {@inheritDoc}
*/
@Override
public ListenableFuture<Void> clearContainer(final String container) {
storageStrategy.clearContainer(container);
return immediateFuture(null);
}
/**
* Override parent method because it uses strange futures and listenables
* that creates problem in the test if more than one test that deletes the
* container is executed
*
* @param container
* @return
*/
@Override
public ListenableFuture<Void> deleteContainer(final String container) {
deleteAndVerifyContainerGone(container);
return immediateFuture(null);
}
@Override
public ListenableFuture<Boolean> deleteContainerIfEmpty(final String container) {
Boolean returnVal = true;
if (storageStrategy.containerExists(container)) {
try {
if (Iterables.isEmpty(storageStrategy.getBlobKeysInsideContainer(container)))
storageStrategy.deleteContainer(container);
else
returnVal = false;
} catch (IOException e) {
logger.error(e, "An error occurred loading blobs contained into container %s", container);
Throwables.propagate(e);
}
}
return immediateFuture(returnVal);
}
/**
* {@inheritDoc}
*/
@Override
public ListenableFuture<Boolean> containerExists(final String containerName) {
return immediateFuture(storageStrategy.containerExists(containerName));
}
/**
* {@inheritDoc}
*/
@Override
public ListenableFuture<PageSet<? extends StorageMetadata>> list() {
Iterable<String> containers = storageStrategy.getAllContainerNames();
return Futures.<PageSet<? extends StorageMetadata>> immediateFuture(new PageSetImpl<StorageMetadata>(transform(
containers, new Function<String, StorageMetadata>() {
public StorageMetadata apply(String name) {
MutableStorageMetadata cmd = create();
cmd.setName(name);
cmd.setType(StorageType.CONTAINER);
cmd.setLocation(storageStrategy.getLocation(name));
return cmd;
}
}), null));
}
protected MutableStorageMetadata create() {
return new MutableStorageMetadataImpl();
}
/**
* {@inheritDoc}
*/
@Override
public ListenableFuture<Boolean> createContainerInLocation(final Location location,
final String name) {
boolean result = storageStrategy.createContainerInLocation(name, location);
return immediateFuture(result);
}
private Blob loadBlob(final String container, final String key) {
logger.debug("Opening blob in container: %s - %s", container, key);
return storageStrategy.getBlob(container, key);
}
protected static class DelimiterFilter implements Predicate<StorageMetadata> {
private final String prefix;
private final String delimiter;
public DelimiterFilter(String prefix, String delimiter) {
this.prefix = prefix;
this.delimiter = delimiter;
}
public boolean apply(StorageMetadata metadata) {
if (prefix == null)
return metadata.getName().indexOf(delimiter) == -1;
// ensure we don't accidentally append twice
String toMatch = prefix.endsWith("/") ? prefix : prefix + delimiter;
if (metadata.getName().startsWith(toMatch)) {
String unprefixedName = metadata.getName().replaceFirst(Pattern.quote(toMatch), "");
if (unprefixedName.equals("")) {
// we are the prefix in this case, return false
return false;
}
return unprefixedName.indexOf(delimiter) == -1;
}
return false;
}
}
protected static class CommonPrefixes implements Function<StorageMetadata, String> {
private final String prefix;
private final String delimiter;
public static final String NO_PREFIX = "NO_PREFIX";
public CommonPrefixes(String prefix, String delimiter) {
this.prefix = prefix;
this.delimiter = delimiter;
}
public String apply(StorageMetadata metadata) {
String working = metadata.getName();
if (prefix != null) {
// ensure we don't accidentally append twice
String toMatch = prefix.endsWith("/") ? prefix : prefix + delimiter;
if (working.startsWith(toMatch)) {
working = working.replaceFirst(Pattern.quote(toMatch), "");
}
}
if (working.contains(delimiter)) {
return working.substring(0, working.indexOf(delimiter));
}
return NO_PREFIX;
}
}
public static HttpResponseException returnResponseException(int code) {
HttpResponse response = HttpResponse.builder().statusCode(code).build();
return new HttpResponseException(new HttpCommand(HttpRequest.builder().method("GET").endpoint("http://stub")
.build()), response);
}
/**
* {@inheritDoc}
*/
@Override
public ListenableFuture<String> putBlob(String containerName, Blob blob) {
checkNotNull(containerName, "containerName must be set");
checkNotNull(blob, "blob must be set");
String blobKey = blob.getMetadata().getName();
logger.debug("Put blob with key [%s] to container [%s]", blobKey, containerName);
if (!storageStrategy.containerExists(containerName)) {
return Futures.immediateFailedFuture(cnfe(containerName));
}
try {
return immediateFuture(storageStrategy.putBlob(containerName, blob));
} catch (IOException e) {
String message = e.getMessage();
if (message != null && message.startsWith("MD5 hash code mismatch")) {
HttpResponseException exception = returnResponseException(400);
exception.initCause(e);
throw exception;
}
logger.error(e, "An error occurred storing the new blob with name [%s] to container [%s].", blobKey,
containerName);
throw Throwables.propagate(e);
}
}
private void copyPayloadHeadersToBlob(Payload payload, Blob blob) {
blob.getAllHeaders().putAll(contentMetadataCodec.toHeaders(payload.getContentMetadata()));
}
/**
* {@inheritDoc}
*/
@Override
public ListenableFuture<Boolean> blobExists(final String containerName, final String key) {
if (!storageStrategy.containerExists(containerName))
return immediateFailedFuture(cnfe(containerName));
return immediateFuture(storageStrategy.blobExists(containerName, key));
}
/**
* {@inheritDoc}
*/
@Override
public ListenableFuture<Blob> getBlob(final String containerName, final String key, GetOptions options) {
logger.debug("Retrieving blob with key %s from container %s", key, containerName);
// If the container doesn't exist, an exception is thrown
if (!storageStrategy.containerExists(containerName)) {
logger.debug("Container %s does not exist", containerName);
return immediateFailedFuture(cnfe(containerName));
}
// If the blob doesn't exist, a null object is returned
if (!storageStrategy.blobExists(containerName, key)) {
logger.debug("Item %s does not exist in container %s", key, containerName);
return immediateFuture(null);
}
Blob blob = loadBlob(containerName, key);
if (options != null) {
if (options.getIfMatch() != null) {
if (!blob.getMetadata().getETag().equals(options.getIfMatch()))
return immediateFailedFuture(returnResponseException(412));
}
if (options.getIfNoneMatch() != null) {
if (blob.getMetadata().getETag().equals(options.getIfNoneMatch()))
return immediateFailedFuture(returnResponseException(304));
}
if (options.getIfModifiedSince() != null) {
Date modifiedSince = options.getIfModifiedSince();
if (blob.getMetadata().getLastModified().before(modifiedSince)) {
HttpResponse response = HttpResponse.builder().statusCode(304).build();
return immediateFailedFuture(new HttpResponseException(String.format("%1$s is before %2$s", blob
.getMetadata().getLastModified(), modifiedSince), null, response));
}
}
if (options.getIfUnmodifiedSince() != null) {
Date unmodifiedSince = options.getIfUnmodifiedSince();
if (blob.getMetadata().getLastModified().after(unmodifiedSince)) {
HttpResponse response = HttpResponse.builder().statusCode(412).build();
return immediateFailedFuture(new HttpResponseException(String.format("%1$s is after %2$s", blob
.getMetadata().getLastModified(), unmodifiedSince), null, response));
}
}
blob = copyBlob(blob);
if (options.getRanges() != null && options.getRanges().size() > 0) {
byte[] data;
try {
data = ByteStreams2.toByteArrayAndClose(blob.getPayload().openStream());
} catch (IOException e) {
return immediateFailedFuture(new RuntimeException(e));
}
ByteArrayOutputStream out = new ByteArrayOutputStream();
for (String s : options.getRanges()) {
// HTTP uses a closed interval while Java array indexing uses a
// half-open interval.
int offset = 0;
int last = data.length - 1;
if (s.startsWith("-")) {
offset = last - Integer.parseInt(s.substring(1)) + 1;
} else if (s.endsWith("-")) {
offset = Integer.parseInt(s.substring(0, s.length() - 1));
} else if (s.contains("-")) {
String[] firstLast = s.split("\\-");
offset = Integer.parseInt(firstLast[0]);
last = Integer.parseInt(firstLast[1]);
} else {
return immediateFailedFuture(new IllegalArgumentException("illegal range: " + s));
}
if (offset > last) {
return immediateFailedFuture(new IllegalArgumentException("illegal range: " + s));
}
if (last + 1 > data.length) {
last = data.length - 1;
}
out.write(data, offset, last - offset + 1);
}
ContentMetadata cmd = blob.getPayload().getContentMetadata();
byte[] byteArray = out.toByteArray();
blob.setPayload(byteArray);
HttpUtils.copy(cmd, blob.getPayload().getContentMetadata());
blob.getPayload().getContentMetadata().setContentLength(Long.valueOf(byteArray.length));
}
}
checkNotNull(blob.getPayload(), "payload " + blob);
return immediateFuture(blob);
}
/**
* {@inheritDoc}
*/
@Override
public ListenableFuture<BlobMetadata> blobMetadata(final String container, final String key) {
try {
Blob blob = getBlob(container, key).get();
return immediateFuture(blob != null ? (BlobMetadata) BlobStoreUtils.copy(blob.getMetadata()) : null);
} catch (Exception e) {
if (size(filter(getCausalChain(e), KeyNotFoundException.class)) >= 1)
return immediateFuture(null);
return immediateFailedFuture(e);
}
}
private Blob copyBlob(Blob blob) {
Blob returnVal = blobFactory.create(BlobStoreUtils.copy(blob.getMetadata()));
returnVal.setPayload(blob.getPayload());
copyPayloadHeadersToBlob(blob.getPayload(), returnVal);
return returnVal;
}
@Override
protected boolean deleteAndVerifyContainerGone(final String container) {
storageStrategy.deleteContainer(container);
return storageStrategy.containerExists(container);
}
@Override
public ListenableFuture<String> putBlob(String container, Blob blob, PutOptions options) {
// TODO implement options
return putBlob(container, blob);
}
@Override
public ListenableFuture<Boolean> createContainerInLocation(Location location, String container,
CreateContainerOptions options) {
if (options.isPublicRead())
throw new UnsupportedOperationException("publicRead");
return createContainerInLocation(location, container);
}
}

View File

@ -16,8 +16,568 @@
*/
package org.jclouds.blobstore.config;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.base.Throwables.getCausalChain;
import static com.google.common.base.Throwables.propagate;
import static com.google.common.collect.Iterables.find;
import static com.google.common.collect.Iterables.size;
import static com.google.common.collect.Iterables.transform;
import static com.google.common.collect.Sets.filter;
import static com.google.common.collect.Sets.newTreeSet;
import static org.jclouds.blobstore.options.ListContainerOptions.Builder.recursive;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.Date;
import java.util.Set;
import java.util.SortedSet;
import java.util.regex.Pattern;
import javax.annotation.Resource;
import javax.inject.Inject;
import javax.inject.Singleton;
import org.jclouds.blobstore.BlobStore;
import org.jclouds.blobstore.BlobStoreContext;
import org.jclouds.blobstore.ContainerNotFoundException;
import org.jclouds.blobstore.KeyNotFoundException;
import org.jclouds.blobstore.LocalStorageStrategy;
import org.jclouds.blobstore.domain.Blob;
import org.jclouds.blobstore.domain.BlobBuilder;
import org.jclouds.blobstore.domain.BlobMetadata;
import org.jclouds.blobstore.domain.MutableBlobMetadata;
import org.jclouds.blobstore.domain.MutableStorageMetadata;
import org.jclouds.blobstore.domain.PageSet;
import org.jclouds.blobstore.domain.StorageMetadata;
import org.jclouds.blobstore.domain.StorageType;
import org.jclouds.blobstore.domain.internal.MutableStorageMetadataImpl;
import org.jclouds.blobstore.domain.internal.PageSetImpl;
import org.jclouds.blobstore.domain.internal.StorageMetadataImpl;
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.strategy.IfDirectoryReturnNameStrategy;
import org.jclouds.blobstore.util.BlobStoreUtils;
import org.jclouds.blobstore.util.BlobUtils;
import org.jclouds.collect.Memoized;
import org.jclouds.domain.Location;
import org.jclouds.http.HttpCommand;
import org.jclouds.http.HttpRequest;
import org.jclouds.http.HttpResponse;
import org.jclouds.http.HttpResponseException;
import org.jclouds.http.HttpUtils;
import org.jclouds.io.ByteStreams2;
import org.jclouds.io.ContentMetadata;
import org.jclouds.io.ContentMetadataCodec;
import org.jclouds.io.Payload;
import org.jclouds.logging.Logger;
public interface LocalBlobStore extends BlobStore {
import com.google.common.base.Function;
import com.google.common.base.Predicate;
import com.google.common.base.Supplier;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
@Singleton
public final class LocalBlobStore implements BlobStore {
@Resource
private Logger logger = Logger.NULL;
private final BlobStoreContext context;
private final BlobUtils blobUtils;
private final Supplier<Set<? extends Location>> locations;
private final ContentMetadataCodec contentMetadataCodec;
private final IfDirectoryReturnNameStrategy ifDirectoryReturnName;
private final Blob.Factory blobFactory;
private final LocalStorageStrategy storageStrategy;
@Inject
LocalBlobStore(BlobStoreContext context,
BlobUtils blobUtils,
@Memoized Supplier<Set<? extends Location>> locations,
ContentMetadataCodec contentMetadataCodec,
IfDirectoryReturnNameStrategy ifDirectoryReturnName,
Blob.Factory blobFactory, LocalStorageStrategy storageStrategy) {
this.context = checkNotNull(context, "context");
this.blobUtils = checkNotNull(blobUtils, "blobUtils");
this.locations = checkNotNull(locations, "locations");
this.blobFactory = blobFactory;
this.contentMetadataCodec = contentMetadataCodec;
this.ifDirectoryReturnName = ifDirectoryReturnName;
this.storageStrategy = storageStrategy;
}
@Override
public BlobStoreContext getContext() {
return context;
}
@Override
public BlobBuilder blobBuilder(String name) {
return blobUtils.blobBuilder().name(name);
}
/** This implementation invokes {@link #list(String, ListContainerOptions)} */
@Override
public PageSet<? extends StorageMetadata> list(String containerName) {
return this.list(containerName, ListContainerOptions.NONE);
}
/**
* This implementation invokes {@link #countBlobs} with the
* {@link ListContainerOptions#recursive} option.
*/
@Override
public long countBlobs(String containerName) {
return countBlobs(containerName, recursive());
}
/**
* This implementation invokes {@link BlobUtils#countBlobs}
*/
@Override
public long countBlobs(final String containerName, final ListContainerOptions options) {
return blobUtils.countBlobs(containerName, options);
}
/**
* This implementation invokes {@link #clearContainer} with the
* {@link ListContainerOptions#recursive} option.
*/
@Override
public void clearContainer(String containerName) {
clearContainer(containerName, recursive());
}
@Override
public void clearContainer(String containerName, ListContainerOptions options) {
blobUtils.clearContainer(containerName, options);
}
@Override
public void deleteDirectory(final String containerName, final String directory) {
blobUtils.deleteDirectory(containerName, directory);
}
@Override
public boolean directoryExists(String containerName, String directory) {
return blobUtils.directoryExists(containerName, directory);
}
@Override
public void createDirectory(String containerName, String directory) {
if (!blobUtils.directoryExists(containerName, directory)) {
blobUtils.createDirectory(containerName, directory);
}
}
/**
* This implementation invokes {@link #getBlob(String,String, GetOptions)}
*/
@Override
public Blob getBlob(String containerName, String key) {
return getBlob(containerName, key, GetOptions.NONE);
}
/**
* This implementation invokes {@link #deleteAndVerifyContainerGone}
*/
@Override
public void deleteContainer(String containerName) {
deleteAndVerifyContainerGone(containerName);
}
@Override
public Set<? extends Location> listAssignableLocations() {
return locations.get();
}
/**
* default maxResults is 1000
*/
@Override
public PageSet<? extends StorageMetadata> list(final String containerName, ListContainerOptions options) {
// Check if the container exists
if (!storageStrategy.containerExists(containerName))
throw cnfe(containerName);
// Loading blobs from container
Iterable<String> blobBelongingToContainer = null;
try {
blobBelongingToContainer = storageStrategy.getBlobKeysInsideContainer(containerName);
} catch (IOException e) {
logger.error(e, "An error occurred loading blobs contained into container %s", containerName);
propagate(e);
}
SortedSet<StorageMetadata> contents = newTreeSet(transform(blobBelongingToContainer,
new Function<String, StorageMetadata>() {
public StorageMetadata apply(String key) {
if (!storageStrategy.blobExists(containerName, key)) {
// handle directory
return new StorageMetadataImpl(StorageType.FOLDER, /*id=*/ null, key,
/*location=*/ null, /*uri=*/ null, /*eTag=*/ null, /*creationDate=*/ null,
/*lastModified=*/ null, ImmutableMap.<String, String>of());
}
Blob oldBlob = loadBlob(containerName, key);
checkState(oldBlob != null, "blob " + key + " is not present although it was in the list of "
+ containerName);
checkState(oldBlob.getMetadata() != null, "blob " + containerName + "/" + key + " has no metadata");
MutableBlobMetadata md = BlobStoreUtils.copy(oldBlob.getMetadata());
String directoryName = ifDirectoryReturnName.execute(md);
if (directoryName != null) {
md.setName(directoryName);
md.setType(StorageType.RELATIVE_PATH);
}
return md;
}
}));
String marker = null;
if (options != null) {
if (options.getMarker() != null) {
final String finalMarker = options.getMarker();
StorageMetadata lastMarkerMetadata = find(contents, new Predicate<StorageMetadata>() {
public boolean apply(StorageMetadata metadata) {
return metadata.getName().compareTo(finalMarker) > 0;
}
});
contents = contents.tailSet(lastMarkerMetadata);
}
final String prefix = options.getDir();
if (prefix != null) {
contents = newTreeSet(filter(contents, new Predicate<StorageMetadata>() {
public boolean apply(StorageMetadata o) {
return o != null && o.getName().startsWith(prefix) && !o.getName().equals(prefix);
}
}));
}
int maxResults = options.getMaxResults() != null ? options.getMaxResults() : 1000;
if (!contents.isEmpty()) {
StorageMetadata lastElement = contents.last();
contents = newTreeSet(Iterables.limit(contents, maxResults));
if (!contents.contains(lastElement)) {
// Partial listing
marker = contents.last().getName();
}
}
if (!options.isRecursive()) {
String delimiter = storageStrategy.getSeparator();
SortedSet<String> commonPrefixes = newTreeSet(
transform(contents, new CommonPrefixes(prefix, delimiter)));
commonPrefixes.remove(CommonPrefixes.NO_PREFIX);
contents = newTreeSet(filter(contents, new DelimiterFilter(prefix, delimiter)));
for (String o : commonPrefixes) {
MutableStorageMetadata md = new MutableStorageMetadataImpl();
md.setType(StorageType.RELATIVE_PATH);
md.setName(o);
contents.add(md);
}
}
// trim metadata, if the response isn't supposed to be detailed.
if (!options.isDetailed()) {
for (StorageMetadata md : contents) {
md.getUserMetadata().clear();
}
}
}
return new PageSetImpl<StorageMetadata>(contents, marker);
}
private ContainerNotFoundException cnfe(final String name) {
return new ContainerNotFoundException(name, String.format(
"container %s not in %s", name,
storageStrategy.getAllContainerNames()));
}
@Override
public void removeBlob(String containerName, final String key) {
if (!storageStrategy.containerExists(containerName)) {
throw cnfe(containerName);
}
storageStrategy.removeBlob(containerName, key);
}
@Override
public boolean deleteContainerIfEmpty(String containerName) {
boolean returnVal = true;
if (storageStrategy.containerExists(containerName)) {
try {
if (Iterables.isEmpty(storageStrategy.getBlobKeysInsideContainer(containerName)))
storageStrategy.deleteContainer(containerName);
else
returnVal = false;
} catch (IOException e) {
logger.error(e, "An error occurred loading blobs contained into container %s", containerName);
throw propagate(e);
}
}
return returnVal;
}
@Override
public boolean containerExists(String containerName) {
return storageStrategy.containerExists(containerName);
}
@Override
public PageSet<? extends StorageMetadata> list() {
Iterable<String> containers = storageStrategy.getAllContainerNames();
return new PageSetImpl<StorageMetadata>(transform(
containers, new Function<String, StorageMetadata>() {
public StorageMetadata apply(String name) {
MutableStorageMetadata cmd = create();
cmd.setName(name);
cmd.setType(StorageType.CONTAINER);
cmd.setLocation(storageStrategy.getLocation(name));
return cmd;
}
}), null);
}
private MutableStorageMetadata create() {
return new MutableStorageMetadataImpl();
}
@Override
public boolean createContainerInLocation(Location location, String name) {
return storageStrategy.createContainerInLocation(name, location);
}
private Blob loadBlob(final String container, final String key) {
logger.debug("Opening blob in container: %s - %s", container, key);
return storageStrategy.getBlob(container, key);
}
private static class DelimiterFilter implements Predicate<StorageMetadata> {
private final String prefix;
private final String delimiter;
public DelimiterFilter(String prefix, String delimiter) {
this.prefix = prefix;
this.delimiter = delimiter;
}
public boolean apply(StorageMetadata metadata) {
if (prefix == null)
return metadata.getName().indexOf(delimiter) == -1;
// ensure we don't accidentally append twice
String toMatch = prefix.endsWith("/") ? prefix : prefix + delimiter;
if (metadata.getName().startsWith(toMatch)) {
String unprefixedName = metadata.getName().replaceFirst(Pattern.quote(toMatch), "");
if (unprefixedName.equals("")) {
// we are the prefix in this case, return false
return false;
}
return unprefixedName.indexOf(delimiter) == -1;
}
return false;
}
}
private static class CommonPrefixes implements Function<StorageMetadata, String> {
private final String prefix;
private final String delimiter;
public static final String NO_PREFIX = "NO_PREFIX";
public CommonPrefixes(String prefix, String delimiter) {
this.prefix = prefix;
this.delimiter = delimiter;
}
public String apply(StorageMetadata metadata) {
String working = metadata.getName();
if (prefix != null) {
// ensure we don't accidentally append twice
String toMatch = prefix.endsWith("/") ? prefix : prefix + delimiter;
if (working.startsWith(toMatch)) {
working = working.replaceFirst(Pattern.quote(toMatch), "");
}
}
if (working.contains(delimiter)) {
return working.substring(0, working.indexOf(delimiter));
}
return NO_PREFIX;
}
}
private static HttpResponseException returnResponseException(int code) {
HttpResponse response = HttpResponse.builder().statusCode(code).build();
return new HttpResponseException(new HttpCommand(HttpRequest.builder().method("GET").endpoint("http://stub")
.build()), response);
}
@Override
public String putBlob(String containerName, Blob blob) {
checkNotNull(containerName, "containerName must be set");
checkNotNull(blob, "blob must be set");
String blobKey = blob.getMetadata().getName();
logger.debug("Put blob with key [%s] to container [%s]", blobKey, containerName);
if (!storageStrategy.containerExists(containerName)) {
throw cnfe(containerName);
}
try {
return storageStrategy.putBlob(containerName, blob);
} catch (IOException e) {
String message = e.getMessage();
if (message != null && message.startsWith("MD5 hash code mismatch")) {
HttpResponseException exception = returnResponseException(400);
exception.initCause(e);
throw exception;
}
logger.error(e, "An error occurred storing the new blob with name [%s] to container [%s].", blobKey,
containerName);
throw propagate(e);
}
}
private void copyPayloadHeadersToBlob(Payload payload, Blob blob) {
blob.getAllHeaders().putAll(contentMetadataCodec.toHeaders(payload.getContentMetadata()));
}
@Override
public boolean blobExists(String containerName, String key) {
if (!storageStrategy.containerExists(containerName))
throw cnfe(containerName);
return storageStrategy.blobExists(containerName, key);
}
@Override
public Blob getBlob(String containerName, String key, GetOptions options) {
logger.debug("Retrieving blob with key %s from container %s", key, containerName);
// If the container doesn't exist, an exception is thrown
if (!storageStrategy.containerExists(containerName)) {
logger.debug("Container %s does not exist", containerName);
throw cnfe(containerName);
}
// If the blob doesn't exist, a null object is returned
if (!storageStrategy.blobExists(containerName, key)) {
logger.debug("Item %s does not exist in container %s", key, containerName);
return null;
}
Blob blob = loadBlob(containerName, key);
if (options != null) {
if (options.getIfMatch() != null) {
if (!blob.getMetadata().getETag().equals(options.getIfMatch()))
throw returnResponseException(412);
}
if (options.getIfNoneMatch() != null) {
if (blob.getMetadata().getETag().equals(options.getIfNoneMatch()))
throw returnResponseException(304);
}
if (options.getIfModifiedSince() != null) {
Date modifiedSince = options.getIfModifiedSince();
if (blob.getMetadata().getLastModified().before(modifiedSince)) {
HttpResponse response = HttpResponse.builder().statusCode(304).build();
throw new HttpResponseException(String.format("%1$s is before %2$s", blob
.getMetadata().getLastModified(), modifiedSince), null, response);
}
}
if (options.getIfUnmodifiedSince() != null) {
Date unmodifiedSince = options.getIfUnmodifiedSince();
if (blob.getMetadata().getLastModified().after(unmodifiedSince)) {
HttpResponse response = HttpResponse.builder().statusCode(412).build();
throw new HttpResponseException(String.format("%1$s is after %2$s", blob
.getMetadata().getLastModified(), unmodifiedSince), null, response);
}
}
blob = copyBlob(blob);
if (options.getRanges() != null && !options.getRanges().isEmpty()) {
byte[] data;
try {
data = ByteStreams2.toByteArrayAndClose(blob.getPayload().openStream());
} catch (IOException e) {
throw new RuntimeException(e);
}
ByteArrayOutputStream out = new ByteArrayOutputStream();
for (String s : options.getRanges()) {
// HTTP uses a closed interval while Java array indexing uses a
// half-open interval.
int offset = 0;
int last = data.length - 1;
if (s.startsWith("-")) {
offset = last - Integer.parseInt(s.substring(1)) + 1;
} else if (s.endsWith("-")) {
offset = Integer.parseInt(s.substring(0, s.length() - 1));
} else if (s.contains("-")) {
String[] firstLast = s.split("\\-");
offset = Integer.parseInt(firstLast[0]);
last = Integer.parseInt(firstLast[1]);
} else {
throw new IllegalArgumentException("illegal range: " + s);
}
if (offset > last) {
throw new IllegalArgumentException("illegal range: " + s);
}
if (last + 1 > data.length) {
last = data.length - 1;
}
out.write(data, offset, last - offset + 1);
}
ContentMetadata cmd = blob.getPayload().getContentMetadata();
byte[] byteArray = out.toByteArray();
blob.setPayload(byteArray);
HttpUtils.copy(cmd, blob.getPayload().getContentMetadata());
blob.getPayload().getContentMetadata().setContentLength(Long.valueOf(byteArray.length));
}
}
checkNotNull(blob.getPayload(), "payload " + blob);
return blob;
}
@Override
public BlobMetadata blobMetadata(String containerName, String key) {
try {
Blob blob = getBlob(containerName, key);
return blob != null ? (BlobMetadata) BlobStoreUtils.copy(blob.getMetadata()) : null;
} catch (Exception e) {
if (size(Iterables.filter(getCausalChain(e), KeyNotFoundException.class)) >= 1)
return null;
throw propagate(e);
}
}
private Blob copyBlob(Blob blob) {
Blob returnVal = blobFactory.create(BlobStoreUtils.copy(blob.getMetadata()));
returnVal.setPayload(blob.getPayload());
copyPayloadHeadersToBlob(blob.getPayload(), returnVal);
return returnVal;
}
private boolean deleteAndVerifyContainerGone(final String container) {
storageStrategy.deleteContainer(container);
return storageStrategy.containerExists(container);
}
@Override
public String putBlob(String containerName, Blob blob, PutOptions options) {
// TODO implement options
return putBlob(containerName, blob);
}
@Override
public boolean createContainerInLocation(Location location, String container, CreateContainerOptions options) {
if (options.isPublicRead())
throw new UnsupportedOperationException("publicRead");
return createContainerInLocation(location, container);
}
}

View File

@ -16,12 +16,8 @@
*/
package org.jclouds.blobstore.config;
import static org.jclouds.rest.config.BinderUtils.bindSyncToAsyncApi;
import org.jclouds.blobstore.AsyncBlobStore;
import org.jclouds.blobstore.BlobRequestSigner;
import org.jclouds.blobstore.BlobStore;
import org.jclouds.blobstore.LocalAsyncBlobStore;
import org.jclouds.blobstore.LocalBlobRequestSigner;
import org.jclouds.blobstore.LocalStorageStrategy;
import org.jclouds.blobstore.TransientStorageStrategy;
@ -29,15 +25,9 @@ import org.jclouds.blobstore.attr.ConsistencyModel;
import com.google.inject.AbstractModule;
/**
* Configures the {@link TransientBlobStoreContext}; requires {@link TransientAsyncBlobStore} bound.
*/
public class TransientBlobStoreContextModule extends AbstractModule {
@Override
protected void configure() {
bind(AsyncBlobStore.class).to(LocalAsyncBlobStore.class).asEagerSingleton();
// forward all requests from TransientBlobStore to TransientAsyncBlobStore. needs above binding as cannot proxy a class
bindSyncToAsyncApi(binder(), LocalBlobStore.class, AsyncBlobStore.class);
install(new BlobStoreObjectModule());
bind(BlobStore.class).to(LocalBlobStore.class);
bind(ConsistencyModel.class).toInstance(ConsistencyModel.STRICT);

View File

@ -1,310 +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.blobstore.internal;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
import static org.jclouds.blobstore.options.ListContainerOptions.Builder.recursive;
import static org.jclouds.util.Predicates2.retry;
import java.util.Set;
import java.util.concurrent.Callable;
import javax.inject.Inject;
import javax.inject.Named;
import org.jclouds.Constants;
import org.jclouds.blobstore.AsyncBlobStore;
import org.jclouds.blobstore.BlobStoreContext;
import org.jclouds.blobstore.ContainerNotFoundException;
import org.jclouds.blobstore.domain.Blob;
import org.jclouds.blobstore.domain.BlobBuilder;
import org.jclouds.blobstore.domain.PageSet;
import org.jclouds.blobstore.domain.StorageMetadata;
import org.jclouds.blobstore.options.ListContainerOptions;
import org.jclouds.blobstore.util.BlobUtils;
import org.jclouds.collect.Memoized;
import org.jclouds.domain.Location;
import com.google.common.base.Predicate;
import com.google.common.base.Supplier;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
/**
*
* @deprecated will be removed in jclouds 1.7, as async interfaces are no longer
* supported. Please use {@link org.jclouds.blobstore.BlobStore}
*/
@Deprecated
public abstract class BaseAsyncBlobStore implements AsyncBlobStore {
protected final BlobStoreContext context;
protected final BlobUtils blobUtils;
protected final ListeningExecutorService userExecutor;
protected final Supplier<Location> defaultLocation;
protected final Supplier<Set<? extends Location>> locations;
@Inject
protected BaseAsyncBlobStore(BlobStoreContext context, BlobUtils blobUtils,
@Named(Constants.PROPERTY_USER_THREADS) ListeningExecutorService userExecutor, Supplier<Location> defaultLocation,
@Memoized Supplier<Set<? extends Location>> locations) {
this.context = checkNotNull(context, "context");
this.blobUtils = checkNotNull(blobUtils, "blobUtils");
this.userExecutor = checkNotNull(userExecutor, "userExecutor");
this.defaultLocation = checkNotNull(defaultLocation, "defaultLocation");
this.locations = checkNotNull(locations, "locations");
}
@Override
public BlobStoreContext getContext() {
return context;
}
/**
* invokes {@link BlobUtilsImpl#blobBuilder }
*/
@Override
public BlobBuilder blobBuilder(String name) {
return blobUtils.blobBuilder().name(name);
}
/**
* This implementation invokes
* {@link #list(String,org.jclouds.blobstore.options.ListContainerOptions)}
*
* @param container
* container name
*/
@Override
public ListenableFuture<PageSet<? extends StorageMetadata>> list(String container) {
return this.list(container, org.jclouds.blobstore.options.ListContainerOptions.NONE);
}
/**
* This implementation invokes {@link #countBlobs} with the
* {@link ListContainerOptions#recursive} option.
*
* @param container
* container name
*/
@Override
public ListenableFuture<Long> countBlobs(String container) {
return countBlobs(container, recursive());
}
/**
* This implementation invokes {@link BlobUtilsImpl#countBlobs}
*
* @param container
* container name
*/
@Override
public ListenableFuture<Long> countBlobs(final String containerName, final ListContainerOptions options) {
return userExecutor.submit(new Callable<Long>() {
public Long call() throws Exception {
return blobUtils.countBlobs(containerName, options);
}
@Override
public String toString() {
return "countBlobs(" + containerName + ")";
}
});
}
/**
* This implementation invokes {@link #clearContainer} with the
* {@link ListContainerOptions#recursive} option.
*
* @param container
* container name
*/
@Override
public ListenableFuture<Void> clearContainer(final String container) {
return clearContainer(container, recursive());
}
/**
* This implementation invokes {@link BlobUtilsImpl#clearContainer}
*
* @param container
* container name
*/
@Override
public ListenableFuture<Void> clearContainer(final String containerName, final ListContainerOptions options) {
return userExecutor.submit(new Callable<Void>() {
public Void call() throws Exception {
blobUtils.clearContainer(containerName, options);
return null;
}
@Override
public String toString() {
return "clearContainer(" + containerName + ")";
}
});
}
/**
* This implementation invokes {@link BlobUtilsImpl#deleteDirectory}.
*
* @param container
* container name
*/
@Override
public ListenableFuture<Void> deleteDirectory(final String containerName, final String directory) {
return userExecutor.submit(new Callable<Void>() {
public Void call() throws Exception {
blobUtils.deleteDirectory(containerName, directory);
return null;
}
@Override
public String toString() {
return "deleteDirectory(" + containerName + "," + directory + ")";
}
});
}
/**
* This implementation invokes {@link BlobUtilsImpl#directoryExists}
*
* @param container
* container name
* @param directory
* virtual path
*/
public ListenableFuture<Boolean> directoryExists(final String containerName, final String directory) {
return userExecutor.submit(new Callable<Boolean>() {
public Boolean call() throws Exception {
return blobUtils.directoryExists(containerName, directory);
}
@Override
public String toString() {
return "directoryExists(" + containerName + "," + directory + ")";
}
});
}
/**
* This implementation invokes {@link BlobUtilsImpl#createDirectory}
*
* @param container
* container name
* @param directory
* virtual path
*/
public ListenableFuture<Void> createDirectory(final String containerName, final String directory) {
return blobUtils.directoryExists(containerName, directory) ? Futures.immediateFuture((Void) null)
: userExecutor.submit(new Callable<Void>() {
public Void call() throws Exception {
blobUtils.createDirectory(containerName, directory);
return null;
}
@Override
public String toString() {
return "createDirectory(" + containerName + "," + directory + ")";
}
});
}
/**
* This implementation invokes
* {@link #getBlob(String,String,org.jclouds.blobstore.options.GetOptions)}
*
* @param container
* container name
* @param key
* blob key
*/
@Override
public ListenableFuture<Blob> getBlob(String container, String key) {
return getBlob(container, key, org.jclouds.blobstore.options.GetOptions.NONE);
}
/**
* This implementation invokes {@link #deleteAndEnsurePathGone}
*
* @param container
* bucket name
*/
@Override
public ListenableFuture<Void> deleteContainer(final String container) {
return userExecutor.submit(new Callable<Void>() {
public Void call() throws Exception {
deletePathAndEnsureGone(container);
return null;
}
@Override
public String toString() {
return "deleteContainer(" + container + ")";
}
});
}
@Override
public ListenableFuture<Boolean> deleteContainerIfEmpty(final String container) {
return userExecutor.submit(new Callable<Boolean>() {
public Boolean call() throws Exception {
return deleteAndVerifyContainerGone(container);
}
@Override
public String toString() {
return "deleteContainerIfEmpty(" + container + ")";
}
});
}
protected void deletePathAndEnsureGone(String path) {
checkState(retry(new Predicate<String>() {
public boolean apply(String in) {
try {
blobUtils.clearContainer(in, recursive());
return deleteAndVerifyContainerGone(in);
} catch (ContainerNotFoundException e) {
return true;
}
}
}, 30000).apply(path), "%s still exists after deleting!", path);
}
@Override
public ListenableFuture<Set<? extends Location>> listAssignableLocations() {
return Futures.<Set<? extends Location>> immediateFuture(locations.get());
}
/**
* Delete a container if it is empty.
*
* @param container what to delete
* @return true if the container was deleted or does not exist
*/
protected abstract boolean deleteAndVerifyContainerGone(String container);
}

View File

@ -23,6 +23,7 @@ import java.io.IOException;
import javax.inject.Provider;
import org.jclouds.apis.ApiMetadata;
import org.jclouds.blobstore.config.LocalBlobStore;
import org.jclouds.blobstore.domain.Blob;
import org.jclouds.blobstore.domain.BlobBuilder;
import org.jclouds.http.HttpRequest;
@ -39,7 +40,7 @@ import com.google.common.io.ByteSource;
*/
// NOTE:without testName, this will not call @Before* and fail w/NPE during surefire
@Test(groups = "unit", testName = "TransientBlobRequestSignerTest")
public class TransientBlobRequestSignerTest extends BaseAsyncClientTest<LocalAsyncBlobStore> {
public class TransientBlobRequestSignerTest extends BaseAsyncClientTest<LocalBlobStore> {
private BlobRequestSigner signer;
private Provider<BlobBuilder> blobFactory;

View File

@ -24,11 +24,9 @@ import org.jclouds.azureblob.AzureBlobClient;
import org.jclouds.azureblob.blobstore.AzureBlobRequestSigner;
import org.jclouds.azureblob.blobstore.AzureBlobStore;
import org.jclouds.azureblob.domain.PublicAccess;
import org.jclouds.blobstore.AsyncBlobStore;
import org.jclouds.blobstore.BlobRequestSigner;
import org.jclouds.blobstore.BlobStore;
import org.jclouds.blobstore.attr.ConsistencyModel;
import org.jclouds.blobstore.internal.SubmissionAsyncBlobStore;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
@ -42,7 +40,6 @@ public class AzureBlobStoreContextModule extends AbstractModule {
@Override
protected void configure() {
bind(ConsistencyModel.class).toInstance(ConsistencyModel.STRICT);
bind(AsyncBlobStore.class).to(SubmissionAsyncBlobStore.class).in(Scopes.SINGLETON);
bind(BlobStore.class).to(AzureBlobStore.class).in(Scopes.SINGLETON);
bind(BlobRequestSigner.class).to(AzureBlobRequestSigner.class);
}

View File

@ -26,10 +26,8 @@ import javax.annotation.Resource;
import javax.inject.Inject;
import javax.inject.Singleton;
import org.jclouds.blobstore.AsyncBlobStore;
import org.jclouds.blobstore.BlobStore;
import org.jclouds.blobstore.attr.ConsistencyModel;
import org.jclouds.blobstore.internal.SubmissionAsyncBlobStore;
import org.jclouds.hpcloud.objectstorage.HPCloudObjectStorageApi;
import org.jclouds.hpcloud.objectstorage.blobstore.HPCloudObjectStorageBlobStore;
import org.jclouds.hpcloud.objectstorage.blobstore.functions.HPCloudObjectStorageObjectToBlobMetadata;
@ -46,7 +44,6 @@ import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import com.google.inject.Provides;
import com.google.inject.Scopes;
public class HPCloudObjectStorageBlobStoreContextModule extends SwiftBlobStoreContextModule {
@ -96,7 +93,6 @@ public class HPCloudObjectStorageBlobStoreContextModule extends SwiftBlobStoreCo
protected void configure() {
bind(ConsistencyModel.class).toInstance(ConsistencyModel.EVENTUAL);
bind(BlobStore.class).to(HPCloudObjectStorageBlobStore.class);
bind(AsyncBlobStore.class).to(SubmissionAsyncBlobStore.class).in(Scopes.SINGLETON);
bind(ObjectToBlobMetadata.class).to(HPCloudObjectStorageObjectToBlobMetadata.class);
}
}