Skip to content

Commit 28afdce

Browse files
committed
Revert ""HADOOP-16910. ABFS Streams to update FileSystem.Statistics counters on IO."
This reverts commit e2c7ac7. Change-Id: I5b5a93f5a36cdb0c3d56d1b3f747c318f089de20
1 parent 5250cd6 commit 28afdce

File tree

7 files changed

+7
-261
lines changed

7 files changed

+7
-261
lines changed

hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -188,7 +188,7 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi
188188
Path qualifiedPath = makeQualified(f);
189189

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

252252
try {
253-
OutputStream outputStream = abfsStore.openFileForWrite(qualifiedPath, statistics, false);
253+
OutputStream outputStream = abfsStore.openFileForWrite(qualifiedPath, false);
254254
return new FSDataOutputStream(outputStream, statistics);
255255
} catch(AzureBlobFileSystemException ex) {
256256
checkException(f, ex);

hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java

Lines changed: 3 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -412,10 +412,8 @@ public void deleteFilesystem() throws AzureBlobFileSystemException {
412412
}
413413
}
414414

415-
public OutputStream createFile(final Path path,
416-
final FileSystem.Statistics statistics,
417-
final boolean overwrite, final FsPermission permission,
418-
final FsPermission umask) throws AzureBlobFileSystemException {
415+
public OutputStream createFile(final Path path, final boolean overwrite, final FsPermission permission,
416+
final FsPermission umask) throws AzureBlobFileSystemException {
419417
try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) {
420418
boolean isNamespaceEnabled = getIsNamespaceEnabled();
421419
LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}",
@@ -438,7 +436,6 @@ public OutputStream createFile(final Path path,
438436

439437
return new AbfsOutputStream(
440438
client,
441-
statistics,
442439
AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path),
443440
0,
444441
abfsConfiguration.getWriteBufferSize(),
@@ -499,7 +496,7 @@ public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statist
499496
}
500497
}
501498

