Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -188,7 +188,7 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi
Path qualifiedPath = makeQualified(f);

try {
OutputStream outputStream = abfsStore.createFile(qualifiedPath, overwrite,
OutputStream outputStream = abfsStore.createFile(qualifiedPath, statistics, overwrite,
permission == null ? FsPermission.getFileDefault() : permission, FsPermission.getUMask(getConf()));
return new FSDataOutputStream(outputStream, statistics);
} catch(AzureBlobFileSystemException ex) {
Expand Down Expand Up @@ -250,7 +250,7 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr
Path qualifiedPath = makeQualified(f);

try {
OutputStream outputStream = abfsStore.openFileForWrite(qualifiedPath, false);
OutputStream outputStream = abfsStore.openFileForWrite(qualifiedPath, statistics, false);
return new FSDataOutputStream(outputStream, statistics);
} catch(AzureBlobFileSystemException ex) {
checkException(f, ex);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,8 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
Expand Down Expand Up @@ -97,8 +99,6 @@
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.http.client.utils.URIBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_EQUALS;
import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_FORWARD_SLASH;
Expand Down Expand Up @@ -391,8 +391,10 @@ public void deleteFilesystem() throws AzureBlobFileSystemException {
}
}

public OutputStream createFile(final Path path, final boolean overwrite, final FsPermission permission,
final FsPermission umask) throws AzureBlobFileSystemException {
public OutputStream createFile(final Path path,
final FileSystem.Statistics statistics,
final boolean overwrite, final FsPermission permission,
final FsPermission umask) throws AzureBlobFileSystemException {
try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) {
boolean isNamespaceEnabled = getIsNamespaceEnabled();
LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}",
Expand All @@ -409,12 +411,13 @@ public OutputStream createFile(final Path path, final boolean overwrite, final F
perfInfo.registerResult(op.getResult()).registerSuccess(true);

return new AbfsOutputStream(
client,
AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path),
0,
abfsConfiguration.getWriteBufferSize(),
abfsConfiguration.isFlushEnabled(),
abfsConfiguration.isOutputStreamFlushDisabled());
client,
statistics,
AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path),
0,
abfsConfiguration.getWriteBufferSize(),
abfsConfiguration.isFlushEnabled(),
abfsConfiguration.isOutputStreamFlushDisabled());
}
}

Expand Down Expand Up @@ -468,7 +471,7 @@ public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statist
}
}

