inserted first implementation of filesystem storage

This commit is contained in:
rainbowbreeze 2010-08-31 16:51:22 +02:00
parent e05b2b560f
commit 8ca932cb26
21 changed files with 3393 additions and 1 deletions

View File

@ -49,5 +49,10 @@
<artifactId>jclouds-rackspace</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>jclouds-filesystem</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
</project>

View File

@ -118,4 +118,7 @@ googlestorage.contextbuilder=org.jclouds.aws.s3.S3ContextBuilder
googlestorage.propertiesbuilder=org.jclouds.aws.s3.GoogleStoragePropertiesBuilder
transient.contextbuilder=org.jclouds.blobstore.TransientBlobStoreContextBuilder
transient.propertiesbuilder=org.jclouds.blobstore.TransientBlobStorePropertiesBuilder
transient.propertiesbuilder=org.jclouds.blobstore.TransientBlobStorePropertiesBuilder
filesystem.contextbuilder=org.jclouds.filesystem.FilesystemBlobStoreContextBuilder
filesystem.propertiesbuilder=org.jclouds.filesystem.FilesystemBlobStorePropertiesBuilder

57
filesystem/pom.xml Normal file
View File

@ -0,0 +1,57 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
Copyright (C) 2010 Cloud Conscious, LLC. <info@cloudconscious.com>
====================================================================
Licensed 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.
====================================================================
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4.0.0.xsd" >
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.jclouds</groupId>
<artifactId>jclouds-project</artifactId>
<version>1.0-SNAPSHOT</version>
<relativePath>../project/pom.xml</relativePath>
</parent>
<artifactId>jclouds-filesystem</artifactId>
<name>jcloud filesystem core</name>
<description>jclouds components to access filesystem</description>
<dependencies>
<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>jclouds-blobstore</artifactId>
<version>${project.version}</version>
<type>jar</type>
</dependency>
<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>jclouds-core</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
<version>1.4</version>
</dependency>
</dependencies>
</project>

View File

