mirror of https://github.com/apache/jclouds.git
JCLOUDS-40 remove all implementations of AsyncBlobStore except Submission in preparation for complete removal.
This commit is contained in:
parent
7e04f6236d
commit
7047874ad9
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue