HADOOP-17774. S3A bytesRead FS statistic showing twice the correct value (#3144)

Contributed by: Mehakmeet Singh
This commit is contained in:
Mehakmeet Singh 2021-07-02 18:33:16 +05:30 committed by GitHub
parent 390f8603d3
commit ea259f236c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
3 changed files with 86 additions and 1 deletions

View File

@ -712,6 +712,16 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
return instrumentation;
}
/**
* Get FS Statistic for this S3AFS instance.
*
* @return FS statistic instance.
*/
@VisibleForTesting
public FileSystem.Statistics getFsStatistics() {
return statistics;
}
/**
* Get current listing instance.
* @return this instance's listing.

View File

@ -988,6 +988,7 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
closed.incrementAndGet();
bytesDiscardedInClose.addAndGet(remainingInCurrentRequest);
totalBytesRead.addAndGet(remainingInCurrentRequest);
filesystemStatistics.incrementBytesRead(remainingInCurrentRequest);
}
}
@ -1144,7 +1145,6 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
// increment the filesystem statistics for this thread.
if (filesystemStatistics != null) {
long t = getTotalBytesRead();
filesystemStatistics.incrementBytesRead(t);
filesystemStatistics.incrementBytesReadByDistance(DISTANCE, t);
}
}

View File

@ -0,0 +1,75 @@
/*
* 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.s3a.statistics;
import java.io.IOException;
import org.junit.Test;
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.s3a.AbstractS3ATestBase;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
import org.apache.hadoop.fs.statistics.IOStatisticAssertions;
import org.apache.hadoop.fs.statistics.StreamStatisticNames;
public class ITestS3AFileSystemStatistic extends AbstractS3ATestBase {
private static final int ONE_KB = 1024;
private static final int TWO_KB = 2 * ONE_KB;
/**
* Verify the fs statistic bytesRead after reading from 2 different
* InputStreams for the same filesystem instance.
*/
@Test
public void testBytesReadWithStream() throws IOException {
S3AFileSystem fs = getFileSystem();
Path filePath = path(getMethodName());
byte[] oneKbBuf = new byte[ONE_KB];
// Writing 1KB in a file.
try (FSDataOutputStream out = fs.create(filePath)) {
out.write(oneKbBuf);
// Verify if correct number of bytes were written.
IOStatisticAssertions.assertThatStatisticCounter(out.getIOStatistics(),
StreamStatisticNames.STREAM_WRITE_BYTES)
.describedAs("Bytes written by OutputStream "
+ "should match the actual bytes")
.isEqualTo(ONE_KB);
}
// Reading 1KB from first InputStream.
try (FSDataInputStream in = fs.open(filePath, ONE_KB)) {
in.readFully(0, oneKbBuf);
}
// Reading 1KB from second InputStream.
try (FSDataInputStream in2 = fs.open(filePath, ONE_KB)) {
in2.readFully(0, oneKbBuf);
}
FileSystem.Statistics fsStats = fs.getFsStatistics();
// Verifying that total bytes read by FS is equal to 2KB.
assertEquals("Mismatch in number of FS bytes read by InputStreams", TWO_KB,
fsStats.getBytesRead());
}
}