502-
public OutputStream openFileForWrite(final Path path, final FileSystem.Statistics statistics, final boolean overwrite) throws
499+
public OutputStream openFileForWrite(final Path path, final boolean overwrite) throws
503500
AzureBlobFileSystemException {
504501
try (AbfsPerfInfo perfInfo = startTracking("openFileForWrite", "getPathStatus")) {
505502
LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {}",
@@ -532,7 +529,6 @@ public OutputStream openFileForWrite(final Path path, final FileSystem.Statistic
532529

533530
return new AbfsOutputStream(
534531
client,
535-
statistics,
536532
AbfsHttpConstants.FORWARD_SLASH + getRelativePath(path),
537533
offset,
538534
abfsConfiguration.getWriteBufferSize(),

hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java

Lines changed: 0 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -101,7 +101,6 @@ public synchronized int read(final byte[] b, final int off, final int len) throw
101101
int currentLen = len;
102102
int lastReadBytes;
103103
int totalReadBytes = 0;
104-
incrementReadOps();
105104
do {
106105
lastReadBytes = readOneBlock(b, currentOff, currentLen);
107106
if (lastReadBytes > 0) {
@@ -202,7 +201,6 @@ private int readInternal(final long position, final byte[] b, final int offset,
202201
// try reading from buffers first
203202
receivedBytes = ReadBufferManager.getBufferManager().getBlock(this, position, length, b);
204203
if (receivedBytes > 0) {
205-
incrementReadOps();
206204
return receivedBytes;
207205
}
208206

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

258-
/**
259-
* Increment Read Operations.
260-
*/
261-
private void incrementReadOps() {
262-
if (statistics != null) {
263-
statistics.incrementReadOps(1);
264-
}
265-
}
266-
267255
/**
268256
* Seek to given position in stream.
269257
* @param n position to seek to

hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java

Lines changed: 0 additions & 15 deletions
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,6 @@
3939
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException;
4040
import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException;
4141
import org.apache.hadoop.io.ElasticByteBufferPool;
42-
import org.apache.hadoop.fs.FileSystem.Statistics;
4342
import org.apache.hadoop.fs.FSExceptionMessages;
4443
import org.apache.hadoop.fs.StreamCapabilities;
4544
import org.apache.hadoop.fs.Syncable;
@@ -81,11 +80,8 @@ public class AbfsOutputStream extends OutputStream implements Syncable, StreamCa
8180
private final ElasticByteBufferPool byteBufferPool
8281
= new ElasticByteBufferPool();
8382

84-
private final Statistics statistics;
85-
8683
public AbfsOutputStream(
8784
final AbfsClient client,
88-
final Statistics statistics,
8985
final String path,
9086
final long position,
9187
final int bufferSize,
@@ -94,7 +90,6 @@ public AbfsOutputStream(
9490
final boolean supportAppendWithFlush,
9591
final boolean appendBlob) {
9692
this.client = client;
97-
this.statistics = statistics;
9893
this.path = path;
9994
this.position = position;
10095
this.closed = false;
@@ -192,16 +187,6 @@ public synchronized void write(final byte[] data, final int off, final int lengt
192187

193188
writableBytes = bufferSize - bufferIndex;
194189
}
195-
incrementWriteOps();
196-
}
197-
198-
/**
199-
* Increment Write Operations.
200-
*/
201-
private void incrementWriteOps() {
202-
if (statistics != null) {
203-
statistics.incrementWriteOps(1);
204-
}
205190
}
206191

207192
/**

hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsTestWithTimeout.java

Lines changed: 0 additions & 59 deletions
Original file line numberDiff line numberDiff line change
@@ -17,19 +17,12 @@
1717
*/
1818
package org.apache.hadoop.fs.azurebfs;
1919

20-
import java.io.IOException;
21-
2220
import org.junit.Assert;
2321
import org.junit.Before;
2422
import org.junit.BeforeClass;
2523
import org.junit.Rule;
2624
import org.junit.rules.TestName;
2725
import org.junit.rules.Timeout;
28-
import org.slf4j.Logger;
29-
import org.slf4j.LoggerFactory;
30-
31-
import org.apache.hadoop.fs.FSDataInputStream;
32-
import org.apache.hadoop.fs.Path;
3326

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

@@ -38,9 +31,6 @@
3831
* This class does not attempt to bind to Azure.
3932
*/
4033
public class AbstractAbfsTestWithTimeout extends Assert {
41-
private static final Logger LOG =
42-
LoggerFactory.getLogger(AbstractAbfsTestWithTimeout.class);
43-
4434
/**
4535
* The name of the current method.
4636
*/
@@ -77,53 +67,4 @@ public void nameThread() {
7767
protected int getTestTimeoutMillis() {
7868
return TEST_TIMEOUT;
7969
}
80-
81-
/**
82-
* Describe a test in the logs.
83-
*
84-
* @param text text to print
85-
* @param args arguments to format in the printing
86-
*/
87-
protected void describe(String text, Object... args) {
88-
LOG.info("\n\n{}: {}\n",
89-
methodName.getMethodName(),
90-
String.format(text, args));
91-
}
92-
93-
/**
94-
* Validate Contents written on a file in Abfs.
95-
*
96-
* @param fs AzureBlobFileSystem
97-
* @param path Path of the file
98-
* @param originalByteArray original byte array
99-
* @return if content is validated true else, false
100-
* @throws IOException
101-
*/
102-
protected boolean validateContent(AzureBlobFileSystem fs, Path path,
103-
byte[] originalByteArray)
104-
throws IOException {
105-
int pos = 0;
106-
int lenOfOriginalByteArray = originalByteArray.length;
107-
108-
try (FSDataInputStream in = fs.open(path)) {
109-
byte valueOfContentAtPos = (byte) in.read();
110-
111-
while (valueOfContentAtPos != -1 && pos < lenOfOriginalByteArray) {
112-
if (originalByteArray[pos] != valueOfContentAtPos) {
113-
assertEquals("Mismatch in content validation at position {}", pos,
114-
originalByteArray[pos], valueOfContentAtPos);
115-
return false;
116-
}
117-
valueOfContentAtPos = (byte) in.read();
118-
pos++;
119-
}
120-
if (valueOfContentAtPos != -1) {
121-
assertEquals("Expected end of file", -1, valueOfContentAtPos);
122-
return false;
123-
}
124-
return true;
125-
}
126-
127-
}
128-
12970
}

hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsStreamStatistics.java

Lines changed: 0 additions & 157 deletions
This file was deleted.

0 commit comments

Comments
 (0)