@ -0,0 +1,664 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.filesystem;
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.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.Lists.newArrayList;
import static com.google.common.collect.Lists.partition;
import static com.google.common.collect.Maps.newHashMap;
import static com.google.common.collect.Sets.filter;
import static com.google.common.collect.Sets.newTreeSet;
import static com.google.common.io.ByteStreams.toByteArray;
import static com.google.common.util.concurrent.Futures.immediateFailedFuture;
import static com.google.common.util.concurrent.Futures.immediateFuture;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.ObjectInput;
import java.io.ObjectInputStream;
import java.io.ObjectOutput;
import java.io.ObjectOutputStream;
import java.io.File;
import java.net.URI;
import java.util.Collection;
import java.util.Date;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.Map.Entry;
import java.util.concurrent.ExecutorService;
import javax.inject.Inject;
import javax.inject.Named;
import javax.ws.rs.core.HttpHeaders;
import org.jclouds.Constants;
import org.jclouds.blobstore.domain.Blob;
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.Blob.Factory;
import org.jclouds.blobstore.domain.internal.MutableStorageMetadataImpl;
import org.jclouds.blobstore.domain.internal.PageSetImpl;
import org.jclouds.blobstore.functions.HttpGetOptionsListToGetOptions;
import org.jclouds.blobstore.internal.BaseAsyncBlobStore;
import org.jclouds.blobstore.options.GetOptions;
import org.jclouds.blobstore.options.ListContainerOptions;
import org.jclouds.blobstore.strategy.IfDirectoryReturnNameStrategy;
import org.jclouds.blobstore.util.BlobUtils;
import org.jclouds.crypto.Crypto;
import org.jclouds.crypto.CryptoStreams;
import org.jclouds.date.DateService;
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.options.HttpRequestOptions;
import org.jclouds.io.Payloads;
import org.jclouds.io.payloads.ByteArrayPayload;
import org.jclouds.io.payloads.DelegatingPayload;
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.Iterables;
import com.google.common.collect.Multimaps;
import com.google.common.io.Closeables;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.inject.internal.Nullable;
import javax.annotation.Resource;
import org.jclouds.blobstore.BlobStoreContext;
import org.jclouds.blobstore.ContainerNotFoundException;
import org.jclouds.blobstore.KeyNotFoundException;
import org.jclouds.blobstore.domain.internal.MutableBlobMetadataImpl;
import org.jclouds.filesystem.utils.FilesystemStorageStrategy;
import org.jclouds.io.Payload;
import org.jclouds.io.payloads.FilePayload;
import org.jclouds.logging.Logger;
/**
*
* Preconditions:
* Blob name cannot start with / char (or \ under windows)
*
* @author Alfredo "Rainbowbreeze" Morresi
*/
public class FilesystemAsyncBlobStore extends BaseAsyncBlobStore {
private static final String BACK_SLASH = "\\";
@Resource
protected Logger logger = Logger.NULL;
protected final DateService dateService;
protected final Crypto crypto;
protected final HttpGetOptionsListToGetOptions httpGetOptionsConverter;
protected final IfDirectoryReturnNameStrategy ifDirectoryReturnName;
protected final Factory blobFactory;
protected final FilesystemStorageStrategy storageStrategy;
@Inject
protected FilesystemAsyncBlobStore(BlobStoreContext context, DateService dateService, Crypto crypto,
HttpGetOptionsListToGetOptions httpGetOptionsConverter,
IfDirectoryReturnNameStrategy ifDirectoryReturnName, Blob.Factory blobFactory, BlobUtils blobUtils,
@Named(Constants.PROPERTY_USER_THREADS) ExecutorService service,
Supplier<Location> defaultLocation,
Supplier<Set<? extends Location>> locations,
FilesystemStorageStrategy storageStrategy) {
super(context, blobUtils, service, defaultLocation, locations);
//super(context, blobUtils, service, null, null);
this.blobFactory = blobFactory;
this.dateService = dateService;
this.crypto = crypto;
this.httpGetOptionsConverter = httpGetOptionsConverter;
this.ifDirectoryReturnName = ifDirectoryReturnName;
this.storageStrategy = checkNotNull(storageStrategy, "Storage strategy");
}
/**
* default maxResults is 1000
*/
@Override
public ListenableFuture<PageSet<? extends StorageMetadata>> list(final String container, ListContainerOptions options) {
// Check if the container exists
if (!containerExistsSyncImpl(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) {
Blob oldBlob = loadFileBlob(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 = 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().equals(finalMarker);
}
});
contents = contents.tailSet(lastMarkerMetadata);
contents.remove(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));
}
}));
}
Integer maxResults = options.getMaxResults() != null ? options.getMaxResults() : 1000;
if (contents.size() > 0) {
SortedSet<StorageMetadata> contentsSlice = firstSliceOfSize(contents, maxResults);
if (!contentsSlice.contains(contents.last())) {
// Partial listing
marker = contentsSlice.last().getName();
} else {
marker = null;
}
contents = contentsSlice;
}
final String delimiter = options.isRecursive() ? null : File.separator;
if (delimiter != null) {
SortedSet<String> commonPrefixes = null;
Iterable<String> iterable = transform(contents, new CommonPrefixes(prefix != null ? prefix : null, delimiter));
commonPrefixes = iterable != null ? newTreeSet(iterable) : new TreeSet<String>();
commonPrefixes.remove(CommonPrefixes.NO_PREFIX);
contents = newTreeSet(filter(contents, new DelimiterFilter(prefix != null ? prefix : null, delimiter)));
Iterables.<StorageMetadata> addAll(contents, transform(commonPrefixes,
new Function<String, StorageMetadata>() {
public StorageMetadata apply(String o) {
MutableStorageMetadata md = new MutableStorageMetadataImpl();
md.setType(StorageType.RELATIVE_PATH);
md.setName(o);
return 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 filesystem", name));
}
public static MutableBlobMetadata copy(MutableBlobMetadata in) {
ByteArrayOutputStream bout = new ByteArrayOutputStream();
ObjectOutput os;
try {
os = new ObjectOutputStream(bout);
os.writeObject(in);
ObjectInput is = new ObjectInputStream(new ByteArrayInputStream(bout.toByteArray()));
MutableBlobMetadata metadata = (MutableBlobMetadata) is.readObject();
convertUserMetadataKeysToLowercase(metadata);
return metadata;
} catch (Exception e) {
propagate(e);
assert false : "exception should have propagated: " + e;
return null;
}
}
private static void convertUserMetadataKeysToLowercase(MutableBlobMetadata metadata) {
Map<String, String> lowerCaseUserMetadata = newHashMap();
for (Entry<String, String> entry : metadata.getUserMetadata().entrySet()) {
lowerCaseUserMetadata.put(entry.getKey().toLowerCase(), entry.getValue());
}
metadata.setUserMetadata(lowerCaseUserMetadata);
}
public static MutableBlobMetadata copy(MutableBlobMetadata in, String newKey) {
MutableBlobMetadata newMd = copy(in);
newMd.setName(newKey);
return newMd;
}
/**
* {@inheritDoc}
*/
@Override
public ListenableFuture<Void> removeBlob(final String container, final String key) {
storageStrategy.removeBlob(container, key);
return immediateFuture(null);
}
/**
* {@inheritDoc}
*/
@Override
public ListenableFuture<Boolean> containerExists(final String containerName) {
boolean exists = containerExistsSyncImpl(containerName);
return immediateFuture(exists);
}
/**
* {@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);
return cmd;
}
}), null));
}
protected MutableStorageMetadata create() {
return new MutableStorageMetadataImpl();
}
/**
* {@inheritDoc}
*/
@Override
public ListenableFuture<Boolean> createContainerInLocation(final Location location, final String name) {
boolean result = storageStrategy.createContainer(name);
return immediateFuture(result);
}
public String getFirstQueryOrNull(String string, @Nullable HttpRequestOptions options) {
if (options == null)
return null;
Collection<String> values = options.buildQueryParameters().get(string);
return (values != null && values.size() >= 1) ? values.iterator().next() : null;
}
/**
* Load the blob with the given key belonging to the container with the given
* name. There must exist a resource on the file system whose complete name
* is given concatenating the container name and the key
*
* @param container it's the name of the container the blob belongs to
* @param key it's the key of the blob
*
* @return the blob belonging to the given container with the given key
*/
private Blob loadFileBlob(final String container, final String key) {
logger.debug("Opening blob in container: %s - %s", container, key);
File blobPayload = storageStrategy.getFileForBlobKey(container, key);
Payload payload = new FilePayload(blobPayload);
// Loading object metadata
MutableBlobMetadata metadata = new MutableBlobMetadataImpl();
metadata.setName(key);
metadata.setLastModified(new Date(blobPayload.lastModified()));
metadata.setSize(blobPayload.length());
// TODO What about the MD5? are we supposed to calculate it each time we load
//the file?
try {
payload = Payloads.calculateMD5(payload);
} catch (IOException e) {
logger.error("An error occurred calculating MD5 for blob %s from container ", key, container);
Throwables.propagateIfPossible(e);
}
metadata.setContentType("");
String eTag = CryptoStreams.hex(payload.getContentMD5());
metadata.setETag(eTag);
// Creating new blob object
Blob blob = blobFactory.create(metadata);
blob.setPayload(blobPayload);
return blob;
}
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(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(toMatch, "");
}
}
if (working.contains(delimiter)) {
return working.substring(0, working.indexOf(delimiter));
}
return NO_PREFIX;
}
}
public static <T extends Comparable<?>> SortedSet<T> firstSliceOfSize(Iterable<T> elements, int size) {
List<List<T>> slices = partition(newArrayList(elements), size);
return newTreeSet(slices.get(0));
}
public static HttpResponseException returnResponseException(int code) {
HttpResponse response = null;
response = new HttpResponse(code, null, null);
return new HttpResponseException(new HttpCommand() {
public int getRedirectCount() {
return 0;
}
public int incrementRedirectCount() {
return 0;
}
public boolean isReplayable() {
return false;
}
public Exception getException() {
return null;
}
public int getFailureCount() {
return 0;
}
public HttpRequest getRequest() {
return new HttpRequest("GET", URI.create("http://stub"));
}
public int incrementFailureCount() {
return 0;
}
public void setException(Exception exception) {
}
}, response);
}
/**
* {@inheritDoc}
*/
@Override
public ListenableFuture<String> putBlob(String containerName, Blob object) {
String blobKey = object.getMetadata().getName();
logger.debug("Put object with key [%s] to container [%s]", blobKey, containerName);
String eTag = getEtag(object);
try {
//TODO
//must override existing file?
storageStrategy.writePayloadOnFile(containerName, blobKey, object.getPayload());
} catch (IOException e) {
logger.error(e, "An error occurred storing the new object with name [%s] to container [%s].",
blobKey,
containerName);
Throwables.propagate(e);
}
return immediateFuture(eTag);
}
/**
* {@inheritDoc}
*/
@Override
public ListenableFuture<Boolean> blobExists(final String containerName, final String key) {
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(!containerExistsSyncImpl(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 = loadFileBlob(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 = new HttpResponse(304, null, null);
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 = new HttpResponse(412, null, null);
return immediateFailedFuture(new HttpResponseException(String.format("%1$s is after %2$s", blob
.getMetadata().getLastModified(), unmodifiedSince), null, response));
}
}
if (options.getRanges() != null && options.getRanges().size() > 0) {
byte[] data;
try {
data = toByteArray(blob.getPayload().getInput());
} catch (IOException e) {
return immediateFailedFuture(new RuntimeException(e));
}
ByteArrayOutputStream out = new ByteArrayOutputStream();
for (String s : options.getRanges()) {
if (s.startsWith("-")) {
int length = Integer.parseInt(s.substring(1));
out.write(data, data.length - length, length);
} else if (s.endsWith("-")) {
int offset = Integer.parseInt(s.substring(0, s.length() - 1));
out.write(data, offset, data.length - offset);
} else if (s.contains("-")) {
String[] firstLast = s.split("\\-");
int offset = Integer.parseInt(firstLast[0]);
int last = Integer.parseInt(firstLast[1]);
int length = (last < data.length) ? last + 1 : data.length - offset;
out.write(data, offset, length);
} else {
return immediateFailedFuture(new IllegalArgumentException("first and last were null!"));
}
}
blob.setPayload(out.toByteArray());
blob.getMetadata().setSize(new Long(data.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) copy(blob.getMetadata()) : null);
} catch (Exception e) {
if (size(filter(getCausalChain(e), KeyNotFoundException.class)) >= 1)
return immediateFuture(null);
return immediateFailedFuture(e);
}
}
@Override
protected boolean deleteAndVerifyContainerGone(String container) {
storageStrategy.deleteContainer(container);
return containerExistsSyncImpl(container);
}
/**
* 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(String container) {
deleteAndVerifyContainerGone(container);
return immediateFuture(null);
}
/**
* Each container is a directory, so in order to check if a container exists
* the corresponding directory must exists. Synchronous implementation
* @param containerName
* @return
*/
private boolean containerExistsSyncImpl(String containerName) {
return storageStrategy.containerExists(containerName);
}
/**
*
* Calculates the object MD5 and returns it as eTag
* @param object
* @return
*/
private String getEtag(Blob object) {
try {
Payloads.calculateMD5(object, crypto.md5());
} catch (IOException ex) {
logger.error(ex, "An error occurred calculating MD5 for object with name %s.",
object.getMetadata().getName());
Throwables.propagate(ex);
}
String eTag = CryptoStreams.hex(object.getPayload().getContentMD5());
return eTag;
}
}

View File

@ -0,0 +1,59 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.filesystem;
import com.google.inject.Module;
import java.util.List;
import java.util.Properties;
import org.jclouds.blobstore.AsyncBlobStore;
import org.jclouds.blobstore.BlobStore;
import org.jclouds.blobstore.BlobStoreContextBuilder;
import org.jclouds.filesystem.config.FilesystemBlobStoreContextModule;
import org.jclouds.filesystem.config.FilesystemBlobStoreModule;
/**
*
* @author Alfredo "Rainbowbreeze" Morresi
*/
public class FilesystemBlobStoreContextBuilder extends
BlobStoreContextBuilder<BlobStore, AsyncBlobStore> {
/**
* This is only to have the same syntax.
*
*/
public FilesystemBlobStoreContextBuilder() {
this(new Properties());
}
public FilesystemBlobStoreContextBuilder(Properties props) {
super(BlobStore.class, AsyncBlobStore.class, props);
}
@Override
public void addContextModule(List<Module> modules) {
modules.add(new FilesystemBlobStoreContextModule());
}
@Override
protected void addClientModule(List<Module> modules) {
modules.add(new FilesystemBlobStoreModule());
}
}

View File

@ -0,0 +1,40 @@
/*
* To change this template, choose Tools | Templates
* and open the template in the editor.
*/
package org.jclouds.filesystem;
import static org.jclouds.Constants.PROPERTY_API_VERSION;
import static org.jclouds.Constants.PROPERTY_ENDPOINT;
import static org.jclouds.Constants.PROPERTY_IO_WORKER_THREADS;
import static org.jclouds.Constants.PROPERTY_IDENTITY;
import static org.jclouds.Constants.PROPERTY_USER_THREADS;
import java.util.Properties;
import org.jclouds.PropertiesBuilder;
/**
*
* @author rainbowbreeze
*/
public class FilesystemBlobStorePropertiesBuilder extends PropertiesBuilder {
@Override
protected Properties defaultProperties() {
Properties properties = super.defaultProperties();
properties.setProperty(PROPERTY_ENDPOINT, "http://localhost/transient");
properties.setProperty(PROPERTY_API_VERSION, "1");
properties.setProperty(PROPERTY_IDENTITY, System.getProperty("user.name"));
properties.setProperty(PROPERTY_USER_THREADS, "0");
properties.setProperty(PROPERTY_IO_WORKER_THREADS, "0");
System.out.println("Properties:"+properties );
return properties;
}
public FilesystemBlobStorePropertiesBuilder(Properties properties) {
super(properties);
}
}

View File

@ -0,0 +1,36 @@
/**
*
* Copyright (C) 2009 Global Cloud Specialists, Inc. <info@globalcloudspecialists.com>
*
* ====================================================================
* 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.filesystem.config;
import java.util.concurrent.TimeUnit;
import org.jclouds.blobstore.BlobStore;
import org.jclouds.concurrent.Timeout;
/**
*
* @author Alfredo "Rainbowbreeze" Morresi
*/
@Timeout(duration = 30, timeUnit = TimeUnit.SECONDS) interface FilesystemBlobStore extends BlobStore {
}

View File

@ -0,0 +1,102 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.filesystem.config;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.common.collect.ImmutableSet;
import com.google.inject.AbstractModule;
import com.google.inject.Provides;
import com.google.inject.Scopes;
import com.google.inject.Singleton;
import com.google.inject.TypeLiteral;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import org.jclouds.blobstore.AsyncBlobStore;
import org.jclouds.blobstore.BlobStore;
import org.jclouds.blobstore.BlobStoreContext;
import org.jclouds.blobstore.attr.ConsistencyModel;
import org.jclouds.blobstore.config.BlobStoreMapModule;
import org.jclouds.blobstore.config.BlobStoreObjectModule;
import org.jclouds.blobstore.domain.Blob;
import org.jclouds.blobstore.internal.BlobStoreContextImpl;
import org.jclouds.blobstore.util.BlobUtils;
import org.jclouds.domain.Location;
import org.jclouds.domain.LocationScope;
import org.jclouds.domain.internal.LocationImpl;
import org.jclouds.filesystem.utils.FileSystemBlobUtilsImpl;
import org.jclouds.filesystem.utils.FilesystemStorageStrategy;
import org.jclouds.filesystem.utils.FilesystemStorageStrategyImpl;
/**
*
* @author Alfredo "Rainbowbreeze" Morresi
*/
public class FilesystemBlobStoreContextModule extends AbstractModule {
@Override
protected void configure() {
bind(new TypeLiteral<BlobStoreContext>() {
}).to(new TypeLiteral<BlobStoreContextImpl<FilesystemBlobStore, AsyncBlobStore>>() {
}).in(Scopes.SINGLETON);
install(new BlobStoreObjectModule());
install(new BlobStoreMapModule());
bind(ConsistencyModel.class).toInstance(ConsistencyModel.STRICT);
bind(FilesystemStorageStrategy.class).to(FilesystemStorageStrategyImpl.class);
bind(BlobUtils.class).to(FileSystemBlobUtilsImpl.class);
}
/* @Provides
@Singleton
Set<Location> provideLocations(Location defaultLocation) {
return ImmutableSet.of(defaultLocation);
}
*
*/
@Provides
@Singleton
BlobStore provide(FilesystemBlobStore in) {
return in;
}
/*@Provides
@Singleton
Location provideDefaultLocation() {
return new LocationImpl(LocationScope.PROVIDER, "filesystem", "filesystem", null);
}
*
*/
@Provides
@Singleton
Supplier<Set<? extends Location>> provideLocations(Supplier<Location> defaultLocation) {
return Suppliers.<Set<? extends Location>> ofInstance(ImmutableSet.of(defaultLocation.get()));
}
@Provides
@Singleton
Supplier<Location> provideDefaultLocation() {
return Suppliers
.<Location> ofInstance(new LocationImpl(LocationScope.PROVIDER, "filesystem", "filesystem", null));
}
}

View File

@ -0,0 +1,46 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.filesystem.config;
import org.jclouds.blobstore.AsyncBlobStore;
import org.jclouds.filesystem.FilesystemAsyncBlobStore;
import org.jclouds.rest.config.RestClientModule;
/**
*
* @author Alfredo "Rainbowbreeze" Morresi
*/
public class FilesystemBlobStoreModule extends RestClientModule<FilesystemBlobStore, AsyncBlobStore> {
public FilesystemBlobStoreModule() {
super(FilesystemBlobStore.class, AsyncBlobStore.class);
}
@Override
protected void configure() {
super.configure();
}
@Override
protected void bindAsyncClient() {
bind(AsyncBlobStore.class).to(FilesystemAsyncBlobStore.class).asEagerSingleton();
}
}

View File

@ -0,0 +1,31 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.filesystem.config;
/**
* Common constants used in filesystem provider
*
* @author Alfredo "Rainbowbreeze" Morresi
*/
public class FilesystemConstants {
/** Specify the base directory where provider starts its file operations - must exists */
public static final String PROPERTY_BASEDIR = "FileSystemAsyncBlobStore-basedir";
}

View File

@ -0,0 +1,77 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.filesystem.utils;
import com.google.inject.Inject;
import org.jclouds.blobstore.AsyncBlobStore;
import org.jclouds.blobstore.domain.Blob;
import org.jclouds.blobstore.options.ListContainerOptions;
import org.jclouds.blobstore.util.BlobUtils;
import static com.google.common.base.Preconditions.checkNotNull;
/**
* Implements the {@link BlobUtils} interfaced and act as a bridge to
* {@link FilesystemStorageStrategy} when used inside {@link AsyncBlobStore}
*
* @author Alfredo "Rainbowbreeze" Morresi
*/
public class FileSystemBlobUtilsImpl implements BlobUtils {
protected final FilesystemStorageStrategy storageStrategy;
@Inject
public FileSystemBlobUtilsImpl(
FilesystemStorageStrategy storageStrategy) {
this.storageStrategy = checkNotNull(storageStrategy, "Filesystem Storage Strategy");
}
@Override
public Blob newBlob(String name) {
return storageStrategy.newBlob(name);
}
@Override
public boolean directoryExists(String containerName, String directory) {
return storageStrategy.directoryExists(containerName, directory);
}
@Override
public void createDirectory(String containerName, String directory) {
storageStrategy.createDirectory(containerName, directory);
}
@Override
public long countBlobs(String container, ListContainerOptions options) {
return storageStrategy.countBlobs(container, options);
}
@Override
public void clearContainer(String container, ListContainerOptions options) {
storageStrategy.clearContainer(container, options);
}
@Override
public void deleteDirectory(String container, String directory) {
storageStrategy.deleteDirectory(container, directory);
}
}

View File

@ -0,0 +1,164 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.filesystem.utils;
import java.io.File;
import java.io.IOException;
import org.jclouds.blobstore.domain.Blob;
import org.jclouds.blobstore.options.ListContainerOptions;
import org.jclouds.io.Payload;
/**
* Strategy for filesystem operations related to container and blob
*
* @author Alfredo "Rainbowbreeze" Morresi
*/
public interface FilesystemStorageStrategy {
/**
* Creates a new container
*
* @param container
* @return
*/
boolean createContainer(String container);
/**
* Deletes a container and all its content
* @param container
*/
void deleteContainer(String container);
/**
* Checks if a container exists
* @param container
* @return
*/
boolean containerExists(String container);
/**
* Empty the container of its content (files and subdirectories), but doesn't
* delete the container itself
* @param container
*/
void clearContainer(final String container);
/**
* Like {@link #clearContainer(String)} except you can use options to do things like recursive
* deletes, or clear at a different path than root.
*
* @param container
* what to clear
* @param options
* recursion and path to clear
*/
void clearContainer(String container, ListContainerOptions options);
/**
* Return an iterator that reports all the containers under base path
* @return
*/
Iterable<String> getAllContainerNames();
/**
* Determines if a directory exists
*
* @param container
* container where the directory resides
* @param directory
* full path to the directory
*/
boolean directoryExists(String container, String directory);
/**
* Creates a folder or a directory marker depending on the service
*
* @param container
* container to create the directory in
* @param directory
* full path to the directory
*/
void createDirectory(String container, String directory);
/**
* Deletes a folder or a directory marker depending on the service
*
* @param container
* container to delete the directory from
* @param directory
* full path to the directory to delete
*/
void deleteDirectory(String container, String directory);
/**
* Creates a new blob
* @param name
* @return
*/
Blob newBlob(String name);
/**
*
* @param containerName
* @param key
* @return
*/
boolean blobExists(String containerName, String key);
/**
* Returns all the blobs key inside a container
* @param container
* @return
* @throws IOException
*/
Iterable<String> getBlobKeysInsideContainer(String container) throws IOException;
/**
* Counts number of blobs inside a container
* @param container
* @param options
* @return
*/
long countBlobs(String container, ListContainerOptions options);
/**
* Returns a {@link File} object that links to the blob
* @param container
* @param key
* @return
*/
File getFileForBlobKey(String container, String key);
/**
*
* @param container
* @param key
*/
void removeBlob(final String container, final String key);
/**
* Write a {@link Blob} {@link Payload} into a file
* @param fileName
* @param payload
* @throws IOException
*/
void writePayloadOnFile(String containerName, String key, Payload payload) throws IOException;
}

View File

@ -0,0 +1,491 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.filesystem.utils;
import java.io.OutputStream;
import java.util.Set;
import java.util.HashSet;
import org.jclouds.blobstore.domain.Blob;
import org.jclouds.blobstore.options.ListContainerOptions;
import org.jclouds.io.Payload;
import java.io.InputStream;
import java.io.FileOutputStream;
import java.io.FileFilter;
import org.apache.commons.io.filefilter.DirectoryFileFilter;
import java.util.Iterator;
import com.google.common.base.Throwables;
import java.io.IOException;
import org.apache.commons.io.FileUtils;
import java.io.File;
import javax.annotation.Resource;
import com.google.inject.Inject;
import com.google.inject.name.Named;
import org.jclouds.filesystem.config.FilesystemConstants;
import org.jclouds.logging.Logger;
import static com.google.common.base.Preconditions.checkNotNull;
/**
*
* @author Alfredo "Rainbowbreeze" Morresi
*/
public class FilesystemStorageStrategyImpl implements FilesystemStorageStrategy {
private static final String BACK_SLASH = "\\";
/** The buffer size used to copy an InputStream to an OutputStream */
private static final int COPY_BUFFER_SIZE = 1024;
@Resource
protected Logger logger = Logger.NULL;
protected final Blob.Factory blobFactory;
protected final String baseDirectory;
@Inject
protected FilesystemStorageStrategyImpl(
Blob.Factory blobFactory,
@Named(FilesystemConstants.PROPERTY_BASEDIR) String baseDir) {
this.blobFactory = checkNotNull(blobFactory, "filesystem storage strategy blobfactory");
this.baseDirectory = checkNotNull(baseDir, "filesystem storage strategy base directory");
}
@Override
public boolean containerExists(String container) {
return directoryExists(container, null);
}
@Override
public boolean blobExists(String containerName, String key) {
return buildPathAndChecksIfFileExists(containerName, key);
}
@Override
public boolean createContainer(String container) {
logger.debug("Creating container %s", container);
return createDirectoryWithResult(container, null);
}
@Override
public void deleteContainer(String container) {
deleteDirectory(container, null);
}
/**
* Empty the directory of its content (files and subdirectories)
* @param container
*/
@Override
public void clearContainer(final String container) {
clearContainer(container, ListContainerOptions.Builder.recursive());
}
@Override
public void clearContainer(String container, ListContainerOptions options) {
//TODO
//now all is deleted, check it based on options
try {
File containerFile = openFolder(container);
File[] children = containerFile.listFiles();
if (null != children) {
for(File child:children) {
FileUtils.forceDelete(child);
}
}
} catch(IOException e) {
logger.error(e,"An error occurred while clearing container %s", container);
Throwables.propagate(e);
}
}
@Override
public Blob newBlob(String name) {
Blob blob = blobFactory.create(null);
blob.getMetadata().setName(name);
return blob;
}
@Override
public void removeBlob(final String container, final String key) {
String fileName = buildPathStartingFromBaseDir(container, key);
logger.debug("Deleting blob %s", fileName);
File fileToBeDeleted = new File(fileName);
fileToBeDeleted.delete();
//now examins if the key of the blob is a complex key (with a directory structure)
//and eventually remove empty directory
removeDirectoriesTreeOfBlobKey(container, key);
}
/**
* Return an iterator that reports all the containers under base path
* @return
*/
@Override
public Iterable<String> getAllContainerNames() {
Iterable<String> containers = new Iterable<String>() {
@Override
public Iterator<String> iterator() {
return new FileIterator(
buildPathStartingFromBaseDir(), DirectoryFileFilter.INSTANCE);
}
};
return containers;
}
/**
* Returns a {@link File} object that links to the blob
* @param container
* @param blobKey
* @return
*/
@Override
public File getFileForBlobKey(String container, String blobKey) {
String fileName = buildPathStartingFromBaseDir(container, blobKey);
File blobFile = new File(fileName);
return blobFile;
}
/**
* Write a {@link Blob} {@link Payload} into a file
* @param containerName
* @param blobKey
* @param payload
* @throws IOException
*/
@Override
public void writePayloadOnFile(String containerName, String blobKey, Payload payload) throws IOException {
File outputFile = null;
OutputStream output = null;
InputStream input = null;
try {
outputFile = getFileForBlobKey(containerName, blobKey);
File parentDirectory = outputFile.getParentFile();
if (!parentDirectory.exists()) {
if (!parentDirectory.mkdirs()) {
throw new IOException("An error occurred creating directory [" + parentDirectory.getName() + "].");
}
}
output = new FileOutputStream(outputFile);
input = payload.getInput();
copy(input, output);
} catch (IOException ex) {
if (outputFile != null) {
outputFile.delete();
}
throw ex;
} finally {
if (input != null) {
try {
input.close();
} catch (IOException ex) {
// Does nothing
}
}
if (output != null) {
try {
output.close();
} catch (IOException ex) {
// Does nothing
}
}
}
}
/**
* Returns all the blobs key inside a container
* @param container
* @return
* @throws IOException
*/
@Override
public Iterable<String> getBlobKeysInsideContainer(String container) throws IOException {
//check if container exists
//TODO maybe an error is more appropriate
if (!containerExists(container)) {
return new HashSet<String>();
}
File containerFile = openFolder(container);
final int containerPathLenght = containerFile.getAbsolutePath().length() + 1;
Set<String> blobNames = new HashSet<String>() {
@Override
public boolean add(String e) {
return super.add(e.substring(containerPathLenght));
}
};
populateBlobKeysInContainer(containerFile, blobNames);
return blobNames;
}
@Override
public boolean directoryExists(String container, String directory) {
return buildPathAndChecksIfDirectoryExists(container, directory);
}
@Override
public void createDirectory(String container, String directory) {
createDirectoryWithResult(container, directory);
}
@Override
public void deleteDirectory(String container, String directory) {
//create complete dir path
String fullDirPath = buildPathStartingFromBaseDir(container, directory);
try {
FileUtils.forceDelete(new File(fullDirPath));
} catch (IOException ex) {
logger.error("An error occurred removing directory %s.", fullDirPath);
Throwables.propagate(ex);
}
}
@Override
public long countBlobs(String container, ListContainerOptions options) {
//TODO
throw new UnsupportedOperationException("Not supported yet.");
}
//---------------------------------------------------------- Private methods
private boolean buildPathAndChecksIfFileExists(String...tokens) {
String path = buildPathStartingFromBaseDir(tokens);
File file = new File(path);
boolean exists = file.exists() || file.isFile();
return exists;
}
/**
* Check if the file system resource whose name is obtained applying buildPath
* on the input path tokens is a directory, otherwise a RuntimeException is thrown
*
* @param tokens the tokens that make up the name of the resource on the
* file system
*/
private boolean buildPathAndChecksIfDirectoryExists(String...tokens) {
String path = buildPathStartingFromBaseDir(tokens);
File file = new File(path);
boolean exists = file.exists() || file.isDirectory();
return exists;
}
/**
* Facility method used to concatenate path tokens normalizing separators
* @param pathTokens all the string in the proper order that must be concatenated
* in order to obtain the filename
* @return the resulting string
*/
protected String buildPathStartingFromBaseDir(String...pathTokens) {
String normalizedToken = removeFileSeparatorFromBorders(normalize(baseDirectory));
StringBuilder completePath = new StringBuilder(normalizedToken);
if(pathTokens!=null && pathTokens.length>0) {
for(int i=0; i<pathTokens.length; i++) {
if(pathTokens[i]!=null) {
normalizedToken = removeFileSeparatorFromBorders(normalize(pathTokens[i]));
completePath.append(File.separator).append(normalizedToken);
}
}
}
return completePath.toString();
}
/**
* Substitutes all the file separator occurrences in the path with a file
* separator for the current operative system
* @param pathToBeNormalized
* @return
*/
private String normalize(String pathToBeNormalized) {
if(null != pathToBeNormalized && pathToBeNormalized.contains(BACK_SLASH)) {
if(!BACK_SLASH.equals(File.separator)) {
return pathToBeNormalized.replaceAll(BACK_SLASH, File.separator);
}
}
return pathToBeNormalized;
}
/**
* Remove leading and trailing {@link File.separator} character from the
* string.
* @param pathToBeCleaned
* @return
*/
private String removeFileSeparatorFromBorders(String pathToBeCleaned) {
if (null == pathToBeCleaned || pathToBeCleaned.equals("")) return pathToBeCleaned;
int beginIndex = 0;
int endIndex = pathToBeCleaned.length();
//search for separator chars
if (pathToBeCleaned.substring(0, 1).equals(File.separator)) beginIndex = 1;
if (pathToBeCleaned.substring(pathToBeCleaned.length() - 1).equals(File.separator)) endIndex--;
return pathToBeCleaned.substring(beginIndex, endIndex);
}
/**
* Removes recursively the directory structure of a complex blob key, only
* if the directory is empty
* @param container
* @param normalizedKey
*/
private void removeDirectoriesTreeOfBlobKey(String container, String blobKey) {
String normalizedBlobKey = normalize(blobKey);
//exists is no path is present in the blobkey
if (!normalizedBlobKey.contains(File.separator)) return;
File file = new File(normalizedBlobKey);
//TODO
//"/media/data/works/java/amazon/jclouds/master/filesystem/aa/bb/cc/dd/eef6f0c8-0206-460b-8870-352e6019893c.txt"
String parentPath = file.getParent();
//no need to manage "/" parentPath, because "/" cannot be used as start
//char of blobkey
if (null != parentPath || "".equals(parentPath)) {
//remove parent directory only it's empty
File directory = new File(buildPathStartingFromBaseDir(container, parentPath));
String[] children = directory.list();
if (null == children || children.length == 0) {
directory.delete();
//recursively call for removing other path
removeDirectoriesTreeOfBlobKey(container, parentPath);
}
}
}
private File openFolder(String folderName) throws IOException {
String baseFolderName = buildPathStartingFromBaseDir(folderName);
File folder = new File(baseFolderName);
if(folder.exists()) {
if(!folder.isDirectory()) {
throw new IOException("Resource " + baseFolderName + " isn't a folder.");
}
}
return folder;
}
private class FileIterator implements Iterator<String>{
int currentFileIndex = 0;
File[] children = new File[0];
File currentFile = null;
public FileIterator(String fileName, FileFilter filter) {
File file = new File(fileName);
if(file.exists() && file.isDirectory()) {
children = file.listFiles(filter);
}
}
@Override
public boolean hasNext() {
return currentFileIndex<children.length;
}
@Override
public String next() {
currentFile = children[currentFileIndex++];
return currentFile.getName();
}
@Override
public void remove() {
if(currentFile!=null && currentFile.exists()) {
if(!currentFile.delete()) {
throw new RuntimeException("An error occurred deleting "+currentFile.getName());
}
}
}
}
private void populateBlobKeysInContainer(File directory, Set<String> blobNames) {
File[] children = directory.listFiles();
for(File child:children) {
if(child.isFile()) {
blobNames.add(child.getAbsolutePath());
} else if(child.isDirectory()) {
populateBlobKeysInContainer(child, blobNames);
}
}
}
/**
* Creates a directory and returns the result
* @param container
* @param directory
* @return true if the directory was created, otherwise false
*/
protected boolean createDirectoryWithResult(String container, String directory) {
String directoryFullName = buildPathStartingFromBaseDir(container, directory);
logger.debug("Creating directory %s", directoryFullName);
//cannot use directoryFullName, because the following method rebuild
//another time the path starting from base directory
if (buildPathAndChecksIfDirectoryExists(container, directory)) {
logger.debug("Directory %s already exists", directoryFullName);
return false;
}
File directoryToCreate = new File(directoryFullName);
boolean result = directoryToCreate.mkdirs();
return result;
}
/**
* Copy from an InputStream to an OutputStream.
*
* @param input The InputStream
* @param output The OutputStream
* @return the number of bytes copied
* @throws IOException if an error occurs
*/
private long copy(InputStream input, OutputStream output)
throws IOException {
byte[] buffer = new byte[COPY_BUFFER_SIZE];
long count = 0;
while (true) {
int read = input.read(buffer);
if (read < 0) {
break;
}
count += read;
output.write(buffer, 0, read);
}
output.flush();
return count;
}
}

View File

@ -0,0 +1,27 @@
#
#
# Copyright (C) 2010 Cloud Conscious, LLC. <info@cloudconscious.com>
#
# ====================================================================
# Licensed 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.
# ====================================================================
#
# Set the default logging level for all loggers to WARNING
.level = INFO
handlers = java.util.logging.ConsoleHandler
java.util.logging.ConsoleHandler.level = ALL
java.util.logging.ConsoleHandler.formatter = java.util.logging.SimpleFormatter
org.jclouds.filesystem.FilesystemAsyncBlobStore.level=ALL
org.jclouds.filesystem.FilesystemAsyncBlobStore.handler=java.util.logging.ConsoleHandler

View File

@ -0,0 +1,832 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.filesystem;
import org.jclouds.filesystem.utils.TestUtils;
import org.jclouds.filesystem.config.FilesystemConstants;
import com.google.inject.CreationException;
import org.jclouds.blobstore.options.GetOptions;
import java.util.Iterator;
import java.util.Set;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.util.HashSet;
import java.util.Properties;
import org.apache.commons.io.FileUtils;
import org.jclouds.blobstore.BlobStore;
import org.jclouds.blobstore.BlobStoreContext;
import org.jclouds.blobstore.BlobStoreContextFactory;
import org.jclouds.blobstore.ContainerNotFoundException;
import org.jclouds.blobstore.domain.Blob;
import org.jclouds.blobstore.domain.BlobMetadata;
import org.jclouds.blobstore.domain.MutableBlobMetadata;
import org.jclouds.blobstore.domain.PageSet;
import org.jclouds.blobstore.domain.StorageMetadata;
import org.jclouds.blobstore.domain.StorageType;
import org.jclouds.blobstore.options.ListContainerOptions;
import org.testng.annotations.*;
import static org.testng.Assert.*;
/**
* Test class for {@link FilesystemAsyncBlobStore} class
*
* @author Alfredo "Rainbowbreeze" Morresi
*/
@Test(groups = "unit", testName = "filesystem.FilesystemAsyncBlobStoreTest", sequential = true)
public class FilesystemAsyncBlobStoreTest {
private static final String CONTAINER_NAME = "funambol-test";
private static final String TARGET_BASE_DIR = "./target/basedir/";
private static final String TARGET_CONTAINER_NAME = TARGET_BASE_DIR + CONTAINER_NAME;
private static final String LOGGING_CONFIG_KEY
= "java.util.logging.config.file";
private static final String LOGGING_CONFIG_VALUE
= "src/main/resources/logging.properties";
private static final String PROVIDER = "filesystem";
private static final String KEY1 = "";
private static final String KEY2 = "";
static {
System.setProperty(LOGGING_CONFIG_KEY,
LOGGING_CONFIG_VALUE);
}
private BlobStoreContext context = null;
private BlobStore blobStore = null;
private Set<File> resourcesToBeDeleted = new HashSet<File>();
@BeforeMethod
protected void setUp() throws Exception {
/* Enumeration<String> loggerNames = LogManager.getLogManager().getLoggerNames();
while(loggerNames.hasMoreElements()) {
String loggerName = loggerNames.nextElement();
System.out.println("Logger "+loggerName);
System.out.println("Livello "+LogManager.getLogManager().getLogger(loggerName).getLevel());
}*/
//create context per filesystem container
Properties prop = new Properties();
prop.setProperty(FilesystemConstants.PROPERTY_BASEDIR, TARGET_BASE_DIR);
context = (BlobStoreContext) new BlobStoreContextFactory().createContext(
PROVIDER, prop);
//create a container in the default location
blobStore = context.getBlobStore();
resourcesToBeDeleted.add(new File(TARGET_BASE_DIR));
}
@AfterMethod
protected void tearDown() {
context.close();
context = null;
// freeing filesystem resources used for tests
Iterator<File> resourceToDelete = resourcesToBeDeleted.iterator();
while(resourceToDelete.hasNext()) {
File fileToDelete = resourceToDelete.next();
try {
FileUtils.forceDelete(fileToDelete);
} catch (IOException ex) {
System.err.println("Error deleting folder ["+fileToDelete.getName()+"].");
}
resourceToDelete.remove();
}
}
/**
* Checks if context parameters are managed in the correct way
*
*/
public void testParameters() {
//no base directory declared in properties
try {
Properties props = new Properties();
BlobStoreContext testContext = (BlobStoreContext) new BlobStoreContextFactory().createContext(
PROVIDER, props);
fail("No error if base directory is not specified");
} catch (CreationException e) {
}
//no base directory declared in properties
try {
Properties props = new Properties();
props.setProperty(FilesystemConstants.PROPERTY_BASEDIR, null);
BlobStoreContext testContext = (BlobStoreContext) new BlobStoreContextFactory().createContext(
PROVIDER, props);
fail("No error if base directory is null in the option");
} catch (NullPointerException e) {
}
}
/**
* Test of list method of the root context
*/
public void testList_Root() throws IOException {
PageSet<StorageMetadata> containersRetrieved;
Set<String> containersCreated = new HashSet<String>();
// Testing list with no containers
containersRetrieved = (PageSet<StorageMetadata>) blobStore.list();
assertTrue(containersRetrieved.isEmpty(), "List operation returns a not empty set of container");
// Testing list with some containers
String[] containerNames = new String[]{"34343", "aaaa", "bbbbb"};
containersCreated = new HashSet<String>();
for(String containerName:containerNames) {
blobStore.createContainerInLocation(null, containerName);
containersCreated.add(containerName);
}
containersRetrieved = (PageSet<StorageMetadata>) blobStore.list();
assertEquals(containersCreated.size(), containersRetrieved.size(), "Different numbers of container");
for(StorageMetadata data:containersRetrieved) {
String containerName = data.getName();
if(!containersCreated.remove(containerName)) {
fail("Container list contains unexpected value ["+containerName+"]");
}
}
assertTrue(containersCreated.isEmpty(), "List operation doesn't return all values.");
for(String containerName:containerNames) {
//delete all creaded containers
blobStore.deleteContainer(containerName);
}
containersRetrieved = (PageSet<StorageMetadata>) blobStore.list();
assertTrue(containersRetrieved.isEmpty(), "List operation returns a not empty set of container");
}
/**
* Test of list method, of class FilesystemAsyncBlobStore.
*/
public void testList_NoOptionSingleContainer() throws IOException {
// Testing list for a not existing container
try {
blobStore.list(CONTAINER_NAME);
fail("Found a not existing container");
} catch(ContainerNotFoundException e) {
}
blobStore.createContainerInLocation(null, CONTAINER_NAME);
// Testing list for an empty container
checkForContainerContent(CONTAINER_NAME, null);
//creates blobs in first container
Set<String> blobsExpected = TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[] {
"bbb" + File.separator + "ccc" + File.separator + "ddd" + File.separator + "1234.jpg",
"4rrr.jpg",
"rrr" + File.separator + "sss" + File.separator + "788.jpg",
"xdc" + File.separator + "wert.kpg" }
);
checkForContainerContent(CONTAINER_NAME, blobsExpected);
}
public void testList_NotExistingContainer() {
// Testing list for a not existing container
try {
blobStore.list(CONTAINER_NAME);
fail("Found a not existing container");
} catch(ContainerNotFoundException e) {
//ok if arriver here
}
}
/**
* Test of list method, of class FilesystemAsyncBlobStore.
*/
public void testList_NoOptionDoubleContainer() throws IOException {
final String CONTAINER_NAME2 = "container2";
//create first container
blobStore.createContainerInLocation(null, CONTAINER_NAME);
//checks for empty container
checkForContainerContent(CONTAINER_NAME, null);
//create second container
blobStore.createContainerInLocation(null, CONTAINER_NAME2);
//checks for empty
checkForContainerContent(CONTAINER_NAME2, null);
//creates blobs in first container
Set<String> blobNamesCreatedInContainer1 = TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[] {
"bbb" + File.separator + "ccc" + File.separator + "ddd" + File.separator + "1234.jpg",
TestUtils.createRandomBlobKey(),
"rrr" + File.separator + "sss" + File.separator + "788.jpg",
"xdc" + File.separator + "wert.kpg"}
);
//creates blobs in second container
blobStore.createContainerInLocation(null, CONTAINER_NAME2);
Set<String> blobNamesCreatedInContainer2 = TestUtils.createBlobsInContainer(
CONTAINER_NAME2,
new String[] {
"asd" + File.separator + "bbb" + File.separator + "ccc" + File.separator + "ddd" + File.separator + "1234.jpg",
TestUtils.createRandomBlobKey(),
"rrr" + File.separator + "sss" + File.separator + "788.jpg",
"xdc" + File.separator + "wert.kpg" }
);
//test blobs in first container
checkForContainerContent(CONTAINER_NAME, blobNamesCreatedInContainer1);
//test blobs in second container
checkForContainerContent(CONTAINER_NAME2, blobNamesCreatedInContainer2);
}
/**
* TODO
* Should throws an exception?
*/
public void testClearContainer_NotExistingContainer(){
blobStore.clearContainer(CONTAINER_NAME);
}
/**
* Integration test, because clearContainer is not redefined in
* {@link FilesystemAsyncBlobStore} class
*/
public void testClearContainer_NoOptions() throws IOException {
final String CONTAINER_NAME2 = "containerToClear";
//create containers
blobStore.createContainerInLocation(null, CONTAINER_NAME);
blobStore.createContainerInLocation(null, CONTAINER_NAME2);
//creates blobs in first container
Set<String> blobNamesCreatedInContainer1 = TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[] {
"bbb" + File.separator + "ccc" + File.separator + "ddd" + File.separator + "1234.jpg",
TestUtils.createRandomBlobKey(),
"rrr" + File.separator + "sss" + File.separator + "788.jpg",
"xdc" + File.separator + "wert.kpg"}
);
//creates blobs in second container
blobStore.createContainerInLocation(null, CONTAINER_NAME2);
Set<String> blobNamesCreatedInContainer2 = TestUtils.createBlobsInContainer(
CONTAINER_NAME2,
new String[] {
"asd" + File.separator + "bbb" + File.separator + "ccc" + File.separator + "ddd" + File.separator + "1234.jpg",
TestUtils.createRandomBlobKey(),
"rrr" + File.separator + "sss" + File.separator + "788.jpg",
"xdc" + File.separator + "wert.kpg" }
);
//test blobs in containers
checkForContainerContent(CONTAINER_NAME, blobNamesCreatedInContainer1);
checkForContainerContent(CONTAINER_NAME2, blobNamesCreatedInContainer2);
//delete blobs in first container
blobStore.clearContainer(CONTAINER_NAME);
checkForContainerContent(CONTAINER_NAME, null);
checkForContainerContent(CONTAINER_NAME2, blobNamesCreatedInContainer2);
//delete blobs in second container
blobStore.clearContainer(CONTAINER_NAME2);
checkForContainerContent(CONTAINER_NAME2, null);
}
/**
* Integration test, because countBlobs is not redefined in
* {@link FilesystemAsyncBlobStore} class
*/
public void testCountBlobs_NotExistingContainer() {
try {
blobStore.countBlobs(PROVIDER);
fail("Magically the method was implemented... Wow!");
} catch (UnsupportedOperationException e) {
}
}
/**
* Integration test, because countBlobs is not redefined in
* {@link FilesystemAsyncBlobStore} class
*/
public void testCountBlobs_NoOptionsEmptyContainer() {
blobStore.createContainerInLocation(null, CONTAINER_NAME);
try {
blobStore.countBlobs(PROVIDER);
fail("Magically the method was implemented... Wow!");
} catch (UnsupportedOperationException e) {
}
}
/**
* Integration test, because countBlobs is not redefined in
* {@link FilesystemAsyncBlobStore} class
*/
public void testCountBlobs_NoOptions() {
blobStore.createContainerInLocation(null, CONTAINER_NAME);
try {
blobStore.countBlobs(PROVIDER);
fail("Magically the method was implemented... Wow!");
} catch (UnsupportedOperationException e) {
}
}
public void testRemoveBlob_SimpleBlobKey() throws IOException {
final String BLOB_KEY = TestUtils.createRandomBlobKey(null, ".txt");
boolean result;
blobStore.createContainerInLocation(null, CONTAINER_NAME);
//checks that blob doesn't exists
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY);
assertFalse(result, "Blob exists");
//create the blob
TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[] { BLOB_KEY }
);
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY);
assertTrue(result, "Blob exists");
//remove it
blobStore.removeBlob(CONTAINER_NAME, BLOB_KEY);
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY);
assertFalse(result, "Blob still exists");
TestUtils.fileExists(TARGET_CONTAINER_NAME + File.separator + BLOB_KEY, false);
}
public void testRemoveBlob_TwoSimpleBlobKeys() throws IOException {
final String BLOB_KEY1 = TestUtils.createRandomBlobKey(null, null);
final String BLOB_KEY2 = TestUtils.createRandomBlobKey(null, null);
boolean result;
//create the container and checks that blob doesn't exists
blobStore.createContainerInLocation(null, CONTAINER_NAME);
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY1);
assertFalse(result, "Blob1 exists");
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY2);
assertFalse(result, "Blob2 exists");
//create the blob
TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[] { BLOB_KEY1, BLOB_KEY2 }
);
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY1);
assertTrue(result, "Blob " + BLOB_KEY1 + " doesn't exist");
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY2);
assertTrue(result, "Blob " + BLOB_KEY2 + " doesn't exist");
//remove first blob
blobStore.removeBlob(CONTAINER_NAME, BLOB_KEY1);
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY1);
assertFalse(result, "Blob1 still exists");
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY2);
assertTrue(result, "Blob2 doesn't exist");
TestUtils.fileExists(TARGET_CONTAINER_NAME + File.separator + BLOB_KEY1, false);
TestUtils.fileExists(TARGET_CONTAINER_NAME + File.separator + BLOB_KEY2, true);
//remove second blob
blobStore.removeBlob(CONTAINER_NAME, BLOB_KEY2);
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY2);
assertFalse(result, "Blob2 still exists");
TestUtils.fileExists(TARGET_CONTAINER_NAME + File.separator + BLOB_KEY2, false);
}
/**
* Test of removeBlob method, with only one blob with a complex path as key
*/
public void testRemoveBlob_ComplexBlobKey() throws IOException {
final String BLOB_KEY = TestUtils.createRandomBlobKey("aa/bb/cc/dd/", null);
boolean result;
//checks that blob doesn't exists
blobStore.createContainerInLocation(null, CONTAINER_NAME);
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY);
assertFalse(result, "Blob exists");
TestUtils.fileExists(TARGET_CONTAINER_NAME + File.separator + BLOB_KEY, false);
//create the blob
TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[] { BLOB_KEY }
);
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY);
assertTrue(result, "Blob doesn't exist");
//remove it
blobStore.removeBlob(CONTAINER_NAME, BLOB_KEY);
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY);
assertFalse(result, "Blob still exists");
//file removed
TestUtils.fileExists(TARGET_CONTAINER_NAME + File.separator + BLOB_KEY, false);
//also the entire directory structure was removed
TestUtils.directoryExists(TARGET_CONTAINER_NAME + "/aa", false);
}
/**
* Test of removeBlob method, with two blobs with a complex path as key and
* when first blob is removed, not all of its key's path is removed, because
* it is shared with the second blob's key
*/
public void testRemoveBlob_TwoComplexBlobKeys() throws IOException {
final String BLOB_KEY1 = TestUtils.createRandomBlobKey("aa/bb/cc/dd/", null);
final String BLOB_KEY2 = TestUtils.createRandomBlobKey("aa/bb/ee/ff/", null);
boolean result;
blobStore.createContainerInLocation(null, CONTAINER_NAME);
//checks that blob doesn't exist
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY1);
assertFalse(result, "Blob1 exists");
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY2);
assertFalse(result, "Blob2 exists");
//create the blobs
TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[] { BLOB_KEY1, BLOB_KEY2 }
);
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY1);
assertTrue(result, "Blob " + BLOB_KEY1 + " doesn't exist");
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY2);
assertTrue(result, "Blob " + BLOB_KEY2 + " doesn't exist");
//remove first blob
blobStore.removeBlob(CONTAINER_NAME, BLOB_KEY1);
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY1);
assertFalse(result, "Blob still exists");
//first file deleted, not the second
TestUtils.fileExists(TARGET_CONTAINER_NAME + File.separator + BLOB_KEY1, false);
TestUtils.fileExists(TARGET_CONTAINER_NAME + File.separator + BLOB_KEY2, true);
//only partial directory structure was removed, because it shares a path
//with the second blob created
TestUtils.directoryExists(TARGET_CONTAINER_NAME + "/aa/bb/cc/dd", false);
TestUtils.directoryExists(TARGET_CONTAINER_NAME + "/aa/bb", true);
//remove second blob
blobStore.removeBlob(CONTAINER_NAME, BLOB_KEY2);
result = blobStore.blobExists(CONTAINER_NAME, BLOB_KEY2);
assertFalse(result, "Blob still exists");
TestUtils.fileExists(TARGET_CONTAINER_NAME + File.separator + BLOB_KEY2, false);
//now all the directory structure is empty
TestUtils.directoryExists(TARGET_CONTAINER_NAME + "/aa", false);
}
/**
* Test of containerExists method, of class FilesystemAsyncBlobStore.
*/
public void testContainerExists() throws IOException {
boolean result;
result = blobStore.containerExists(CONTAINER_NAME);
assertFalse(result, "Container exists");
//create container
TestUtils.createContainerAsDirectory(CONTAINER_NAME);
result = blobStore.containerExists(CONTAINER_NAME);
assertTrue(result, "Container doesn't exist");
}
/**
* Test of createContainerInLocation method, of class FilesystemAsyncBlobStore.
*/
public void testCreateContainerInLocation() throws IOException {
final String CONTAINER_NAME2 = "funambol-test-2";
final String TARGET_CONTAINER_NAME2 = TARGET_BASE_DIR + CONTAINER_NAME2;
boolean result;
result = blobStore.containerExists(CONTAINER_NAME);
assertFalse(result, "Container exists");
result = blobStore.createContainerInLocation(null, CONTAINER_NAME);
assertTrue(result, "Container not created");
result = blobStore.containerExists(CONTAINER_NAME);
assertTrue(result, "Container doesn't exist");
TestUtils.directoryExists(TARGET_CONTAINER_NAME, true);
result = blobStore.containerExists(CONTAINER_NAME2);
assertFalse(result, "Container exists");
result = blobStore.createContainerInLocation(null, CONTAINER_NAME2);
assertTrue(result, "Container not created");
result = blobStore.containerExists(CONTAINER_NAME2);
assertTrue(result, "Container doesn't exist");
TestUtils.directoryExists(TARGET_BASE_DIR + CONTAINER_NAME2, true);
//clean the environment
FileUtils.forceDelete(new File(TARGET_CONTAINER_NAME2));
}
/**
* Test of putBlob method, of class FilesystemAsyncBlobStore.
* with a simple filename - no path in the filename, eg
* filename.jpg
*/
public void testPutBlobSimpleName() {
blobStore.createContainerInLocation(null, CONTAINER_NAME);
putBlobAndCheckIt(TestUtils.createRandomBlobKey("putBlob-", ".jpg"));
putBlobAndCheckIt(TestUtils.createRandomBlobKey("putBlob-", ".jpg"));
}
/**
* Test of putBlob method with a complex key, with path in the filename, eg
* picture/filename.jpg
*/
public void testPutBlobComplexName1() {
blobStore.createContainerInLocation(null, CONTAINER_NAME);
putBlobAndCheckIt(TestUtils.createRandomBlobKey("picture/putBlob-", ".jpg"));
putBlobAndCheckIt(TestUtils.createRandomBlobKey("video/putBlob-", ".jpg"));
putBlobAndCheckIt(TestUtils.createRandomBlobKey("putBlob-", ".jpg"));
putBlobAndCheckIt(TestUtils.createRandomBlobKey("video/putBlob-", ".jpg"));
}
/**
* Test of putBlob method with a complex key, with path in the filename, eg
* picture/filename.jpg
*/
public void testPutBlobComplexName2() {
blobStore.createContainerInLocation(null, CONTAINER_NAME);
putBlobAndCheckIt(TestUtils.createRandomBlobKey("aa/bb/cc/dd/ee/putBlob-", ".jpg"));
putBlobAndCheckIt(TestUtils.createRandomBlobKey("aa/bb/cc/dd/ee/putBlob-", ".jpg"));
putBlobAndCheckIt(TestUtils.createRandomBlobKey("putBlob-", ".jpg"));
}
/**
* Test of blobExists method, of class FilesystemAsyncBlobStore.
*/
public void testBlobExists() throws IOException {
boolean result;
String blobKey;
//when location doesn't exists
blobKey = TestUtils.createRandomBlobKey();
result = blobStore.blobExists(CONTAINER_NAME, blobKey);
assertFalse(result, "Blob exists");
//when location exists
blobStore.createContainerInLocation(null, CONTAINER_NAME);
result = blobStore.blobExists(CONTAINER_NAME, blobKey);
assertFalse(result, "Blob exists");
//create blob
TestUtils.createBlobAsFile(CONTAINER_NAME, blobKey, TestUtils.getImageForBlobPayload());
result = blobStore.blobExists(CONTAINER_NAME, blobKey);
assertTrue(result, "Blob doesn't exist");
//complex path test
blobKey = TestUtils.createRandomBlobKey("ss/asdas/", "");
result = blobStore.blobExists(CONTAINER_NAME, blobKey);
assertFalse(result, "Blob exists");
TestUtils.createBlobAsFile(CONTAINER_NAME, blobKey, TestUtils.getImageForBlobPayload());
result = blobStore.blobExists(CONTAINER_NAME, blobKey);
assertTrue(result, "Blob doesn't exist");
}
public void testGetBlob_NotExistingContainer() {
try {
blobStore.getBlob(CONTAINER_NAME, TestUtils.createRandomBlobKey(), null);
fail("Retrieve must fail, container does not exist.");
} catch(ContainerNotFoundException e) {
//correct if arrive here
}
}
/**
* Test of getBlob method, of class FilesystemAsyncBlobStore.
*/
public void testGetBlob() throws IOException {
String blobKey = TestUtils.createRandomBlobKey();
GetOptions options = null;
Blob resultBlob;
blobStore.createContainerInLocation(null, CONTAINER_NAME);
resultBlob = blobStore.getBlob(CONTAINER_NAME, blobKey, options);
assertNull(resultBlob, "Blob exists");
//create blob
TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[]{blobKey});
resultBlob = blobStore.getBlob(CONTAINER_NAME, blobKey, options);
assertNotNull(resultBlob, "Blob exists");
//checks file content
InputStream expectedFile = new FileInputStream(TARGET_CONTAINER_NAME + File.separator + blobKey);
InputStream currentFile = resultBlob.getPayload().getInput();
assertTrue(TestUtils.isSame(expectedFile, currentFile), "Blob payload differs from file content");
//metadata are verified in the test for blobMetadata, so no need to
//perform a complete test here
assertNotNull(resultBlob.getMetadata(), "Metadata null");
MutableBlobMetadata metadata = resultBlob.getMetadata();
assertEquals(blobKey, metadata.getName(), "Wrong blob metadata");
}
public void testBlobMetadata_withDefaultMetadata() throws IOException {
String BLOB_KEY = TestUtils.createRandomBlobKey(null, null);
//create the blob
TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[]{ BLOB_KEY }
);
BlobMetadata metadata = blobStore.blobMetadata(CONTAINER_NAME, BLOB_KEY);
assertNotNull(metadata, "Metadata null");
assertEquals(metadata.getName(), BLOB_KEY, "Wrong blob name");
assertEquals(metadata.getType(), StorageType.BLOB, "Wrong blob type");
assertEquals(metadata.getContentType(), "", "Wrong blob content-type");
assertEquals(metadata.getContentMD5(), null, "Wrong blob MD5");
assertEquals(metadata.getLocation(), null, "Wrong blob location");
assertEquals(metadata.getProviderId(), null, "Wrong blob provider id");
assertEquals(metadata.getUri(), null, "Wrong blob URI");
assertNotNull(metadata.getUserMetadata(), "No blob UserMetadata");
assertEquals(metadata.getUserMetadata().size(), 0, "Wrong blob UserMetadata");
//metadata.getLastModified()
File file = new File(TARGET_CONTAINER_NAME + File.separator + BLOB_KEY);
assertEquals(metadata.getSize(), new Long(file.length()), "Wrong blob size");
//don't know how to calculate ETAG
//assertEquals(metadata.getETag(), "105cf4e6c052d65352dabd20028ff102", "Wrong blob ETag");
}
public void testDeleteContainer_NotExistingContainer(){
try {
blobStore.deleteContainer(CONTAINER_NAME);
fail("No error when container doesn't exist");
} catch (Exception e) {
}
}
public void testDeleteContainer_EmptyContanier(){
boolean result;
blobStore.createContainerInLocation(null, CONTAINER_NAME);
result = blobStore.containerExists(CONTAINER_NAME);
assertTrue(result, "Container doesn't exists");
TestUtils.directoryExists(TARGET_CONTAINER_NAME, true);
//delete container
blobStore.deleteContainer(CONTAINER_NAME);
result = blobStore.containerExists(CONTAINER_NAME);
assertFalse(result, "Container still exists");
TestUtils.directoryExists(TARGET_CONTAINER_NAME, false);
}
public void testDeleteContainer() throws IOException{
boolean result;
String CONTAINER_NAME2 = "container-to-delete";
String TARGET_CONTAINER_NAME2 = TARGET_BASE_DIR + CONTAINER_NAME2;
blobStore.createContainerInLocation(null, CONTAINER_NAME);
blobStore.createContainerInLocation(null, CONTAINER_NAME2);
result = blobStore.containerExists(CONTAINER_NAME);
assertTrue(result, "Container [" + CONTAINER_NAME + "] doesn't exists");
TestUtils.directoryExists(TARGET_CONTAINER_NAME, true);
result = blobStore.containerExists(CONTAINER_NAME2);
assertTrue(result, "Container [" + CONTAINER_NAME2 + "] doesn't exists");
TestUtils.directoryExists(TARGET_CONTAINER_NAME2, true);
//create blobs inside container
TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[]{
TestUtils.createRandomBlobKey("testutils-", null),
TestUtils.createRandomBlobKey("testutils-", null),
TestUtils.createRandomBlobKey("ab123s" + File.separator + "testutils-", null),
});
TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[]{
TestUtils.createRandomBlobKey("testutils-", null),
TestUtils.createRandomBlobKey("testutils-", null),
TestUtils.createRandomBlobKey("asda123s" + File.separator + "testutils-", null),
TestUtils.createRandomBlobKey("123-_3s" + File.separator + "testutils-", null),
});
//delete first container
blobStore.deleteContainer(CONTAINER_NAME);
result = blobStore.containerExists(CONTAINER_NAME);
assertFalse(result, "Container [" + CONTAINER_NAME + "] still exists");
TestUtils.directoryExists(TARGET_CONTAINER_NAME, false);
result = blobStore.containerExists(CONTAINER_NAME2);
assertTrue(result, "Container [" + CONTAINER_NAME2 + "] still exists");
TestUtils.directoryExists(TARGET_CONTAINER_NAME2, true);
//delete second container
blobStore.deleteContainer(CONTAINER_NAME2);
result = blobStore.containerExists(CONTAINER_NAME2);
assertFalse(result, "Container [" + CONTAINER_NAME2 + "] still exists");
TestUtils.directoryExists(TARGET_CONTAINER_NAME2, false);
}
//---------------------------------------------------------- Private Methods
/**
* Creates a {@link Blob} object filled with data from a file
* @param keyName
* @param fileContent
* @return
*/
private Blob createBlob(String keyName, File filePayload) {
Blob blob = blobStore.newBlob(keyName);
blob.setPayload(filePayload);
return blob;
}
/**
* Tests if container contains only the expected blobs
* @param containerName
* @param expectedBlobKeys
*/
private void checkForContainerContent(final String containerName, Set<String> expectedBlobKeys) {
ListContainerOptions options = ListContainerOptions.Builder.recursive();
PageSet<StorageMetadata> blobsRetrieved = (PageSet<StorageMetadata>) blobStore.list(containerName, options);
//nothing expected
if (null == expectedBlobKeys) {
assertTrue(blobsRetrieved.isEmpty(), "Wrong blob number retrieved in the containter [" + containerName + "]");
return;
}
//copies values
Set<String> expectedBlobKeysCopy = new HashSet<String>();
for (String value:expectedBlobKeys){
expectedBlobKeysCopy.add(value);
}
assertEquals(blobsRetrieved.size(), expectedBlobKeysCopy.size(), "Wrong blob number retrieved in the containter [" + containerName + "]");
for (StorageMetadata data : blobsRetrieved) {
String blobName = data.getName();
if (!expectedBlobKeysCopy.remove(blobName)) {
fail("List for container [" + containerName + "] contains unexpected value [" + blobName + "]");
}
}
assertTrue(expectedBlobKeysCopy.isEmpty(), "List operation for container [" + containerName + "] doesn't return all values.");
}
/**
* Create a blob with putBlob method
*/
private void putBlobAndCheckIt(String blobKey) {
Blob blob;
TestUtils.fileExists(TARGET_CONTAINER_NAME + File.separator + blobKey, false);
//create the blob
blob = createBlob(blobKey, TestUtils.getImageForBlobPayload());
String eTag = blobStore.putBlob(CONTAINER_NAME, blob);
assertNotNull(eTag, "putBlob result null");
assertNotSame(eTag, "", "putBlob result empty");
//checks if the blob exists
TestUtils.fileExists(TARGET_CONTAINER_NAME + File.separator + blobKey, true);
}
}