public OutputStream openFileForWrite(final Path path, final boolean overwrite) throws
public OutputStream openFileForWrite(final Path path, final FileSystem.Statistics statistics, final boolean overwrite) throws
AzureBlobFileSystemException {
try (AbfsPerfInfo perfInfo = startTracking("openFileForWrite", "getPathStatus")) {
LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {}",
Expand All @@ -495,12 +498,13 @@ public OutputStream openFileForWrite(final Path path, final boolean overwrite) t
perfInfo.registerSuccess(true);

return new AbfsOutputStream(
client,
AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path),
offset,
abfsConfiguration.getWriteBufferSize(),
abfsConfiguration.isFlushEnabled(),
abfsConfiguration.isOutputStreamFlushDisabled());
client,
statistics,
AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path),
offset,
abfsConfiguration.getWriteBufferSize(),
abfsConfiguration.isFlushEnabled(),
abfsConfiguration.isOutputStreamFlushDisabled());
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,7 @@ public synchronized int read(final byte[] b, final int off, final int len) throw
int currentLen = len;
int lastReadBytes;
int totalReadBytes = 0;
incrementReadOps();
do {
lastReadBytes = readOneBlock(b, currentOff, currentLen);
if (lastReadBytes > 0) {
Expand Down Expand Up @@ -201,6 +202,7 @@ private int readInternal(final long position, final byte[] b, final int offset,
// try reading from buffers first
receivedBytes = ReadBufferManager.getBufferManager().getBlock(this, position, length, b);
if (receivedBytes > 0) {
incrementReadOps();
return receivedBytes;
}

Expand Down Expand Up @@ -236,6 +238,7 @@ int readRemote(long position, byte[] b, int offset, int length) throws IOExcepti
try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "readRemote", "read")) {
op = client.read(path, position, b, offset, length, tolerateOobAppends ? "*" : eTag);
perfInfo.registerResult(op.getResult()).registerSuccess(true);
incrementReadOps();
} catch (AzureBlobFileSystemException ex) {
if (ex instanceof AbfsRestOperationException) {
AbfsRestOperationException ere = (AbfsRestOperationException) ex;
Expand All @@ -252,6 +255,15 @@ int readRemote(long position, byte[] b, int offset, int length) throws IOExcepti
return (int) bytesRead;
}

/**
* Increment Read Operations.
*/
private void incrementReadOps() {
if (statistics != null) {
statistics.incrementReadOps(1);
}
}

/**
* Seek to given position in stream.
* @param n position to seek to
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
import org.apache.hadoop.io.ElasticByteBufferPool;
import org.apache.hadoop.fs.FileSystem.Statistics;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.StreamCapabilities;
import org.apache.hadoop.fs.Syncable;
Expand Down Expand Up @@ -78,14 +79,18 @@ public class AbfsOutputStream extends OutputStream implements Syncable, StreamCa
private final ElasticByteBufferPool byteBufferPool
= new ElasticByteBufferPool();

private final Statistics statistics;

public AbfsOutputStream(
final AbfsClient client,
final Statistics statistics,
final String path,
final long position,
final int bufferSize,
final boolean supportFlush,
final boolean disableOutputStreamFlush) {
this.client = client;
this.statistics = statistics;
this.path = path;
this.position = position;
this.closed = false;
Expand Down Expand Up @@ -181,6 +186,16 @@ public synchronized void write(final byte[] data, final int off, final int lengt

writableBytes = bufferSize - bufferIndex;
}
incrementWriteOps();
}

/**
* Increment Write Operations.
*/
private void incrementWriteOps() {
if (statistics != null) {
statistics.incrementWriteOps(1);
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,19 @@
*/
package org.apache.hadoop.fs.azurebfs;

import java.io.IOException;

import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.rules.TestName;
import org.junit.rules.Timeout;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.Path;

import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_TIMEOUT;

Expand All @@ -31,6 +38,9 @@
* This class does not attempt to bind to Azure.
*/
public class AbstractAbfsTestWithTimeout extends Assert {
private static final Logger LOG =
LoggerFactory.getLogger(AbstractAbfsTestWithTimeout.class);

/**
* The name of the current method.
*/
Expand Down Expand Up @@ -67,4 +77,53 @@ public void nameThread() {
protected int getTestTimeoutMillis() {
return TEST_TIMEOUT;
}

/**
* Describe a test in the logs.
*
* @param text text to print
* @param args arguments to format in the printing
*/
protected void describe(String text, Object... args) {
LOG.info("\n\n{}: {}\n",
methodName.getMethodName(),
String.format(text, args));
}

/**
* Validate Contents written on a file in Abfs.
*
* @param fs AzureBlobFileSystem
* @param path Path of the file
* @param originalByteArray original byte array
* @return if content is validated true else, false
* @throws IOException
*/
protected boolean validateContent(AzureBlobFileSystem fs, Path path,
byte[] originalByteArray)
throws IOException {
int pos = 0;
int lenOfOriginalByteArray = originalByteArray.length;

try (FSDataInputStream in = fs.open(path)) {
byte valueOfContentAtPos = (byte) in.read();

while (valueOfContentAtPos != -1 && pos < lenOfOriginalByteArray) {
if (originalByteArray[pos] != valueOfContentAtPos) {
assertEquals("Mismatch in content validation at position {}", pos,
originalByteArray[pos], valueOfContentAtPos);
return false;
}
valueOfContentAtPos = (byte) in.read();
pos++;
}
if (valueOfContentAtPos != -1) {
assertEquals("Expected end of file", -1, valueOfContentAtPos);
return false;
}
return true;
}

}

}
Loading