Skip to content

Commit

Permalink
Skip Tests
Browse files Browse the repository at this point in the history
  • Loading branch information
fuatbasik committed Dec 12, 2024
1 parent f78df73 commit 87fa86d
Show file tree
Hide file tree
Showing 11 changed files with 92 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,11 @@

package org.apache.hadoop.fs.s3a;

import java.io.EOFException;
import java.io.IOException;

import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.fs.StreamCapabilities;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -31,7 +33,7 @@
import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamFactory;
import software.amazon.s3.analyticsaccelerator.util.S3URI;

public class S3ASeekableStream extends FSInputStream {
public class S3ASeekableStream extends FSInputStream implements StreamCapabilities {

private S3SeekableInputStream inputStream;
private long lastReadCurrentPos = 0;
Expand All @@ -44,6 +46,17 @@ public S3ASeekableStream(String bucket, String key, S3SeekableInputStreamFactory
this.key = key;
}

/**
* Indicates whether the given {@code capability} is supported by this stream.
*
* @param capability the capability to check.
* @return true if the given {@code capability} is supported by this stream, false otherwise.
*/
@Override
public boolean hasCapability(String capability) {
return false;
}

@Override
public int read() throws IOException {
throwIfClosed();
Expand All @@ -53,6 +66,10 @@ public int read() throws IOException {
@Override
public void seek(long pos) throws IOException {
throwIfClosed();
if (pos < 0) {
throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK
+ " " + pos);
}
inputStream.seek(pos);
}

Expand Down Expand Up @@ -92,6 +109,12 @@ public boolean seekToNewSource(long l) throws IOException {
return false;
}

@Override
public int available() throws IOException {
throwIfClosed();
return super.available();
}

@Override
public void close() throws IOException {
if (inputStream != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,7 @@
import org.apache.hadoop.fs.s3a.S3ATestUtils;

import static org.apache.hadoop.fs.s3a.Constants.CONNECTION_EXPECT_CONTINUE;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.setPerformanceFlags;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;

/**
* S3A contract tests creating files.
Expand Down Expand Up @@ -88,6 +87,13 @@ protected Configuration createConfiguration() {
return conf;
}

@Override
public void testOverwriteExistingFile() throws Throwable {
super.testOverwriteExistingFile();
skipIfAnalyticsAcceleratorEnabled(this.createConfiguration());

}

@Override
public void testOverwriteNonEmptyDirectory() throws Throwable {
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import static org.apache.hadoop.fs.s3a.Constants.*;
import static org.apache.hadoop.fs.s3a.S3ATestConstants.SCALE_TEST_TIMEOUT_MILLIS;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.StorageStatistics;
Expand Down Expand Up @@ -51,6 +52,12 @@ protected Configuration createConfiguration() {
return newConf;
}

@Override
public void setup() throws Exception {
super.setup();
skipIfAnalyticsAcceleratorEnabled(createConfiguration());
}

@Override
protected boolean shouldUseDirectWrite() {
return true;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyFileContents;
import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
import static org.apache.hadoop.fs.s3a.S3ATestConstants.S3A_TEST_TIMEOUT;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled;

/**
* S3A contract tests covering rename.
Expand All @@ -45,6 +46,12 @@ public class ITestS3AContractRename extends AbstractContractRenameTest {
public static final Logger LOG = LoggerFactory.getLogger(
ITestS3AContractRename.class);

@Override
public void setup() throws Exception {
super.setup();
skipIfAnalyticsAcceleratorEnabled(getContract().getConf());

}
@Override
protected int getTestTimeoutMillis() {
return S3A_TEST_TIMEOUT;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@
import static org.apache.hadoop.fs.contract.ContractTestUtils.range;
import static org.apache.hadoop.fs.contract.ContractTestUtils.returnBuffersToPoolPostRead;
import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString;
import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture;
Expand All @@ -84,6 +85,16 @@ protected AbstractFSContract createContract(Configuration conf) {
return new S3AContract(conf);
}

/**
* Analytics Accelerator Library for Amazon S3 does not support Vectored Reads
* @throws Exception
*/
@Override
public void setup() throws Exception {
skipIfAnalyticsAcceleratorEnabled(createConfiguration());
super.setup();
}

/**
* Verify response to a vector read request which is beyond the
* real length of the file.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,9 +38,7 @@
import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_KEY;
import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM;
import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_KEY;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeSkipRootTests;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;

/**
Expand Down Expand Up @@ -93,6 +91,7 @@ protected Configuration createConfiguration() {
@Override
public void setup() throws Exception {
super.setup();
skipIfAnalyticsAcceleratorEnabled(getConfiguration());
assumeEnabled();
// although not a root dir test, this confuses paths enough it shouldn't be run in
// parallel with other jobs
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.disablePrefetching;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES;
Expand Down Expand Up @@ -77,6 +78,8 @@ protected Configuration createConfiguration() {
public void setup() throws Exception {
super.setup();
executor = HadoopExecutors.newFixedThreadPool(SMALL_THREADS);
skipIfAnalyticsAcceleratorEnabled(getContract().getConf());

}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@
import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY;
import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_MAX_BLOCKS_COUNT;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValues;
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue;
Expand All @@ -68,6 +69,12 @@ public static Collection<Object[]> params() {
});
}

@Override
public void setup() throws Exception {
super.setup();
skipIfAnalyticsAcceleratorEnabled(createConfiguration());
}

public ITestS3APrefetchingLruEviction(final String maxBlocks) {
super(true);
this.maxBlocks = maxBlocks;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -574,6 +574,20 @@ public static boolean isS3ExpressTestBucket(final Configuration conf) {
return S3ExpressStorage.isS3ExpressStore(getTestBucketName(conf), "");
}

/**
* Skip a test if the Analytics Accelerator Library for Amazon S3 is enabled.
* @param configuration configuration to probe
*/
public static void skipIfAnalyticsAcceleratorEnabled(
Configuration configuration) {
assume("Skipping test as Analytics Accelerator Library does not support this functionality",
!isAnalyticsAcceleratorEnabled(configuration));
}

public static boolean isAnalyticsAcceleratorEnabled(final Configuration conf) {
return conf.getBoolean(ANALYTICS_ACCELERATOR_ENABLED_KEY, ANALYTICS_ACCELERATOR_ENABLED_DEFAULT);
}

/**
* Skip a test if the filesystem lacks a required capability.
* @param fs filesystem
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
import org.junit.runners.Parameterized;

import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled;
import static org.apache.hadoop.fs.s3a.S3AUtils.listAndFilter;
import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*;
import static org.apache.hadoop.fs.s3a.commit.impl.CommitUtilsWithMR.getMagicJobPath;
Expand Down Expand Up @@ -77,6 +78,7 @@ protected String getCommitterName() {
@Override
public void setup() throws Exception {
super.setup();
skipIfAnalyticsAcceleratorEnabled(getConfiguration());
CommitUtils.verifyIsMagicCommitFS(getFileSystem());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.hadoop.fs.contract.s3a.S3AContract;
import org.apache.hadoop.fs.statistics.StreamStatisticNames;

import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfAnalyticsAcceleratorEnabled;
import static org.apache.hadoop.fs.statistics.StreamStatisticNames.*;

/**
Expand Down Expand Up @@ -78,4 +79,10 @@ public List<String> outputStreamStatisticKeys() {
STREAM_WRITE_EXCEPTIONS);
}

@Override
public void setup() throws Exception {
super.setup();
skipIfAnalyticsAcceleratorEnabled(getContract().getConf());
}

}

0 comments on commit 87fa86d

Please sign in to comment.