HADOOP-10809. hadoop-azure: page blob support. Contributed by Dexter Bradshaw, Mostafa Elhemali, Eric Hanson, and Mike Liddell.
(cherry picked from commit 2217e2f8ff
)
This commit is contained in:
parent
9a2e4f4f98
commit
5a737026cc
|
@ -79,6 +79,9 @@ Release 2.7.0 - UNRELEASED
|
|||
HADOOP-11238. Update the NameNode's Group Cache in the background when
|
||||
possible (Chris Li via Colin P. McCabe)
|
||||
|
||||
HADOOP-10809. hadoop-azure: page blob support. (Dexter Bradshaw,
|
||||
Mostafa Elhemali, Eric Hanson, and Mike Liddell via cnauroth)
|
||||
|
||||
BUG FIXES
|
||||
|
||||
HADOOP-11236. NFS: Fix javadoc warning in RpcProgram.java (Abhiraj Butala via harsh)
|
||||
|
|
|
@ -78,6 +78,54 @@ For live tests, set the following in azure-test.xml:
|
|||
1. "fs.azure.test.account.name -> {azureStorageAccountName}
|
||||
2. "fs.azure.account.key.{AccountName} -> {fullStorageKey}"
|
||||
|
||||
===================================
|
||||
Page Blob Support and Configuration
|
||||
===================================
|
||||
|
||||
The Azure Blob Storage interface for Hadoop supports two kinds of blobs, block blobs
|
||||
and page blobs. Block blobs are the default kind of blob and are good for most
|
||||
big-data use cases, like input data for Hive, Pig, analytical map-reduce jobs etc.
|
||||
Page blob handling in hadoop-azure was introduced to support HBase log files.
|
||||
Page blobs can be written any number of times, whereas block blobs can only be
|
||||
appended to 50,000 times before you run out of blocks and your writes will fail.
|
||||
That won't work for HBase logs, so page blob support was introduced to overcome
|
||||
this limitation.
|
||||
|
||||
Page blobs can be used for other purposes beyond just HBase log files though.
|
||||
They support the Hadoop FileSystem interface. Page blobs can be up to 1TB in
|
||||
size, larger than the maximum 200GB size for block blobs.
|
||||
|
||||
In order to have the files you create be page blobs, you must set the configuration
|
||||
variable fs.azure.page.blob.dir to a comma-separated list of folder names.
|
||||
E.g.
|
||||
|
||||
/hbase/WALs,/hbase/oldWALs,/data/mypageblobfiles
|
||||
|
||||
You can set this to simply / to make all files page blobs.
|
||||
|
||||
The configuration option fs.azure.page.blob.size is the default initial
|
||||
size for a page blob. It must be 128MB or greater, and no more than 1TB,
|
||||
specified as an integer number of bytes.
|
||||
|
||||
====================
|
||||
Atomic Folder Rename
|
||||
====================
|
||||
|
||||
Azure storage stores files as a flat key/value store without formal support
|
||||
for folders. The hadoop-azure file system layer simulates folders on top
|
||||
of Azure storage. By default, folder rename in the hadoop-azure file system
|
||||
layer is not atomic. That means that a failure during a folder rename
|
||||
could, for example, leave some folders in the original directory and
|
||||
some in the new one.
|
||||
|
||||
HBase depends on atomic folder rename. Hence, a configuration setting was
|
||||
introduced called fs.azure.atomic.rename.dir that allows you to specify a
|
||||
comma-separated list of directories to receive special treatment so that
|
||||
folder rename is made atomic. The default value of this setting is just /hbase.
|
||||
Redo will be applied to finish a folder rename that fails. A file
|
||||
<folderName>-renamePending.json may appear temporarily and is the record of
|
||||
the intention of the rename operation, to allow redo in event of a failure.
|
||||
|
||||
=============
|
||||
Findbugs
|
||||
=============
|
||||
|
|
|
@ -15,5 +15,36 @@
|
|||
limitations under the License.
|
||||
-->
|
||||
<FindBugsFilter>
|
||||
<!-- It is okay to skip up to end of file. No need to check return value. -->
|
||||
<Match>
|
||||
<Class name="org.apache.hadoop.fs.azure.AzureNativeFileSystemStore" />
|
||||
<Method name="retrieve" />
|
||||
<Bug pattern="SR_NOT_CHECKED" />
|
||||
<Priority value="2" />
|
||||
</Match>
|
||||
|
||||
<!-- Returning fully loaded array to iterate through is a convenience
|
||||
and helps performance. -->
|
||||
<Match>
|
||||
<Class name="org.apache.hadoop.fs.azure.NativeAzureFileSystem$FolderRenamePending" />
|
||||
<Method name="getFiles" />
|
||||
<Bug pattern="EI_EXPOSE_REP" />
|
||||
<Priority value="2" />
|
||||
</Match>
|
||||
|
||||
<!-- Need to start keep-alive thread for SelfRenewingLease in constructor. -->
|
||||
<Match>
|
||||
<Class name="org.apache.hadoop.fs.azure.SelfRenewingLease" />
|
||||
<Bug pattern="SC_START_IN_CTOR" />
|
||||
<Priority value="2" />
|
||||
</Match>
|
||||
|
||||
<!-- Using a key set iterator is fine because this is not a performance-critical
|
||||
method. -->
|
||||
<Match>
|
||||
<Class name="org.apache.hadoop.fs.azure.PageBlobOutputStream" />
|
||||
<Method name="logAllStackTraces" />
|
||||
<Bug pattern="WMI_WRONG_MAP_ITERATOR" />
|
||||
<Priority value="2" />
|
||||
</Match>
|
||||
</FindBugsFilter>
|
||||
|
|
|
@ -108,7 +108,10 @@
|
|||
<property name="max" value="3000"/>
|
||||
</module>
|
||||
|
||||
<module name="ParameterNumber"/>
|
||||
<module name="ParameterNumber">
|
||||
<property name="max" value="8"/>
|
||||
</module>
|
||||
|
||||
|
||||
|
||||
<!-- Checks for whitespace -->
|
||||
|
@ -152,7 +155,7 @@
|
|||
<module name="IllegalInstantiation"/>
|
||||
<module name="InnerAssignment"/>
|
||||
<module name="MagicNumber">
|
||||
<property name="ignoreNumbers" value="-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 10, 1000"/>
|
||||
<property name="ignoreNumbers" value="-1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 10, 255, 1000, 1024"/>
|
||||
</module>
|
||||
<module name="MissingSwitchDefault"/>
|
||||
<module name="RedundantThrows"/>
|
||||
|
|
|
@ -0,0 +1,217 @@
|
|||
/**
|
||||
* 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.apache.hadoop.fs.azure;
|
||||
|
||||
/**
|
||||
* A simple generic stack implementation using linked lists. The stack
|
||||
* implementation has five main operations:
|
||||
* <ul>
|
||||
* <li>push -- adds an element to the top of the stack</li>
|
||||
* <li>pop -- removes an element from the top of the stack and returns a
|
||||
* reference to it</li>
|
||||
* <li>peek -- peek returns an element from the top of the stack without
|
||||
* removing it</li>
|
||||
* <li>isEmpty -- tests whether the stack is empty</li>
|
||||
* <li>size -- returns the size of the stack</li>
|
||||
* <li>toString -- returns a string representation of the stack.</li>
|
||||
* </ul>
|
||||
*/
|
||||
|
||||
public class AzureLinkedStack<E> {
|
||||
/*
|
||||
* Linked node for Azure stack collection.
|
||||
*/
|
||||
private static class AzureLinkedNode<E> {
|
||||
private E element; // Linked element on the list.
|
||||
private AzureLinkedNode<E> next;// Reference to the next linked element on
|
||||
// list.
|
||||
|
||||
/*
|
||||
* The constructor builds the linked node with no successor
|
||||
*
|
||||
* @param element : The value of the element to be stored with this node.
|
||||
*/
|
||||
private AzureLinkedNode(E anElement) {
|
||||
element = anElement;
|
||||
next = null;
|
||||
}
|
||||
|
||||
/*
|
||||
* Constructor builds a linked node with a specified successor. The
|
||||
* successor may be null.
|
||||
*
|
||||
* @param anElement : new element to be created.
|
||||
*
|
||||
* @param nextElement: successor to the new element.
|
||||
*/
|
||||
private AzureLinkedNode(E anElement, AzureLinkedNode<E> nextElement) {
|
||||
element = anElement;
|
||||
next = nextElement;
|
||||
}
|
||||
|
||||
/*
|
||||
* Get the element stored in the linked node.
|
||||
*
|
||||
* @return E : element stored in linked node.
|
||||
*/
|
||||
private E getElement() {
|
||||
return element;
|
||||
}
|
||||
|
||||
/*
|
||||
* Get the successor node to the element.
|
||||
*
|
||||
* @return E : reference to the succeeding node on the list.
|
||||
*/
|
||||
private AzureLinkedNode<E> getNext() {
|
||||
return next;
|
||||
}
|
||||
}
|
||||
|
||||
private int count; // The number of elements stored on the stack.
|
||||
private AzureLinkedNode<E> top; // Top of the stack.
|
||||
|
||||
/*
|
||||
* Constructor creating an empty stack.
|
||||
*/
|
||||
public AzureLinkedStack() {
|
||||
// Simply initialize the member variables.
|
||||
//
|
||||
count = 0;
|
||||
top = null;
|
||||
}
|
||||
|
||||
/*
|
||||
* Adds an element to the top of the stack.
|
||||
*
|
||||
* @param element : element pushed to the top of the stack.
|
||||
*/
|
||||
public void push(E element) {
|
||||
// Create a new node containing a reference to be placed on the stack.
|
||||
// Set the next reference to the new node to point to the current top
|
||||
// of the stack. Set the top reference to point to the new node. Finally
|
||||
// increment the count of nodes on the stack.
|
||||
//
|
||||
AzureLinkedNode<E> newNode = new AzureLinkedNode<E>(element, top);
|
||||
top = newNode;
|
||||
count++;
|
||||
}
|
||||
|
||||
/*
|
||||
* Removes the element at the top of the stack and returns a reference to it.
|
||||
*
|
||||
* @return E : element popped from the top of the stack.
|
||||
*
|
||||
* @throws Exception on pop from an empty stack.
|
||||
*/
|
||||
public E pop() throws Exception {
|
||||
// Make sure the stack is not empty. If it is empty, throw a StackEmpty
|
||||
// exception.
|
||||
//
|
||||
if (isEmpty()) {
|
||||
throw new Exception("AzureStackEmpty");
|
||||
}
|
||||
|
||||
// Set a temporary reference equal to the element at the top of the stack,
|
||||
// decrement the count of elements and return reference to the temporary.
|
||||
//
|
||||
E element = top.getElement();
|
||||
top = top.getNext();
|
||||
count--;
|
||||
|
||||
// Return the reference to the element that was at the top of the stack.
|
||||
//
|
||||
return element;
|
||||
}
|
||||
|
||||
/*
|
||||
* Return the top element of the stack without removing it.
|
||||
*
|
||||
* @return E
|
||||
*
|
||||
* @throws Exception on peek into an empty stack.
|
||||
*/
|
||||
public E peek() throws Exception {
|
||||
// Make sure the stack is not empty. If it is empty, throw a StackEmpty
|
||||
// exception.
|
||||
//
|
||||
if (isEmpty()) {
|
||||
throw new Exception("AzureStackEmpty");
|
||||
}
|
||||
|
||||
// Set a temporary reference equal to the element at the top of the stack
|
||||
// and return the temporary.
|
||||
//
|
||||
E element = top.getElement();
|
||||
return element;
|
||||
}
|
||||
|
||||
/*
|
||||
* Determines whether the stack is empty
|
||||
*
|
||||
* @return boolean true if the stack is empty and false otherwise.
|
||||
*/
|
||||
public boolean isEmpty() {
|
||||
if (0 == size()) {
|
||||
// Zero-sized stack so the stack is empty.
|
||||
//
|
||||
return true;
|
||||
}
|
||||
|
||||
// The stack is not empty.
|
||||
//
|
||||
return false;
|
||||
}
|
||||
|
||||
/*
|
||||
* Determines the size of the stack
|
||||
*
|
||||
* @return int: Count of the number of elements in the stack.
|
||||
*/
|
||||
public int size() {
|
||||
return count;
|
||||
}
|
||||
|
||||
/*
|
||||
* Returns a string representation of the stack.
|
||||
*
|
||||
* @return String String representation of all elements in the stack.
|
||||
*/
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
|
||||
AzureLinkedNode<E> current = top;
|
||||
for (int i = 0; i < size(); i++) {
|
||||
E element = current.getElement();
|
||||
sb.append(element.toString());
|
||||
current = current.getNext();
|
||||
|
||||
// Insert commas between strings except after the last string.
|
||||
//
|
||||
if (size() - 1 > i) {
|
||||
sb.append(", ");
|
||||
}
|
||||
}
|
||||
|
||||
// Return the string.
|
||||
//
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -53,6 +53,10 @@ interface NativeFileSystemStore {
|
|||
DataOutputStream storefile(String key, PermissionStatus permissionStatus)
|
||||
throws AzureException;
|
||||
|
||||
boolean isPageBlobKey(String key);
|
||||
|
||||
boolean isAtomicRenameKey(String key);
|
||||
|
||||
void storeEmptyLinkFile(String key, String tempBlobKey,
|
||||
PermissionStatus permissionStatus) throws AzureException;
|
||||
|
||||
|
@ -74,6 +78,9 @@ interface NativeFileSystemStore {
|
|||
|
||||
void rename(String srcKey, String dstKey) throws IOException;
|
||||
|
||||
void rename(String srcKey, String dstKey, boolean acquireLease, SelfRenewingLease existingLease)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Delete all keys with the given prefix. Used for testing.
|
||||
*
|
||||
|
@ -91,8 +98,13 @@ interface NativeFileSystemStore {
|
|||
|
||||
void close();
|
||||
|
||||
void updateFolderLastModifiedTime(String key) throws AzureException;
|
||||
|
||||
void updateFolderLastModifiedTime(String key, Date lastModified)
|
||||
void updateFolderLastModifiedTime(String key, SelfRenewingLease folderLease)
|
||||
throws AzureException;
|
||||
|
||||
void updateFolderLastModifiedTime(String key, Date lastModified,
|
||||
SelfRenewingLease folderLease) throws AzureException;
|
||||
|
||||
void delete(String key, SelfRenewingLease lease) throws IOException;
|
||||
|
||||
SelfRenewingLease acquireLease(String key) throws AzureException;
|
||||
}
|
||||
|
|
|
@ -0,0 +1,58 @@
|
|||
/**
|
||||
* 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.apache.hadoop.fs.azure;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import com.microsoft.windowsazure.storage.blob.BlobRequestOptions;
|
||||
|
||||
/**
|
||||
* Constants and helper methods for ASV's custom data format in page blobs.
|
||||
*/
|
||||
final class PageBlobFormatHelpers {
|
||||
public static final short PAGE_SIZE = 512;
|
||||
public static final short PAGE_HEADER_SIZE = 2;
|
||||
public static final short PAGE_DATA_SIZE = PAGE_SIZE - PAGE_HEADER_SIZE;
|
||||
|
||||
// Hide constructor for utility class.
|
||||
private PageBlobFormatHelpers() {
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Stores the given short as a two-byte array.
|
||||
*/
|
||||
public static byte[] fromShort(short s) {
|
||||
return ByteBuffer.allocate(2).putShort(s).array();
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves a short from the given two bytes.
|
||||
*/
|
||||
public static short toShort(byte firstByte, byte secondByte) {
|
||||
return ByteBuffer.wrap(new byte[] { firstByte, secondByte })
|
||||
.getShort();
|
||||
}
|
||||
|
||||
public static BlobRequestOptions withMD5Checking() {
|
||||
BlobRequestOptions options = new BlobRequestOptions();
|
||||
options.setUseTransactionalContentMD5(true);
|
||||
return options;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,455 @@
|
|||
/**
|
||||
* 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.apache.hadoop.fs.azure;
|
||||
|
||||
import static org.apache.hadoop.fs.azure.PageBlobFormatHelpers.PAGE_DATA_SIZE;
|
||||
import static org.apache.hadoop.fs.azure.PageBlobFormatHelpers.PAGE_HEADER_SIZE;
|
||||
import static org.apache.hadoop.fs.azure.PageBlobFormatHelpers.PAGE_SIZE;
|
||||
import static org.apache.hadoop.fs.azure.PageBlobFormatHelpers.toShort;
|
||||
import static org.apache.hadoop.fs.azure.PageBlobFormatHelpers.withMD5Checking;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.azure.StorageInterface.CloudPageBlobWrapper;
|
||||
|
||||
import com.microsoft.windowsazure.storage.OperationContext;
|
||||
import com.microsoft.windowsazure.storage.StorageException;
|
||||
import com.microsoft.windowsazure.storage.blob.BlobRequestOptions;
|
||||
import com.microsoft.windowsazure.storage.blob.PageRange;
|
||||
|
||||
/**
|
||||
* An input stream that reads file data from a page blob stored
|
||||
* using ASV's custom format.
|
||||
*/
|
||||
|
||||
final class PageBlobInputStream extends InputStream {
|
||||
private static final Log LOG = LogFactory.getLog(PageBlobInputStream.class);
|
||||
|
||||
// The blob we're reading from.
|
||||
private final CloudPageBlobWrapper blob;
|
||||
// The operation context to use for storage requests.
|
||||
private final OperationContext opContext;
|
||||
// The number of pages remaining to be read from the server.
|
||||
private long numberOfPagesRemaining;
|
||||
// The current byte offset to start reading from the server next,
|
||||
// equivalent to (total number of pages we've read) * (page size).
|
||||
private long currentOffsetInBlob;
|
||||
// The buffer holding the current data we last read from the server.
|
||||
private byte[] currentBuffer;
|
||||
// The current byte offset we're at in the buffer.
|
||||
private int currentOffsetInBuffer;
|
||||
// Maximum number of pages to get per any one request.
|
||||
private static final int MAX_PAGES_PER_DOWNLOAD =
|
||||
4 * 1024 * 1024 / PAGE_SIZE;
|
||||
// Whether the stream has been closed.
|
||||
private boolean closed = false;
|
||||
// Total stream size, or -1 if not initialized.
|
||||
long pageBlobSize = -1;
|
||||
// Current position in stream of valid data.
|
||||
long filePosition = 0;
|
||||
|
||||
/**
|
||||
* Helper method to extract the actual data size of a page blob.
|
||||
* This typically involves 2 service requests (one for page ranges, another
|
||||
* for the last page's data).
|
||||
*
|
||||
* @param blob The blob to get the size from.
|
||||
* @param opContext The operation context to use for the requests.
|
||||
* @return The total data size of the blob in bytes.
|
||||
* @throws IOException If the format is corrupt.
|
||||
* @throws StorageException If anything goes wrong in the requests.
|
||||
*/
|
||||
public static long getPageBlobSize(CloudPageBlobWrapper blob,
|
||||
OperationContext opContext) throws IOException, StorageException {
|
||||
// Get the page ranges for the blob. There should be one range starting
|
||||
// at byte 0, but we tolerate (and ignore) ranges after the first one.
|
||||
ArrayList<PageRange> pageRanges =
|
||||
blob.downloadPageRanges(new BlobRequestOptions(), opContext);
|
||||
if (pageRanges.size() == 0) {
|
||||
return 0;
|
||||
}
|
||||
if (pageRanges.get(0).getStartOffset() != 0) {
|
||||
// Not expected: we always upload our page blobs as a contiguous range
|
||||
// starting at byte 0.
|
||||
throw badStartRangeException(blob, pageRanges.get(0));
|
||||
}
|
||||
long totalRawBlobSize = pageRanges.get(0).getEndOffset() + 1;
|
||||
|
||||
// Get the last page.
|
||||
long lastPageStart = totalRawBlobSize - PAGE_SIZE;
|
||||
ByteArrayOutputStream baos =
|
||||
new ByteArrayOutputStream(PageBlobFormatHelpers.PAGE_SIZE);
|
||||
blob.downloadRange(lastPageStart, PAGE_SIZE, baos,
|
||||
new BlobRequestOptions(), opContext);
|
||||
|
||||
byte[] lastPage = baos.toByteArray();
|
||||
short lastPageSize = getPageSize(blob, lastPage, 0);
|
||||
long totalNumberOfPages = totalRawBlobSize / PAGE_SIZE;
|
||||
return (totalNumberOfPages - 1) * PAGE_DATA_SIZE + lastPageSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs a stream over the given page blob.
|
||||
*/
|
||||
public PageBlobInputStream(CloudPageBlobWrapper blob,
|
||||
OperationContext opContext)
|
||||
throws IOException {
|
||||
this.blob = blob;
|
||||
this.opContext = opContext;
|
||||
ArrayList<PageRange> allRanges;
|
||||
try {
|
||||
allRanges =
|
||||
blob.downloadPageRanges(new BlobRequestOptions(), opContext);
|
||||
} catch (StorageException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
if (allRanges.size() > 0) {
|
||||
if (allRanges.get(0).getStartOffset() != 0) {
|
||||
throw badStartRangeException(blob, allRanges.get(0));
|
||||
}
|
||||
if (allRanges.size() > 1) {
|
||||
LOG.warn(String.format(
|
||||
"Blob %s has %d page ranges beyond the first range. "
|
||||
+ "Only reading the first range.",
|
||||
blob.getUri(), allRanges.size() - 1));
|
||||
}
|
||||
numberOfPagesRemaining =
|
||||
(allRanges.get(0).getEndOffset() + 1) / PAGE_SIZE;
|
||||
} else {
|
||||
numberOfPagesRemaining = 0;
|
||||
}
|
||||
}
|
||||
|
||||
/** Return the size of the remaining available bytes
|
||||
* if the size is less than or equal to {@link Integer#MAX_VALUE},
|
||||
* otherwise, return {@link Integer#MAX_VALUE}.
|
||||
*
|
||||
* This is to match the behavior of DFSInputStream.available(),
|
||||
* which some clients may rely on (HBase write-ahead log reading in
|
||||
* particular).
|
||||
*/
|
||||
@Override
|
||||
public synchronized int available() throws IOException {
|
||||
if (closed) {
|
||||
throw new IOException("Stream closed");
|
||||
}
|
||||
if (pageBlobSize == -1) {
|
||||
try {
|
||||
pageBlobSize = getPageBlobSize(blob, opContext);
|
||||
} catch (StorageException e) {
|
||||
throw new IOException("Unable to get page blob size.", e);
|
||||
}
|
||||
}
|
||||
|
||||
final long remaining = pageBlobSize - filePosition;
|
||||
return remaining <= Integer.MAX_VALUE ?
|
||||
(int) remaining : Integer.MAX_VALUE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void close() throws IOException {
|
||||
closed = true;
|
||||
}
|
||||
|
||||
private boolean dataAvailableInBuffer() {
|
||||
return currentBuffer != null
|
||||
&& currentOffsetInBuffer < currentBuffer.length;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check our buffer and download more from the server if needed.
|
||||
* @return true if there's more data in the buffer, false if we're done.
|
||||
* @throws IOException
|
||||
*/
|
||||
private synchronized boolean ensureDataInBuffer() throws IOException {
|
||||
if (dataAvailableInBuffer()) {
|
||||
// We still have some data in our buffer.
|
||||
return true;
|
||||
}
|
||||
currentBuffer = null;
|
||||
if (numberOfPagesRemaining == 0) {
|
||||
// No more data to read.
|
||||
return false;
|
||||
}
|
||||
final long pagesToRead = Math.min(MAX_PAGES_PER_DOWNLOAD,
|
||||
numberOfPagesRemaining);
|
||||
final int bufferSize = (int) (pagesToRead * PAGE_SIZE);
|
||||
|
||||
// Download page to current buffer.
|
||||
try {
|
||||
// Create a byte array output stream to capture the results of the
|
||||
// download.
|
||||
ByteArrayOutputStream baos = new ByteArrayOutputStream(bufferSize);
|
||||
blob.downloadRange(currentOffsetInBlob, bufferSize, baos,
|
||||
withMD5Checking(), opContext);
|
||||
currentBuffer = baos.toByteArray();
|
||||
} catch (StorageException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
numberOfPagesRemaining -= pagesToRead;
|
||||
currentOffsetInBlob += bufferSize;
|
||||
currentOffsetInBuffer = PAGE_HEADER_SIZE;
|
||||
|
||||
// Since we just downloaded a new buffer, validate its consistency.
|
||||
validateCurrentBufferConsistency();
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
private void validateCurrentBufferConsistency()
|
||||
throws IOException {
|
||||
if (currentBuffer.length % PAGE_SIZE != 0) {
|
||||
throw new AssertionError("Unexpected buffer size: "
|
||||
+ currentBuffer.length);
|
||||
}
|
||||
int numberOfPages = currentBuffer.length / PAGE_SIZE;
|
||||
for (int page = 0; page < numberOfPages; page++) {
|
||||
short currentPageSize = getPageSize(blob, currentBuffer,
|
||||
page * PAGE_SIZE);
|
||||
// Calculate the number of pages that exist after this one
|
||||
// in the blob.
|
||||
long totalPagesAfterCurrent =
|
||||
(numberOfPages - page - 1) + numberOfPagesRemaining;
|
||||
// Only the last page is allowed to be not filled completely.
|
||||
if (currentPageSize < PAGE_DATA_SIZE
|
||||
&& totalPagesAfterCurrent > 0) {
|
||||
throw fileCorruptException(blob, String.format(
|
||||
"Page with partial data found in the middle (%d pages from the"
|
||||
+ " end) that only has %d bytes of data.",
|
||||
totalPagesAfterCurrent, currentPageSize));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Reads the page size from the page header at the given offset.
|
||||
private static short getPageSize(CloudPageBlobWrapper blob,
|
||||
byte[] data, int offset) throws IOException {
|
||||
short pageSize = toShort(data[offset], data[offset + 1]);
|
||||
if (pageSize < 0 || pageSize > PAGE_DATA_SIZE) {
|
||||
throw fileCorruptException(blob, String.format(
|
||||
"Unexpected page size in the header: %d.",
|
||||
pageSize));
|
||||
}
|
||||
return pageSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized int read(byte[] outputBuffer, int offset, int len)
|
||||
throws IOException {
|
||||
int numberOfBytesRead = 0;
|
||||
while (len > 0) {
|
||||
if (!ensureDataInBuffer()) {
|
||||
filePosition += numberOfBytesRead;
|
||||
return numberOfBytesRead;
|
||||
}
|
||||
int bytesRemainingInCurrentPage = getBytesRemainingInCurrentPage();
|
||||
int numBytesToRead = Math.min(len, bytesRemainingInCurrentPage);
|
||||
System.arraycopy(currentBuffer, currentOffsetInBuffer, outputBuffer,
|
||||
offset, numBytesToRead);
|
||||
numberOfBytesRead += numBytesToRead;
|
||||
offset += numBytesToRead;
|
||||
len -= numBytesToRead;
|
||||
if (numBytesToRead == bytesRemainingInCurrentPage) {
|
||||
// We've finished this page, move on to the next.
|
||||
advancePagesInBuffer(1);
|
||||
} else {
|
||||
currentOffsetInBuffer += numBytesToRead;
|
||||
}
|
||||
}
|
||||
filePosition += numberOfBytesRead;
|
||||
return numberOfBytesRead;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
byte[] oneByte = new byte[1];
|
||||
if (read(oneByte) == 0) {
|
||||
return -1;
|
||||
}
|
||||
return oneByte[0];
|
||||
}
|
||||
|
||||
/**
|
||||
* Skips over and discards n bytes of data from this input stream.
|
||||
* @param n the number of bytes to be skipped.
|
||||
* @return the actual number of bytes skipped.
|
||||
*/
|
||||
@Override
|
||||
public synchronized long skip(long n) throws IOException {
|
||||
long skipped = skipImpl(n);
|
||||
filePosition += skipped; // track the position in the stream
|
||||
return skipped;
|
||||
}
|
||||
|
||||
private long skipImpl(long n) throws IOException {
|
||||
|
||||
if (n == 0) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
// First skip within the current buffer as much as possible.
|
||||
long skippedWithinBuffer = skipWithinBuffer(n);
|
||||
if (skippedWithinBuffer > n) {
|
||||
// TO CONSIDER: Using a contracts framework such as Google's cofoja for
|
||||
// these post-conditions.
|
||||
throw new AssertionError(String.format(
|
||||
"Bug in skipWithinBuffer: it skipped over %d bytes when asked to "
|
||||
+ "skip %d bytes.", skippedWithinBuffer, n));
|
||||
}
|
||||
n -= skippedWithinBuffer;
|
||||
long skipped = skippedWithinBuffer;
|
||||
|
||||
// Empty the current buffer, we're going beyond it.
|
||||
currentBuffer = null;
|
||||
|
||||
// Skip over whole pages as necessary without retrieving them from the
|
||||
// server.
|
||||
long pagesToSkipOver = Math.min(
|
||||
n / PAGE_DATA_SIZE,
|
||||
numberOfPagesRemaining - 1);
|
||||
numberOfPagesRemaining -= pagesToSkipOver;
|
||||
currentOffsetInBlob += pagesToSkipOver * PAGE_SIZE;
|
||||
skipped += pagesToSkipOver * PAGE_DATA_SIZE;
|
||||
n -= pagesToSkipOver * PAGE_DATA_SIZE;
|
||||
if (n == 0) {
|
||||
return skipped;
|
||||
}
|
||||
|
||||
// Now read in at the current position, and skip within current buffer.
|
||||
if (!ensureDataInBuffer()) {
|
||||
return skipped;
|
||||
}
|
||||
return skipped + skipWithinBuffer(n);
|
||||
}
|
||||
|
||||
/**
|
||||
* Skip over n bytes within the current buffer or just over skip the whole
|
||||
* buffer if n is greater than the bytes remaining in the buffer.
|
||||
* @param n The number of data bytes to skip.
|
||||
* @return The number of bytes actually skipped.
|
||||
* @throws IOException if data corruption found in the buffer.
|
||||
*/
|
||||
private long skipWithinBuffer(long n) throws IOException {
|
||||
if (!dataAvailableInBuffer()) {
|
||||
return 0;
|
||||
}
|
||||
long skipped = 0;
|
||||
// First skip within the current page.
|
||||
skipped = skipWithinCurrentPage(n);
|
||||
if (skipped > n) {
|
||||
throw new AssertionError(String.format(
|
||||
"Bug in skipWithinCurrentPage: it skipped over %d bytes when asked"
|
||||
+ " to skip %d bytes.", skipped, n));
|
||||
}
|
||||
n -= skipped;
|
||||
if (n == 0 || !dataAvailableInBuffer()) {
|
||||
return skipped;
|
||||
}
|
||||
|
||||
// Calculate how many whole pages (pages before the possibly partially
|
||||
// filled last page) remain.
|
||||
int currentPageIndex = currentOffsetInBuffer / PAGE_SIZE;
|
||||
int numberOfPagesInBuffer = currentBuffer.length / PAGE_SIZE;
|
||||
int wholePagesRemaining = numberOfPagesInBuffer - currentPageIndex - 1;
|
||||
|
||||
if (n < (PAGE_DATA_SIZE * wholePagesRemaining)) {
|
||||
// I'm within one of the whole pages remaining, skip in there.
|
||||
advancePagesInBuffer((int) (n / PAGE_DATA_SIZE));
|
||||
currentOffsetInBuffer += n % PAGE_DATA_SIZE;
|
||||
return n + skipped;
|
||||
}
|
||||
|
||||
// Skip over the whole pages.
|
||||
advancePagesInBuffer(wholePagesRemaining);
|
||||
skipped += wholePagesRemaining * PAGE_DATA_SIZE;
|
||||
n -= wholePagesRemaining * PAGE_DATA_SIZE;
|
||||
|
||||
// At this point we know we need to skip to somewhere in the last page,
|
||||
// or just go to the end.
|
||||
return skipWithinCurrentPage(n) + skipped;
|
||||
}
|
||||
|
||||
/**
|
||||
* Skip over n bytes within the current page or just over skip the whole
|
||||
* page if n is greater than the bytes remaining in the page.
|
||||
* @param n The number of data bytes to skip.
|
||||
* @return The number of bytes actually skipped.
|
||||
* @throws IOException if data corruption found in the buffer.
|
||||
*/
|
||||
private long skipWithinCurrentPage(long n) throws IOException {
|
||||
int remainingBytesInCurrentPage = getBytesRemainingInCurrentPage();
|
||||
if (n < remainingBytesInCurrentPage) {
|
||||
currentOffsetInBuffer += n;
|
||||
return n;
|
||||
} else {
|
||||
advancePagesInBuffer(1);
|
||||
return remainingBytesInCurrentPage;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the number of bytes remaining within the current page in the buffer.
|
||||
* @return The number of bytes remaining.
|
||||
* @throws IOException if data corruption found in the buffer.
|
||||
*/
|
||||
private int getBytesRemainingInCurrentPage() throws IOException {
|
||||
if (!dataAvailableInBuffer()) {
|
||||
return 0;
|
||||
}
|
||||
// Calculate our current position relative to the start of the current
|
||||
// page.
|
||||
int currentDataOffsetInPage =
|
||||
(currentOffsetInBuffer % PAGE_SIZE) - PAGE_HEADER_SIZE;
|
||||
int pageBoundary = getCurrentPageStartInBuffer();
|
||||
// Get the data size of the current page from the header.
|
||||
short sizeOfCurrentPage = getPageSize(blob, currentBuffer, pageBoundary);
|
||||
return sizeOfCurrentPage - currentDataOffsetInPage;
|
||||
}
|
||||
|
||||
private static IOException badStartRangeException(CloudPageBlobWrapper blob,
|
||||
PageRange startRange) {
|
||||
return fileCorruptException(blob, String.format(
|
||||
"Page blobs for ASV should always use a page range starting at byte 0. "
|
||||
+ "This starts at byte %d.",
|
||||
startRange.getStartOffset()));
|
||||
}
|
||||
|
||||
private void advancePagesInBuffer(int numberOfPages) {
|
||||
currentOffsetInBuffer =
|
||||
getCurrentPageStartInBuffer()
|
||||
+ (numberOfPages * PAGE_SIZE)
|
||||
+ PAGE_HEADER_SIZE;
|
||||
}
|
||||
|
||||
private int getCurrentPageStartInBuffer() {
|
||||
return PAGE_SIZE * (currentOffsetInBuffer / PAGE_SIZE);
|
||||
}
|
||||
|
||||
private static IOException fileCorruptException(CloudPageBlobWrapper blob,
|
||||
String reason) {
|
||||
return new IOException(String.format(
|
||||
"The page blob: '%s' is corrupt or has an unexpected format: %s.",
|
||||
blob.getUri(), reason));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,497 @@
|
|||
/**
|
||||
* 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.apache.hadoop.fs.azure;
|
||||
|
||||
import static org.apache.hadoop.fs.azure.PageBlobFormatHelpers.PAGE_DATA_SIZE;
|
||||
import static org.apache.hadoop.fs.azure.PageBlobFormatHelpers.PAGE_HEADER_SIZE;
|
||||
import static org.apache.hadoop.fs.azure.PageBlobFormatHelpers.PAGE_SIZE;
|
||||
import static org.apache.hadoop.fs.azure.PageBlobFormatHelpers.fromShort;
|
||||
import static org.apache.hadoop.fs.azure.PageBlobFormatHelpers.withMD5Checking;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.util.Arrays;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.hadoop.fs.Syncable;
|
||||
import org.apache.hadoop.fs.azure.StorageInterface.CloudPageBlobWrapper;
|
||||
import org.apache.commons.lang.exception.ExceptionUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.microsoft.windowsazure.storage.OperationContext;
|
||||
import com.microsoft.windowsazure.storage.StorageException;
|
||||
import com.microsoft.windowsazure.storage.blob.BlobRequestOptions;
|
||||
|
||||
|
||||
/**
|
||||
* An output stream that write file data to a page blob stored using ASV's
|
||||
* custom format.
|
||||
*/
|
||||
final class PageBlobOutputStream extends OutputStream implements Syncable {
|
||||
/**
|
||||
* The maximum number of raw bytes Azure Storage allows us to upload in a
|
||||
* single request (4 MB).
|
||||
*/
|
||||
private static final int MAX_RAW_BYTES_PER_REQUEST = 4 * 1024 * 1024;
|
||||
/**
|
||||
* The maximum number of pages Azure Storage allows us to upload in a
|
||||
* single request.
|
||||
*/
|
||||
private static final int MAX_PAGES_IN_REQUEST =
|
||||
MAX_RAW_BYTES_PER_REQUEST / PAGE_SIZE;
|
||||
/**
|
||||
* The maximum number of data bytes (header not included) we can upload
|
||||
* in a single request. I'm limiting it to (N - 1) pages to account for
|
||||
* the possibility that we may have to rewrite the previous request's
|
||||
* last page.
|
||||
*/
|
||||
private static final int MAX_DATA_BYTES_PER_REQUEST =
|
||||
PAGE_DATA_SIZE * (MAX_PAGES_IN_REQUEST - 1);
|
||||
|
||||
private final CloudPageBlobWrapper blob;
|
||||
private final OperationContext opContext;
|
||||
|
||||
/**
|
||||
* If the IO thread encounters an error, it'll store it here.
|
||||
*/
|
||||
private volatile IOException lastError;
|
||||
|
||||
/**
|
||||
* The current byte offset we're at in the blob (how many bytes we've
|
||||
* uploaded to the server).
|
||||
*/
|
||||
private long currentBlobOffset;
|
||||
/**
|
||||
* The data in the last page that we wrote to the server, in case we have to
|
||||
* overwrite it in the new request.
|
||||
*/
|
||||
private byte[] previousLastPageDataWritten = new byte[0];
|
||||
/**
|
||||
* The current buffer we're writing to before sending to the server.
|
||||
*/
|
||||
private ByteArrayOutputStream outBuffer;
|
||||
/**
|
||||
* The task queue for writing to the server.
|
||||
*/
|
||||
private final LinkedBlockingQueue<Runnable> ioQueue;
|
||||
/**
|
||||
* The thread pool we're using for writing to the server. Note that the IO
|
||||
* write is NOT designed for parallelism, so there can only be one thread
|
||||
* in that pool (I'm using the thread pool mainly for the lifetime management
|
||||
* capabilities, otherwise I'd have just used a simple Thread).
|
||||
*/
|
||||
private final ThreadPoolExecutor ioThreadPool;
|
||||
|
||||
// The last task given to the ioThreadPool to execute, to allow
|
||||
// waiting until it's done.
|
||||
private WriteRequest lastQueuedTask;
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(AzureNativeFileSystemStore.class);
|
||||
|
||||
// Set the minimum page blob file size to 128MB, which is >> the default block size of 32MB.
|
||||
// This default block size is often used as the hbase.regionserver.hlog.blocksize.
|
||||
// The goal is to have a safe minimum size for HBase log files to allow them
|
||||
// to be filled and rolled without exceeding the minimum size. A larger size can be
|
||||
// used by setting the fs.azure.page.blob.size configuration variable.
|
||||
public static final long PAGE_BLOB_MIN_SIZE = 128L * 1024L * 1024L;
|
||||
|
||||
/**
|
||||
* Constructs an output stream over the given page blob.
|
||||
*
|
||||
* @param blob the blob that this stream is associated with.
|
||||
* @param opContext an object used to track the execution of the operation
|
||||
* @throws StorageException if anything goes wrong creating the blob.
|
||||
*/
|
||||
public PageBlobOutputStream(final CloudPageBlobWrapper blob,
|
||||
final OperationContext opContext,
|
||||
final Configuration conf) throws StorageException {
|
||||
this.blob = blob;
|
||||
this.outBuffer = new ByteArrayOutputStream();
|
||||
this.opContext = opContext;
|
||||
this.lastQueuedTask = null;
|
||||
this.ioQueue = new LinkedBlockingQueue<Runnable>();
|
||||
|
||||
// As explained above: the IO writes are not designed for parallelism,
|
||||
// so we only have one thread in this thread pool.
|
||||
this.ioThreadPool = new ThreadPoolExecutor(1, 1, 2, TimeUnit.SECONDS,
|
||||
ioQueue);
|
||||
|
||||
|
||||
|
||||
// Make page blob files have a size that is the greater of a
|
||||
// minimum size, or the value of fs.azure.page.blob.size from configuration.
|
||||
long pageBlobConfigSize = conf.getLong("fs.azure.page.blob.size", 0);
|
||||
LOG.debug("Read value of fs.azure.page.blob.size as " + pageBlobConfigSize
|
||||
+ " from configuration (0 if not present).");
|
||||
long pageBlobSize = Math.max(PAGE_BLOB_MIN_SIZE, pageBlobConfigSize);
|
||||
|
||||
// Ensure that the pageBlobSize is a multiple of page size.
|
||||
if (pageBlobSize % PAGE_SIZE != 0) {
|
||||
pageBlobSize += PAGE_SIZE - pageBlobSize % PAGE_SIZE;
|
||||
}
|
||||
blob.create(pageBlobSize, new BlobRequestOptions(), opContext);
|
||||
}
|
||||
|
||||
private void checkStreamState() throws IOException {
|
||||
if (lastError != null) {
|
||||
throw lastError;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes this output stream and releases any system resources associated with
|
||||
* this stream. If any data remains in the buffer it is committed to the
|
||||
* service.
|
||||
*/
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
LOG.debug("Closing page blob output stream.");
|
||||
flush();
|
||||
checkStreamState();
|
||||
ioThreadPool.shutdown();
|
||||
try {
|
||||
LOG.debug(ioThreadPool.toString());
|
||||
if (!ioThreadPool.awaitTermination(10, TimeUnit.MINUTES)) {
|
||||
LOG.debug("Timed out after 10 minutes waiting for IO requests to finish");
|
||||
logAllStackTraces();
|
||||
LOG.debug(ioThreadPool.toString());
|
||||
throw new IOException("Timed out waiting for IO requests to finish");
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
LOG.debug("Caught InterruptedException");
|
||||
|
||||
// Restore the interrupted status
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
|
||||
this.lastError = new IOException("Stream is already closed.");
|
||||
}
|
||||
|
||||
// Log the stacks of all threads.
|
||||
private void logAllStackTraces() {
|
||||
Map liveThreads = Thread.getAllStackTraces();
|
||||
for (Iterator i = liveThreads.keySet().iterator(); i.hasNext(); ) {
|
||||
Thread key = (Thread) i.next();
|
||||
LOG.debug("Thread " + key.getName());
|
||||
StackTraceElement[] trace = (StackTraceElement[]) liveThreads.get(key);
|
||||
for (int j = 0; j < trace.length; j++) {
|
||||
LOG.debug("\tat " + trace[j]);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A single write request for data to write to Azure storage.
|
||||
*/
|
||||
private class WriteRequest implements Runnable {
|
||||
private final byte[] dataPayload;
|
||||
private final CountDownLatch doneSignal = new CountDownLatch(1);
|
||||
|
||||
public WriteRequest(byte[] dataPayload) {
|
||||
this.dataPayload = dataPayload;
|
||||
}
|
||||
|
||||
public void waitTillDone() throws InterruptedException {
|
||||
doneSignal.await();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
LOG.debug("before runInternal()");
|
||||
runInternal();
|
||||
LOG.debug("after runInternal()");
|
||||
} finally {
|
||||
doneSignal.countDown();
|
||||
}
|
||||
}
|
||||
|
||||
private void runInternal() {
|
||||
if (lastError != null) {
|
||||
// We're already in an error state, no point doing anything.
|
||||
return;
|
||||
}
|
||||
if (dataPayload.length == 0) {
|
||||
// Nothing to do.
|
||||
return;
|
||||
}
|
||||
|
||||
// Since we have to rewrite the last request's last page's data
|
||||
// (may be empty), total data size is our data plus whatever was
|
||||
// left from there.
|
||||
final int totalDataBytes = dataPayload.length
|
||||
+ previousLastPageDataWritten.length;
|
||||
// Calculate the total number of pages we're writing to the server.
|
||||
final int numberOfPages = (totalDataBytes / PAGE_DATA_SIZE)
|
||||
+ (totalDataBytes % PAGE_DATA_SIZE == 0 ? 0 : 1);
|
||||
// Fill up the raw bytes we're writing.
|
||||
byte[] rawPayload = new byte[numberOfPages * PAGE_SIZE];
|
||||
// Keep track of the size of the last page we uploaded.
|
||||
int currentLastPageDataSize = -1;
|
||||
for (int page = 0; page < numberOfPages; page++) {
|
||||
// Our current byte offset in the data.
|
||||
int dataOffset = page * PAGE_DATA_SIZE;
|
||||
// Our current byte offset in the raw buffer.
|
||||
int rawOffset = page * PAGE_SIZE;
|
||||
// The size of the data in the current page.
|
||||
final short currentPageDataSize = (short) Math.min(PAGE_DATA_SIZE,
|
||||
totalDataBytes - dataOffset);
|
||||
// Save off this page's size as the potential last page's size.
|
||||
currentLastPageDataSize = currentPageDataSize;
|
||||
|
||||
// Write out the page size in the header.
|
||||
final byte[] header = fromShort(currentPageDataSize);
|
||||
System.arraycopy(header, 0, rawPayload, rawOffset, header.length);
|
||||
rawOffset += header.length;
|
||||
|
||||
int bytesToCopyFromDataPayload = currentPageDataSize;
|
||||
if (dataOffset < previousLastPageDataWritten.length) {
|
||||
// First write out the last page's data.
|
||||
final int bytesToCopyFromLastPage = Math.min(currentPageDataSize,
|
||||
previousLastPageDataWritten.length - dataOffset);
|
||||
System.arraycopy(previousLastPageDataWritten, dataOffset,
|
||||
rawPayload, rawOffset, bytesToCopyFromLastPage);
|
||||
bytesToCopyFromDataPayload -= bytesToCopyFromLastPage;
|
||||
rawOffset += bytesToCopyFromLastPage;
|
||||
dataOffset += bytesToCopyFromLastPage;
|
||||
}
|
||||
|
||||
if (dataOffset >= previousLastPageDataWritten.length) {
|
||||
// Then write the current payload's data.
|
||||
System.arraycopy(dataPayload,
|
||||
dataOffset - previousLastPageDataWritten.length,
|
||||
rawPayload, rawOffset, bytesToCopyFromDataPayload);
|
||||
}
|
||||
}
|
||||
|
||||
// Raw payload constructed, ship it off to the server.
|
||||
writePayloadToServer(rawPayload);
|
||||
|
||||
// Post-send bookkeeping.
|
||||
currentBlobOffset += rawPayload.length;
|
||||
if (currentLastPageDataSize < PAGE_DATA_SIZE) {
|
||||
// Partial page, save it off so it's overwritten in the next request.
|
||||
final int startOffset = (numberOfPages - 1) * PAGE_SIZE + PAGE_HEADER_SIZE;
|
||||
previousLastPageDataWritten = Arrays.copyOfRange(rawPayload,
|
||||
startOffset,
|
||||
startOffset + currentLastPageDataSize);
|
||||
// Since we're rewriting this page, set our current offset in the server
|
||||
// to that page's beginning.
|
||||
currentBlobOffset -= PAGE_SIZE;
|
||||
} else {
|
||||
// It wasn't a partial page, we won't need to rewrite it.
|
||||
previousLastPageDataWritten = new byte[0];
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes the given raw payload to Azure Storage at the current blob
|
||||
* offset.
|
||||
*/
|
||||
private void writePayloadToServer(byte[] rawPayload) {
|
||||
final ByteArrayInputStream wrapperStream =
|
||||
new ByteArrayInputStream(rawPayload);
|
||||
LOG.debug("writing payload of " + rawPayload.length + " bytes to Azure page blob");
|
||||
try {
|
||||
long start = System.currentTimeMillis();
|
||||
blob.uploadPages(wrapperStream, currentBlobOffset, rawPayload.length,
|
||||
withMD5Checking(), PageBlobOutputStream.this.opContext);
|
||||
long end = System.currentTimeMillis();
|
||||
LOG.trace("Azure uploadPages time for " + rawPayload.length + " bytes = " + (end - start));
|
||||
} catch (IOException ex) {
|
||||
LOG.debug(ExceptionUtils.getStackTrace(ex));
|
||||
lastError = ex;
|
||||
} catch (StorageException ex) {
|
||||
LOG.debug(ExceptionUtils.getStackTrace(ex));
|
||||
lastError = new IOException(ex);
|
||||
}
|
||||
if (lastError != null) {
|
||||
LOG.debug("Caught error in PageBlobOutputStream#writePayloadToServer()");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private synchronized void flushIOBuffers() {
|
||||
if (outBuffer.size() == 0) {
|
||||
return;
|
||||
}
|
||||
lastQueuedTask = new WriteRequest(outBuffer.toByteArray());
|
||||
ioThreadPool.execute(lastQueuedTask);
|
||||
outBuffer = new ByteArrayOutputStream();
|
||||
}
|
||||
|
||||
/**
|
||||
* Flushes this output stream and forces any buffered output bytes to be
|
||||
* written out. If any data remains in the buffer it is committed to the
|
||||
* service. Data is queued for writing but not forced out to the service
|
||||
* before the call returns.
|
||||
*/
|
||||
@Override
|
||||
public void flush() throws IOException {
|
||||
checkStreamState();
|
||||
flushIOBuffers();
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes b.length bytes from the specified byte array to this output stream.
|
||||
*
|
||||
* @param data
|
||||
* the byte array to write.
|
||||
*
|
||||
* @throws IOException
|
||||
* if an I/O error occurs. In particular, an IOException may be
|
||||
* thrown if the output stream has been closed.
|
||||
*/
|
||||
@Override
|
||||
public void write(final byte[] data) throws IOException {
|
||||
write(data, 0, data.length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes length bytes from the specified byte array starting at offset to
|
||||
* this output stream.
|
||||
*
|
||||
* @param data
|
||||
* the byte array to write.
|
||||
* @param offset
|
||||
* the start offset in the data.
|
||||
* @param length
|
||||
* the number of bytes to write.
|
||||
* @throws IOException
|
||||
* if an I/O error occurs. In particular, an IOException may be
|
||||
* thrown if the output stream has been closed.
|
||||
*/
|
||||
@Override
|
||||
public void write(final byte[] data, final int offset, final int length)
|
||||
throws IOException {
|
||||
if (offset < 0 || length < 0 || length > data.length - offset) {
|
||||
throw new IndexOutOfBoundsException();
|
||||
}
|
||||
|
||||
writeInternal(data, offset, length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes the specified byte to this output stream. The general contract for
|
||||
* write is that one byte is written to the output stream. The byte to be
|
||||
* written is the eight low-order bits of the argument b. The 24 high-order
|
||||
* bits of b are ignored.
|
||||
*
|
||||
* @param byteVal
|
||||
* the byteValue to write.
|
||||
* @throws IOException
|
||||
* if an I/O error occurs. In particular, an IOException may be
|
||||
* thrown if the output stream has been closed.
|
||||
*/
|
||||
@Override
|
||||
public void write(final int byteVal) throws IOException {
|
||||
write(new byte[] { (byte) (byteVal & 0xFF) });
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes the data to the buffer and triggers writes to the service as needed.
|
||||
*
|
||||
* @param data
|
||||
* the byte array to write.
|
||||
* @param offset
|
||||
* the start offset in the data.
|
||||
* @param length
|
||||
* the number of bytes to write.
|
||||
* @throws IOException
|
||||
* if an I/O error occurs. In particular, an IOException may be
|
||||
* thrown if the output stream has been closed.
|
||||
*/
|
||||
private synchronized void writeInternal(final byte[] data, int offset,
|
||||
int length) throws IOException {
|
||||
while (length > 0) {
|
||||
checkStreamState();
|
||||
final int availableBufferBytes = MAX_DATA_BYTES_PER_REQUEST
|
||||
- this.outBuffer.size();
|
||||
final int nextWrite = Math.min(availableBufferBytes, length);
|
||||
|
||||
outBuffer.write(data, offset, nextWrite);
|
||||
offset += nextWrite;
|
||||
length -= nextWrite;
|
||||
|
||||
if (outBuffer.size() > MAX_DATA_BYTES_PER_REQUEST) {
|
||||
throw new RuntimeException("Internal error: maximum write size " +
|
||||
Integer.toString(MAX_DATA_BYTES_PER_REQUEST) + "exceeded.");
|
||||
}
|
||||
|
||||
if (outBuffer.size() == MAX_DATA_BYTES_PER_REQUEST) {
|
||||
flushIOBuffers();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Force all data in the output stream to be written to Azure storage.
|
||||
* Wait to return until this is complete.
|
||||
*/
|
||||
@Override
|
||||
public synchronized void hsync() throws IOException {
|
||||
LOG.debug("Entering PageBlobOutputStream#hsync().");
|
||||
long start = System.currentTimeMillis();
|
||||
flush();
|
||||
LOG.debug(ioThreadPool.toString());
|
||||
try {
|
||||
if (lastQueuedTask != null) {
|
||||
lastQueuedTask.waitTillDone();
|
||||
}
|
||||
} catch (InterruptedException e1) {
|
||||
|
||||
// Restore the interrupted status
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
LOG.debug("Leaving PageBlobOutputStream#hsync(). Total hsync duration = "
|
||||
+ (System.currentTimeMillis() - start) + " msec.");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
public void hflush() throws IOException {
|
||||
|
||||
// hflush is required to force data to storage, so call hsync,
|
||||
// which does that.
|
||||
hsync();
|
||||
}
|
||||
|
||||
@Deprecated
|
||||
public void sync() throws IOException {
|
||||
|
||||
// Sync has been deprecated in favor of hflush.
|
||||
hflush();
|
||||
}
|
||||
|
||||
// For unit testing purposes: kill the IO threads.
|
||||
@VisibleForTesting
|
||||
void killIoThreads() {
|
||||
ioThreadPool.shutdownNow();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,202 @@
|
|||
/**
|
||||
* 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.apache.hadoop.fs.azure;
|
||||
|
||||
import org.apache.commons.lang.exception.ExceptionUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.azure.StorageInterface.CloudBlobWrapper;
|
||||
|
||||
import com.microsoft.windowsazure.storage.AccessCondition;
|
||||
import com.microsoft.windowsazure.storage.StorageException;
|
||||
import com.microsoft.windowsazure.storage.blob.CloudBlob;
|
||||
|
||||
/**
|
||||
* An Azure blob lease that automatically renews itself indefinitely
|
||||
* using a background thread. Use it to synchronize distributed processes,
|
||||
* or to prevent writes to the blob by other processes that don't
|
||||
* have the lease.
|
||||
*
|
||||
* Creating a new Lease object blocks the caller until the Azure blob lease is
|
||||
* acquired.
|
||||
*
|
||||
* Attempting to get a lease on a non-existent blob throws StorageException.
|
||||
*
|
||||
* Call free() to release the Lease.
|
||||
*
|
||||
* You can use this Lease like a distributed lock. If the holder process
|
||||
* dies, the lease will time out since it won't be renewed.
|
||||
*/
|
||||
public class SelfRenewingLease {
|
||||
|
||||
private CloudBlobWrapper blobWrapper;
|
||||
private Thread renewer;
|
||||
private volatile boolean leaseFreed;
|
||||
private String leaseID = null;
|
||||
private static final int LEASE_TIMEOUT = 60; // Lease timeout in seconds
|
||||
|
||||
// Time to wait to renew lease in milliseconds
|
||||
public static final int LEASE_RENEWAL_PERIOD = 40000;
|
||||
private static final Log LOG = LogFactory.getLog(SelfRenewingLease.class);
|
||||
|
||||
// Used to allocate thread serial numbers in thread name
|
||||
private static volatile int threadNumber = 0;
|
||||
|
||||
|
||||
// Time to wait to retry getting the lease in milliseconds
|
||||
private static final int LEASE_ACQUIRE_RETRY_INTERVAL = 2000;
|
||||
|
||||
public SelfRenewingLease(CloudBlobWrapper blobWrapper)
|
||||
throws StorageException {
|
||||
|
||||
this.leaseFreed = false;
|
||||
this.blobWrapper = blobWrapper;
|
||||
|
||||
// Keep trying to get the lease until you get it.
|
||||
CloudBlob blob = blobWrapper.getBlob();
|
||||
while(leaseID == null) {
|
||||
try {
|
||||
leaseID = blob.acquireLease(LEASE_TIMEOUT, null);
|
||||
} catch (StorageException e) {
|
||||
|
||||
// Throw again if we don't want to keep waiting.
|
||||
// We expect it to be that the lease is already present,
|
||||
// or in some cases that the blob does not exist.
|
||||
if (!e.getErrorCode().equals("LeaseAlreadyPresent")) {
|
||||
LOG.info(
|
||||
"Caught exception when trying to get lease on blob "
|
||||
+ blobWrapper.getUri().toString() + ". " + e.getMessage());
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
if (leaseID == null) {
|
||||
try {
|
||||
Thread.sleep(LEASE_ACQUIRE_RETRY_INTERVAL);
|
||||
} catch (InterruptedException e) {
|
||||
|
||||
// Restore the interrupted status
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
}
|
||||
renewer = new Thread(new Renewer());
|
||||
|
||||
// A Renewer running should not keep JVM from exiting, so make it a daemon.
|
||||
renewer.setDaemon(true);
|
||||
renewer.setName("AzureLeaseRenewer-" + threadNumber++);
|
||||
renewer.start();
|
||||
LOG.debug("Acquired lease " + leaseID + " on " + blob.getUri()
|
||||
+ " managed by thread " + renewer.getName());
|
||||
}
|
||||
|
||||
/**
|
||||
* Free the lease and stop the keep-alive thread.
|
||||
* @throws StorageException
|
||||
*/
|
||||
public void free() throws StorageException {
|
||||
AccessCondition accessCondition = AccessCondition.generateEmptyCondition();
|
||||
accessCondition.setLeaseID(leaseID);
|
||||
try {
|
||||
blobWrapper.getBlob().releaseLease(accessCondition);
|
||||
} catch (StorageException e) {
|
||||
if (e.getErrorCode().equals("BlobNotFound")) {
|
||||
|
||||
// Don't do anything -- it's okay to free a lease
|
||||
// on a deleted file. The delete freed the lease
|
||||
// implicitly.
|
||||
} else {
|
||||
|
||||
// This error is not anticipated, so re-throw it.
|
||||
LOG.warn("Unanticipated exception when trying to free lease " + leaseID
|
||||
+ " on " + blobWrapper.getStorageUri());
|
||||
throw(e);
|
||||
}
|
||||
} finally {
|
||||
|
||||
// Even if releasing the lease fails (e.g. because the file was deleted),
|
||||
// make sure to record that we freed the lease, to terminate the
|
||||
// keep-alive thread.
|
||||
leaseFreed = true;
|
||||
LOG.debug("Freed lease " + leaseID + " on " + blobWrapper.getUri()
|
||||
+ " managed by thread " + renewer.getName());
|
||||
}
|
||||
}
|
||||
|
||||
public boolean isFreed() {
|
||||
return leaseFreed;
|
||||
}
|
||||
|
||||
public String getLeaseID() {
|
||||
return leaseID;
|
||||
}
|
||||
|
||||
public CloudBlob getCloudBlob() {
|
||||
return blobWrapper.getBlob();
|
||||
}
|
||||
|
||||
private class Renewer implements Runnable {
|
||||
|
||||
/**
|
||||
* Start a keep-alive thread that will continue to renew
|
||||
* the lease until it is freed or the process dies.
|
||||
*/
|
||||
@Override
|
||||
public void run() {
|
||||
LOG.debug("Starting lease keep-alive thread.");
|
||||
AccessCondition accessCondition =
|
||||
AccessCondition.generateEmptyCondition();
|
||||
accessCondition.setLeaseID(leaseID);
|
||||
|
||||
while(!leaseFreed) {
|
||||
try {
|
||||
Thread.sleep(LEASE_RENEWAL_PERIOD);
|
||||
} catch (InterruptedException e) {
|
||||
LOG.debug("Keep-alive thread for lease " + leaseID +
|
||||
" interrupted.");
|
||||
|
||||
// Restore the interrupted status
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
try {
|
||||
if (!leaseFreed) {
|
||||
blobWrapper.getBlob().renewLease(accessCondition);
|
||||
|
||||
// It'll be very rare to renew the lease (most will be short)
|
||||
// so log that we did it, to help with system debugging.
|
||||
LOG.info("Renewed lease " + leaseID + " on "
|
||||
+ getCloudBlob().getUri());
|
||||
}
|
||||
} catch (StorageException e) {
|
||||
if (!leaseFreed) {
|
||||
|
||||
// Free the lease so we don't leave this thread running forever.
|
||||
leaseFreed = true;
|
||||
|
||||
// Normally leases should be freed and there should be no
|
||||
// exceptions, so log a warning.
|
||||
LOG.warn("Attempt to renew lease " + leaseID + " on "
|
||||
+ getCloudBlob().getUri()
|
||||
+ " failed, but lease not yet freed. Reason: " +
|
||||
e.getMessage());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -68,12 +68,14 @@ public class SelfThrottlingIntercept {
|
|||
|
||||
private final float readFactor;
|
||||
private final float writeFactor;
|
||||
private final OperationContext operationContext;
|
||||
|
||||
// Concurrency: access to non-final members must be thread-safe
|
||||
private long lastE2Elatency;
|
||||
|
||||
public SelfThrottlingIntercept(OperationContext operationContext,
|
||||
float readFactor, float writeFactor) {
|
||||
this.operationContext = operationContext;
|
||||
this.readFactor = readFactor;
|
||||
this.writeFactor = writeFactor;
|
||||
}
|
||||
|
|
|
@ -31,7 +31,8 @@ import org.apache.hadoop.util.Shell;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ShellDecryptionKeyProvider extends SimpleKeyProvider {
|
||||
static final String KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT = "fs.azure.shellkeyprovider.script";
|
||||
static final String KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT =
|
||||
"fs.azure.shellkeyprovider.script";
|
||||
|
||||
@Override
|
||||
public String getStorageAccountKey(String accountName, Configuration conf)
|
||||
|
|
|
@ -28,7 +28,8 @@ import org.apache.hadoop.conf.Configuration;
|
|||
@InterfaceAudience.Private
|
||||
public class SimpleKeyProvider implements KeyProvider {
|
||||
|
||||
protected static final String KEY_ACCOUNT_KEY_PREFIX = "fs.azure.account.key.";
|
||||
protected static final String KEY_ACCOUNT_KEY_PREFIX =
|
||||
"fs.azure.account.key.";
|
||||
|
||||
@Override
|
||||
public String getStorageAccountKey(String accountName, Configuration conf)
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.io.InputStream;
|
|||
import java.io.OutputStream;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.EnumSet;
|
||||
import java.util.HashMap;
|
||||
|
||||
|
@ -36,8 +37,10 @@ import com.microsoft.windowsazure.storage.StorageException;
|
|||
import com.microsoft.windowsazure.storage.blob.BlobListingDetails;
|
||||
import com.microsoft.windowsazure.storage.blob.BlobProperties;
|
||||
import com.microsoft.windowsazure.storage.blob.BlobRequestOptions;
|
||||
import com.microsoft.windowsazure.storage.blob.CloudBlob;
|
||||
import com.microsoft.windowsazure.storage.blob.CopyState;
|
||||
import com.microsoft.windowsazure.storage.blob.ListBlobItem;
|
||||
import com.microsoft.windowsazure.storage.blob.PageRange;
|
||||
|
||||
/**
|
||||
* This is a very thin layer over the methods exposed by the Windows Azure
|
||||
|
@ -323,23 +326,39 @@ abstract class StorageInterface {
|
|||
* @throws URISyntaxException
|
||||
* If URI syntax exception occurred.
|
||||
*/
|
||||
public abstract CloudBlockBlobWrapper getBlockBlobReference(
|
||||
public abstract CloudBlobWrapper getBlockBlobReference(
|
||||
String relativePath) throws URISyntaxException, StorageException;
|
||||
}
|
||||
|
||||
/**
|
||||
* A thin wrapper over the {@link CloudBlockBlob} class that simply redirects
|
||||
* calls to the real object except in unit tests.
|
||||
* Returns a wrapper for a CloudPageBlob.
|
||||
*
|
||||
* @param relativePath
|
||||
* A <code>String</code> that represents the name of the blob, relative to the container
|
||||
*
|
||||
* @throws StorageException
|
||||
* If a storage service error occurred.
|
||||
*
|
||||
* @throws URISyntaxException
|
||||
* If URI syntax exception occurred.
|
||||
*/
|
||||
public abstract CloudBlobWrapper getPageBlobReference(String relativePath)
|
||||
throws URISyntaxException, StorageException;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* A thin wrapper over the {@link CloudBlob} class that simply redirects calls
|
||||
* to the real object except in unit tests.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public abstract static class CloudBlockBlobWrapper implements ListBlobItem {
|
||||
public interface CloudBlobWrapper extends ListBlobItem {
|
||||
/**
|
||||
* Returns the URI for this blob.
|
||||
*
|
||||
* @return A <code>java.net.URI</code> object that represents the URI for
|
||||
* the blob.
|
||||
*/
|
||||
public abstract URI getUri();
|
||||
URI getUri();
|
||||
|
||||
/**
|
||||
* Returns the metadata for the blob.
|
||||
|
@ -347,7 +366,7 @@ abstract class StorageInterface {
|
|||
* @return A <code>java.util.HashMap</code> object that represents the
|
||||
* metadata for the blob.
|
||||
*/
|
||||
public abstract HashMap<String, String> getMetadata();
|
||||
HashMap<String, String> getMetadata();
|
||||
|
||||
/**
|
||||
* Sets the metadata for the blob.
|
||||
|
@ -356,29 +375,27 @@ abstract class StorageInterface {
|
|||
* A <code>java.util.HashMap</code> object that contains the
|
||||
* metadata being assigned to the blob.
|
||||
*/
|
||||
public abstract void setMetadata(HashMap<String, String> metadata);
|
||||
void setMetadata(HashMap<String, String> metadata);
|
||||
|
||||
/**
|
||||
* Copies an existing blob's contents, properties, and metadata to this
|
||||
* instance of the <code>CloudBlob</code> class, using the specified
|
||||
* operation context.
|
||||
* Copies an existing blob's contents, properties, and metadata to this instance of the <code>CloudBlob</code>
|
||||
* class, using the specified operation context.
|
||||
*
|
||||
* @param sourceBlob
|
||||
* A <code>CloudBlob</code> object that represents the source blob
|
||||
* to copy.
|
||||
* @param source
|
||||
* A <code>java.net.URI</code> The URI of a source blob.
|
||||
* @param opContext
|
||||
* An {@link OperationContext} object that represents the context
|
||||
* for the current operation. This object is used to track requests
|
||||
* to the storage service, and to provide additional runtime
|
||||
* information about the operation.
|
||||
* An {@link OperationContext} object that represents the context for the current operation. This object
|
||||
* is used to track requests to the storage service, and to provide additional runtime information about
|
||||
* the operation.
|
||||
*
|
||||
* @throws StorageException
|
||||
* If a storage service error occurred.
|
||||
* @throws URISyntaxException
|
||||
*
|
||||
*/
|
||||
public abstract void startCopyFromBlob(CloudBlockBlobWrapper sourceBlob,
|
||||
OperationContext opContext) throws StorageException, URISyntaxException;
|
||||
public abstract void startCopyFromBlob(URI source,
|
||||
OperationContext opContext)
|
||||
throws StorageException, URISyntaxException;
|
||||
|
||||
/**
|
||||
* Returns the blob's copy state.
|
||||
|
@ -386,7 +403,36 @@ abstract class StorageInterface {
|
|||
* @return A {@link CopyState} object that represents the copy state of the
|
||||
* blob.
|
||||
*/
|
||||
public abstract CopyState getCopyState();
|
||||
CopyState getCopyState();
|
||||
|
||||
/**
|
||||
* Downloads a range of bytes from the blob to the given byte buffer, using the specified request options and
|
||||
* operation context.
|
||||
*
|
||||
* @param offset
|
||||
* The byte offset to use as the starting point for the source.
|
||||
* @param length
|
||||
* The number of bytes to read.
|
||||
* @param buffer
|
||||
* The byte buffer, as an array of bytes, to which the blob bytes are downloaded.
|
||||
* @param bufferOffset
|
||||
* The byte offset to use as the starting point for the target.
|
||||
* @param options
|
||||
* A {@link BlobRequestOptions} object that specifies any additional options for the request. Specifying
|
||||
* <code>null</code> will use the default request options from the associated service client (
|
||||
* {@link CloudBlobClient}).
|
||||
* @param opContext
|
||||
* An {@link OperationContext} object that represents the context for the current operation. This object
|
||||
* is used to track requests to the storage service, and to provide additional runtime information about
|
||||
* the operation.
|
||||
*
|
||||
* @throws StorageException
|
||||
* If a storage service error occurred.
|
||||
*/
|
||||
void downloadRange(final long offset, final long length,
|
||||
final OutputStream outStream, final BlobRequestOptions options,
|
||||
final OperationContext opContext)
|
||||
throws StorageException, IOException;
|
||||
|
||||
/**
|
||||
* Deletes the blob using the specified operation context.
|
||||
|
@ -407,7 +453,7 @@ abstract class StorageInterface {
|
|||
* @throws StorageException
|
||||
* If a storage service error occurred.
|
||||
*/
|
||||
public abstract void delete(OperationContext opContext)
|
||||
void delete(OperationContext opContext, SelfRenewingLease lease)
|
||||
throws StorageException;
|
||||
|
||||
/**
|
||||
|
@ -419,13 +465,13 @@ abstract class StorageInterface {
|
|||
* to the storage service, and to provide additional runtime
|
||||
* information about the operation.
|
||||
*
|
||||
* @return <code>true</code> if the blob exists, other wise
|
||||
* @return <code>true</code> if the blob exists, otherwise
|
||||
* <code>false</code>.
|
||||
*
|
||||
* @throws StorageException
|
||||
* f a storage service error occurred.
|
||||
* If a storage service error occurred.
|
||||
*/
|
||||
public abstract boolean exists(OperationContext opContext)
|
||||
boolean exists(OperationContext opContext)
|
||||
throws StorageException;
|
||||
|
||||
/**
|
||||
|
@ -446,7 +492,7 @@ abstract class StorageInterface {
|
|||
* @throws StorageException
|
||||
* If a storage service error occurred.
|
||||
*/
|
||||
public abstract void downloadAttributes(OperationContext opContext)
|
||||
void downloadAttributes(OperationContext opContext)
|
||||
throws StorageException;
|
||||
|
||||
/**
|
||||
|
@ -455,7 +501,7 @@ abstract class StorageInterface {
|
|||
* @return A {@link BlobProperties} object that represents the properties of
|
||||
* the blob.
|
||||
*/
|
||||
public abstract BlobProperties getProperties();
|
||||
BlobProperties getProperties();
|
||||
|
||||
/**
|
||||
* Opens a blob input stream to download the blob using the specified
|
||||
|
@ -476,48 +522,9 @@ abstract class StorageInterface {
|
|||
* @throws StorageException
|
||||
* If a storage service error occurred.
|
||||
*/
|
||||
public abstract InputStream openInputStream(BlobRequestOptions options,
|
||||
InputStream openInputStream(BlobRequestOptions options,
|
||||
OperationContext opContext) throws StorageException;
|
||||
|
||||
/**
|
||||
* Creates and opens an output stream to write data to the block blob using
|
||||
* the specified operation context.
|
||||
*
|
||||
* @param opContext
|
||||
* An {@link OperationContext} object that represents the context
|
||||
* for the current operation. This object is used to track requests
|
||||
* to the storage service, and to provide additional runtime
|
||||
* information about the operation.
|
||||
*
|
||||
* @return A {@link BlobOutputStream} object used to write data to the blob.
|
||||
*
|
||||
* @throws StorageException
|
||||
* If a storage service error occurred.
|
||||
*/
|
||||
public abstract OutputStream openOutputStream(BlobRequestOptions options,
|
||||
OperationContext opContext) throws StorageException;
|
||||
|
||||
/**
|
||||
* Uploads the source stream data to the blob, using the specified operation
|
||||
* context.
|
||||
*
|
||||
* @param sourceStream
|
||||
* An <code>InputStream</code> object that represents the input
|
||||
* stream to write to the block blob.
|
||||
* @param opContext
|
||||
* An {@link OperationContext} object that represents the context
|
||||
* for the current operation. This object is used to track requests
|
||||
* to the storage service, and to provide additional runtime
|
||||
* information about the operation.
|
||||
*
|
||||
* @throws IOException
|
||||
* If an I/O error occurred.
|
||||
* @throws StorageException
|
||||
* If a storage service error occurred.
|
||||
*/
|
||||
public abstract void upload(InputStream sourceStream,
|
||||
OperationContext opContext) throws StorageException, IOException;
|
||||
|
||||
/**
|
||||
* Uploads the blob's metadata to the storage service using the specified
|
||||
* lease ID, request options, and operation context.
|
||||
|
@ -531,12 +538,15 @@ abstract class StorageInterface {
|
|||
* @throws StorageException
|
||||
* If a storage service error occurred.
|
||||
*/
|
||||
public abstract void uploadMetadata(OperationContext opContext)
|
||||
void uploadMetadata(OperationContext opContext)
|
||||
throws StorageException;
|
||||
|
||||
public abstract void uploadProperties(OperationContext opContext)
|
||||
void uploadProperties(OperationContext opContext,
|
||||
SelfRenewingLease lease)
|
||||
throws StorageException;
|
||||
|
||||
SelfRenewingLease acquireLease() throws StorageException;
|
||||
|
||||
/**
|
||||
* Sets the minimum read block size to use with this Blob.
|
||||
*
|
||||
|
@ -545,7 +555,7 @@ abstract class StorageInterface {
|
|||
* while using a {@link BlobInputStream} object, ranging from 512
|
||||
* bytes to 64 MB, inclusive.
|
||||
*/
|
||||
public abstract void setStreamMinimumReadSizeInBytes(
|
||||
void setStreamMinimumReadSizeInBytes(
|
||||
int minimumReadSizeBytes);
|
||||
|
||||
/**
|
||||
|
@ -560,7 +570,121 @@ abstract class StorageInterface {
|
|||
* If <code>writeBlockSizeInBytes</code> is less than 1 MB or
|
||||
* greater than 4 MB.
|
||||
*/
|
||||
public abstract void setWriteBlockSizeInBytes(int writeBlockSizeBytes);
|
||||
void setWriteBlockSizeInBytes(int writeBlockSizeBytes);
|
||||
|
||||
CloudBlob getBlob();
|
||||
}
|
||||
|
||||
/**
|
||||
* A thin wrapper over the {@link CloudBlockBlob} class that simply redirects calls
|
||||
* to the real object except in unit tests.
|
||||
*/
|
||||
public abstract interface CloudBlockBlobWrapper
|
||||
extends CloudBlobWrapper {
|
||||
/**
|
||||
* Creates and opens an output stream to write data to the block blob using the specified
|
||||
* operation context.
|
||||
*
|
||||
* @param opContext
|
||||
* An {@link OperationContext} object that represents the context for the current operation. This object
|
||||
* is used to track requests to the storage service, and to provide additional runtime information about
|
||||
* the operation.
|
||||
*
|
||||
* @return A {@link BlobOutputStream} object used to write data to the blob.
|
||||
*
|
||||
* @throws StorageException
|
||||
* If a storage service error occurred.
|
||||
*/
|
||||
OutputStream openOutputStream(
|
||||
BlobRequestOptions options,
|
||||
OperationContext opContext) throws StorageException;
|
||||
}
|
||||
|
||||
/**
|
||||
* A thin wrapper over the {@link CloudPageBlob} class that simply redirects calls
|
||||
* to the real object except in unit tests.
|
||||
*/
|
||||
public abstract interface CloudPageBlobWrapper
|
||||
extends CloudBlobWrapper {
|
||||
/**
|
||||
* Creates a page blob using the specified request options and operation context.
|
||||
*
|
||||
* @param length
|
||||
* The size, in bytes, of the page blob.
|
||||
* @param options
|
||||
* A {@link BlobRequestOptions} object that specifies any additional options for the request. Specifying
|
||||
* <code>null</code> will use the default request options from the associated service client (
|
||||
* {@link CloudBlobClient}).
|
||||
* @param opContext
|
||||
* An {@link OperationContext} object that represents the context for the current operation. This object
|
||||
* is used to track requests to the storage service, and to provide additional runtime information about
|
||||
* the operation.
|
||||
*
|
||||
* @throws IllegalArgumentException
|
||||
* If the length is not a multiple of 512.
|
||||
*
|
||||
* @throws StorageException
|
||||
* If a storage service error occurred.
|
||||
*/
|
||||
void create(final long length, BlobRequestOptions options,
|
||||
OperationContext opContext) throws StorageException;
|
||||
|
||||
|
||||
/**
|
||||
* Uploads a range of contiguous pages, up to 4 MB in size, at the specified offset in the page blob, using the
|
||||
* specified lease ID, request options, and operation context.
|
||||
*
|
||||
* @param sourceStream
|
||||
* An <code>InputStream</code> object that represents the input stream to write to the page blob.
|
||||
* @param offset
|
||||
* The offset, in number of bytes, at which to begin writing the data. This value must be a multiple of
|
||||
* 512.
|
||||
* @param length
|
||||
* The length, in bytes, of the data to write. This value must be a multiple of 512.
|
||||
* @param options
|
||||
* A {@link BlobRequestOptions} object that specifies any additional options for the request. Specifying
|
||||
* <code>null</code> will use the default request options from the associated service client (
|
||||
* {@link CloudBlobClient}).
|
||||
* @param opContext
|
||||
* An {@link OperationContext} object that represents the context for the current operation. This object
|
||||
* is used to track requests to the storage service, and to provide additional runtime information about
|
||||
* the operation.
|
||||
*
|
||||
* @throws IllegalArgumentException
|
||||
* If the offset or length are not multiples of 512, or if the length is greater than 4 MB.
|
||||
* @throws IOException
|
||||
* If an I/O exception occurred.
|
||||
* @throws StorageException
|
||||
* If a storage service error occurred.
|
||||
*/
|
||||
void uploadPages(final InputStream sourceStream, final long offset,
|
||||
final long length, BlobRequestOptions options,
|
||||
OperationContext opContext) throws StorageException, IOException;
|
||||
|
||||
/**
|
||||
* Returns a collection of page ranges and their starting and ending byte offsets using the specified request
|
||||
* options and operation context.
|
||||
*
|
||||
* @param options
|
||||
* A {@link BlobRequestOptions} object that specifies any additional options for the request. Specifying
|
||||
* <code>null</code> will use the default request options from the associated service client (
|
||||
* {@link CloudBlobClient}).
|
||||
* @param opContext
|
||||
* An {@link OperationContext} object that represents the context for the current operation. This object
|
||||
* is used to track requests to the storage service, and to provide additional runtime information about
|
||||
* the operation.
|
||||
*
|
||||
* @return An <code>ArrayList</code> object that represents the set of page ranges and their starting and ending
|
||||
* byte offsets.
|
||||
*
|
||||
* @throws StorageException
|
||||
* If a storage service error occurred.
|
||||
*/
|
||||
|
||||
ArrayList<PageRange> downloadPageRanges(BlobRequestOptions options,
|
||||
OperationContext opContext) throws StorageException;
|
||||
|
||||
void uploadMetadata(OperationContext opContext)
|
||||
throws StorageException;
|
||||
}
|
||||
}
|
|
@ -23,6 +23,7 @@ import java.io.InputStream;
|
|||
import java.io.OutputStream;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.EnumSet;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
|
@ -39,13 +40,16 @@ import com.microsoft.windowsazure.storage.StorageUri;
|
|||
import com.microsoft.windowsazure.storage.blob.BlobListingDetails;
|
||||
import com.microsoft.windowsazure.storage.blob.BlobProperties;
|
||||
import com.microsoft.windowsazure.storage.blob.BlobRequestOptions;
|
||||
import com.microsoft.windowsazure.storage.blob.CloudBlob;
|
||||
import com.microsoft.windowsazure.storage.blob.CloudBlobClient;
|
||||
import com.microsoft.windowsazure.storage.blob.CloudBlobContainer;
|
||||
import com.microsoft.windowsazure.storage.blob.CloudBlobDirectory;
|
||||
import com.microsoft.windowsazure.storage.blob.CloudBlockBlob;
|
||||
import com.microsoft.windowsazure.storage.blob.CloudPageBlob;
|
||||
import com.microsoft.windowsazure.storage.blob.CopyState;
|
||||
import com.microsoft.windowsazure.storage.blob.DeleteSnapshotsOption;
|
||||
import com.microsoft.windowsazure.storage.blob.ListBlobItem;
|
||||
import com.microsoft.windowsazure.storage.blob.PageRange;
|
||||
|
||||
/**
|
||||
* A real implementation of the Azure interaction layer that just redirects
|
||||
|
@ -129,6 +133,8 @@ class StorageInterfaceImpl extends StorageInterface {
|
|||
return new CloudBlobDirectoryWrapperImpl((CloudBlobDirectory) unwrapped);
|
||||
} else if (unwrapped instanceof CloudBlockBlob) {
|
||||
return new CloudBlockBlobWrapperImpl((CloudBlockBlob) unwrapped);
|
||||
} else if (unwrapped instanceof CloudPageBlob) {
|
||||
return new CloudPageBlobWrapperImpl((CloudPageBlob) unwrapped);
|
||||
} else {
|
||||
return unwrapped;
|
||||
}
|
||||
|
@ -244,129 +250,217 @@ class StorageInterfaceImpl extends StorageInterface {
|
|||
}
|
||||
|
||||
@Override
|
||||
public CloudBlockBlobWrapper getBlockBlobReference(String relativePath)
|
||||
public CloudBlobWrapper getBlockBlobReference(String relativePath)
|
||||
throws URISyntaxException, StorageException {
|
||||
|
||||
return new CloudBlockBlobWrapperImpl(
|
||||
container.getBlockBlobReference(relativePath));
|
||||
}
|
||||
return new CloudBlockBlobWrapperImpl(container.getBlockBlobReference(relativePath));
|
||||
}
|
||||
|
||||
//
|
||||
// CloudBlockBlobWrapperImpl
|
||||
//
|
||||
@InterfaceAudience.Private
|
||||
static class CloudBlockBlobWrapperImpl extends CloudBlockBlobWrapper {
|
||||
private final CloudBlockBlob blob;
|
||||
@Override
|
||||
public CloudBlobWrapper getPageBlobReference(String relativePath)
|
||||
throws URISyntaxException, StorageException {
|
||||
return new CloudPageBlobWrapperImpl(
|
||||
container.getPageBlobReference(relativePath));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
abstract static class CloudBlobWrapperImpl implements CloudBlobWrapper {
|
||||
private final CloudBlob blob;
|
||||
|
||||
@Override
|
||||
public CloudBlob getBlob() {
|
||||
return blob;
|
||||
}
|
||||
|
||||
public URI getUri() {
|
||||
return blob.getUri();
|
||||
return getBlob().getUri();
|
||||
}
|
||||
|
||||
public CloudBlockBlobWrapperImpl(CloudBlockBlob blob) {
|
||||
protected CloudBlobWrapperImpl(CloudBlob blob) {
|
||||
this.blob = blob;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HashMap<String, String> getMetadata() {
|
||||
return blob.getMetadata();
|
||||
return getBlob().getMetadata();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startCopyFromBlob(CloudBlockBlobWrapper sourceBlob,
|
||||
OperationContext opContext) throws StorageException, URISyntaxException {
|
||||
|
||||
blob.startCopyFromBlob(((CloudBlockBlobWrapperImpl) sourceBlob).blob,
|
||||
null, null, null, opContext);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void delete(OperationContext opContext) throws StorageException {
|
||||
blob.delete(DeleteSnapshotsOption.NONE, null, null, opContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean exists(OperationContext opContext) throws StorageException {
|
||||
return blob.exists(null, null, opContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void downloadAttributes(OperationContext opContext)
|
||||
public void delete(OperationContext opContext, SelfRenewingLease lease)
|
||||
throws StorageException {
|
||||
blob.downloadAttributes(null, null, opContext);
|
||||
getBlob().delete(DeleteSnapshotsOption.NONE, getLeaseCondition(lease),
|
||||
null, opContext);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return and access condition for this lease, or else null if
|
||||
* there's no lease.
|
||||
*/
|
||||
private AccessCondition getLeaseCondition(SelfRenewingLease lease) {
|
||||
AccessCondition leaseCondition = null;
|
||||
if (lease != null) {
|
||||
leaseCondition = AccessCondition.generateLeaseCondition(lease.getLeaseID());
|
||||
}
|
||||
return leaseCondition;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean exists(OperationContext opContext)
|
||||
throws StorageException {
|
||||
return getBlob().exists(null, null, opContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void downloadAttributes(
|
||||
OperationContext opContext) throws StorageException {
|
||||
getBlob().downloadAttributes(null, null, opContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BlobProperties getProperties() {
|
||||
return blob.getProperties();
|
||||
return getBlob().getProperties();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setMetadata(HashMap<String, String> metadata) {
|
||||
blob.setMetadata(metadata);
|
||||
getBlob().setMetadata(metadata);
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputStream openInputStream(BlobRequestOptions options,
|
||||
public InputStream openInputStream(
|
||||
BlobRequestOptions options,
|
||||
OperationContext opContext) throws StorageException {
|
||||
return blob.openInputStream(null, options, opContext);
|
||||
return getBlob().openInputStream(null, options, opContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public OutputStream openOutputStream(BlobRequestOptions options,
|
||||
public OutputStream openOutputStream(
|
||||
BlobRequestOptions options,
|
||||
OperationContext opContext) throws StorageException {
|
||||
return blob.openOutputStream(null, options, opContext);
|
||||
return ((CloudBlockBlob) getBlob()).openOutputStream(null, options, opContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void upload(InputStream sourceStream, OperationContext opContext)
|
||||
throws StorageException, IOException {
|
||||
blob.upload(sourceStream, 0, null, null, opContext);
|
||||
getBlob().upload(sourceStream, 0, null, null, opContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CloudBlobContainer getContainer() throws URISyntaxException,
|
||||
StorageException {
|
||||
return blob.getContainer();
|
||||
return getBlob().getContainer();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CloudBlobDirectory getParent() throws URISyntaxException,
|
||||
StorageException {
|
||||
return blob.getParent();
|
||||
return getBlob().getParent();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void uploadMetadata(OperationContext opContext)
|
||||
throws StorageException {
|
||||
blob.uploadMetadata(null, null, opContext);
|
||||
getBlob().uploadMetadata(null, null, opContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void uploadProperties(OperationContext opContext)
|
||||
public void uploadProperties(OperationContext opContext, SelfRenewingLease lease)
|
||||
throws StorageException {
|
||||
blob.uploadProperties(null, null, opContext);
|
||||
|
||||
// Include lease in request if lease not null.
|
||||
getBlob().uploadProperties(getLeaseCondition(lease), null, opContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setStreamMinimumReadSizeInBytes(int minimumReadSizeBytes) {
|
||||
blob.setStreamMinimumReadSizeInBytes(minimumReadSizeBytes);
|
||||
getBlob().setStreamMinimumReadSizeInBytes(minimumReadSizeBytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setWriteBlockSizeInBytes(int writeBlockSizeBytes) {
|
||||
blob.setStreamWriteSizeInBytes(writeBlockSizeBytes);
|
||||
getBlob().setStreamWriteSizeInBytes(writeBlockSizeBytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public StorageUri getStorageUri() {
|
||||
return blob.getStorageUri();
|
||||
return getBlob().getStorageUri();
|
||||
}
|
||||
|
||||
@Override
|
||||
public CopyState getCopyState() {
|
||||
return blob.getCopyState();
|
||||
return getBlob().getCopyState();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startCopyFromBlob(URI source,
|
||||
OperationContext opContext)
|
||||
throws StorageException, URISyntaxException {
|
||||
getBlob().startCopyFromBlob(source,
|
||||
null, null, null, opContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void downloadRange(long offset, long length, OutputStream outStream,
|
||||
BlobRequestOptions options, OperationContext opContext)
|
||||
throws StorageException, IOException {
|
||||
|
||||
getBlob().downloadRange(offset, length, outStream, null, options, opContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SelfRenewingLease acquireLease() throws StorageException {
|
||||
return new SelfRenewingLease(this);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
//
|
||||
// CloudBlockBlobWrapperImpl
|
||||
//
|
||||
|
||||
static class CloudBlockBlobWrapperImpl extends CloudBlobWrapperImpl implements CloudBlockBlobWrapper {
|
||||
public CloudBlockBlobWrapperImpl(CloudBlockBlob blob) {
|
||||
super(blob);
|
||||
}
|
||||
|
||||
public OutputStream openOutputStream(
|
||||
BlobRequestOptions options,
|
||||
OperationContext opContext) throws StorageException {
|
||||
return ((CloudBlockBlob) getBlob()).openOutputStream(null, options, opContext);
|
||||
}
|
||||
|
||||
public void upload(InputStream sourceStream, OperationContext opContext)
|
||||
throws StorageException, IOException {
|
||||
getBlob().upload(sourceStream, 0, null, null, opContext);
|
||||
}
|
||||
|
||||
public void uploadProperties(OperationContext opContext)
|
||||
throws StorageException {
|
||||
getBlob().uploadProperties(null, null, opContext);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
static class CloudPageBlobWrapperImpl extends CloudBlobWrapperImpl implements CloudPageBlobWrapper {
|
||||
public CloudPageBlobWrapperImpl(CloudPageBlob blob) {
|
||||
super(blob);
|
||||
}
|
||||
|
||||
public void create(final long length, BlobRequestOptions options,
|
||||
OperationContext opContext) throws StorageException {
|
||||
((CloudPageBlob) getBlob()).create(length, null, options, opContext);
|
||||
}
|
||||
|
||||
public void uploadPages(final InputStream sourceStream, final long offset,
|
||||
final long length, BlobRequestOptions options, OperationContext opContext)
|
||||
throws StorageException, IOException {
|
||||
((CloudPageBlob) getBlob()).uploadPages(sourceStream, offset, length, null,
|
||||
options, opContext);
|
||||
}
|
||||
|
||||
public ArrayList<PageRange> downloadPageRanges(BlobRequestOptions options,
|
||||
OperationContext opContext) throws StorageException {
|
||||
return ((CloudPageBlob) getBlob()).downloadPageRanges(
|
||||
null, options, opContext);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,56 @@
|
|||
/**
|
||||
* 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.apache.hadoop.fs.azure;
|
||||
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
|
||||
import org.apache.hadoop.fs.Syncable;
|
||||
|
||||
/**
|
||||
* Support the Syncable interface on top of a DataOutputStream.
|
||||
* This allows passing the sync/hflush/hsync calls through to the
|
||||
* wrapped stream passed in to the constructor. This is required
|
||||
* for HBase when wrapping a PageBlobOutputStream used as a write-ahead log.
|
||||
*/
|
||||
public class SyncableDataOutputStream extends DataOutputStream implements Syncable {
|
||||
|
||||
public SyncableDataOutputStream(OutputStream out) {
|
||||
super(out);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void hflush() throws IOException {
|
||||
if (out instanceof Syncable) {
|
||||
((Syncable) out).hflush();
|
||||
} else {
|
||||
out.flush();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void hsync() throws IOException {
|
||||
if (out instanceof Syncable) {
|
||||
((Syncable) out).hsync();
|
||||
} else {
|
||||
out.flush();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -27,7 +27,6 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.DelegateToFileSystem;
|
||||
|
||||
|
||||
/**
|
||||
* WASB implementation of AbstractFileSystem.
|
||||
* This impl delegates to the old FileSystem
|
||||
|
|
|
@ -381,7 +381,6 @@ public final class AzureFileSystemInstrumentation implements MetricsSource {
|
|||
*/
|
||||
public long getCurrentMaximumDownloadBandwidth() {
|
||||
return currentMaximumDownloadBytesPerSecond;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -33,8 +33,7 @@ import com.microsoft.windowsazure.storage.StorageEvent;
|
|||
|
||||
/**
|
||||
* An event listener to the ResponseReceived event from Azure Storage that will
|
||||
* update metrics appropriately.
|
||||
*
|
||||
* update metrics appropriately when it gets that event.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public final class ResponseReceivedMetricUpdater extends StorageEvent<ResponseReceivedEvent> {
|
||||
|
|
|
@ -0,0 +1,17 @@
|
|||
# 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.
|
||||
|
||||
org.apache.hadoop.fs.azure.NativeAzureFileSystem
|
||||
org.apache.hadoop.fs.azure.NativeAzureFileSystem$Secure
|
|
@ -78,6 +78,8 @@ public final class AzureBlobStorageTestAccount {
|
|||
|
||||
private static final String KEY_DISABLE_THROTTLING = "fs.azure.disable.bandwidth.throttling";
|
||||
private static final String KEY_READ_TOLERATE_CONCURRENT_APPEND = "fs.azure.io.read.tolerate.concurrent.append";
|
||||
public static final String DEFAULT_PAGE_BLOB_DIRECTORY = "pageBlobs";
|
||||
public static final String DEFAULT_ATOMIC_RENAME_DIRECTORIES = "/atomicRenameDir1,/atomicRenameDir2";
|
||||
|
||||
private CloudStorageAccount account;
|
||||
private CloudBlobContainer container;
|
||||
|
@ -85,12 +87,14 @@ public final class AzureBlobStorageTestAccount {
|
|||
private NativeAzureFileSystem fs;
|
||||
private AzureNativeFileSystemStore storage;
|
||||
private MockStorageInterface mockStorage;
|
||||
private String pageBlobDirectory;
|
||||
private static final ConcurrentLinkedQueue<MetricsRecord> allMetrics =
|
||||
new ConcurrentLinkedQueue<MetricsRecord>();
|
||||
|
||||
private static boolean metricsConfigSaved = false;
|
||||
|
||||
private AzureBlobStorageTestAccount(NativeAzureFileSystem fs,
|
||||
CloudStorageAccount account, CloudBlobContainer container) {
|
||||
CloudStorageAccount account,
|
||||
CloudBlobContainer container) {
|
||||
this.account = account;
|
||||
this.container = container;
|
||||
this.fs = fs;
|
||||
|
@ -158,6 +162,14 @@ public final class AzureBlobStorageTestAccount {
|
|||
return toMockUri(path.toUri().getRawPath().substring(1));
|
||||
}
|
||||
|
||||
public static Path pageBlobPath() {
|
||||
return new Path("/" + DEFAULT_PAGE_BLOB_DIRECTORY);
|
||||
}
|
||||
|
||||
public static Path pageBlobPath(String fileName) {
|
||||
return new Path(pageBlobPath(), fileName);
|
||||
}
|
||||
|
||||
public Number getLatestMetricValue(String metricName, Number defaultValue)
|
||||
throws IndexOutOfBoundsException{
|
||||
boolean found = false;
|
||||
|
@ -206,8 +218,10 @@ public final class AzureBlobStorageTestAccount {
|
|||
* The blob key (no initial slash).
|
||||
* @return The blob reference.
|
||||
*/
|
||||
public CloudBlockBlob getBlobReference(String blobKey) throws Exception {
|
||||
return container.getBlockBlobReference(String.format(blobKey));
|
||||
public CloudBlockBlob getBlobReference(String blobKey)
|
||||
throws Exception {
|
||||
return container.getBlockBlobReference(
|
||||
String.format(blobKey));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -233,26 +247,58 @@ public final class AzureBlobStorageTestAccount {
|
|||
getBlobReference(blobKey).releaseLease(accessCondition);
|
||||
}
|
||||
|
||||
private static void saveMetricsConfigFile() {
|
||||
if (!metricsConfigSaved) {
|
||||
new org.apache.hadoop.metrics2.impl.ConfigBuilder()
|
||||
.add("azure-file-system.sink.azuretestcollector.class",
|
||||
StandardCollector.class.getName())
|
||||
.save("hadoop-metrics2-azure-file-system.properties");
|
||||
metricsConfigSaved = true;
|
||||
}
|
||||
}
|
||||
|
||||
public static AzureBlobStorageTestAccount createMock() throws Exception {
|
||||
return createMock(new Configuration());
|
||||
}
|
||||
|
||||
public static AzureBlobStorageTestAccount createMock(Configuration conf)
|
||||
throws Exception {
|
||||
public static AzureBlobStorageTestAccount createMock(Configuration conf) throws Exception {
|
||||
saveMetricsConfigFile();
|
||||
configurePageBlobDir(conf);
|
||||
configureAtomicRenameDir(conf);
|
||||
AzureNativeFileSystemStore store = new AzureNativeFileSystemStore();
|
||||
MockStorageInterface mockStorage = new MockStorageInterface();
|
||||
store.setAzureStorageInteractionLayer(mockStorage);
|
||||
NativeAzureFileSystem fs = new NativeAzureFileSystem(store);
|
||||
addWasbToConfiguration(conf);
|
||||
setMockAccountKey(conf);
|
||||
// register the fs provider.
|
||||
|
||||
fs.initialize(new URI(MOCK_WASB_URI), conf);
|
||||
AzureBlobStorageTestAccount testAcct = new AzureBlobStorageTestAccount(fs,
|
||||
mockStorage);
|
||||
AzureBlobStorageTestAccount testAcct =
|
||||
new AzureBlobStorageTestAccount(fs, mockStorage);
|
||||
return testAcct;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the page blob directories configuration to the default if it is not
|
||||
* already set. Some tests may set it differently (e.g. the page blob
|
||||
* tests in TestNativeAzureFSPageBlobLive).
|
||||
* @param conf The configuration to conditionally update.
|
||||
*/
|
||||
private static void configurePageBlobDir(Configuration conf) {
|
||||
if (conf.get(AzureNativeFileSystemStore.KEY_PAGE_BLOB_DIRECTORIES) == null) {
|
||||
conf.set(AzureNativeFileSystemStore.KEY_PAGE_BLOB_DIRECTORIES,
|
||||
"/" + DEFAULT_PAGE_BLOB_DIRECTORY);
|
||||
}
|
||||
}
|
||||
|
||||
/** Do the same for the atomic rename directories configuration */
|
||||
private static void configureAtomicRenameDir(Configuration conf) {
|
||||
if (conf.get(AzureNativeFileSystemStore.KEY_ATOMIC_RENAME_DIRECTORIES) == null) {
|
||||
conf.set(AzureNativeFileSystemStore.KEY_ATOMIC_RENAME_DIRECTORIES,
|
||||
DEFAULT_ATOMIC_RENAME_DIRECTORIES);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a test account that goes against the storage emulator.
|
||||
*
|
||||
|
@ -260,18 +306,20 @@ public final class AzureBlobStorageTestAccount {
|
|||
*/
|
||||
public static AzureBlobStorageTestAccount createForEmulator()
|
||||
throws Exception {
|
||||
saveMetricsConfigFile();
|
||||
NativeAzureFileSystem fs = null;
|
||||
CloudBlobContainer container = null;
|
||||
Configuration conf = createTestConfiguration();
|
||||
if (!conf.getBoolean(USE_EMULATOR_PROPERTY_NAME, false)) {
|
||||
// Not configured to test against the storage emulator.
|
||||
System.out.println("Skipping emulator Azure test because configuration "
|
||||
+ "doesn't indicate that it's running."
|
||||
+ " Please see README.txt for guidance.");
|
||||
System.out
|
||||
.println("Skipping emulator Azure test because configuration " +
|
||||
"doesn't indicate that it's running." +
|
||||
" Please see RunningLiveWasbTests.txt for guidance.");
|
||||
return null;
|
||||
}
|
||||
CloudStorageAccount account = CloudStorageAccount
|
||||
.getDevelopmentStorageAccount();
|
||||
CloudStorageAccount account =
|
||||
CloudStorageAccount.getDevelopmentStorageAccount();
|
||||
fs = new NativeAzureFileSystem();
|
||||
String containerName = String.format("wasbtests-%s-%tQ",
|
||||
System.getProperty("user.name"), new Date());
|
||||
|
@ -285,14 +333,18 @@ public final class AzureBlobStorageTestAccount {
|
|||
fs.initialize(accountUri, conf);
|
||||
|
||||
// Create test account initializing the appropriate member variables.
|
||||
AzureBlobStorageTestAccount testAcct = new AzureBlobStorageTestAccount(fs,
|
||||
account, container);
|
||||
//
|
||||
AzureBlobStorageTestAccount testAcct =
|
||||
new AzureBlobStorageTestAccount(fs, account, container);
|
||||
|
||||
return testAcct;
|
||||
}
|
||||
|
||||
public static AzureBlobStorageTestAccount createOutOfBandStore(
|
||||
int uploadBlockSize, int downloadBlockSize) throws Exception {
|
||||
|
||||
saveMetricsConfigFile();
|
||||
|
||||
CloudBlobContainer container = null;
|
||||
Configuration conf = createTestConfiguration();
|
||||
CloudStorageAccount account = createTestAccount(conf);
|
||||
|
@ -337,8 +389,9 @@ public final class AzureBlobStorageTestAccount {
|
|||
testStorage.initialize(accountUri, conf, instrumentation);
|
||||
|
||||
// Create test account initializing the appropriate member variables.
|
||||
AzureBlobStorageTestAccount testAcct = new AzureBlobStorageTestAccount(
|
||||
testStorage, account, container);
|
||||
//
|
||||
AzureBlobStorageTestAccount testAcct =
|
||||
new AzureBlobStorageTestAccount(testStorage, account, container);
|
||||
|
||||
return testAcct;
|
||||
}
|
||||
|
@ -416,11 +469,11 @@ public final class AzureBlobStorageTestAccount {
|
|||
}
|
||||
}
|
||||
|
||||
private static Configuration createTestConfiguration() {
|
||||
public static Configuration createTestConfiguration() {
|
||||
return createTestConfiguration(null);
|
||||
}
|
||||
|
||||
protected static Configuration createTestConfiguration(Configuration conf) {
|
||||
private static Configuration createTestConfiguration(Configuration conf) {
|
||||
if (conf == null) {
|
||||
conf = new Configuration();
|
||||
}
|
||||
|
@ -429,16 +482,9 @@ public final class AzureBlobStorageTestAccount {
|
|||
return conf;
|
||||
}
|
||||
|
||||
// for programmatic setting of the wasb configuration.
|
||||
// note that tests can also get the
|
||||
public static void addWasbToConfiguration(Configuration conf) {
|
||||
conf.set("fs.wasb.impl", "org.apache.hadoop.fs.azure.NativeAzureFileSystem");
|
||||
conf.set("fs.wasbs.impl",
|
||||
"org.apache.hadoop.fs.azure.NativeAzureFileSystem");
|
||||
}
|
||||
|
||||
static CloudStorageAccount createTestAccount() throws URISyntaxException,
|
||||
KeyProviderException {
|
||||
static CloudStorageAccount createTestAccount()
|
||||
throws URISyntaxException, KeyProviderException
|
||||
{
|
||||
return createTestAccount(createTestConfiguration());
|
||||
}
|
||||
|
||||
|
@ -447,8 +493,8 @@ public final class AzureBlobStorageTestAccount {
|
|||
String testAccountName = conf.get(TEST_ACCOUNT_NAME_PROPERTY_NAME);
|
||||
if (testAccountName == null) {
|
||||
System.out
|
||||
.println("Skipping live Azure test because of missing test account."
|
||||
+ " Please see README.txt for guidance.");
|
||||
.println("Skipping live Azure test because of missing test account." +
|
||||
" Please see RunningLiveWasbTests.txt for guidance.");
|
||||
return null;
|
||||
}
|
||||
return createStorageAccount(testAccountName, conf, false);
|
||||
|
@ -466,9 +512,12 @@ public final class AzureBlobStorageTestAccount {
|
|||
public static AzureBlobStorageTestAccount create(String containerNameSuffix,
|
||||
EnumSet<CreateOptions> createOptions, Configuration initialConfiguration)
|
||||
throws Exception {
|
||||
saveMetricsConfigFile();
|
||||
NativeAzureFileSystem fs = null;
|
||||
CloudBlobContainer container = null;
|
||||
Configuration conf = createTestConfiguration(initialConfiguration);
|
||||
configurePageBlobDir(conf);
|
||||
configureAtomicRenameDir(conf);
|
||||
CloudStorageAccount account = createTestAccount(conf);
|
||||
if (account == null) {
|
||||
return null;
|
||||
|
@ -510,15 +559,18 @@ public final class AzureBlobStorageTestAccount {
|
|||
fs.initialize(accountUri, conf);
|
||||
|
||||
// Create test account initializing the appropriate member variables.
|
||||
AzureBlobStorageTestAccount testAcct = new AzureBlobStorageTestAccount(fs,
|
||||
account, container);
|
||||
//
|
||||
AzureBlobStorageTestAccount testAcct =
|
||||
new AzureBlobStorageTestAccount(fs, account, container);
|
||||
|
||||
return testAcct;
|
||||
}
|
||||
|
||||
private static String generateContainerName() throws Exception {
|
||||
String containerName = String.format("wasbtests-%s-%tQ",
|
||||
System.getProperty("user.name"), new Date());
|
||||
String containerName =
|
||||
String.format ("wasbtests-%s-%tQ",
|
||||
System.getProperty("user.name"),
|
||||
new Date());
|
||||
return containerName;
|
||||
}
|
||||
|
||||
|
@ -548,12 +600,16 @@ public final class AzureBlobStorageTestAccount {
|
|||
|
||||
if (readonly) {
|
||||
// Set READ permissions
|
||||
sasPolicy.setPermissions(EnumSet.of(SharedAccessBlobPermissions.READ,
|
||||
sasPolicy.setPermissions(EnumSet.of(
|
||||
SharedAccessBlobPermissions.READ,
|
||||
SharedAccessBlobPermissions.LIST));
|
||||
} else {
|
||||
// Set READ and WRITE permissions.
|
||||
sasPolicy.setPermissions(EnumSet.of(SharedAccessBlobPermissions.READ,
|
||||
SharedAccessBlobPermissions.WRITE, SharedAccessBlobPermissions.LIST));
|
||||
//
|
||||
sasPolicy.setPermissions(EnumSet.of(
|
||||
SharedAccessBlobPermissions.READ,
|
||||
SharedAccessBlobPermissions.WRITE,
|
||||
SharedAccessBlobPermissions.LIST));
|
||||
}
|
||||
|
||||
// Create the container permissions.
|
||||
|
@ -590,8 +646,11 @@ public final class AzureBlobStorageTestAccount {
|
|||
SharedAccessBlobPolicy sasPolicy = new SharedAccessBlobPolicy();
|
||||
|
||||
// Set READ and WRITE permissions.
|
||||
sasPolicy.setPermissions(EnumSet.of(SharedAccessBlobPermissions.READ,
|
||||
SharedAccessBlobPermissions.WRITE, SharedAccessBlobPermissions.LIST,
|
||||
//
|
||||
sasPolicy.setPermissions(EnumSet.of(
|
||||
SharedAccessBlobPermissions.READ,
|
||||
SharedAccessBlobPermissions.WRITE,
|
||||
SharedAccessBlobPermissions.LIST,
|
||||
SharedAccessBlobPermissions.DELETE));
|
||||
|
||||
// Create the container permissions.
|
||||
|
@ -725,8 +784,9 @@ public final class AzureBlobStorageTestAccount {
|
|||
|
||||
// Create test account initializing the appropriate member variables.
|
||||
// Set the container value to null for the default root container.
|
||||
AzureBlobStorageTestAccount testAcct = new AzureBlobStorageTestAccount(fs,
|
||||
account, blobRoot);
|
||||
//
|
||||
AzureBlobStorageTestAccount testAcct = new AzureBlobStorageTestAccount(
|
||||
fs, account, blobRoot);
|
||||
|
||||
// Return to caller with test account.
|
||||
return testAcct;
|
||||
|
@ -806,4 +866,11 @@ public final class AzureBlobStorageTestAccount {
|
|||
}
|
||||
}
|
||||
|
||||
public void setPageBlobDirectory(String directory) {
|
||||
this.pageBlobDirectory = directory;
|
||||
}
|
||||
|
||||
public String getPageBlobDirectory() {
|
||||
return pageBlobDirectory;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -41,12 +41,15 @@ public class InMemoryBlockBlobStore {
|
|||
private final String key;
|
||||
private final HashMap<String, String> metadata;
|
||||
private final int contentLength;
|
||||
private final boolean isPageBlob;
|
||||
|
||||
|
||||
ListBlobEntry(String key, HashMap<String, String> metadata,
|
||||
int contentLength) {
|
||||
int contentLength, boolean isPageBlob) {
|
||||
this.key = key;
|
||||
this.metadata = metadata;
|
||||
this.contentLength = contentLength;
|
||||
this.isPageBlob = isPageBlob;
|
||||
}
|
||||
|
||||
public String getKey() {
|
||||
|
@ -60,6 +63,10 @@ public class InMemoryBlockBlobStore {
|
|||
public int getContentLength() {
|
||||
return contentLength;
|
||||
}
|
||||
|
||||
public boolean isPageBlob() {
|
||||
return isPageBlob;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -77,10 +84,13 @@ public class InMemoryBlockBlobStore {
|
|||
ArrayList<ListBlobEntry> list = new ArrayList<ListBlobEntry>();
|
||||
for (Map.Entry<String, Entry> entry : blobs.entrySet()) {
|
||||
if (entry.getKey().startsWith(prefix)) {
|
||||
list.add(new ListBlobEntry(entry.getKey(),
|
||||
includeMetadata ? new HashMap<String, String>(
|
||||
entry.getValue().metadata) : null,
|
||||
entry.getValue().content.length));
|
||||
list.add(new ListBlobEntry(
|
||||
entry.getKey(),
|
||||
includeMetadata ?
|
||||
new HashMap<String, String>(entry.getValue().metadata) :
|
||||
null,
|
||||
entry.getValue().content.length,
|
||||
entry.getValue().isPageBlob));
|
||||
}
|
||||
}
|
||||
return list;
|
||||
|
@ -92,19 +102,49 @@ public class InMemoryBlockBlobStore {
|
|||
|
||||
@SuppressWarnings("unchecked")
|
||||
public synchronized void setContent(String key, byte[] value,
|
||||
HashMap<String, String> metadata) {
|
||||
blobs
|
||||
.put(key, new Entry(value, (HashMap<String, String>) metadata.clone()));
|
||||
HashMap<String, String> metadata, boolean isPageBlob,
|
||||
long length) {
|
||||
blobs.put(key, new Entry(value, (HashMap<String, String>)metadata.clone(),
|
||||
isPageBlob, length));
|
||||
}
|
||||
|
||||
public OutputStream upload(final String key,
|
||||
@SuppressWarnings("unchecked")
|
||||
public synchronized void setMetadata(String key,
|
||||
HashMap<String, String> metadata) {
|
||||
blobs.get(key).metadata = (HashMap<String, String>) metadata.clone();
|
||||
}
|
||||
|
||||
public OutputStream uploadBlockBlob(final String key,
|
||||
final HashMap<String, String> metadata) {
|
||||
setContent(key, new byte[0], metadata);
|
||||
setContent(key, new byte[0], metadata, false, 0);
|
||||
return new ByteArrayOutputStream() {
|
||||
@Override
|
||||
public void flush() throws IOException {
|
||||
public void flush()
|
||||
throws IOException {
|
||||
super.flush();
|
||||
setContent(key, toByteArray(), metadata);
|
||||
byte[] tempBytes = toByteArray();
|
||||
setContent(key, tempBytes, metadata, false, tempBytes.length);
|
||||
}
|
||||
@Override
|
||||
public void close()
|
||||
throws IOException {
|
||||
super.close();
|
||||
byte[] tempBytes = toByteArray();
|
||||
setContent(key, tempBytes, metadata, false, tempBytes.length);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public OutputStream uploadPageBlob(final String key,
|
||||
final HashMap<String, String> metadata,
|
||||
final long length) {
|
||||
setContent(key, new byte[0], metadata, true, length);
|
||||
return new ByteArrayOutputStream() {
|
||||
@Override
|
||||
public void flush()
|
||||
throws IOException {
|
||||
super.flush();
|
||||
setContent(key, toByteArray(), metadata, true, length);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -137,10 +177,16 @@ public class InMemoryBlockBlobStore {
|
|||
private static class Entry {
|
||||
private byte[] content;
|
||||
private HashMap<String, String> metadata;
|
||||
private boolean isPageBlob;
|
||||
@SuppressWarnings("unused") // TODO: use it
|
||||
private long length;
|
||||
|
||||
public Entry(byte[] content, HashMap<String, String> metadata) {
|
||||
public Entry(byte[] content, HashMap<String, String> metadata,
|
||||
boolean isPageBlob, long length) {
|
||||
this.content = content;
|
||||
this.metadata = metadata;
|
||||
this.isPageBlob = isPageBlob;
|
||||
this.length = length;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,7 +33,7 @@ import java.util.TimeZone;
|
|||
|
||||
import org.apache.commons.httpclient.URIException;
|
||||
import org.apache.commons.httpclient.util.URIUtil;
|
||||
import org.apache.commons.io.output.ByteArrayOutputStream;
|
||||
import org.apache.commons.lang.NotImplementedException;
|
||||
|
||||
import com.microsoft.windowsazure.storage.CloudStorageAccount;
|
||||
import com.microsoft.windowsazure.storage.OperationContext;
|
||||
|
@ -44,10 +44,15 @@ import com.microsoft.windowsazure.storage.StorageUri;
|
|||
import com.microsoft.windowsazure.storage.blob.BlobListingDetails;
|
||||
import com.microsoft.windowsazure.storage.blob.BlobProperties;
|
||||
import com.microsoft.windowsazure.storage.blob.BlobRequestOptions;
|
||||
import com.microsoft.windowsazure.storage.blob.CloudBlob;
|
||||
import com.microsoft.windowsazure.storage.blob.CloudBlobContainer;
|
||||
import com.microsoft.windowsazure.storage.blob.CloudBlobDirectory;
|
||||
import com.microsoft.windowsazure.storage.blob.CopyState;
|
||||
import com.microsoft.windowsazure.storage.blob.ListBlobItem;
|
||||
import com.microsoft.windowsazure.storage.blob.PageRange;
|
||||
|
||||
import javax.ws.rs.core.UriBuilder;
|
||||
import javax.ws.rs.core.UriBuilderException;
|
||||
|
||||
/**
|
||||
* A mock implementation of the Azure Storage interaction layer for unit tests.
|
||||
|
@ -55,7 +60,8 @@ import com.microsoft.windowsazure.storage.blob.ListBlobItem;
|
|||
*/
|
||||
public class MockStorageInterface extends StorageInterface {
|
||||
private InMemoryBlockBlobStore backingStore;
|
||||
private final ArrayList<PreExistingContainer> preExistingContainers = new ArrayList<MockStorageInterface.PreExistingContainer>();
|
||||
private final ArrayList<PreExistingContainer> preExistingContainers =
|
||||
new ArrayList<MockStorageInterface.PreExistingContainer>();
|
||||
private String baseUriString;
|
||||
|
||||
public InMemoryBlockBlobStore getBackingStore() {
|
||||
|
@ -107,6 +113,33 @@ public class MockStorageInterface extends StorageInterface {
|
|||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility function used to convert a given URI to a decoded string
|
||||
* representation sent to the backing store. URIs coming as input
|
||||
* to this class will be encoded by the URI class, and we want
|
||||
* the underlying storage to store keys in their original UTF-8 form.
|
||||
*/
|
||||
private static String convertUriToDecodedString(URI uri) {
|
||||
try {
|
||||
String result = URIUtil.decode(uri.toString());
|
||||
return result;
|
||||
} catch (URIException e) {
|
||||
throw new AssertionError("Failed to decode URI: " + uri.toString());
|
||||
}
|
||||
}
|
||||
|
||||
private static URI convertKeyToEncodedUri(String key) {
|
||||
try {
|
||||
String encodedKey = URIUtil.encodePath(key);
|
||||
URI uri = new URI(encodedKey);
|
||||
return uri;
|
||||
} catch (URISyntaxException e) {
|
||||
throw new AssertionError("Failed to encode key: " + key);
|
||||
} catch (URIException e) {
|
||||
throw new AssertionError("Failed to encode key: " + key);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public CloudBlobContainerWrapper getContainerReference(String name)
|
||||
throws URISyntaxException, StorageException {
|
||||
|
@ -196,6 +229,12 @@ public class MockStorageInterface extends StorageInterface {
|
|||
false)), null, 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public CloudPageBlobWrapper getPageBlobReference(String blobAddressUri)
|
||||
throws URISyntaxException, StorageException {
|
||||
return new MockCloudPageBlobWrapper(new URI(blobAddressUri), null, 0);
|
||||
}
|
||||
|
||||
// helper to create full URIs for directory and blob.
|
||||
// use withTrailingSlash=true to get a good path for a directory.
|
||||
private String fullUriString(String relativePath, boolean withTrailingSlash) {
|
||||
|
@ -260,24 +299,41 @@ public class MockStorageInterface extends StorageInterface {
|
|||
BlobRequestOptions options, OperationContext opContext)
|
||||
throws URISyntaxException, StorageException {
|
||||
ArrayList<ListBlobItem> ret = new ArrayList<ListBlobItem>();
|
||||
String fullPrefix = prefix == null ? uri.toString() : new URI(
|
||||
uri.getScheme(), uri.getAuthority(), uri.getPath() + prefix,
|
||||
uri.getQuery(), uri.getFragment()).toString();
|
||||
boolean includeMetadata = listingDetails
|
||||
.contains(BlobListingDetails.METADATA);
|
||||
URI searchUri = null;
|
||||
if (prefix == null) {
|
||||
searchUri = uri;
|
||||
} else {
|
||||
try {
|
||||
searchUri = UriBuilder.fromUri(uri).path(prefix).build();
|
||||
} catch (UriBuilderException e) {
|
||||
throw new AssertionError("Failed to encode path: " + prefix);
|
||||
}
|
||||
}
|
||||
|
||||
String fullPrefix = convertUriToDecodedString(searchUri);
|
||||
boolean includeMetadata = listingDetails.contains(BlobListingDetails.METADATA);
|
||||
HashSet<String> addedDirectories = new HashSet<String>();
|
||||
for (InMemoryBlockBlobStore.ListBlobEntry current : backingStore
|
||||
.listBlobs(fullPrefix, includeMetadata)) {
|
||||
for (InMemoryBlockBlobStore.ListBlobEntry current : backingStore.listBlobs(
|
||||
fullPrefix, includeMetadata)) {
|
||||
int indexOfSlash = current.getKey().indexOf('/', fullPrefix.length());
|
||||
if (useFlatBlobListing || indexOfSlash < 0) {
|
||||
ret.add(new MockCloudBlockBlobWrapper(new URI(current.getKey()),
|
||||
current.getMetadata(), current.getContentLength()));
|
||||
if (current.isPageBlob()) {
|
||||
ret.add(new MockCloudPageBlobWrapper(
|
||||
convertKeyToEncodedUri(current.getKey()),
|
||||
current.getMetadata(),
|
||||
current.getContentLength()));
|
||||
} else {
|
||||
ret.add(new MockCloudBlockBlobWrapper(
|
||||
convertKeyToEncodedUri(current.getKey()),
|
||||
current.getMetadata(),
|
||||
current.getContentLength()));
|
||||
}
|
||||
} else {
|
||||
String directoryName = current.getKey().substring(0, indexOfSlash);
|
||||
if (!addedDirectories.contains(directoryName)) {
|
||||
addedDirectories.add(current.getKey());
|
||||
ret.add(new MockCloudBlobDirectoryWrapper(new URI(directoryName
|
||||
+ "/")));
|
||||
ret.add(new MockCloudBlobDirectoryWrapper(new URI(
|
||||
directoryName + "/")));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -286,35 +342,35 @@ public class MockStorageInterface extends StorageInterface {
|
|||
|
||||
@Override
|
||||
public StorageUri getStorageUri() {
|
||||
throw new UnsupportedOperationException();
|
||||
throw new NotImplementedException();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
abstract class MockCloudBlobWrapper implements CloudBlobWrapper {
|
||||
protected final URI uri;
|
||||
protected HashMap<String, String> metadata =
|
||||
new HashMap<String, String>();
|
||||
protected BlobProperties properties;
|
||||
|
||||
class MockCloudBlockBlobWrapper extends CloudBlockBlobWrapper {
|
||||
private URI uri;
|
||||
private HashMap<String, String> metadata = new HashMap<String, String>();
|
||||
private BlobProperties properties;
|
||||
|
||||
public MockCloudBlockBlobWrapper(URI uri, HashMap<String, String> metadata,
|
||||
protected MockCloudBlobWrapper(URI uri, HashMap<String, String> metadata,
|
||||
int length) {
|
||||
this.uri = uri;
|
||||
this.metadata = metadata;
|
||||
this.properties = new BlobProperties();
|
||||
this.properties.setLength(length);
|
||||
this.properties.setLastModified(Calendar.getInstance(
|
||||
TimeZone.getTimeZone("UTC")).getTime());
|
||||
this.properties.setLastModified(
|
||||
Calendar.getInstance(TimeZone.getTimeZone("UTC")).getTime());
|
||||
}
|
||||
|
||||
private void refreshProperties(boolean getMetadata) {
|
||||
if (backingStore.exists(uri.toString())) {
|
||||
byte[] content = backingStore.getContent(uri.toString());
|
||||
protected void refreshProperties(boolean getMetadata) {
|
||||
if (backingStore.exists(convertUriToDecodedString(uri))) {
|
||||
byte[] content = backingStore.getContent(convertUriToDecodedString(uri));
|
||||
properties = new BlobProperties();
|
||||
properties.setLength(content.length);
|
||||
properties.setLastModified(Calendar.getInstance(
|
||||
TimeZone.getTimeZone("UTC")).getTime());
|
||||
properties.setLastModified(
|
||||
Calendar.getInstance(TimeZone.getTimeZone("UTC")).getTime());
|
||||
if (getMetadata) {
|
||||
metadata = backingStore.getMetadata(uri.toString());
|
||||
metadata = backingStore.getMetadata(convertUriToDecodedString(uri));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -347,11 +403,11 @@ public class MockStorageInterface extends StorageInterface {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void startCopyFromBlob(CloudBlockBlobWrapper sourceBlob,
|
||||
public void startCopyFromBlob(URI source,
|
||||
OperationContext opContext) throws StorageException, URISyntaxException {
|
||||
backingStore.copy(sourceBlob.getUri().toString(), uri.toString());
|
||||
// it would be best if backingStore.properties.CopyState were tracked
|
||||
// If implemented, update azureNativeFileSystemStore.waitForCopyToComplete
|
||||
backingStore.copy(convertUriToDecodedString(source), convertUriToDecodedString(uri));
|
||||
//TODO: set the backingStore.properties.CopyState and
|
||||
// update azureNativeFileSystemStore.waitForCopyToComplete
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -360,13 +416,14 @@ public class MockStorageInterface extends StorageInterface {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void delete(OperationContext opContext) throws StorageException {
|
||||
backingStore.delete(uri.toString());
|
||||
public void delete(OperationContext opContext, SelfRenewingLease lease)
|
||||
throws StorageException {
|
||||
backingStore.delete(convertUriToDecodedString(uri));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean exists(OperationContext opContext) throws StorageException {
|
||||
return backingStore.exists(uri.toString());
|
||||
return backingStore.exists(convertUriToDecodedString(uri));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -383,37 +440,90 @@ public class MockStorageInterface extends StorageInterface {
|
|||
@Override
|
||||
public InputStream openInputStream(BlobRequestOptions options,
|
||||
OperationContext opContext) throws StorageException {
|
||||
return new ByteArrayInputStream(backingStore.getContent(uri.toString()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public OutputStream openOutputStream(BlobRequestOptions options,
|
||||
OperationContext opContext) throws StorageException {
|
||||
return backingStore.upload(uri.toString(), metadata);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void upload(InputStream sourceStream, OperationContext opContext)
|
||||
throws StorageException, IOException {
|
||||
ByteArrayOutputStream allContent = new ByteArrayOutputStream();
|
||||
allContent.write(sourceStream);
|
||||
backingStore.setContent(uri.toString(), allContent.toByteArray(),
|
||||
metadata);
|
||||
refreshProperties(false);
|
||||
allContent.close();
|
||||
return new ByteArrayInputStream(
|
||||
backingStore.getContent(convertUriToDecodedString(uri)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void uploadMetadata(OperationContext opContext)
|
||||
throws StorageException {
|
||||
backingStore.setContent(uri.toString(),
|
||||
backingStore.getContent(uri.toString()), metadata);
|
||||
backingStore.setMetadata(convertUriToDecodedString(uri), metadata);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void uploadProperties(OperationContext opContext)
|
||||
public void downloadRange(long offset, long length, OutputStream os,
|
||||
BlobRequestOptions options, OperationContext opContext)
|
||||
throws StorageException {
|
||||
refreshProperties(false);
|
||||
throw new NotImplementedException();
|
||||
}
|
||||
}
|
||||
|
||||
class MockCloudBlockBlobWrapper extends MockCloudBlobWrapper
|
||||
implements CloudBlockBlobWrapper {
|
||||
public MockCloudBlockBlobWrapper(URI uri, HashMap<String, String> metadata,
|
||||
int length) {
|
||||
super(uri, metadata, length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public OutputStream openOutputStream(BlobRequestOptions options,
|
||||
OperationContext opContext) throws StorageException {
|
||||
return backingStore.uploadBlockBlob(convertUriToDecodedString(uri),
|
||||
metadata);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setStreamMinimumReadSizeInBytes(int minimumReadSizeBytes) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setWriteBlockSizeInBytes(int writeBlockSizeBytes) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public StorageUri getStorageUri() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void uploadProperties(OperationContext context, SelfRenewingLease lease) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public SelfRenewingLease acquireLease() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CloudBlob getBlob() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
class MockCloudPageBlobWrapper extends MockCloudBlobWrapper
|
||||
implements CloudPageBlobWrapper {
|
||||
public MockCloudPageBlobWrapper(URI uri, HashMap<String, String> metadata,
|
||||
int length) {
|
||||
super(uri, metadata, length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void create(long length, BlobRequestOptions options,
|
||||
OperationContext opContext) throws StorageException {
|
||||
throw new NotImplementedException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void uploadPages(InputStream sourceStream, long offset, long length,
|
||||
BlobRequestOptions options, OperationContext opContext)
|
||||
throws StorageException, IOException {
|
||||
throw new NotImplementedException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ArrayList<PageRange> downloadPageRanges(BlobRequestOptions options,
|
||||
OperationContext opContext) throws StorageException {
|
||||
throw new NotImplementedException();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -426,8 +536,23 @@ public class MockStorageInterface extends StorageInterface {
|
|||
|
||||
@Override
|
||||
public StorageUri getStorageUri() {
|
||||
throw new UnsupportedOperationException();
|
||||
throw new NotImplementedException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void uploadProperties(OperationContext opContext,
|
||||
SelfRenewingLease lease)
|
||||
throws StorageException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public SelfRenewingLease acquireLease() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CloudBlob getBlob() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,22 @@
|
|||
========================================================================
|
||||
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.
|
||||
=========================================================================
|
||||
|
||||
In order to run Windows Azure Storage Blob (WASB) unit tests against a live
|
||||
Azure Storage account, you need to provide test account details in a configuration
|
||||
file called azure-test.xml. See hadoop-tools/hadoop-azure/README.txt for details
|
||||
on configuration, and how to run the tests.
|
|
@ -22,11 +22,10 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.fail;
|
||||
import static org.junit.Assume.assumeNotNull;
|
||||
|
||||
import java.io.DataInputStream;
|
||||
import java.io.DataOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.*;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.hadoop.fs.azure.AzureException;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.fs.permission.PermissionStatus;
|
||||
import org.junit.After;
|
||||
|
@ -100,13 +99,14 @@ public class TestAzureConcurrentOutOfBandIo {
|
|||
public void run() {
|
||||
byte[] dataBlockWrite = new byte[UPLOAD_BLOCK_SIZE];
|
||||
|
||||
DataOutputStream outputStream = null;
|
||||
OutputStream outputStream = null;
|
||||
|
||||
try {
|
||||
for (int i = 0; !done; i++) {
|
||||
// Write two 4 MB blocks to the blob.
|
||||
//
|
||||
outputStream = writerStorageAccount.getStore().storefile(key,
|
||||
outputStream = writerStorageAccount.getStore().storefile(
|
||||
key,
|
||||
new PermissionStatus("", "", FsPermission.getDefault()));
|
||||
|
||||
Arrays.fill(dataBlockWrite, (byte) (i % 256));
|
||||
|
@ -137,10 +137,9 @@ public class TestAzureConcurrentOutOfBandIo {
|
|||
|
||||
// Write to blob to make sure it exists.
|
||||
//
|
||||
// Write five 4 MB blocks to the blob. To ensure there is data in the blob
|
||||
// before reading. This eliminates the race between the reader and writer
|
||||
// threads.
|
||||
DataOutputStream outputStream = testAccount.getStore().storefile(
|
||||
// Write five 4 MB blocks to the blob. To ensure there is data in the blob before
|
||||
// reading. This eliminates the race between the reader and writer threads.
|
||||
OutputStream outputStream = testAccount.getStore().storefile(
|
||||
"WASB_String.txt",
|
||||
new PermissionStatus("", "", FsPermission.getDefault()));
|
||||
Arrays.fill(dataBlockWrite, (byte) 255);
|
||||
|
@ -160,7 +159,7 @@ public class TestAzureConcurrentOutOfBandIo {
|
|||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
try {
|
||||
inputStream = testAccount.getStore().retrieve("WASB_String.txt", 0);
|
||||
inputStream = testAccount.getStore().retrieve("WASB_String.txt");
|
||||
count = 0;
|
||||
int c = 0;
|
||||
|
||||
|
@ -180,7 +179,7 @@ public class TestAzureConcurrentOutOfBandIo {
|
|||
}
|
||||
|
||||
// Close the stream.
|
||||
if (null != inputStream) {
|
||||
if (null != inputStream){
|
||||
inputStream.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,7 +32,6 @@ import java.util.Arrays;
|
|||
import java.util.HashMap;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore.TestHookOperationContext;
|
||||
|
@ -65,19 +64,18 @@ public class TestAzureFileSystemErrorConditions {
|
|||
*/
|
||||
@Test
|
||||
public void testAccessUnauthorizedPublicContainer() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
AzureBlobStorageTestAccount.addWasbToConfiguration(conf);
|
||||
Path noAccessPath = new Path(
|
||||
"wasb://nonExistentContainer@hopefullyNonExistentAccount/someFile");
|
||||
NativeAzureFileSystem.suppressRetryPolicy();
|
||||
try {
|
||||
FileSystem.get(noAccessPath.toUri(), conf).open(noAccessPath);
|
||||
FileSystem.get(noAccessPath.toUri(), new Configuration())
|
||||
.open(noAccessPath);
|
||||
assertTrue("Should've thrown.", false);
|
||||
} catch (AzureException ex) {
|
||||
assertTrue("Unexpected message in exception " + ex, ex.getMessage()
|
||||
.contains(
|
||||
"Unable to access container nonExistentContainer in account"
|
||||
+ " hopefullyNonExistentAccount"));
|
||||
assertTrue("Unexpected message in exception " + ex,
|
||||
ex.getMessage().contains(
|
||||
"Unable to access container nonExistentContainer in account" +
|
||||
" hopefullyNonExistentAccount"));
|
||||
} finally {
|
||||
NativeAzureFileSystem.resumeRetryPolicy();
|
||||
}
|
||||
|
@ -104,11 +102,11 @@ public class TestAzureFileSystemErrorConditions {
|
|||
fs.listStatus(new Path("/"));
|
||||
passed = true;
|
||||
} catch (AzureException ex) {
|
||||
assertTrue("Unexpected exception message: " + ex, ex.getMessage()
|
||||
.contains("unsupported version: 2090-04-05."));
|
||||
assertTrue("Unexpected exception message: " + ex,
|
||||
ex.getMessage().contains("unsupported version: 2090-04-05."));
|
||||
}
|
||||
assertFalse(
|
||||
"Should've thrown an exception because of the wrong version.", passed);
|
||||
assertFalse("Should've thrown an exception because of the wrong version.",
|
||||
passed);
|
||||
} finally {
|
||||
fs.close();
|
||||
}
|
||||
|
@ -118,8 +116,7 @@ public class TestAzureFileSystemErrorConditions {
|
|||
boolean isTargetConnection(HttpURLConnection connection);
|
||||
}
|
||||
|
||||
private class TransientErrorInjector extends
|
||||
StorageEvent<SendingRequestEvent> {
|
||||
private class TransientErrorInjector extends StorageEvent<SendingRequestEvent> {
|
||||
final ConnectionRecognizer connectionRecognizer;
|
||||
private boolean injectedErrorOnce = false;
|
||||
|
||||
|
@ -129,8 +126,7 @@ public class TestAzureFileSystemErrorConditions {
|
|||
|
||||
@Override
|
||||
public void eventOccurred(SendingRequestEvent eventArg) {
|
||||
HttpURLConnection connection = (HttpURLConnection) eventArg
|
||||
.getConnectionObject();
|
||||
HttpURLConnection connection = (HttpURLConnection)eventArg.getConnectionObject();
|
||||
if (!connectionRecognizer.isTargetConnection(connection)) {
|
||||
return;
|
||||
}
|
||||
|
@ -157,8 +153,8 @@ public class TestAzureFileSystemErrorConditions {
|
|||
@Test
|
||||
public void testTransientErrorOnDelete() throws Exception {
|
||||
// Need to do this test against a live storage account
|
||||
AzureBlobStorageTestAccount testAccount = AzureBlobStorageTestAccount
|
||||
.create();
|
||||
AzureBlobStorageTestAccount testAccount =
|
||||
AzureBlobStorageTestAccount.create();
|
||||
assumeNotNull(testAccount);
|
||||
try {
|
||||
NativeAzureFileSystem fs = testAccount.getFileSystem();
|
||||
|
@ -179,7 +175,7 @@ public class TestAzureFileSystemErrorConditions {
|
|||
private void writeAllThreeFile(NativeAzureFileSystem fs, Path testFile)
|
||||
throws IOException {
|
||||
byte[] buffer = new byte[ALL_THREE_FILE_SIZE];
|
||||
Arrays.fill(buffer, (byte) 3);
|
||||
Arrays.fill(buffer, (byte)3);
|
||||
OutputStream stream = fs.create(testFile);
|
||||
stream.write(buffer);
|
||||
stream.close();
|
||||
|
@ -189,7 +185,8 @@ public class TestAzureFileSystemErrorConditions {
|
|||
throws IOException {
|
||||
byte[] buffer = new byte[ALL_THREE_FILE_SIZE];
|
||||
InputStream inStream = fs.open(testFile);
|
||||
assertEquals(buffer.length, inStream.read(buffer, 0, buffer.length));
|
||||
assertEquals(buffer.length,
|
||||
inStream.read(buffer, 0, buffer.length));
|
||||
inStream.close();
|
||||
for (int i = 0; i < buffer.length; i++) {
|
||||
assertEquals(3, buffer[i]);
|
||||
|
@ -199,8 +196,8 @@ public class TestAzureFileSystemErrorConditions {
|
|||
@Test
|
||||
public void testTransientErrorOnCommitBlockList() throws Exception {
|
||||
// Need to do this test against a live storage account
|
||||
AzureBlobStorageTestAccount testAccount = AzureBlobStorageTestAccount
|
||||
.create();
|
||||
AzureBlobStorageTestAccount testAccount =
|
||||
AzureBlobStorageTestAccount.create();
|
||||
assumeNotNull(testAccount);
|
||||
try {
|
||||
NativeAzureFileSystem fs = testAccount.getFileSystem();
|
||||
|
@ -222,8 +219,8 @@ public class TestAzureFileSystemErrorConditions {
|
|||
@Test
|
||||
public void testTransientErrorOnRead() throws Exception {
|
||||
// Need to do this test against a live storage account
|
||||
AzureBlobStorageTestAccount testAccount = AzureBlobStorageTestAccount
|
||||
.create();
|
||||
AzureBlobStorageTestAccount testAccount =
|
||||
AzureBlobStorageTestAccount.create();
|
||||
assumeNotNull(testAccount);
|
||||
try {
|
||||
NativeAzureFileSystem fs = testAccount.getFileSystem();
|
||||
|
@ -240,16 +237,4 @@ public class TestAzureFileSystemErrorConditions {
|
|||
testAccount.cleanup();
|
||||
}
|
||||
}
|
||||
|
||||
// Tests an error during stream creation (in this case in the seek() implementation
|
||||
// to verify the close-stream-on-error logic.
|
||||
@Test (expected=AzureException.class)
|
||||
public void testErrorDuringRetrieve() throws Exception {
|
||||
NativeAzureFileSystem fs = AzureBlobStorageTestAccount.createMock().getFileSystem();
|
||||
Path testFile = new Path("/testErrorDuringRetrieve");
|
||||
writeAllThreeFile(fs, testFile);
|
||||
|
||||
FSDataInputStream stream = fs.open(testFile);
|
||||
stream.seek(Integer.MAX_VALUE);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -128,7 +128,7 @@ public class TestBlobDataValidation {
|
|||
if (!expectMd5Stored) {
|
||||
throw ex;
|
||||
}
|
||||
StorageException cause = (StorageException) ex.getCause();
|
||||
StorageException cause = (StorageException)ex.getCause();
|
||||
assertNotNull(cause);
|
||||
assertTrue("Unexpected cause: " + cause,
|
||||
cause.getErrorCode().equals(StorageErrorCodeStrings.INVALID_MD5));
|
||||
|
|
|
@ -69,7 +69,8 @@ public class TestBlobMetadata {
|
|||
throws Exception {
|
||||
return String.format(
|
||||
"{\"owner\":\"%s\",\"group\":\"%s\",\"permissions\":\"%s\"}",
|
||||
getExpectedOwner(), NativeAzureFileSystem.AZURE_DEFAULT_GROUP_DEFAULT,
|
||||
getExpectedOwner(),
|
||||
NativeAzureFileSystem.AZURE_DEFAULT_GROUP_DEFAULT,
|
||||
permissionString);
|
||||
}
|
||||
|
||||
|
@ -80,8 +81,8 @@ public class TestBlobMetadata {
|
|||
public void testContainerVersionMetadata() throws Exception {
|
||||
// Do a write operation to trigger version stamp
|
||||
fs.createNewFile(new Path("/foo"));
|
||||
HashMap<String, String> containerMetadata = backingStore
|
||||
.getContainerMetadata();
|
||||
HashMap<String, String> containerMetadata =
|
||||
backingStore.getContainerMetadata();
|
||||
assertNotNull(containerMetadata);
|
||||
assertEquals(AzureNativeFileSystemStore.CURRENT_WASB_VERSION,
|
||||
containerMetadata.get(AzureNativeFileSystemStore.VERSION_METADATA_KEY));
|
||||
|
@ -226,26 +227,32 @@ public class TestBlobMetadata {
|
|||
@Test
|
||||
public void testOldPermissionMetadata() throws Exception {
|
||||
Path selfishFile = new Path("/noOneElse");
|
||||
HashMap<String, String> metadata = new HashMap<String, String>();
|
||||
metadata.put("asv_permission", getExpectedPermissionString("rw-------"));
|
||||
backingStore.setContent(AzureBlobStorageTestAccount.toMockUri(selfishFile),
|
||||
new byte[] {}, metadata);
|
||||
FsPermission justMe = new FsPermission(FsAction.READ_WRITE, FsAction.NONE,
|
||||
FsAction.NONE);
|
||||
HashMap<String, String> metadata =
|
||||
new HashMap<String, String>();
|
||||
metadata.put("asv_permission",
|
||||
getExpectedPermissionString("rw-------"));
|
||||
backingStore.setContent(
|
||||
AzureBlobStorageTestAccount.toMockUri(selfishFile),
|
||||
new byte[] { },
|
||||
metadata, false, 0);
|
||||
FsPermission justMe = new FsPermission(
|
||||
FsAction.READ_WRITE, FsAction.NONE, FsAction.NONE);
|
||||
FileStatus retrievedStatus = fs.getFileStatus(selfishFile);
|
||||
assertNotNull(retrievedStatus);
|
||||
assertEquals(justMe, retrievedStatus.getPermission());
|
||||
assertEquals(getExpectedOwner(), retrievedStatus.getOwner());
|
||||
assertEquals(NativeAzureFileSystem.AZURE_DEFAULT_GROUP_DEFAULT,
|
||||
retrievedStatus.getGroup());
|
||||
FsPermission meAndYou = new FsPermission(FsAction.READ_WRITE,
|
||||
FsAction.READ_WRITE, FsAction.NONE);
|
||||
FsPermission meAndYou = new FsPermission(
|
||||
FsAction.READ_WRITE, FsAction.READ_WRITE, FsAction.NONE);
|
||||
fs.setPermission(selfishFile, meAndYou);
|
||||
metadata = backingStore.getMetadata(AzureBlobStorageTestAccount
|
||||
.toMockUri(selfishFile));
|
||||
metadata =
|
||||
backingStore.getMetadata(
|
||||
AzureBlobStorageTestAccount.toMockUri(selfishFile));
|
||||
assertNotNull(metadata);
|
||||
String storedPermission = metadata.get("hdi_permission");
|
||||
assertEquals(getExpectedPermissionString("rw-rw----"), storedPermission);
|
||||
assertEquals(getExpectedPermissionString("rw-rw----"),
|
||||
storedPermission);
|
||||
assertNull(metadata.get("asv_permission"));
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,160 @@
|
|||
/**
|
||||
* 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.apache.hadoop.fs.azure;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.*;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.*;
|
||||
import org.apache.hadoop.fs.azure.metrics.AzureFileSystemInstrumentation;
|
||||
|
||||
import junit.framework.*;
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
|
||||
/**
|
||||
* A simple benchmark to find out the difference in speed between block
|
||||
* and page blobs.
|
||||
*/
|
||||
public class TestBlobTypeSpeedDifference extends TestCase {
|
||||
/**
|
||||
* Writes data to the given stream of the given size, flushing every
|
||||
* x bytes.
|
||||
*/
|
||||
private static void writeTestFile(OutputStream writeStream,
|
||||
long size, long flushInterval) throws IOException {
|
||||
int bufferSize = (int) Math.min(1000, flushInterval);
|
||||
byte[] buffer = new byte[bufferSize];
|
||||
Arrays.fill(buffer, (byte) 7);
|
||||
int bytesWritten = 0;
|
||||
int bytesUnflushed = 0;
|
||||
while (bytesWritten < size) {
|
||||
int numberToWrite = (int) Math.min(bufferSize, size - bytesWritten);
|
||||
writeStream.write(buffer, 0, numberToWrite);
|
||||
bytesWritten += numberToWrite;
|
||||
bytesUnflushed += numberToWrite;
|
||||
if (bytesUnflushed >= flushInterval) {
|
||||
writeStream.flush();
|
||||
bytesUnflushed = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static class TestResult {
|
||||
final long timeTakenInMs;
|
||||
final long totalNumberOfRequests;
|
||||
|
||||
TestResult(long timeTakenInMs, long totalNumberOfRequests) {
|
||||
this.timeTakenInMs = timeTakenInMs;
|
||||
this.totalNumberOfRequests = totalNumberOfRequests;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes data to the given file of the given size, flushing every
|
||||
* x bytes. Measure performance of that and return it.
|
||||
*/
|
||||
private static TestResult writeTestFile(NativeAzureFileSystem fs, Path path,
|
||||
long size, long flushInterval) throws IOException {
|
||||
AzureFileSystemInstrumentation instrumentation =
|
||||
fs.getInstrumentation();
|
||||
long initialRequests = instrumentation.getCurrentWebResponses();
|
||||
Date start = new Date();
|
||||
OutputStream output = fs.create(path);
|
||||
writeTestFile(output, size, flushInterval);
|
||||
output.close();
|
||||
long finalRequests = instrumentation.getCurrentWebResponses();
|
||||
return new TestResult(new Date().getTime() - start.getTime(),
|
||||
finalRequests - initialRequests);
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes data to a block blob of the given size, flushing every
|
||||
* x bytes. Measure performance of that and return it.
|
||||
*/
|
||||
private static TestResult writeBlockBlobTestFile(NativeAzureFileSystem fs,
|
||||
long size, long flushInterval) throws IOException {
|
||||
return writeTestFile(fs, new Path("/blockBlob"), size, flushInterval);
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes data to a page blob of the given size, flushing every
|
||||
* x bytes. Measure performance of that and return it.
|
||||
*/
|
||||
private static TestResult writePageBlobTestFile(NativeAzureFileSystem fs,
|
||||
long size, long flushInterval) throws IOException {
|
||||
return writeTestFile(fs,
|
||||
AzureBlobStorageTestAccount.pageBlobPath("pageBlob"),
|
||||
size, flushInterval);
|
||||
}
|
||||
|
||||
/**
|
||||
* Runs the benchmark over a small 10 KB file, flushing every 500 bytes.
|
||||
*/
|
||||
@Test
|
||||
public void testTenKbFileFrequentFlush() throws Exception {
|
||||
AzureBlobStorageTestAccount testAccount =
|
||||
AzureBlobStorageTestAccount.create();
|
||||
if (testAccount == null) {
|
||||
return;
|
||||
}
|
||||
try {
|
||||
testForSizeAndFlushInterval(testAccount.getFileSystem(), 10 * 1000, 500);
|
||||
} finally {
|
||||
testAccount.cleanup();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Runs the benchmark for the given file size and flush frequency.
|
||||
*/
|
||||
private static void testForSizeAndFlushInterval(NativeAzureFileSystem fs,
|
||||
final long size, final long flushInterval) throws IOException {
|
||||
for (int i = 0; i < 5; i++) {
|
||||
TestResult pageBlobResults = writePageBlobTestFile(fs, size, flushInterval);
|
||||
System.out.printf(
|
||||
"Page blob upload took %d ms. Total number of requests: %d.\n",
|
||||
pageBlobResults.timeTakenInMs, pageBlobResults.totalNumberOfRequests);
|
||||
TestResult blockBlobResults = writeBlockBlobTestFile(fs, size, flushInterval);
|
||||
System.out.printf(
|
||||
"Block blob upload took %d ms. Total number of requests: %d.\n",
|
||||
blockBlobResults.timeTakenInMs, blockBlobResults.totalNumberOfRequests);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Runs the benchmark for the given file size and flush frequency from the
|
||||
* command line.
|
||||
*/
|
||||
public static void main(String argv[]) throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
long size = 10 * 1000 * 1000;
|
||||
long flushInterval = 2000;
|
||||
if (argv.length > 0) {
|
||||
size = Long.parseLong(argv[0]);
|
||||
}
|
||||
if (argv.length > 1) {
|
||||
flushInterval = Long.parseLong(argv[1]);
|
||||
}
|
||||
testForSizeAndFlushInterval((NativeAzureFileSystem)FileSystem.get(conf),
|
||||
size, flushInterval);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,43 @@
|
|||
/**
|
||||
* 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.apache.hadoop.fs.azure;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
/**
|
||||
* Run the base Azure file system tests strictly on page blobs to make sure fundamental
|
||||
* operations on page blob files and folders work as expected.
|
||||
* These operations include create, delete, rename, list, and so on.
|
||||
*/
|
||||
public class TestNativeAzureFSPageBlobLive extends
|
||||
NativeAzureFileSystemBaseTest {
|
||||
|
||||
@Override
|
||||
protected AzureBlobStorageTestAccount createTestAccount()
|
||||
throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
|
||||
// Configure the page blob directories key so every file created is a page blob.
|
||||
conf.set(AzureNativeFileSystemStore.KEY_PAGE_BLOB_DIRECTORIES, "/");
|
||||
|
||||
// Configure the atomic rename directories key so every folder will have
|
||||
// atomic rename applied.
|
||||
conf.set(AzureNativeFileSystemStore.KEY_ATOMIC_RENAME_DIRECTORIES, "/");
|
||||
return AzureBlobStorageTestAccount.create(conf);
|
||||
}
|
||||
}
|
|
@ -26,6 +26,7 @@ import static org.junit.Assert.assertTrue;
|
|||
import java.io.OutputStream;
|
||||
import java.io.PrintWriter;
|
||||
import java.io.StringWriter;
|
||||
import java.net.URLDecoder;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
|
@ -67,8 +68,8 @@ public class TestNativeAzureFileSystemConcurrency {
|
|||
HashMap<String, String> metadata = backingStore
|
||||
.getMetadata(AzureBlobStorageTestAccount.toMockUri(filePath));
|
||||
assertNotNull(metadata);
|
||||
String linkValue = metadata
|
||||
.get(AzureNativeFileSystemStore.LINK_BACK_TO_UPLOAD_IN_PROGRESS_METADATA_KEY);
|
||||
String linkValue = metadata.get(AzureNativeFileSystemStore.LINK_BACK_TO_UPLOAD_IN_PROGRESS_METADATA_KEY);
|
||||
linkValue = URLDecoder.decode(linkValue, "UTF-8");
|
||||
assertNotNull(linkValue);
|
||||
assertTrue(backingStore.exists(AzureBlobStorageTestAccount
|
||||
.toMockUri(linkValue)));
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.fs.azure;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystemContractBaseTest;
|
||||
import org.junit.Ignore;
|
||||
|
||||
public class TestNativeAzureFileSystemContractLive extends
|
||||
FileSystemContractBaseTest {
|
||||
|
@ -47,4 +48,29 @@ public class TestNativeAzureFileSystemContractLive extends
|
|||
super.runTest();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The following tests are failing on Azure and the Azure
|
||||
* file system code needs to be modified to make them pass.
|
||||
* A separate work item has been opened for this.
|
||||
*/
|
||||
@Ignore
|
||||
public void testMoveFileUnderParent() throws Throwable {
|
||||
}
|
||||
|
||||
@Ignore
|
||||
public void testRenameFileToSelf() throws Throwable {
|
||||
}
|
||||
|
||||
@Ignore
|
||||
public void testRenameChildDirForbidden() throws Exception {
|
||||
}
|
||||
|
||||
@Ignore
|
||||
public void testMoveDirUnderParent() throws Throwable {
|
||||
}
|
||||
|
||||
@Ignore
|
||||
public void testRenameDirToSelf() throws Throwable {
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.fs.azure;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystemContractBaseTest;
|
||||
import org.junit.Ignore;
|
||||
|
||||
public class TestNativeAzureFileSystemContractMocked extends
|
||||
FileSystemContractBaseTest {
|
||||
|
@ -28,4 +29,28 @@ public class TestNativeAzureFileSystemContractMocked extends
|
|||
fs = AzureBlobStorageTestAccount.createMock().getFileSystem();
|
||||
}
|
||||
|
||||
/**
|
||||
* The following tests are failing on Azure and the Azure
|
||||
* file system code needs to be modified to make them pass.
|
||||
* A separate work item has been opened for this.
|
||||
*/
|
||||
@Ignore
|
||||
public void testMoveFileUnderParent() throws Throwable {
|
||||
}
|
||||
|
||||
@Ignore
|
||||
public void testRenameFileToSelf() throws Throwable {
|
||||
}
|
||||
|
||||
@Ignore
|
||||
public void testRenameChildDirForbidden() throws Exception {
|
||||
}
|
||||
|
||||
@Ignore
|
||||
public void testMoveDirUnderParent() throws Throwable {
|
||||
}
|
||||
|
||||
@Ignore
|
||||
public void testRenameDirToSelf() throws Throwable {
|
||||
}
|
||||
}
|
||||
|
|
|
@ -123,11 +123,12 @@ public class TestNativeAzureFileSystemFileNameCheck {
|
|||
assertFalse(runWasbFsck(testFolder1));
|
||||
|
||||
// negative test
|
||||
InMemoryBlockBlobStore backingStore = testAccount.getMockStorage()
|
||||
.getBackingStore();
|
||||
backingStore.setContent(AzureBlobStorageTestAccount
|
||||
.toMockUri("testFolder1/testFolder2/test2:2"), new byte[] { 1, 2 },
|
||||
new HashMap<String, String>());
|
||||
InMemoryBlockBlobStore backingStore
|
||||
= testAccount.getMockStorage().getBackingStore();
|
||||
backingStore.setContent(
|
||||
AzureBlobStorageTestAccount.toMockUri("testFolder1/testFolder2/test2:2"),
|
||||
new byte[] { 1, 2 },
|
||||
new HashMap<String, String>(), false, 0);
|
||||
assertTrue(runWasbFsck(testFolder1));
|
||||
}
|
||||
|
||||
|
|
|
@ -18,6 +18,12 @@
|
|||
|
||||
package org.apache.hadoop.fs.azure;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
/*
|
||||
* Tests the Native Azure file system (WASB) against an actual blob store if
|
||||
* provided in the environment.
|
||||
|
@ -29,4 +35,73 @@ public class TestNativeAzureFileSystemLive extends
|
|||
protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
|
||||
return AzureBlobStorageTestAccount.create();
|
||||
}
|
||||
|
||||
/**
|
||||
* Check that isPageBlobKey works as expected. This assumes that
|
||||
* in the test configuration, the list of supported page blob directories
|
||||
* only includes "pageBlobs". That's why this test is made specific
|
||||
* to this subclass.
|
||||
*/
|
||||
@Test
|
||||
public void testIsPageBlobKey() {
|
||||
AzureNativeFileSystemStore store = ((NativeAzureFileSystem) fs).getStore();
|
||||
|
||||
// Use literal strings so it's easier to understand the tests.
|
||||
// In case the constant changes, we want to know about it so we can update this test.
|
||||
assertEquals(AzureBlobStorageTestAccount.DEFAULT_PAGE_BLOB_DIRECTORY, "pageBlobs");
|
||||
|
||||
// URI prefix for test environment.
|
||||
String uriPrefix = "file:///";
|
||||
|
||||
// negative tests
|
||||
String[] negativeKeys = { "", "/", "bar", "bar/", "bar/pageBlobs", "bar/pageBlobs/foo",
|
||||
"bar/pageBlobs/foo/", "/pageBlobs/", "/pageBlobs", "pageBlobs", "pageBlobsxyz/" };
|
||||
for (String s : negativeKeys) {
|
||||
assertFalse(store.isPageBlobKey(s));
|
||||
assertFalse(store.isPageBlobKey(uriPrefix + s));
|
||||
}
|
||||
|
||||
// positive tests
|
||||
String[] positiveKeys = { "pageBlobs/", "pageBlobs/foo/", "pageBlobs/foo/bar/" };
|
||||
for (String s : positiveKeys) {
|
||||
assertTrue(store.isPageBlobKey(s));
|
||||
assertTrue(store.isPageBlobKey(uriPrefix + s));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that isAtomicRenameKey() works as expected.
|
||||
*/
|
||||
@Test
|
||||
public void testIsAtomicRenameKey() {
|
||||
|
||||
AzureNativeFileSystemStore store = ((NativeAzureFileSystem) fs).getStore();
|
||||
|
||||
// We want to know if the default configuration changes so we can fix
|
||||
// this test.
|
||||
assertEquals(AzureBlobStorageTestAccount.DEFAULT_ATOMIC_RENAME_DIRECTORIES,
|
||||
"/atomicRenameDir1,/atomicRenameDir2");
|
||||
|
||||
// URI prefix for test environment.
|
||||
String uriPrefix = "file:///";
|
||||
|
||||
// negative tests
|
||||
String[] negativeKeys = { "", "/", "bar", "bar/", "bar/hbase",
|
||||
"bar/hbase/foo", "bar/hbase/foo/", "/hbase/", "/hbase", "hbase",
|
||||
"hbasexyz/", "foo/atomicRenameDir1/"};
|
||||
for (String s : negativeKeys) {
|
||||
assertFalse(store.isAtomicRenameKey(s));
|
||||
assertFalse(store.isAtomicRenameKey(uriPrefix + s));
|
||||
}
|
||||
|
||||
// Positive tests. The directories for atomic rename are /hbase
|
||||
// plus the ones in the configuration (DEFAULT_ATOMIC_RENAME_DIRECTORIES
|
||||
// for this test).
|
||||
String[] positiveKeys = { "hbase/", "hbase/foo/", "hbase/foo/bar/",
|
||||
"atomicRenameDir1/foo/", "atomicRenameDir2/bar/"};
|
||||
for (String s : positiveKeys) {
|
||||
assertTrue(store.isAtomicRenameKey(s));
|
||||
assertTrue(store.isAtomicRenameKey(uriPrefix + s));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,6 +18,9 @@
|
|||
|
||||
package org.apache.hadoop.fs.azure;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.junit.Ignore;
|
||||
|
||||
public class TestNativeAzureFileSystemMocked extends
|
||||
NativeAzureFileSystemBaseTest {
|
||||
|
||||
|
@ -25,4 +28,36 @@ public class TestNativeAzureFileSystemMocked extends
|
|||
protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
|
||||
return AzureBlobStorageTestAccount.createMock();
|
||||
}
|
||||
|
||||
// Ignore the following tests because taking a lease requires a real
|
||||
// (not mock) file system store. These tests don't work on the mock.
|
||||
@Override
|
||||
@Ignore
|
||||
public void testLeaseAsDistributedLock() {
|
||||
}
|
||||
|
||||
@Override
|
||||
@Ignore
|
||||
public void testSelfRenewingLease() {
|
||||
}
|
||||
|
||||
@Override
|
||||
@Ignore
|
||||
public void testRedoFolderRenameAll() {
|
||||
}
|
||||
|
||||
@Override
|
||||
@Ignore
|
||||
public void testCreateNonRecursive() {
|
||||
}
|
||||
|
||||
@Override
|
||||
@Ignore
|
||||
public void testSelfRenewingLeaseFileDelete() {
|
||||
}
|
||||
|
||||
@Override
|
||||
@Ignore
|
||||
public void testRenameRedoFolderAlreadyDone() throws IOException{
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,8 +27,16 @@ import org.apache.hadoop.fs.Path;
|
|||
public class TestNativeAzureFileSystemOperationsMocked extends
|
||||
FSMainOperationsBaseTest {
|
||||
|
||||
public TestNativeAzureFileSystemOperationsMocked() {
|
||||
super("/tmp/TestNativeAzureFileSystemOperationsMocked");
|
||||
private static final String TEST_ROOT_DIR =
|
||||
"/tmp/TestNativeAzureFileSystemOperationsMocked";
|
||||
|
||||
public TestNativeAzureFileSystemOperationsMocked (){
|
||||
super(TEST_ROOT_DIR);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
fSys = AzureBlobStorageTestAccount.createMock().getFileSystem();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -42,4 +50,29 @@ public class TestNativeAzureFileSystemOperationsMocked extends
|
|||
+ " doesn't honor directory permissions.");
|
||||
assumeTrue(!Path.WINDOWS);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTestRootDir() {
|
||||
return TEST_ROOT_DIR;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getTestRootPath(FileSystem fSys) {
|
||||
return fSys.makeQualified(new Path(TEST_ROOT_DIR));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getTestRootPath(FileSystem fSys, String pathString) {
|
||||
return fSys.makeQualified(new Path(TEST_ROOT_DIR, pathString));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getAbsoluteTestRootPath(FileSystem fSys) {
|
||||
Path testRootPath = new Path(TEST_ROOT_DIR);
|
||||
if (testRootPath.isAbsolute()) {
|
||||
return testRootPath;
|
||||
} else {
|
||||
return new Path(fSys.getWorkingDirectory(), TEST_ROOT_DIR);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,186 @@
|
|||
/**
|
||||
* 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.apache.hadoop.fs.azure;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* Tests for the upload, buffering and flush logic in WASB.
|
||||
*/
|
||||
public class TestNativeAzureFileSystemUploadLogic {
|
||||
private AzureBlobStorageTestAccount testAccount;
|
||||
|
||||
// Just an arbitrary number so that the values I write have a predictable
|
||||
// pattern: 0, 1, 2, .. , 45, 46, 0, 1, 2, ...
|
||||
static final int byteValuePeriod = 47;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
testAccount = AzureBlobStorageTestAccount.createMock();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
if (testAccount != null) {
|
||||
testAccount.cleanup();
|
||||
testAccount = null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Various scenarios to test in how often we flush data while uploading.
|
||||
*/
|
||||
private enum FlushFrequencyVariation {
|
||||
/**
|
||||
* Flush before even a single in-memory buffer is full.
|
||||
*/
|
||||
BeforeSingleBufferFull,
|
||||
/**
|
||||
* Flush after a single in-memory buffer is full.
|
||||
*/
|
||||
AfterSingleBufferFull,
|
||||
/**
|
||||
* Flush after all the in-memory buffers got full and were
|
||||
* automatically flushed to the backing store.
|
||||
*/
|
||||
AfterAllRingBufferFull,
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that we upload consistently if we flush after every little
|
||||
* bit of data.
|
||||
*/
|
||||
@Test
|
||||
@Ignore /* flush() no longer does anything. @@TODO: implement a force-flush and reinstate this test */
|
||||
public void testConsistencyAfterSmallFlushes() throws Exception {
|
||||
testConsistencyAfterManyFlushes(FlushFrequencyVariation.BeforeSingleBufferFull);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that we upload consistently if we flush after every medium-sized
|
||||
* bit of data.
|
||||
*/
|
||||
@Test
|
||||
@Ignore /* flush() no longer does anything. @@TODO: implement a force-flush and reinstate this test */
|
||||
public void testConsistencyAfterMediumFlushes() throws Exception {
|
||||
testConsistencyAfterManyFlushes(FlushFrequencyVariation.AfterSingleBufferFull);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that we upload consistently if we flush after every large chunk
|
||||
* of data.
|
||||
*/
|
||||
@Test
|
||||
@Ignore /* flush() no longer does anything. @@TODO: implement a force-flush and reinstate this test */
|
||||
public void testConsistencyAfterLargeFlushes() throws Exception {
|
||||
testConsistencyAfterManyFlushes(FlushFrequencyVariation.AfterAllRingBufferFull);
|
||||
}
|
||||
|
||||
/**
|
||||
* Makes sure the data in the given input is what I'd expect.
|
||||
* @param inStream The input stream.
|
||||
* @param expectedSize The expected size of the data in there.
|
||||
*/
|
||||
private void assertDataInStream(InputStream inStream, int expectedSize)
|
||||
throws Exception {
|
||||
int byteRead;
|
||||
int countBytes = 0;
|
||||
while ((byteRead = inStream.read()) != -1) {
|
||||
assertEquals(countBytes % byteValuePeriod, byteRead);
|
||||
countBytes++;
|
||||
}
|
||||
assertEquals(expectedSize, countBytes);
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks that the data in the given file is what I'd expect.
|
||||
* @param file The file to check.
|
||||
* @param expectedSize The expected size of the data in there.
|
||||
*/
|
||||
private void assertDataInFile(Path file, int expectedSize) throws Exception {
|
||||
InputStream inStream = testAccount.getFileSystem().open(file);
|
||||
assertDataInStream(inStream, expectedSize);
|
||||
inStream.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks that the data in the current temporary upload blob
|
||||
* is what I'd expect.
|
||||
* @param expectedSize The expected size of the data in there.
|
||||
*/
|
||||
private void assertDataInTempBlob(int expectedSize) throws Exception {
|
||||
// Look for the temporary upload blob in the backing store.
|
||||
InMemoryBlockBlobStore backingStore =
|
||||
testAccount.getMockStorage().getBackingStore();
|
||||
String tempKey = null;
|
||||
for (String key : backingStore.getKeys()) {
|
||||
if (key.contains(NativeAzureFileSystem.AZURE_TEMP_FOLDER)) {
|
||||
// Assume this is the one we're looking for.
|
||||
tempKey = key;
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertNotNull(tempKey);
|
||||
InputStream inStream = new ByteArrayInputStream(backingStore.getContent(tempKey));
|
||||
assertDataInStream(inStream, expectedSize);
|
||||
inStream.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests the given scenario for uploading a file while flushing
|
||||
* periodically and making sure the data is always consistent
|
||||
* with what I'd expect.
|
||||
* @param variation The variation/scenario to test.
|
||||
*/
|
||||
private void testConsistencyAfterManyFlushes(FlushFrequencyVariation variation)
|
||||
throws Exception {
|
||||
Path uploadedFile = new Path("/uploadedFile");
|
||||
OutputStream outStream = testAccount.getFileSystem().create(uploadedFile);
|
||||
final int totalSize = 9123;
|
||||
int flushPeriod;
|
||||
switch (variation) {
|
||||
case BeforeSingleBufferFull: flushPeriod = 300; break;
|
||||
case AfterSingleBufferFull: flushPeriod = 600; break;
|
||||
case AfterAllRingBufferFull: flushPeriod = 1600; break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Unknown variation: " + variation);
|
||||
}
|
||||
for (int i = 0; i < totalSize; i++) {
|
||||
outStream.write(i % byteValuePeriod);
|
||||
if ((i + 1) % flushPeriod == 0) {
|
||||
outStream.flush();
|
||||
assertDataInTempBlob(i + 1);
|
||||
}
|
||||
}
|
||||
outStream.close();
|
||||
assertDataInFile(uploadedFile, totalSize);
|
||||
}
|
||||
}
|
|
@ -57,10 +57,14 @@ public class TestOutOfBandAzureBlobOperations {
|
|||
}
|
||||
|
||||
private void createEmptyBlobOutOfBand(String path) {
|
||||
backingStore.setContent(AzureBlobStorageTestAccount.toMockUri(path),
|
||||
new byte[] { 1, 2 }, new HashMap<String, String>());
|
||||
backingStore.setContent(
|
||||
AzureBlobStorageTestAccount.toMockUri(path),
|
||||
new byte[] { 1, 2 },
|
||||
new HashMap<String, String>(),
|
||||
false, 0);
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
@Test
|
||||
public void testImplicitFolderListed() throws Exception {
|
||||
createEmptyBlobOutOfBand("root/b");
|
||||
|
@ -69,20 +73,20 @@ public class TestOutOfBandAzureBlobOperations {
|
|||
FileStatus[] obtained = fs.listStatus(new Path("/root/b"));
|
||||
assertNotNull(obtained);
|
||||
assertEquals(1, obtained.length);
|
||||
assertFalse(obtained[0].isDirectory());
|
||||
assertFalse(obtained[0].isDir());
|
||||
assertEquals("/root/b", obtained[0].getPath().toUri().getPath());
|
||||
|
||||
// List the directory
|
||||
obtained = fs.listStatus(new Path("/root"));
|
||||
assertNotNull(obtained);
|
||||
assertEquals(1, obtained.length);
|
||||
assertFalse(obtained[0].isDirectory());
|
||||
assertFalse(obtained[0].isDir());
|
||||
assertEquals("/root/b", obtained[0].getPath().toUri().getPath());
|
||||
|
||||
// Get the directory's file status
|
||||
FileStatus dirStatus = fs.getFileStatus(new Path("/root"));
|
||||
assertNotNull(dirStatus);
|
||||
assertTrue(dirStatus.isDirectory());
|
||||
assertTrue(dirStatus.isDir());
|
||||
assertEquals("/root", dirStatus.getPath().toUri().getPath());
|
||||
}
|
||||
|
||||
|
@ -102,6 +106,7 @@ public class TestOutOfBandAzureBlobOperations {
|
|||
assertTrue(fs.exists(new Path("/root")));
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
@Test
|
||||
public void testFileAndImplicitFolderSameName() throws Exception {
|
||||
createEmptyBlobOutOfBand("root/b");
|
||||
|
@ -109,7 +114,7 @@ public class TestOutOfBandAzureBlobOperations {
|
|||
FileStatus[] listResult = fs.listStatus(new Path("/root/b"));
|
||||
// File should win.
|
||||
assertEquals(1, listResult.length);
|
||||
assertFalse(listResult[0].isDirectory());
|
||||
assertFalse(listResult[0].isDir());
|
||||
try {
|
||||
// Trying to delete root/b/c would cause a dilemma for WASB, so
|
||||
// it should throw.
|
||||
|
|
|
@ -163,6 +163,27 @@ public class TestOutOfBandAzureBlobOperationsLive {
|
|||
fs.rename(srcFilePath, destFilePath);
|
||||
}
|
||||
|
||||
// Verify that you can rename a file which is the only file in an implicit folder in the
|
||||
// WASB file system.
|
||||
// scenario for this particular test described at MONARCH-HADOOP-892
|
||||
@Test
|
||||
public void outOfBandSingleFile_rename() throws Exception {
|
||||
|
||||
//NOTE: manual use of CloubBlockBlob targets working directory explicitly.
|
||||
// WASB driver methods prepend working directory implicitly.
|
||||
String workingDir = "user/" + UserGroupInformation.getCurrentUser().getShortUserName() + "/";
|
||||
CloudBlockBlob blob = testAccount.getBlobReference(workingDir + "testFolder5/a/input/file");
|
||||
BlobOutputStream s = blob.openOutputStream();
|
||||
s.close();
|
||||
|
||||
Path srcFilePath = new Path("testFolder5/a/input/file");
|
||||
assertTrue(fs.exists(srcFilePath));
|
||||
|
||||
Path destFilePath = new Path("testFolder5/file2");
|
||||
fs.rename(srcFilePath, destFilePath);
|
||||
}
|
||||
|
||||
// WASB must force explicit parent directories in create, delete, mkdirs, rename.
|
||||
// scenario for this particular test described at MONARCH-HADOOP-764
|
||||
@Test
|
||||
public void outOfBandFolder_rename_rootLevelFiles() throws Exception {
|
||||
|
|
|
@ -0,0 +1,333 @@
|
|||
/**
|
||||
* 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.apache.hadoop.fs.azure;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assume.assumeNotNull;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.util.Arrays;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.azure.AzureException;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* Write data into a page blob and verify you can read back all of it
|
||||
* or just a part of it.
|
||||
*/
|
||||
public class TestReadAndSeekPageBlobAfterWrite {
|
||||
private static final Log LOG = LogFactory.getLog(TestReadAndSeekPageBlobAfterWrite.class);
|
||||
|
||||
private FileSystem fs;
|
||||
private AzureBlobStorageTestAccount testAccount;
|
||||
private byte[] randomData;
|
||||
|
||||
// Page blob physical page size
|
||||
private static final int PAGE_SIZE = PageBlobFormatHelpers.PAGE_SIZE;
|
||||
|
||||
// Size of data on page (excluding header)
|
||||
private static final int PAGE_DATA_SIZE = PAGE_SIZE - PageBlobFormatHelpers.PAGE_HEADER_SIZE;
|
||||
private static final int MAX_BYTES = 33554432; // maximum bytes in a file that we'll test
|
||||
private static final int MAX_PAGES = MAX_BYTES / PAGE_SIZE; // maximum number of pages we'll test
|
||||
private Random rand = new Random();
|
||||
|
||||
// A key with a prefix under /pageBlobs, which for the test file system will
|
||||
// force use of a page blob.
|
||||
private static final String KEY = "/pageBlobs/file.dat";
|
||||
private static final Path PATH = new Path(KEY); // path of page blob file to read and write
|
||||
|
||||
protected AzureBlobStorageTestAccount createTestAccount() throws Exception {
|
||||
return AzureBlobStorageTestAccount.create();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
testAccount = createTestAccount();
|
||||
if (testAccount != null) {
|
||||
fs = testAccount.getFileSystem();
|
||||
}
|
||||
assumeNotNull(testAccount);
|
||||
|
||||
// Make sure we are using an integral number of pages.
|
||||
assertEquals(0, MAX_BYTES % PAGE_SIZE);
|
||||
|
||||
// load an in-memory array of random data
|
||||
randomData = new byte[PAGE_SIZE * MAX_PAGES];
|
||||
rand.nextBytes(randomData);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
if (testAccount != null) {
|
||||
testAccount.cleanup();
|
||||
testAccount = null;
|
||||
fs = null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Make sure the file name (key) is a page blob file name. If anybody changes that,
|
||||
* we need to come back and update this test class.
|
||||
*/
|
||||
@Test
|
||||
public void testIsPageBlobFileName() {
|
||||
AzureNativeFileSystemStore store = ((NativeAzureFileSystem) fs).getStore();
|
||||
String[] a = KEY.split("/");
|
||||
String key2 = a[1] + "/";
|
||||
assertTrue(store.isPageBlobKey(key2));
|
||||
}
|
||||
|
||||
/**
|
||||
* For a set of different file sizes, write some random data to a page blob,
|
||||
* read it back, and compare that what was read is the same as what was written.
|
||||
*/
|
||||
@Test
|
||||
public void testReadAfterWriteRandomData() throws IOException {
|
||||
|
||||
// local shorthand
|
||||
final int PDS = PAGE_DATA_SIZE;
|
||||
|
||||
// Test for sizes at and near page boundaries
|
||||
int[] dataSizes = {
|
||||
|
||||
// on first page
|
||||
0, 1, 2, 3,
|
||||
|
||||
// Near first physical page boundary (because the implementation
|
||||
// stores PDS + the page header size bytes on each page).
|
||||
PDS - 1, PDS, PDS + 1, PDS + 2, PDS + 3,
|
||||
|
||||
// near second physical page boundary
|
||||
(2 * PDS) - 1, (2 * PDS), (2 * PDS) + 1, (2 * PDS) + 2, (2 * PDS) + 3,
|
||||
|
||||
// near tenth physical page boundary
|
||||
(10 * PDS) - 1, (10 * PDS), (10 * PDS) + 1, (10 * PDS) + 2, (10 * PDS) + 3,
|
||||
|
||||
// test one big size, >> 4MB (an internal buffer size in the code)
|
||||
MAX_BYTES
|
||||
};
|
||||
|
||||
for (int i : dataSizes) {
|
||||
testReadAfterWriteRandomData(i);
|
||||
}
|
||||
}
|
||||
|
||||
private void testReadAfterWriteRandomData(int size) throws IOException {
|
||||
writeRandomData(size);
|
||||
readRandomDataAndVerify(size);
|
||||
}
|
||||
|
||||
/**
|
||||
* Read "size" bytes of data and verify that what was read and what was written
|
||||
* are the same.
|
||||
*/
|
||||
private void readRandomDataAndVerify(int size) throws AzureException, IOException {
|
||||
byte[] b = new byte[size];
|
||||
FSDataInputStream stream = fs.open(PATH);
|
||||
int bytesRead = stream.read(b);
|
||||
stream.close();
|
||||
assertEquals(bytesRead, size);
|
||||
|
||||
// compare the data read to the data written
|
||||
assertTrue(comparePrefix(randomData, b, size));
|
||||
}
|
||||
|
||||
// return true if the beginning "size" values of the arrays are the same
|
||||
private boolean comparePrefix(byte[] a, byte[] b, int size) {
|
||||
if (a.length < size || b.length < size) {
|
||||
return false;
|
||||
}
|
||||
for (int i = 0; i < size; i++) {
|
||||
if (a[i] != b[i]) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
// Write a specified amount of random data to the file path for this test class.
|
||||
private void writeRandomData(int size) throws IOException {
|
||||
OutputStream output = fs.create(PATH);
|
||||
output.write(randomData, 0, size);
|
||||
output.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Write data to a page blob, open it, seek, and then read a range of data.
|
||||
* Then compare that the data read from that range is the same as the data originally written.
|
||||
*/
|
||||
@Test
|
||||
public void testPageBlobSeekAndReadAfterWrite() throws IOException {
|
||||
writeRandomData(PAGE_SIZE * MAX_PAGES);
|
||||
int recordSize = 100;
|
||||
byte[] b = new byte[recordSize];
|
||||
FSDataInputStream stream = fs.open(PATH);
|
||||
|
||||
// Seek to a boundary around the middle of the 6th page
|
||||
int seekPosition = 5 * PAGE_SIZE + 250;
|
||||
stream.seek(seekPosition);
|
||||
|
||||
// Read a record's worth of bytes and verify results
|
||||
int bytesRead = stream.read(b);
|
||||
verifyReadRandomData(b, bytesRead, seekPosition, recordSize);
|
||||
|
||||
// Seek to another spot and read a record greater than a page
|
||||
seekPosition = 10 * PAGE_SIZE + 250;
|
||||
stream.seek(seekPosition);
|
||||
recordSize = 1000;
|
||||
b = new byte[recordSize];
|
||||
bytesRead = stream.read(b);
|
||||
verifyReadRandomData(b, bytesRead, seekPosition, recordSize);
|
||||
|
||||
// Read the last 100 bytes of the file
|
||||
recordSize = 100;
|
||||
seekPosition = PAGE_SIZE * MAX_PAGES - recordSize;
|
||||
stream.seek(seekPosition);
|
||||
b = new byte[recordSize];
|
||||
bytesRead = stream.read(b);
|
||||
verifyReadRandomData(b, bytesRead, seekPosition, recordSize);
|
||||
|
||||
// Read past the end of the file and we should get only partial data.
|
||||
recordSize = 100;
|
||||
seekPosition = PAGE_SIZE * MAX_PAGES - recordSize + 50;
|
||||
stream.seek(seekPosition);
|
||||
b = new byte[recordSize];
|
||||
bytesRead = stream.read(b);
|
||||
assertEquals(50, bytesRead);
|
||||
|
||||
// compare last 50 bytes written with those read
|
||||
byte[] tail = Arrays.copyOfRange(randomData, seekPosition, randomData.length);
|
||||
assertTrue(comparePrefix(tail, b, 50));
|
||||
}
|
||||
|
||||
// Verify that reading a record of data after seeking gives the expected data.
|
||||
private void verifyReadRandomData(byte[] b, int bytesRead, int seekPosition, int recordSize) {
|
||||
byte[] originalRecordData =
|
||||
Arrays.copyOfRange(randomData, seekPosition, seekPosition + recordSize + 1);
|
||||
assertEquals(recordSize, bytesRead);
|
||||
assertTrue(comparePrefix(originalRecordData, b, recordSize));
|
||||
}
|
||||
|
||||
// Test many small flushed writes interspersed with periodic hflush calls.
|
||||
// For manual testing, increase NUM_WRITES to a large number.
|
||||
// The goal for a long-running manual test is to make sure that it finishes
|
||||
// and the close() call does not time out. It also facilitates debugging into
|
||||
// hflush/hsync.
|
||||
@Test
|
||||
public void testManySmallWritesWithHFlush() throws IOException {
|
||||
writeAndReadOneFile(50, 100, 20);
|
||||
}
|
||||
|
||||
/**
|
||||
* Write a total of numWrites * recordLength data to a file, read it back,
|
||||
* and check to make sure what was read is the same as what was written.
|
||||
* The syncInterval is the number of writes after which to call hflush to
|
||||
* force the data to storage.
|
||||
*/
|
||||
private void writeAndReadOneFile(int numWrites, int recordLength, int syncInterval) throws IOException {
|
||||
final int NUM_WRITES = numWrites;
|
||||
final int RECORD_LENGTH = recordLength;
|
||||
final int SYNC_INTERVAL = syncInterval;
|
||||
|
||||
// A lower bound on the minimum time we think it will take to do
|
||||
// a write to Azure storage.
|
||||
final long MINIMUM_EXPECTED_TIME = 20;
|
||||
LOG.info("Writing " + NUM_WRITES * RECORD_LENGTH + " bytes to " + PATH.getName());
|
||||
FSDataOutputStream output = fs.create(PATH);
|
||||
int writesSinceHFlush = 0;
|
||||
try {
|
||||
|
||||
// Do a flush and hflush to exercise case for empty write queue in PageBlobOutputStream,
|
||||
// to test concurrent execution gates.
|
||||
output.flush();
|
||||
output.hflush();
|
||||
for (int i = 0; i < NUM_WRITES; i++) {
|
||||
output.write(randomData, i * RECORD_LENGTH, RECORD_LENGTH);
|
||||
writesSinceHFlush++;
|
||||
output.flush();
|
||||
if ((i % SYNC_INTERVAL) == 0) {
|
||||
long start = Time.monotonicNow();
|
||||
output.hflush();
|
||||
writesSinceHFlush = 0;
|
||||
long end = Time.monotonicNow();
|
||||
|
||||
// A true, round-trip synchronous flush to Azure must take
|
||||
// a significant amount of time or we are not syncing to storage correctly.
|
||||
LOG.debug("hflush duration = " + (end - start) + " msec.");
|
||||
assertTrue(String.format(
|
||||
"hflush duration of %d, less than minimum expected of %d",
|
||||
end - start, MINIMUM_EXPECTED_TIME),
|
||||
end - start >= MINIMUM_EXPECTED_TIME);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
long start = Time.monotonicNow();
|
||||
output.close();
|
||||
long end = Time.monotonicNow();
|
||||
LOG.debug("close duration = " + (end - start) + " msec.");
|
||||
if (writesSinceHFlush > 0) {
|
||||
assertTrue(String.format(
|
||||
"close duration with >= 1 pending write is %d, less than minimum expected of %d",
|
||||
end - start, MINIMUM_EXPECTED_TIME),
|
||||
end - start >= MINIMUM_EXPECTED_TIME);
|
||||
}
|
||||
}
|
||||
|
||||
// Read the data back and check it.
|
||||
FSDataInputStream stream = fs.open(PATH);
|
||||
int SIZE = NUM_WRITES * RECORD_LENGTH;
|
||||
byte[] b = new byte[SIZE];
|
||||
try {
|
||||
stream.seek(0);
|
||||
stream.read(b, 0, SIZE);
|
||||
verifyReadRandomData(b, SIZE, 0, SIZE);
|
||||
} finally {
|
||||
stream.close();
|
||||
}
|
||||
|
||||
// delete the file
|
||||
fs.delete(PATH, false);
|
||||
}
|
||||
|
||||
// Test writing to a large file repeatedly as a stress test.
|
||||
// Set the repetitions to a larger number for manual testing
|
||||
// for a longer stress run.
|
||||
@Test
|
||||
public void testLargeFileStress() throws IOException {
|
||||
int numWrites = 32;
|
||||
int recordSize = 1024 * 1024;
|
||||
int syncInterval = 10;
|
||||
int repetitions = 1;
|
||||
for (int i = 0; i < repetitions; i++) {
|
||||
writeAndReadOneFile(numWrites, recordSize, syncInterval);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -29,6 +29,7 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestWasbFsck {
|
||||
|
@ -63,6 +64,38 @@ public class TestWasbFsck {
|
|||
return count;
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that we recover files properly
|
||||
*/
|
||||
@Test
|
||||
@Ignore /* flush() no longer does anything @@TODO: reinstate an appropriate test of fsck recovery*/
|
||||
public void testRecover() throws Exception {
|
||||
Path danglingFile = new Path("/crashedInTheMiddle");
|
||||
|
||||
// Create a file and leave it dangling and try to recover it.
|
||||
FSDataOutputStream stream = fs.create(danglingFile);
|
||||
stream.write(new byte[] { 1, 2, 3 });
|
||||
stream.flush();
|
||||
|
||||
// Now we should still only see a zero-byte file in this place
|
||||
FileStatus fileStatus = fs.getFileStatus(danglingFile);
|
||||
assertNotNull(fileStatus);
|
||||
assertEquals(0, fileStatus.getLen());
|
||||
assertEquals(1, getNumTempBlobs());
|
||||
|
||||
// Run WasbFsck -move to recover the file.
|
||||
runFsck("-move");
|
||||
|
||||
// Now we should the see the file in lost+found with the data there.
|
||||
fileStatus = fs.getFileStatus(new Path("/lost+found",
|
||||
danglingFile.getName()));
|
||||
assertNotNull(fileStatus);
|
||||
assertEquals(3, fileStatus.getLen());
|
||||
assertEquals(0, getNumTempBlobs());
|
||||
// But not in its original location
|
||||
assertFalse(fs.exists(danglingFile));
|
||||
}
|
||||
|
||||
private void runFsck(String command) throws Exception {
|
||||
Configuration conf = fs.getConf();
|
||||
// Set the dangling cutoff to zero, so every temp blob is considered
|
||||
|
|
|
@ -274,8 +274,8 @@ public class TestWasbUriAndConfiguration {
|
|||
assumeNotNull(firstAccount);
|
||||
assumeNotNull(secondAccount);
|
||||
try {
|
||||
FileSystem firstFs = firstAccount.getFileSystem(), secondFs = secondAccount
|
||||
.getFileSystem();
|
||||
FileSystem firstFs = firstAccount.getFileSystem(),
|
||||
secondFs = secondAccount.getFileSystem();
|
||||
Path testFile = new Path("/testWasb");
|
||||
assertTrue(validateIOStreams(firstFs, testFile));
|
||||
assertTrue(validateIOStreams(secondFs, testFile));
|
||||
|
@ -356,13 +356,16 @@ public class TestWasbUriAndConfiguration {
|
|||
// the actual URI being asv(s)/wasb(s):///, it should work.
|
||||
|
||||
String[] wasbAliases = new String[] { "wasb", "wasbs" };
|
||||
for (String defaultScheme : wasbAliases){
|
||||
for (String defaultScheme : wasbAliases) {
|
||||
for (String wantedScheme : wasbAliases) {
|
||||
testAccount = AzureBlobStorageTestAccount.createMock();
|
||||
Configuration conf = testAccount.getFileSystem().getConf();
|
||||
String authority = testAccount.getFileSystem().getUri().getAuthority();
|
||||
URI defaultUri = new URI(defaultScheme, authority, null, null, null);
|
||||
conf.set("fs.default.name", defaultUri.toString());
|
||||
|
||||
// Add references to file system implementations for wasb and wasbs.
|
||||
conf.addResource("azure-test.xml");
|
||||
URI wantedUri = new URI(wantedScheme + ":///random/path");
|
||||
NativeAzureFileSystem obtained = (NativeAzureFileSystem) FileSystem
|
||||
.get(wantedUri, conf);
|
||||
|
|
|
@ -42,6 +42,7 @@ public final class AzureMetricsTestUtil {
|
|||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Gets the current value of the wasb_bytes_written_last_second counter.
|
||||
*/
|
||||
|
|
|
@ -543,4 +543,31 @@ public class TestAzureFileSystemInstrumentation {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A matcher class for asserting that a long value is in a
|
||||
* given range.
|
||||
*/
|
||||
private static class InRange extends BaseMatcher<Long> {
|
||||
private final long inclusiveLowerLimit;
|
||||
private final long inclusiveUpperLimit;
|
||||
private long obtained;
|
||||
|
||||
public InRange(long inclusiveLowerLimit, long inclusiveUpperLimit) {
|
||||
this.inclusiveLowerLimit = inclusiveLowerLimit;
|
||||
this.inclusiveUpperLimit = inclusiveUpperLimit;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean matches(Object number) {
|
||||
obtained = (Long)number;
|
||||
return obtained >= inclusiveLowerLimit &&
|
||||
obtained <= inclusiveUpperLimit;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void describeTo(Description description) {
|
||||
description.appendText("Between " + inclusiveLowerLimit +
|
||||
" and " + inclusiveUpperLimit + " inclusively");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,14 +20,10 @@ package org.apache.hadoop.fs.azure.metrics;
|
|||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assume.assumeNotNull;
|
||||
|
||||
import java.util.Date;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestBandwidthGaugeUpdater {
|
||||
|
@ -79,47 +75,4 @@ public class TestBandwidthGaugeUpdater {
|
|||
assertEquals(10 * threads.length, AzureMetricsTestUtil.getCurrentBytesRead(instrumentation));
|
||||
updater.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFinalizerThreadShutdown() throws Exception {
|
||||
|
||||
// force cleanup of any existing wasb filesystems
|
||||
System.gc();
|
||||
System.runFinalization();
|
||||
|
||||
int nUpdaterThreadsStart = getWasbThreadCount();
|
||||
assertTrue("Existing WASB threads have not been cleared", nUpdaterThreadsStart == 0);
|
||||
|
||||
final int nFilesystemsToSpawn = 10;
|
||||
AzureBlobStorageTestAccount testAccount = null;
|
||||
|
||||
for(int i = 0; i < nFilesystemsToSpawn; i++){
|
||||
testAccount = AzureBlobStorageTestAccount.createMock();
|
||||
testAccount.getFileSystem();
|
||||
}
|
||||
|
||||
int nUpdaterThreadsAfterSpawn = getWasbThreadCount();
|
||||
Assume.assumeTrue("Background threads should have spawned.", nUpdaterThreadsAfterSpawn == 10);
|
||||
|
||||
testAccount = null; //clear the last reachable reference
|
||||
|
||||
// force cleanup
|
||||
System.gc();
|
||||
System.runFinalization();
|
||||
|
||||
int nUpdaterThreadsAfterCleanup = getWasbThreadCount();
|
||||
assertTrue("Finalizers should have reduced the thread count. ", nUpdaterThreadsAfterCleanup == 0 );
|
||||
}
|
||||
|
||||
private int getWasbThreadCount() {
|
||||
int c = 0;
|
||||
Map<Thread, StackTraceElement[]> stacksStart = Thread.getAllStackTraces();
|
||||
for (Thread t : stacksStart.keySet()){
|
||||
if(t.getName().equals(BandwidthGaugeUpdater.THREAD_NAME))
|
||||
{
|
||||
c++;
|
||||
}
|
||||
}
|
||||
return c;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,18 +16,6 @@
|
|||
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
|
||||
<configuration xmlns:xi="http://www.w3.org/2001/XInclude">
|
||||
|
||||
<property>
|
||||
<name>fs.wasb.impl</name>
|
||||
<value>org.apache.hadoop.fs.azure.NativeAzureFileSystem</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.wasbs.impl</name>
|
||||
<value>org.apache.hadoop.fs.azure.NativeAzureFileSystem</value>
|
||||
</property>
|
||||
|
||||
|
||||
|
||||
<!-- For tests against live azure, provide the following account information -->
|
||||
<!--
|
||||
<property>
|
||||
|
|
Loading…
Reference in New Issue