Skip to content

Commit

Permalink
HADOOP-19354. checkstyle complaints
Browse files Browse the repository at this point in the history
Change-Id: Ic45539f10bb1911346b74a9a9e4e344b03071015
  • Loading branch information
steveloughran committed Jan 27, 2025
1 parent 537805c commit b5346a1
Show file tree
Hide file tree
Showing 4 changed files with 28 additions and 35 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -259,15 +259,15 @@ protected void serviceStart() throws Exception {
* @param capability non-null, non-empty string to query the path for support.
* @return known capabilities
*/
@Override
public boolean hasPathCapability(final Path path, final String capability) {
switch (toLowerCase(capability)) {
case StreamCapabilities.IOSTATISTICS:
return true;
default:
return hasCapability(capability);
}
@Override
public boolean hasPathCapability(final Path path, final String capability) {
switch (toLowerCase(capability)) {
case StreamCapabilities.IOSTATISTICS:
return true;
default:
return hasCapability(capability);
}
}


/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,9 @@
*/
public final class StreamIntegration {

private StreamIntegration() {
}

private static final Logger LOG_DEPRECATION =
LoggerFactory.getLogger(
"org.apache.hadoop.conf.Configuration.deprecation");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,8 @@ public class StreamThreadOptions {
* Create the thread options.
* @param sharedThreads Number of shared threads to included in the bounded pool.
* @param streamThreads How many threads per stream, ignoring vector IO requirements.
* @param createFuturePool Flag to enable creation of a future pool around the bounded thread pool.
* @param createFuturePool Flag to enable creation of a future pool around the
* bounded thread pool.
*/
public StreamThreadOptions(final int sharedThreads,
final int streamThreads,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,7 @@
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MAX;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MEAN;
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.SUFFIX_MIN;
import static org.apache.hadoop.io.Sizes.*;
import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;

/**
Expand All @@ -81,15 +82,15 @@
public class ITestS3AInputStreamPerformance extends S3AScaleTestBase {
private static final Logger LOG = LoggerFactory.getLogger(
ITestS3AInputStreamPerformance.class);
private static final int READAHEAD_128K = 128 * _1KB;
private static final int READAHEAD_128K = S_128K;

private S3AFileSystem s3aFS;
private Path testData;
private FileStatus testDataStatus;
private FSDataInputStream in;
private S3AInputStreamStatistics streamStatistics;
public static final int BLOCK_SIZE = 32 * 1024;
public static final int BIG_BLOCK_SIZE = 256 * 1024;
public static final int BLOCK_SIZE = S_32K;
public static final int BIG_BLOCK_SIZE = S_256K;

private static final IOStatisticsSnapshot IOSTATS = snapshotIOStatistics();

Expand All @@ -116,8 +117,8 @@ protected Configuration createScaleConfiguration() {
@Before
public void openFS() throws IOException {
Configuration conf = getConf();
conf.setInt(SOCKET_SEND_BUFFER, 16 * 1024);
conf.setInt(SOCKET_RECV_BUFFER, 16 * 1024);
conf.setInt(SOCKET_SEND_BUFFER, S_16K);
conf.setInt(SOCKET_RECV_BUFFER, S_16K);
// look up the test file, no requirement to be set.
String testFile = conf.getTrimmed(KEY_CSVTEST_FILE,
PublicDatasetTestUtils.DEFAULT_EXTERNAL_FILE);
Expand Down Expand Up @@ -281,7 +282,7 @@ public void testTimeToOpenAndReadWholeFileBlocks() throws Throwable {
// implicitly rounding down here
long blockCount = len / blockSize;
long totalToRead = blockCount * blockSize;
long minimumBandwidth = 128 * 1024;
long minimumBandwidth = S_128K;
int maxResetCount = 4;
int resetCount = 0;
for (long i = 0; i < blockCount; i++) {
Expand Down Expand Up @@ -471,22 +472,10 @@ protected void executeSeekReadSequence(long blockSize,
logStreamStatistics();
}

public static final int _4K = 4 * 1024;
public static final int _8K = 8 * 1024;
public static final int _16K = 16 * 1024;
public static final int _32K = 32 * 1024;
public static final int _64K = 64 * 1024;
public static final int _128K = 128 * 1024;
public static final int _256K = 256 * 1024;
public static final int _1MB = 1024 * 1024;
public static final int _2MB = 2 * _1MB;
public static final int _10MB = _1MB * 10;
public static final int _5MB = _1MB * 5;

private static final int[][] RANDOM_IO_SEQUENCE = {
{_2MB, _128K},
{_128K, _128K},
{_5MB, _64K},
{S_2M, S_128K},
{S_128K, S_128K},
{S_5M, S_64K},
{_1MB, _1MB},
};

Expand Down Expand Up @@ -534,7 +523,7 @@ private ContractTestUtils.NanoTimer executeRandomIO(S3AInputPolicy policy,
long expectedOpenCount)
throws IOException {
describe("Random IO with policy \"%s\"", policy);
byte[] buffer = new byte[_1MB];
byte[] buffer = new byte[S_1M];
long totalBytesRead = 0;
final long len = testDataStatus.getLen();
in = openTestFile(policy, 0);
Expand Down Expand Up @@ -586,15 +575,15 @@ S3AInputStream getS3aStream() {
public void testRandomReadOverBuffer() throws Throwable {
describe("read over a buffer, making sure that the requests" +
" spans readahead ranges");
int datasetLen = _32K;
int datasetLen = S_32K;
S3AFileSystem fs = getFileSystem();
Path dataFile = path("testReadOverBuffer.bin");
byte[] sourceData = dataset(datasetLen, 0, 64);
// relies on the field 'fs' referring to the R/W FS
writeDataset(fs, dataFile, sourceData, datasetLen, _16K, true);
writeDataset(fs, dataFile, sourceData, datasetLen, S_16K, true);
byte[] buffer = new byte[datasetLen];
int readahead = _8K;
int halfReadahead = _4K;
int readahead = S_8K;
int halfReadahead = S_4K;
in = openDataFile(fs, dataFile, S3AInputPolicy.Random, readahead, datasetLen);

LOG.info("Starting initial reads");
Expand Down

0 comments on commit b5346a1

Please sign in to comment.