View File

@ -0,0 +1,512 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.filesystem.utils;
import java.util.List;
import java.util.Set;
import org.jclouds.blobstore.domain.internal.MutableBlobMetadataImpl;
import org.jclouds.blobstore.domain.internal.BlobImpl;
import org.jclouds.blobstore.domain.MutableBlobMetadata;
import org.jclouds.blobstore.domain.Blob;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
import org.apache.commons.io.FileUtils;
import org.jclouds.blobstore.options.ListContainerOptions;
import org.jclouds.io.payloads.FilePayload;
import org.testng.annotations.*;
import static org.testng.Assert.*;
/**
* Test class for {@link FilesystemStorageStrategyImpl } class
*
* @author Alfredo "Rainbowbreeze" Morresi
*/
@Test(groups = "unit", testName = "filesystem.FilesystemBlobUtilsTest", sequential = true)
public class FilesystemStorageStrategyImplTest {
private static final String CONTAINER_NAME = "funambol-test";
private static final String TARGET_CONTAINER_NAME = TestUtils.TARGET_BASE_DIR + CONTAINER_NAME;
private static final String LOGGING_CONFIG_KEY = "java.util.logging.config.file";
private static final String LOGGING_CONFIG_VALUE = "src/main/resources/logging.properties";
static {
System.setProperty(LOGGING_CONFIG_KEY,
LOGGING_CONFIG_VALUE);
}
private FilesystemStorageStrategy storageStrategy;
@BeforeMethod
protected void setUp() throws Exception {
storageStrategy = new FilesystemStorageStrategyImpl(
new Blob.Factory() {
@Override
public Blob create(MutableBlobMetadata metadata) {
return new BlobImpl(metadata != null ? metadata : new MutableBlobMetadataImpl());
}
},
TestUtils.TARGET_BASE_DIR);
TestUtils.cleanDirectoryContent(TestUtils.TARGET_BASE_DIR);
}
@AfterMethod
protected void tearDown() throws IOException {
TestUtils.cleanDirectoryContent(TestUtils.TARGET_BASE_DIR);
}
public void testCreateDirectory() {
storageStrategy.createDirectory(CONTAINER_NAME, null);
TestUtils.directoryExists(TARGET_CONTAINER_NAME, true);
storageStrategy.createDirectory(CONTAINER_NAME, "subdir");
TestUtils.directoryExists(TARGET_CONTAINER_NAME + File.separator + "subdir", true);
storageStrategy.createDirectory(CONTAINER_NAME, "subdir1" + File.separator);
TestUtils.directoryExists(TARGET_CONTAINER_NAME + File.separator + "subdir1", true);
storageStrategy.createDirectory(CONTAINER_NAME, File.separator + "subdir2");
TestUtils.directoryExists(TARGET_CONTAINER_NAME + File.separator + "subdir2", true);
storageStrategy.createDirectory(CONTAINER_NAME, "subdir3" + File.separator + "subdir4");
TestUtils.directoryExists(TARGET_CONTAINER_NAME + File.separator + "subdir2", true);
}
public void testCreateDirectory_DirectoryAlreadyExists() {
storageStrategy.createDirectory(CONTAINER_NAME, null);
storageStrategy.createDirectory(CONTAINER_NAME, null);
}
public void testCreateDirectory_WrongDirectoryName() {
try {
storageStrategy.createDirectory(CONTAINER_NAME, "$%&!'`\\");
fail("No exception throwed");
} catch(Exception e) {
}
}
public void testCreateContainer() {
boolean result;
TestUtils.directoryExists(TARGET_CONTAINER_NAME, false);
result = storageStrategy.createContainer(CONTAINER_NAME);
assertTrue(result, "Container not created");
TestUtils.directoryExists(TARGET_CONTAINER_NAME, true);
}
public void testCreateContainer_ContainerAlreadyExists() {
boolean result;
TestUtils.directoryExists(TARGET_CONTAINER_NAME, false);
result = storageStrategy.createContainer(CONTAINER_NAME);
assertTrue(result, "Container not created");
result = storageStrategy.createContainer(CONTAINER_NAME);
assertFalse(result, "Container not created");
}
public void testDeleteDirectory() throws IOException {
TestUtils.createContainerAsDirectory(CONTAINER_NAME);
TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[]{
TestUtils.createRandomBlobKey("lev1/lev2/lev3/", ".txt"),
TestUtils.createRandomBlobKey("lev1/lev2/lev4/", ".jpg")
}
);
//delete directory in different ways
storageStrategy.deleteDirectory(CONTAINER_NAME, "lev1/lev2/lev4");
TestUtils.directoryExists(TARGET_CONTAINER_NAME + File.separator + "lev1/lev2/lev4", false);
TestUtils.directoryExists(TARGET_CONTAINER_NAME + File.separator + "lev1/lev2", true);
storageStrategy.deleteDirectory(CONTAINER_NAME, "lev1/lev2/lev3/");
TestUtils.directoryExists(TARGET_CONTAINER_NAME + File.separator + "lev1/lev2/lev3", false);
TestUtils.directoryExists(TARGET_CONTAINER_NAME + File.separator + "lev1/lev2", true);
storageStrategy.deleteDirectory(CONTAINER_NAME, "/lev1");
TestUtils.directoryExists(TARGET_CONTAINER_NAME + File.separator + "lev1", false);
TestUtils.directoryExists(TARGET_CONTAINER_NAME, true);
//delete the directory and all the files inside
TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[]{
TestUtils.createRandomBlobKey("lev1/lev2/lev3/", ".txt"),
TestUtils.createRandomBlobKey("lev1/lev2/lev4/", ".jpg")
}
);
storageStrategy.deleteDirectory(CONTAINER_NAME, null);
TestUtils.directoryExists(TARGET_CONTAINER_NAME, false);
}
public void testDeleteDirectory_ErrorWhenNotExists(){
try {
storageStrategy.deleteDirectory(CONTAINER_NAME, null);
fail("No exception throwed");
} catch(Exception e) {
}
}
public void testDirectoryExists() throws IOException {
final String SUBDIRECTORY_NAME = "ad" + File.separator + "sda" + File.separator + "asd";
boolean result;
result = storageStrategy.directoryExists(CONTAINER_NAME, null);
assertFalse(result, "Directory exist");
//create the container
TestUtils.createContainerAsDirectory(CONTAINER_NAME);
//check if exists
result = storageStrategy.directoryExists(CONTAINER_NAME, null);
assertTrue(result, "Directory doesn't exist");
result = storageStrategy.directoryExists(CONTAINER_NAME + File.separator, null);
assertTrue(result, "Directory doesn't exist");
result = storageStrategy.directoryExists(CONTAINER_NAME, SUBDIRECTORY_NAME);
assertFalse(result, "Directory exist");
//create subdirs inside the container
TestUtils.createContainerAsDirectory(CONTAINER_NAME + File.separator + SUBDIRECTORY_NAME);
//check if exists
result = storageStrategy.directoryExists(CONTAINER_NAME, SUBDIRECTORY_NAME);
assertTrue(result, "Directory doesn't exist");
result = storageStrategy.directoryExists(CONTAINER_NAME, File.separator + SUBDIRECTORY_NAME);
assertTrue(result, "Directory doesn't exist");
result = storageStrategy.directoryExists(CONTAINER_NAME, SUBDIRECTORY_NAME + File.separator);
assertTrue(result, "Directory doesn't exist");
result = storageStrategy.directoryExists(CONTAINER_NAME + File.separator, File.separator + SUBDIRECTORY_NAME);
assertTrue(result, "Directory doesn't exist");
}
public void testClearContainer() throws IOException{
storageStrategy.createContainer(CONTAINER_NAME);
Set<String> blobs = TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[]{
TestUtils.createRandomBlobKey("clean_container-", ".jpg"),
TestUtils.createRandomBlobKey("bf/sd/as/clean_container-", ".jpg")}
);
//test if file exits
for(String blob:blobs) {
TestUtils.fileExists(TARGET_CONTAINER_NAME + File.separator + blob, true);
}
//clear the container
storageStrategy.clearContainer(CONTAINER_NAME);
//test if container still exits
TestUtils.directoryExists(TARGET_CONTAINER_NAME, true);
//test if file was cleared
for(String blob:blobs) {
TestUtils.fileExists(TARGET_CONTAINER_NAME + File.separator + blob, false);
}
}
public void testClearContainer_NotExistingContainer() throws IOException{
//test if container still exits
TestUtils.directoryExists(TARGET_CONTAINER_NAME, false);
//clear the container
storageStrategy.clearContainer(CONTAINER_NAME);
//test if container still exits
TestUtils.directoryExists(TARGET_CONTAINER_NAME, false);
}
public void testClearContainerAndThenDeleteContainer() throws IOException{
storageStrategy.createContainer(CONTAINER_NAME);
Set<String> blobs = TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[]{
TestUtils.createRandomBlobKey("clean_container-", ".jpg"),
TestUtils.createRandomBlobKey("bf/sd/as/clean_container-", ".jpg")}
);
//test if file exits
for(String blob:blobs) {
TestUtils.fileExists(TARGET_CONTAINER_NAME + File.separator + blob, true);
}
//clear the container
storageStrategy.clearContainer(CONTAINER_NAME);
//test if container still exits
TestUtils.directoryExists(TARGET_CONTAINER_NAME, true);
//test if file was cleared
for(String blob:blobs) {
TestUtils.fileExists(TARGET_CONTAINER_NAME + File.separator + blob, false);
}
//delete the container
storageStrategy.deleteContainer(CONTAINER_NAME);
//test if container still exits
TestUtils.directoryExists(TARGET_CONTAINER_NAME, false);
assertFalse(storageStrategy.containerExists(CONTAINER_NAME), "Container still exists");
}
public void testDeleteContainer() throws IOException {
final String BLOB_KEY1 = "blobName.jpg";
final String BLOB_KEY2 = "aa/bb/cc/dd/ee/ff/23/blobName.jpg";
boolean result;
result = storageStrategy.createContainer(CONTAINER_NAME);
//put data inside the container
TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[] {BLOB_KEY1, BLOB_KEY2}
);
storageStrategy.deleteContainer(CONTAINER_NAME);
assertTrue(result, "Cannot delete container");
TestUtils.directoryExists(CONTAINER_NAME, false);
}
public void testDeleteContainer_EmptyContainer() {
boolean result;
result = storageStrategy.createContainer(CONTAINER_NAME);
assertTrue(result, "Cannot create container");
storageStrategy.deleteContainer(CONTAINER_NAME);
TestUtils.directoryExists(CONTAINER_NAME, false);
}
public void testDeleteContainer_ErrorWhenNotExists() {
try {
storageStrategy.deleteContainer(CONTAINER_NAME);
fail("Exception not throwed");
} catch (Exception e) {
}
}
public void testGetAllContainerNames() {
Iterable<String> resultList;
//no container
resultList = storageStrategy.getAllContainerNames();
assertNotNull(resultList, "Result is null");
assertFalse(resultList.iterator().hasNext(), "Containers detected");
//create containers
storageStrategy.createContainer(CONTAINER_NAME + "1");
storageStrategy.createContainer(CONTAINER_NAME + "2");
storageStrategy.createContainer(CONTAINER_NAME + "3");
List<String> containers = new ArrayList<String>();
resultList = storageStrategy.getAllContainerNames();
Iterator<String> containersIterator = resultList.iterator();
while(containersIterator.hasNext()){
containers.add(containersIterator.next());
}
assertEquals(containers.size(), 3, "Different containers number");
assertTrue(containers.contains(CONTAINER_NAME + "1"), "Containers doesn't exist");
assertTrue(containers.contains(CONTAINER_NAME + "2"), "Containers doesn't exist");
assertTrue(containers.contains(CONTAINER_NAME + "3"), "Containers doesn't exist");
}
public void testContainerExists(){
boolean result;
TestUtils.directoryExists(TARGET_CONTAINER_NAME, false);
result = storageStrategy.containerExists(CONTAINER_NAME);
assertFalse(result, "Container exists");
storageStrategy.createContainer(CONTAINER_NAME);
result = storageStrategy.containerExists(CONTAINER_NAME);
assertTrue(result, "Container exists");
}
public void testNewBlob() {
String blobKey;
Blob newBlob;
blobKey = TestUtils.createRandomBlobKey("blobtest-", ".txt");
newBlob = storageStrategy.newBlob(blobKey);
assertNotNull(newBlob, "Created blob was null");
assertNotNull(newBlob.getMetadata(), "Created blob metadata were null");
assertEquals(newBlob.getMetadata().getName(), blobKey, "Created blob name is different");
blobKey = TestUtils.createRandomBlobKey("blobtest-", "");
newBlob = storageStrategy.newBlob(blobKey);
assertEquals(newBlob.getMetadata().getName(), blobKey, "Created blob name is different");
blobKey = TestUtils.createRandomBlobKey("asd/asd/asdasd/afadsf-", "");
newBlob = storageStrategy.newBlob(blobKey);
assertEquals(newBlob.getMetadata().getName(), blobKey, "Created blob name is different");
}
public void testWritePayloadOnFile() throws IOException {
String blobKey;
File sourceFile;
FilePayload filePayload;
blobKey = TestUtils.createRandomBlobKey("writePayload-", ".img");
sourceFile = TestUtils.getImageForBlobPayload();
filePayload = new FilePayload(sourceFile);
//write files
storageStrategy.writePayloadOnFile(CONTAINER_NAME, blobKey, filePayload);
//verify that the files is equal
String blobFullPath = TARGET_CONTAINER_NAME + File.separator + blobKey;
InputStream expectedInput = new FileInputStream(sourceFile);
InputStream currentInput = new FileInputStream(blobFullPath);
assertTrue(TestUtils.isSame(expectedInput, currentInput), "Files aren't equals");
}
public void testWritePayloadOnFile_SourceFileDoesntExist() {
File sourceFile = new File("asdfkjsadkfjasdlfasdflk.asdfasdfas");
try {
FilePayload filePayload = new FilePayload(sourceFile);
fail("Exception not throwed");
} catch (Exception ex) {
}
}
public void testGetFileForBlobKey() {
String blobKey;
File fileForPayload;
String fullPath = (new File(TARGET_CONTAINER_NAME).getAbsolutePath()) + File.separator;
blobKey = TestUtils.createRandomBlobKey("getFileForBlobKey-", ".img");
fileForPayload = storageStrategy.getFileForBlobKey(CONTAINER_NAME, blobKey);
assertNotNull(fileForPayload, "Result File object is null");
assertEquals(fileForPayload.getAbsolutePath(), fullPath + blobKey, "Wrong file path");
blobKey = TestUtils.createRandomBlobKey("asd/vmad/andsnf/getFileForBlobKey-", ".img");
fileForPayload = storageStrategy.getFileForBlobKey(CONTAINER_NAME, blobKey);
assertEquals(fileForPayload.getAbsolutePath(), fullPath + blobKey, "Wrong file path");
}
public void testBlobExists() throws IOException {
String[] sourceBlobKeys = new String[]{
TestUtils.createRandomBlobKey("blobExists-", ".jpg"),
TestUtils.createRandomBlobKey("blobExists-", ".jpg"),
TestUtils.createRandomBlobKey("afasd" + File.separator + "asdma" + File.separator + "blobExists-", ".jpg")
};
for(String blobKey:sourceBlobKeys) {
assertFalse(storageStrategy.blobExists(CONTAINER_NAME, blobKey), "Blob " + blobKey + " exists");
}
TestUtils.createBlobsInContainer(CONTAINER_NAME, sourceBlobKeys);
for(String blobKey:sourceBlobKeys) {
assertTrue(storageStrategy.blobExists(CONTAINER_NAME, blobKey), "Blob " + blobKey + " doesn't exist");
}
}
public void testRemoveBlob() throws IOException {
storageStrategy.createContainer(CONTAINER_NAME);
Set<String> blobKeys = TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[]{
TestUtils.createRandomBlobKey("removeBlob-", ".jpg"),
TestUtils.createRandomBlobKey("removeBlob-", ".jpg"),
TestUtils.createRandomBlobKey("346" + File.separator + "g3sx2" + File.separator + "removeBlob-", ".jpg"),
TestUtils.createRandomBlobKey("346" + File.separator + "g3sx2" + File.separator + "removeBlob-", ".jpg")
});
Set<String> remainingBlobKeys = new HashSet<String>();
for(String key:blobKeys) {
remainingBlobKeys.add(key);
}
for (String blobKeyToRemove:blobKeys) {
storageStrategy.removeBlob(CONTAINER_NAME, blobKeyToRemove);
//checks if the blob was removed
TestUtils.fileExists(blobKeyToRemove, false);
remainingBlobKeys.remove(blobKeyToRemove);
//checks if all other blobs still exists
for(String remainingBlobKey:remainingBlobKeys) {
TestUtils.fileExists(TARGET_CONTAINER_NAME + File.separator + remainingBlobKey, true);
}
}
}
public void testRemoveBlob_ContainerNotExists() {
storageStrategy.removeBlob("asdasdasd", "sdfsdfsdfasd");
}
public void testRemoveBlob_BlobNotExists() {
storageStrategy.createContainer(CONTAINER_NAME);
storageStrategy.removeBlob(CONTAINER_NAME, "sdfsdfsdfasd");
}
public void testGetBlobKeysInsideContainer() throws IOException {
Iterable<String> resultList;
//no container
resultList = storageStrategy.getBlobKeysInsideContainer(CONTAINER_NAME);
assertNotNull(resultList, "Result is null");
assertFalse(resultList.iterator().hasNext(), "Blobs detected");
//create blobs
storageStrategy.createContainer(CONTAINER_NAME);
Set<String> createBlobKeys = TestUtils.createBlobsInContainer(
CONTAINER_NAME,
new String[]{
TestUtils.createRandomBlobKey("GetBlobKeys-", ".jpg"),
TestUtils.createRandomBlobKey("GetBlobKeys-", ".jpg"),
TestUtils.createRandomBlobKey("563" + File.separator + "g3sx2" + File.separator + "removeBlob-", ".jpg"),
TestUtils.createRandomBlobKey("563" + File.separator + "g3sx2" + File.separator + "removeBlob-", ".jpg")
});
storageStrategy.getBlobKeysInsideContainer(CONTAINER_NAME);
List<String> retrievedBlobKeys = new ArrayList<String>();
resultList = storageStrategy.getBlobKeysInsideContainer(CONTAINER_NAME);
Iterator<String> containersIterator = resultList.iterator();
while(containersIterator.hasNext()){
retrievedBlobKeys.add(containersIterator.next());
}
assertEquals(retrievedBlobKeys.size(), createBlobKeys.size(), "Different blobs number");
for(String createdBlobKey:createBlobKeys) {
assertTrue(retrievedBlobKeys.contains(createdBlobKey), "Blob " + createdBlobKey + " not found");
}
}
public void testCountsBlob() {
try {
storageStrategy.countBlobs(CONTAINER_NAME, ListContainerOptions.NONE);
fail("Magically the method was implemented... Wow!");
} catch (UnsupportedOperationException e) {
}
}
//---------------------------------------------------------- Private methods
}

View File

@ -0,0 +1,246 @@
/**
*
* Copyright (C) 2009 Cloud Conscious, LLC. <info@cloudconscious.com>
*
* ====================================================================
* Licensed 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.filesystem.utils;
import java.util.Arrays;
import java.io.InputStream;
import java.util.HashSet;
import java.util.Set;
import java.util.UUID;
import java.io.File;
import java.io.IOException;
import org.apache.commons.io.FileUtils;
import static org.testng.Assert.*;
/**
* Utility class for test
*
* @author Alfredo "Rainbowbreeze" Morresi
*/
public class TestUtils {
private static final String TARGET_RESOURCE_DIR = "." + File.separator + "src" + File.separator + "test" + File.separator + "resources" + File.separator;
/** All the files available for the tests */
private static String[] imageResource = new String[]{
TARGET_RESOURCE_DIR + "image1.jpg",
TARGET_RESOURCE_DIR + "image2.jpg",
TARGET_RESOURCE_DIR + "image3.jpg",
TARGET_RESOURCE_DIR + "image4.jpg"
};
private static int imageResourceIndex = 0;
public static final String TARGET_BASE_DIR = "." + File.separator + "target" + File.separator + "basedir" + File.separator;
/**
* Generate a random blob key simple name (with no path in the key)
* @return
*/
public static String createRandomBlobKey() {
return createRandomBlobKey("", "");
}
/**
* Generate a random blob key simple name (with no path in the key)
* @param prefix a prefix to the id, default "testkey-"
* @param extension a extension for the blob key, default ".jpg"
* @return
*/
public static String createRandomBlobKey(String prefix, String extension) {
String okPrefix = (null != prefix && !"".equals(prefix)) ? prefix : "testkey-";
String okExtension = (null != extension && !"".equals(extension)) ? extension : ".jpg";
return okPrefix + UUID.randomUUID().toString() + okExtension;
}
/**
* Creates blobs in container
* @param containerName
* @param blobNames
* @return a Set with all blobs created
* @throws IOException
*/
public static Set<String> createBlobsInContainer(String containerName, String[] blobNames) throws IOException {
boolean result;
Set<String> blobNamesCreatedInContainer = new HashSet<String>();
for (String blobName : blobNames) {
createBlobAsFile(containerName, blobName, getImageForBlobPayload());
blobNamesCreatedInContainer.add(blobName);
}
return blobNamesCreatedInContainer;
}
/**
* Creates a container object creating a directory
* @param containerName
* @throws IOException
*/
public static void createContainerAsDirectory(String containerName) throws IOException {
FileUtils.forceMkdir(new File(TARGET_BASE_DIR + containerName));
}
/**
*
* @param directoryFullPath the directory path
* @return
*/
public static boolean directoryExists(String directoryFullPath) {
File file = new File(directoryFullPath);
boolean exists = file.exists() || file.isDirectory();
return exists;
}
/**
*
* @param directoryFullPath
* @param checkResult
* @param expectedResult
* @return
*/
public static boolean directoryExists(String directoryFullPath, boolean expectedResult) {
boolean exists = directoryExists(directoryFullPath);
if (expectedResult) {
assertTrue(exists, "Directory " + directoryFullPath + " doens't exists");
} else {
assertFalse(exists, "Directory " + directoryFullPath + " still exists");
}
return exists;
}
public static boolean fileExists(String fileFullName) {
File file = new File(fileFullName);
boolean exists = file.exists() || file.isFile();
return exists;
}
/**
*
* @param fileFullName
* @param checkResult
* @param expectedResult
* @return
*/
public static boolean fileExists(String fileFullName, boolean expectedResult) {
boolean exists = fileExists(fileFullName);
if (expectedResult) {
assertTrue(exists, "File " + fileFullName + " doens't exists");
} else {
assertFalse(exists, "File " + fileFullName + " still exists");
}
return exists;
}
/**
* Empty a directory
*
* @param directoryName
* @throws IOException
*/
public static void cleanDirectoryContent(String directoryName) throws IOException {
File parentDirectory = new File(directoryName);
File[] children = parentDirectory.listFiles();
if (null != children) {
for(File child:children) {
FileUtils.forceDelete(child);
}
}
}
/**
* Create a blob object from a given file
* @param source
* @param containerName
* @param blobKey
* @throws IOException
*/
public static void createBlobAsFile(String containerName, String blobKey, File source) throws IOException {
String filePath;
if (blobKey.startsWith("\\"))
filePath = containerName + blobKey;
else
filePath = containerName + File.separator + blobKey;
FileUtils.copyFile(source, new File(TARGET_BASE_DIR + filePath));
}
/**
* Returns a pointer to an image, cycling between the ones that are available
* @return
*/
public static File getImageForBlobPayload() {
String fileName = imageResource[imageResourceIndex++];
if (imageResourceIndex >= imageResource.length) imageResourceIndex = 0;
return new File(fileName);
}
/**
* Compare two input stream
*
* @param input1 the first stream
* @param input2 the second stream
* @return true if the streams contain the same content, or false otherwise
* @throws IOException
* @throws IllegalArgumentException if the stream is null
*/
public static boolean isSame(InputStream input1, InputStream input2 ) throws IOException {
boolean error = false;
try {
byte[] buffer1 = new byte[1024];
byte[] buffer2 = new byte[1024];
try {
int numRead1 = 0;
int numRead2 = 0;
while (true) {
numRead1 = input1.read(buffer1);
numRead2 = input2.read(buffer2);
if (numRead1 > -1) {
if (numRead2 != numRead1) return false;
// Otherwise same number of bytes read
if (!Arrays.equals(buffer1, buffer2)) return false;
// Otherwise same bytes read, so continue ...
} else {
// Nothing more in stream 1 ...
return numRead2 < 0;
}
}
} finally {
input1.close();
}
} catch (IOException e) {
error = true; // this error should be thrown, even if there is an error closing stream 2
throw e;
} catch (RuntimeException e) {
error = true; // this error should be thrown, even if there is an error closing stream 2
throw e;
} finally {
try {
input2.close();
} catch (IOException e) {
if (!error) throw e;
}
}
}
}

Binary file not shown.

After

Width:  |  Height:  |  Size: 339 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 844 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 641 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 2.8 MiB