From 7ba1a24e83233f282133bf5edc4e45de24d95276 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 2 Oct 2023 18:20:12 +0100 Subject: [PATCH 1/3] HADOOP-18830-cut-s3-select Cut out S3 Select * leave public/unstable constants alone * s3guard tool will fail with error * s3afs. path capability will fail * openFile() will fail with specific error * s3 select doc updated * Cut eventstream jar * New test: ITestSelectUnsupported verifies new failure handling above Everything else cut, including tests. Change-Id: Iff8c1e6562ed2d47abefe2cfef2909c00c78a55d --- hadoop-project/pom.xml | 5 - hadoop-tools/hadoop-aws/pom.xml | 5 - .../apache/hadoop/fs/s3a/S3AFileSystem.java | 129 +- .../hadoop/fs/s3a/S3ObjectAttributes.java | 2 +- .../org/apache/hadoop/fs/s3a/Statistic.java | 4 - .../hadoop/fs/s3a/WriteOperationHelper.java | 75 -- .../apache/hadoop/fs/s3a/WriteOperations.java | 29 - .../hadoop/fs/s3a/api/RequestFactory.java | 9 - .../fs/s3a/audit/AWSRequestAnalyzer.java | 8 - .../hadoop/fs/s3a/impl/ChangeTracker.java | 2 +- .../hadoop/fs/s3a/impl/InternalConstants.java | 2 - .../hadoop/fs/s3a/impl/OpenFileSupport.java | 74 +- .../fs/s3a/impl/OperationCallbacks.java | 2 +- .../fs/s3a/impl/RequestFactoryImpl.java | 15 - .../hadoop/fs/s3a/s3guard/S3GuardTool.java | 12 +- .../fs/s3a/select/BlockingEnumeration.java | 156 --- .../s3a/select/InternalSelectConstants.java | 77 -- .../hadoop/fs/s3a/select/SelectBinding.java | 428 ------- .../hadoop/fs/s3a/select/SelectConstants.java | 9 +- .../select/SelectEventStreamPublisher.java | 124 -- .../fs/s3a/select/SelectInputStream.java | 455 ------- .../s3a/select/SelectObjectContentHelper.java | 114 -- .../hadoop/fs/s3a/select/SelectTool.java | 347 ----- .../hadoop/fs/s3a/select/package-info.java | 7 +- .../markdown/tools/hadoop-aws/encryption.md | 1 - .../markdown/tools/hadoop-aws/s3_select.md | 1117 +---------------- .../site/markdown/tools/hadoop-aws/testing.md | 36 +- .../tools/hadoop-aws/third_party_stores.md | 5 - .../tools/hadoop-aws/troubleshooting_s3a.md | 4 +- .../fs/s3a/impl/TestRequestFactory.java | 1 - .../fs/s3a/select/AbstractS3SelectTest.java | 756 ----------- .../hadoop/fs/s3a/select/ITestS3Select.java | 981 --------------- .../fs/s3a/select/ITestS3SelectCLI.java | 357 ------ .../fs/s3a/select/ITestS3SelectLandsat.java | 435 ------- .../fs/s3a/select/ITestS3SelectMRJob.java | 216 ---- .../fs/s3a/select/ITestSelectUnsupported.java | 100 ++ .../hadoop/fs/s3a/select/StreamPublisher.java | 89 -- .../s3a/select/TestBlockingEnumeration.java | 200 --- .../TestSelectEventStreamPublisher.java | 190 --- .../MinimalWriteOperationHelperCallbacks.java | 13 +- .../fs/s3a/{select => tools}/CsvFile.java | 2 +- 41 files changed, 160 insertions(+), 6433 deletions(-) delete mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/BlockingEnumeration.java delete mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/InternalSelectConstants.java delete mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java delete mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectEventStreamPublisher.java delete mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java delete mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectObjectContentHelper.java delete mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectTool.java delete mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/AbstractS3SelectTest.java delete mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java delete mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectCLI.java delete mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java delete mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestSelectUnsupported.java delete mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/StreamPublisher.java delete mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/TestBlockingEnumeration.java delete mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/TestSelectEventStreamPublisher.java rename hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/{select => tools}/CsvFile.java (98%) diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index d5a6024b4e59c..1a135ded88098 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -1121,11 +1121,6 @@ - - software.amazon.eventstream - eventstream - ${aws.eventstream.version} - org.apache.mina mina-core diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 5ff4223e2d60b..5a0f2356b5e4a 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -508,11 +508,6 @@ bundle compile - - software.amazon.eventstream - eventstream - test - org.assertj assertj-core diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index c5e6e09a835eb..de48c2df15698 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -83,8 +83,6 @@ import software.amazon.awssdk.services.s3.model.PutObjectResponse; import software.amazon.awssdk.services.s3.model.S3Error; import software.amazon.awssdk.services.s3.model.S3Object; -import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; -import software.amazon.awssdk.services.s3.model.SelectObjectContentResponseHandler; import software.amazon.awssdk.services.s3.model.StorageClass; import software.amazon.awssdk.services.s3.model.UploadPartRequest; import software.amazon.awssdk.services.s3.model.UploadPartResponse; @@ -194,8 +192,6 @@ import org.apache.hadoop.fs.s3a.commit.PutTracker; import org.apache.hadoop.fs.s3a.commit.MagicCommitIntegration; import org.apache.hadoop.fs.s3a.impl.ChangeTracker; -import org.apache.hadoop.fs.s3a.select.SelectBinding; -import org.apache.hadoop.fs.s3a.select.SelectConstants; import org.apache.hadoop.fs.s3a.s3guard.S3Guard; import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; @@ -299,7 +295,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, private S3Client s3Client; - /** Async client is used for transfer manager and s3 select. */ + /** Async client is used for transfer manager. */ private S3AsyncClient s3AsyncClient; // initial callback policy is fail-once; it's there just to assist @@ -1725,8 +1721,7 @@ public FSDataInputStream open(Path f, int bufferSize) /** * Opens an FSDataInputStream at the indicated Path. * The {@code fileInformation} parameter controls how the file - * is opened, whether it is normal vs. an S3 select call, - * can a HEAD be skipped, etc. + * is opened, can a HEAD be skipped, etc. * @param path the file to open * @param fileInformation information about the file to open * @throws IOException IO failure. @@ -1853,13 +1848,6 @@ public CompletableFuture submit(final CallableRaisingIOE operation) { private final class WriteOperationHelperCallbacksImpl implements WriteOperationHelper.WriteOperationHelperCallbacks { - @Override - public CompletableFuture selectObjectContent( - SelectObjectContentRequest request, - SelectObjectContentResponseHandler responseHandler) { - return getS3AsyncClient().selectObjectContent(request, responseHandler); - } - @Override public CompleteMultipartUploadResponse completeMultipartUpload( CompleteMultipartUploadRequest request) { @@ -1872,7 +1860,7 @@ public CompleteMultipartUploadResponse completeMultipartUpload( * using FS state as well as the status. * @param fileStatus file status. * @param auditSpan audit span. - * @return a context for read and select operations. + * @return a context for read operations. */ @VisibleForTesting protected S3AReadOpContext createReadContext( @@ -5452,13 +5440,6 @@ public boolean hasPathCapability(final Path path, final String capability) // capability depends on FS configuration return isMagicCommitEnabled(); - case SelectConstants.S3_SELECT_CAPABILITY: - // select is only supported if enabled and client side encryption is - // disabled. - return !isCSEEnabled - && SelectBinding.isSelectEnabled(getConf()) - && !s3ExpressStore; - case CommonPathCapabilities.FS_CHECKSUMS: // capability depends on FS configuration return getConf().getBoolean(ETAG_CHECKSUM_ENABLED, @@ -5572,85 +5553,6 @@ public AWSCredentialProviderList shareCredentials(final String purpose) { return credentials.share(); } - /** - * This is a proof of concept of a select API. - * @param source path to source data - * @param options request configuration from the builder. - * @param fileInformation any passed in information. - * @return the stream of the results - * @throws IOException IO failure - */ - @Retries.RetryTranslated - @AuditEntryPoint - private FSDataInputStream select(final Path source, - final Configuration options, - final OpenFileSupport.OpenFileInformation fileInformation) - throws IOException { - requireSelectSupport(source); - final AuditSpan auditSpan = entryPoint(OBJECT_SELECT_REQUESTS, source); - final Path path = makeQualified(source); - String expression = fileInformation.getSql(); - final S3AFileStatus fileStatus = extractOrFetchSimpleFileStatus(path, - fileInformation); - - // readahead range can be dynamically set - S3ObjectAttributes objectAttributes = createObjectAttributes( - path, fileStatus); - ChangeDetectionPolicy changePolicy = fileInformation.getChangePolicy(); - S3AReadOpContext readContext = createReadContext( - fileStatus, - auditSpan); - fileInformation.applyOptions(readContext); - - if (changePolicy.getSource() != ChangeDetectionPolicy.Source.None - && fileStatus.getEtag() != null) { - // if there is change detection, and the status includes at least an - // etag, - // check that the object metadata lines up with what is expected - // based on the object attributes (which may contain an eTag or - // versionId). - // This is because the select API doesn't offer this. - // (note: this is trouble for version checking as cannot force the old - // version in the final read; nor can we check the etag match) - ChangeTracker changeTracker = - new ChangeTracker(uri.toString(), - changePolicy, - readContext.getS3AStatisticsContext() - .newInputStreamStatistics() - .getChangeTrackerStatistics(), - objectAttributes); - - // will retry internally if wrong version detected - Invoker readInvoker = readContext.getReadInvoker(); - getObjectMetadata(path, changeTracker, readInvoker, "select"); - } - // instantiate S3 Select support using the current span - // as the active span for operations. - SelectBinding selectBinding = new SelectBinding( - createWriteOperationHelper(auditSpan)); - - // build and execute the request - return selectBinding.select( - readContext, - expression, - options, - objectAttributes); - } - - /** - * Verify the FS supports S3 Select. - * @param source source file. - * @throws UnsupportedOperationException if not. - */ - private void requireSelectSupport(final Path source) throws - UnsupportedOperationException { - if (!isCSEEnabled && !SelectBinding.isSelectEnabled(getConf())) { - - throw new UnsupportedOperationException( - SelectConstants.SELECT_UNSUPPORTED); - } - } - /** * Get the file status of the source file. * If in the fileInformation parameter return that @@ -5681,16 +5583,14 @@ private S3AFileStatus extractOrFetchSimpleFileStatus( } /** - * Initiate the open() or select() operation. + * Initiate the open() operation. * This is invoked from both the FileSystem and FileContext APIs. * It's declared as an audit entry point but the span creation is pushed - * down into the open/select methods it ultimately calls. + * down into the open operation s it ultimately calls. * @param rawPath path to the file * @param parameters open file parameters from the builder. - * @return a future which will evaluate to the opened/selected file. + * @return a future which will evaluate to the opened file. * @throws IOException failure to resolve the link. - * @throws PathIOException operation is a select request but S3 select is - * disabled * @throws IllegalArgumentException unknown mandatory key */ @Override @@ -5706,20 +5606,9 @@ public CompletableFuture openFileWithOptions( parameters, getDefaultBlockSize()); CompletableFuture result = new CompletableFuture<>(); - if (!fileInformation.isS3Select()) { - // normal path. - unboundedThreadPool.submit(() -> - LambdaUtils.eval(result, - () -> executeOpen(path, fileInformation))); - } else { - // it is a select statement. - // fail fast if the operation is not available - requireSelectSupport(path); - // submit the query - unboundedThreadPool.submit(() -> - LambdaUtils.eval(result, - () -> select(path, parameters.getOptions(), fileInformation))); - } + unboundedThreadPool.submit(() -> + LambdaUtils.eval(result, + () -> executeOpen(path, fileInformation))); return result; } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java index 4fc5b8658b605..18912d5d3caef 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ObjectAttributes.java @@ -25,7 +25,7 @@ /** * This class holds attributes of an object independent of the * file status type. - * It is used in {@link S3AInputStream} and the select equivalent. + * It is used in {@link S3AInputStream} and elsewhere. * as a way to reduce parameters being passed * to the constructor of such class, * and elsewhere to be a source-neutral representation of a file status. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java index 72fc75b642415..ce3af3de803a4 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java @@ -265,10 +265,6 @@ public enum Statistic { StoreStatisticNames.OBJECT_PUT_BYTES_PENDING, "number of bytes queued for upload/being actively uploaded", TYPE_GAUGE), - OBJECT_SELECT_REQUESTS( - StoreStatisticNames.OBJECT_SELECT_REQUESTS, - "Count of S3 Select requests issued", - TYPE_COUNTER), STREAM_READ_ABORTED( StreamStatisticNames.STREAM_READ_ABORTED, "Count of times the TCP stream was aborted", diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java index f2ece63a854fa..3bbe000bf5b6e 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java @@ -22,7 +22,6 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.util.List; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; import software.amazon.awssdk.core.sync.RequestBody; @@ -33,8 +32,6 @@ import software.amazon.awssdk.services.s3.model.MultipartUpload; import software.amazon.awssdk.services.s3.model.PutObjectRequest; import software.amazon.awssdk.services.s3.model.PutObjectResponse; -import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; -import software.amazon.awssdk.services.s3.model.SelectObjectContentResponseHandler; import software.amazon.awssdk.services.s3.model.UploadPartRequest; import software.amazon.awssdk.services.s3.model.UploadPartResponse; import org.slf4j.Logger; @@ -49,16 +46,11 @@ import org.apache.hadoop.fs.s3a.api.RequestFactory; import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.s3a.impl.StoreContext; -import org.apache.hadoop.fs.s3a.select.SelectEventStreamPublisher; -import org.apache.hadoop.fs.s3a.select.SelectObjectContentHelper; import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; -import org.apache.hadoop.fs.s3a.select.SelectBinding; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.fs.store.audit.AuditSpanSource; -import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.functional.CallableRaisingIOE; -import org.apache.hadoop.util.Preconditions; import static org.apache.hadoop.util.Preconditions.checkNotNull; import static org.apache.hadoop.fs.s3a.Invoker.*; @@ -82,7 +74,6 @@ *
  • Other low-level access to S3 functions, for private use.
  • *
  • Failure handling, including converting exceptions to IOEs.
  • *
  • Integration with instrumentation.
  • - *
  • Evolution to add more low-level operations, such as S3 select.
  • * * * This API is for internal use only. @@ -615,63 +606,6 @@ public Configuration getConf() { return conf; } - public SelectObjectContentRequest.Builder newSelectRequestBuilder(Path path) { - try (AuditSpan span = getAuditSpan()) { - return getRequestFactory().newSelectRequestBuilder( - storeContext.pathToKey(path)); - } - } - - /** - * Execute an S3 Select operation. - * On a failure, the request is only logged at debug to avoid the - * select exception being printed. - * - * @param source source for selection - * @param request Select request to issue. - * @param action the action for use in exception creation - * @return response - * @throws IOException failure - */ - @Retries.RetryTranslated - public SelectEventStreamPublisher select( - final Path source, - final SelectObjectContentRequest request, - final String action) - throws IOException { - // no setting of span here as the select binding is (statically) created - // without any span. - String bucketName = request.bucket(); - Preconditions.checkArgument(bucket.equals(bucketName), - "wrong bucket: %s", bucketName); - if (LOG.isDebugEnabled()) { - LOG.debug("Initiating select call {} {}", - source, request.expression()); - LOG.debug(SelectBinding.toString(request)); - } - return invoker.retry( - action, - source.toString(), - true, - withinAuditSpan(getAuditSpan(), () -> { - try (DurationInfo ignored = - new DurationInfo(LOG, "S3 Select operation")) { - try { - return SelectObjectContentHelper.select( - writeOperationHelperCallbacks, source, request, action); - } catch (Throwable e) { - LOG.error("Failure of S3 Select request against {}", - source); - LOG.debug("S3 Select request against {}:\n{}", - source, - SelectBinding.toString(request), - e); - throw e; - } - } - })); - } - @Override public AuditSpan createSpan(final String operation, @Nullable final String path1, @@ -705,15 +639,6 @@ public RequestFactory getRequestFactory() { */ public interface WriteOperationHelperCallbacks { - /** - * Initiates a select request. - * @param request selectObjectContent request - * @param t selectObjectContent request handler - * @return selectObjectContentResult - */ - CompletableFuture selectObjectContent(SelectObjectContentRequest request, - SelectObjectContentResponseHandler t); - /** * Initiates a complete multi-part upload request. * @param request Complete multi-part upload request diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java index 0fda4921a30da..5ad9c9f9b6482 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java @@ -31,16 +31,13 @@ import software.amazon.awssdk.services.s3.model.MultipartUpload; import software.amazon.awssdk.services.s3.model.PutObjectRequest; import software.amazon.awssdk.services.s3.model.PutObjectResponse; -import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; import software.amazon.awssdk.services.s3.model.UploadPartRequest; import software.amazon.awssdk.services.s3.model.UploadPartResponse; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; -import org.apache.hadoop.fs.s3a.select.SelectEventStreamPublisher; import org.apache.hadoop.fs.store.audit.AuditSpanSource; import org.apache.hadoop.util.functional.CallableRaisingIOE; @@ -274,32 +271,6 @@ UploadPartResponse uploadPart(UploadPartRequest request, RequestBody body, */ Configuration getConf(); - /** - * Create a S3 Select request builder for the destination path. - * This does not build the query. - * @param path pre-qualified path for query - * @return the request builder - */ - SelectObjectContentRequest.Builder newSelectRequestBuilder(Path path); - - /** - * Execute an S3 Select operation. - * On a failure, the request is only logged at debug to avoid the - * select exception being printed. - * - * @param source source for selection - * @param request Select request to issue. - * @param action the action for use in exception creation - * @return response - * @throws IOException failure - */ - @Retries.RetryTranslated - SelectEventStreamPublisher select( - Path source, - SelectObjectContentRequest request, - String action) - throws IOException; - /** * Increment the write operation counter * of the filesystem. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java index 99a898f728166..73ad137a86d3c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java @@ -37,7 +37,6 @@ import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import software.amazon.awssdk.services.s3.model.ObjectIdentifier; import software.amazon.awssdk.services.s3.model.PutObjectRequest; -import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; import software.amazon.awssdk.services.s3.model.StorageClass; import software.amazon.awssdk.services.s3.model.UploadPartRequest; @@ -214,14 +213,6 @@ UploadPartRequest.Builder newUploadPartRequestBuilder( int partNumber, long size) throws PathIOException; - /** - * Create a S3 Select request builder for the destination object. - * This does not build the query. - * @param key object key - * @return the request builder - */ - SelectObjectContentRequest.Builder newSelectRequestBuilder(String key); - /** * Create the (legacy) V1 list request builder. * @param key key to list under diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java index 3df862055d197..e91710a0af3a0 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/audit/AWSRequestAnalyzer.java @@ -35,7 +35,6 @@ import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import software.amazon.awssdk.services.s3.model.ObjectIdentifier; import software.amazon.awssdk.services.s3.model.PutObjectRequest; -import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; import software.amazon.awssdk.services.s3.model.UploadPartCopyRequest; import software.amazon.awssdk.services.s3.model.UploadPartRequest; @@ -50,7 +49,6 @@ import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_DELETE_REQUEST; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_LIST_REQUEST; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_PUT_REQUEST; -import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OBJECT_SELECT_REQUESTS; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.STORE_EXISTS_PROBE; /** @@ -132,12 +130,6 @@ public RequestInfo analyze(SdkRequest request) { return writing(OBJECT_PUT_REQUEST, r.key(), 0); - } else if (request instanceof SelectObjectContentRequest) { - SelectObjectContentRequest r = - (SelectObjectContentRequest) request; - return reading(OBJECT_SELECT_REQUESTS, - r.key(), - 1); } else if (request instanceof UploadPartRequest) { UploadPartRequest r = (UploadPartRequest) request; return writing(MULTIPART_UPLOAD_PART_PUT, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java index 2c9d6857b46a2..0c56ca1f308bb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ChangeTracker.java @@ -223,7 +223,7 @@ public void processResponse(final CopyObjectResponse copyObjectResponse) * cause. * @param e the exception * @param operation the operation performed when the exception was - * generated (e.g. "copy", "read", "select"). + * generated (e.g. "copy", "read"). * @throws RemoteFileChangedException if the remote file has changed. */ public void processException(SdkException e, String operation) throws diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java index 8ebf8c013d10a..1d12a41008b6b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java @@ -113,8 +113,6 @@ private InternalConstants() { /** * The known keys used in a standard openFile call. - * if there's a select marker in there then the keyset - * used becomes that of the select operation. */ @InterfaceStability.Unstable public static final Set S3A_OPENFILE_KEYS; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OpenFileSupport.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OpenFileSupport.java index 4703d63567245..b841e8f786dc4 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OpenFileSupport.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OpenFileSupport.java @@ -35,8 +35,8 @@ import org.apache.hadoop.fs.s3a.S3AInputPolicy; import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus; import org.apache.hadoop.fs.s3a.S3AReadOpContext; -import org.apache.hadoop.fs.s3a.select.InternalSelectConstants; import org.apache.hadoop.fs.s3a.select.SelectConstants; +import org.apache.hadoop.fs.store.LogExactlyOnce; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE; import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH; @@ -68,6 +68,7 @@ public class OpenFileSupport { private static final Logger LOG = LoggerFactory.getLogger(OpenFileSupport.class); + public static final LogExactlyOnce LOG_NO_SQL_SELECT = new LogExactlyOnce(LOG); /** * For use when a value of an split/file length is unknown. */ @@ -153,12 +154,14 @@ public S3AReadOpContext applyDefaultOptions(S3AReadOpContext roc) { /** * Prepare to open a file from the openFile parameters. + * S3Select SQL is rejected if a mandatory opt, ignored if optional. * @param path path to the file * @param parameters open file parameters from the builder. * @param blockSize for fileStatus * @return open file options * @throws IOException failure to resolve the link. * @throws IllegalArgumentException unknown mandatory key + * @throws UnsupportedOperationException for S3 Select options. */ @SuppressWarnings("ChainOfInstanceofChecks") public OpenFileInformation prepareToOpenFile( @@ -167,21 +170,21 @@ public OpenFileInformation prepareToOpenFile( final long blockSize) throws IOException { Configuration options = parameters.getOptions(); Set mandatoryKeys = parameters.getMandatoryKeys(); - String sql = options.get(SelectConstants.SELECT_SQL, null); - boolean isSelect = sql != null; - // choice of keys depends on open type - if (isSelect) { - // S3 Select call adds a large set of supported mandatory keys - rejectUnknownMandatoryKeys( - mandatoryKeys, - InternalSelectConstants.SELECT_OPTIONS, - "for " + path + " in S3 Select operation"); - } else { - rejectUnknownMandatoryKeys( - mandatoryKeys, - InternalConstants.S3A_OPENFILE_KEYS, - "for " + path + " in non-select file I/O"); + // S3 Select is not supported in this release + if (options.get(SelectConstants.SELECT_SQL, null) != null) { + if (mandatoryKeys.contains(SelectConstants.SELECT_SQL)) { + // mandatory option: fail with a specific message. + throw new UnsupportedOperationException(SelectConstants.SELECT_UNSUPPORTED); + } else { + // optional; log once and continue + LOG_NO_SQL_SELECT.warn(SelectConstants.SELECT_UNSUPPORTED); + } } + // choice of keys depends on open type + rejectUnknownMandatoryKeys( + mandatoryKeys, + InternalConstants.S3A_OPENFILE_KEYS, + "for " + path + " in file I/O"); // where does a read end? long fileLength = LENGTH_UNKNOWN; @@ -281,8 +284,6 @@ public OpenFileInformation prepareToOpenFile( } return new OpenFileInformation() - .withS3Select(isSelect) - .withSql(sql) .withAsyncDrainThreshold( builderSupport.getPositiveLong(ASYNC_DRAIN_THRESHOLD, defaultReadAhead)) @@ -329,7 +330,6 @@ private S3AFileStatus createStatus(Path path, long length, long blockSize) { */ public OpenFileInformation openSimpleFile(final int bufferSize) { return new OpenFileInformation() - .withS3Select(false) .withAsyncDrainThreshold(defaultAsyncDrainThreshold) .withBufferSize(bufferSize) .withChangePolicy(changePolicy) @@ -357,15 +357,9 @@ public String toString() { */ public static final class OpenFileInformation { - /** Is this SQL? */ - private boolean isS3Select; - /** File status; may be null. */ private S3AFileStatus status; - /** SQL string if this is a SQL select file. */ - private String sql; - /** Active input policy. */ private S3AInputPolicy inputPolicy; @@ -415,18 +409,10 @@ public OpenFileInformation build() { return this; } - public boolean isS3Select() { - return isS3Select; - } - public S3AFileStatus getStatus() { return status; } - public String getSql() { - return sql; - } - public S3AInputPolicy getInputPolicy() { return inputPolicy; } @@ -454,9 +440,7 @@ public long getSplitEnd() { @Override public String toString() { return "OpenFileInformation{" + - "isSql=" + isS3Select + - ", status=" + status + - ", sql='" + sql + '\'' + + "status=" + status + ", inputPolicy=" + inputPolicy + ", changePolicy=" + changePolicy + ", readAheadRange=" + readAheadRange + @@ -475,16 +459,6 @@ public long getFileLength() { return fileLength; } - /** - * Set builder value. - * @param value new value - * @return the builder - */ - public OpenFileInformation withS3Select(final boolean value) { - isS3Select = value; - return this; - } - /** * Set builder value. * @param value new value @@ -495,16 +469,6 @@ public OpenFileInformation withStatus(final S3AFileStatus value) { return this; } - /** - * Set builder value. - * @param value new value - * @return the builder - */ - public OpenFileInformation withSql(final String value) { - sql = value; - return this; - } - /** * Set builder value. * @param value new value diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java index 9c88870633a35..5a5d537d7a65d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/OperationCallbacks.java @@ -69,7 +69,7 @@ S3ObjectAttributes createObjectAttributes( * Create the read context for reading from the referenced file, * using FS state as well as the status. * @param fileStatus file status. - * @return a context for read and select operations. + * @return a context for read operations. */ S3AReadOpContext createReadContext( FileStatus fileStatus); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java index 17a7189ae220d..c91324da7cb15 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java @@ -43,7 +43,6 @@ import software.amazon.awssdk.services.s3.model.MetadataDirective; import software.amazon.awssdk.services.s3.model.ObjectIdentifier; import software.amazon.awssdk.services.s3.model.PutObjectRequest; -import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; import software.amazon.awssdk.services.s3.model.ServerSideEncryption; import software.amazon.awssdk.services.s3.model.StorageClass; import software.amazon.awssdk.services.s3.model.UploadPartRequest; @@ -585,20 +584,6 @@ public UploadPartRequest.Builder newUploadPartRequestBuilder( return prepareRequest(builder); } - @Override - public SelectObjectContentRequest.Builder newSelectRequestBuilder(String key) { - SelectObjectContentRequest.Builder requestBuilder = - SelectObjectContentRequest.builder().bucket(bucket).key(key); - - EncryptionSecretOperations.getSSECustomerKey(encryptionSecrets).ifPresent(base64customerKey -> { - requestBuilder.sseCustomerAlgorithm(ServerSideEncryption.AES256.name()) - .sseCustomerKey(base64customerKey) - .sseCustomerKeyMD5(Md5Utils.md5AsBase64(Base64.getDecoder().decode(base64customerKey))); - }); - - return prepareRequest(requestBuilder); - } - @Override public ListObjectsRequest.Builder newListObjectsV1RequestBuilder( final String key, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java index 26b6acda30906..51bff4228be0f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java @@ -57,7 +57,7 @@ import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicyImpl; -import org.apache.hadoop.fs.s3a.select.SelectTool; +import org.apache.hadoop.fs.s3a.select.SelectConstants; import org.apache.hadoop.fs.s3a.tools.BucketTool; import org.apache.hadoop.fs.s3a.tools.MarkerTool; import org.apache.hadoop.fs.shell.CommandFormat; @@ -76,6 +76,7 @@ import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.staging.StagingCommitterConstants.FILESYSTEM_TEMP_PATH; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.S3A_DYNAMIC_CAPABILITIES; +import static org.apache.hadoop.fs.s3a.select.SelectConstants.SELECT_UNSUPPORTED; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.retrieveIOStatistics; import static org.apache.hadoop.fs.statistics.StoreStatisticNames.MULTIPART_UPLOAD_ABORTED; @@ -121,7 +122,6 @@ public abstract class S3GuardTool extends Configured implements Tool, "\t" + BucketInfo.NAME + " - " + BucketInfo.PURPOSE + "\n" + "\t" + BucketTool.NAME + " - " + BucketTool.PURPOSE + "\n" + "\t" + MarkerTool.MARKERS + " - " + MarkerTool.PURPOSE + "\n" + - "\t" + SelectTool.NAME + " - " + SelectTool.PURPOSE + "\n" + "\t" + Uploads.NAME + " - " + Uploads.PURPOSE + "\n"; private static final String E_UNSUPPORTED = "This command is no longer supported"; @@ -1004,11 +1004,9 @@ public static int run(Configuration conf, String... args) throws case Uploads.NAME: command = new Uploads(conf); break; - case SelectTool.NAME: - // the select tool is not technically a S3Guard tool, but it's on the CLI - // because this is the defacto S3 CLI. - command = new SelectTool(conf); - break; + case SelectConstants.NAME: + throw new ExitUtil.ExitException( + EXIT_UNSUPPORTED_VERSION, SELECT_UNSUPPORTED); default: printHelp(); throw new ExitUtil.ExitException(E_USAGE, diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/BlockingEnumeration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/BlockingEnumeration.java deleted file mode 100644 index 42000f1017259..0000000000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/BlockingEnumeration.java +++ /dev/null @@ -1,156 +0,0 @@ -/* - * 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.select; - -import java.util.Enumeration; -import java.util.NoSuchElementException; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.LinkedBlockingQueue; - -import org.reactivestreams.Subscriber; -import org.reactivestreams.Subscription; - -import software.amazon.awssdk.core.async.SdkPublisher; -import software.amazon.awssdk.core.exception.SdkException; - -/** - * Implements the {@link Enumeration} interface by subscribing to a - * {@link SdkPublisher} instance. The enumeration will buffer a fixed - * number of elements and only request new ones from the publisher - * when they are consumed. Calls to {@link #hasMoreElements()} and - * {@link #nextElement()} may block while waiting for new elements. - * @param the type of element. - */ -public final class BlockingEnumeration implements Enumeration { - private static final class Signal { - private final T element; - private final Throwable error; - - Signal(T element) { - this.element = element; - this.error = null; - } - - Signal(Throwable error) { - this.element = null; - this.error = error; - } - } - - private final Signal endSignal = new Signal<>((Throwable)null); - private final CompletableFuture subscription = new CompletableFuture<>(); - private final BlockingQueue> signalQueue; - private final int bufferSize; - private Signal current = null; - - /** - * Create an enumeration with a fixed buffer size and an - * optional injected first element. - * @param publisher the publisher feeding the enumeration. - * @param bufferSize the buffer size. - * @param firstElement (optional) first element the enumeration will return. - */ - public BlockingEnumeration(SdkPublisher publisher, - final int bufferSize, - final T firstElement) { - this.signalQueue = new LinkedBlockingQueue<>(); - this.bufferSize = bufferSize; - if (firstElement != null) { - this.current = new Signal<>(firstElement); - } - publisher.subscribe(new EnumerationSubscriber()); - } - - /** - * Create an enumeration with a fixed buffer size. - * @param publisher the publisher feeding the enumeration. - * @param bufferSize the buffer size. - */ - public BlockingEnumeration(SdkPublisher publisher, - final int bufferSize) { - this(publisher, bufferSize, null); - } - - @Override - public boolean hasMoreElements() { - if (current == null) { - try { - current = signalQueue.take(); - } catch (InterruptedException e) { - current = new Signal<>(e); - subscription.thenAccept(Subscription::cancel); - Thread.currentThread().interrupt(); - } - } - if (current.error != null) { - Throwable error = current.error; - current = endSignal; - if (error instanceof Error) { - throw (Error)error; - } else if (error instanceof SdkException) { - throw (SdkException)error; - } else { - throw SdkException.create("Unexpected error", error); - } - } - return current != endSignal; - } - - @Override - public T nextElement() { - if (!hasMoreElements()) { - throw new NoSuchElementException(); - } - T element = current.element; - current = null; - subscription.thenAccept(s -> s.request(1)); - return element; - } - - private final class EnumerationSubscriber implements Subscriber { - - @Override - public void onSubscribe(Subscription s) { - long request = bufferSize; - if (current != null) { - request--; - } - if (request > 0) { - s.request(request); - } - subscription.complete(s); - } - - @Override - public void onNext(T t) { - signalQueue.add(new Signal<>(t)); - } - - @Override - public void onError(Throwable t) { - signalQueue.add(new Signal<>(t)); - } - - @Override - public void onComplete() { - signalQueue.add(endSignal); - } - } -} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/InternalSelectConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/InternalSelectConstants.java deleted file mode 100644 index fbf5226afb82f..0000000000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/InternalSelectConstants.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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.select; - -import java.util.Arrays; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.fs.s3a.impl.InternalConstants; - -import static org.apache.hadoop.fs.s3a.select.SelectConstants.*; - -/** - * Constants for internal use in the org.apache.hadoop.fs.s3a module itself. - * Please don't refer to these outside of this module & its tests. - * If you find you need to then either the code is doing something it - * should not, or these constants need to be uprated to being - * public and stable entries. - */ -@InterfaceAudience.Private -public final class InternalSelectConstants { - - private InternalSelectConstants() { - } - - /** - * An unmodifiable set listing the options - * supported in {@code openFile()}. - */ - public static final Set SELECT_OPTIONS; - - /* - * Build up the options, pulling in the standard set too. - */ - static { - // when adding to this, please keep in alphabetical order after the - // common options and the SQL. - HashSet options = new HashSet<>(Arrays.asList( - SELECT_SQL, - SELECT_ERRORS_INCLUDE_SQL, - SELECT_INPUT_COMPRESSION, - SELECT_INPUT_FORMAT, - SELECT_OUTPUT_FORMAT, - CSV_INPUT_COMMENT_MARKER, - CSV_INPUT_HEADER, - CSV_INPUT_INPUT_FIELD_DELIMITER, - CSV_INPUT_QUOTE_CHARACTER, - CSV_INPUT_QUOTE_ESCAPE_CHARACTER, - CSV_INPUT_RECORD_DELIMITER, - CSV_OUTPUT_FIELD_DELIMITER, - CSV_OUTPUT_QUOTE_CHARACTER, - CSV_OUTPUT_QUOTE_ESCAPE_CHARACTER, - CSV_OUTPUT_QUOTE_FIELDS, - CSV_OUTPUT_RECORD_DELIMITER - )); - options.addAll(InternalConstants.S3A_OPENFILE_KEYS); - SELECT_OPTIONS = Collections.unmodifiableSet(options); - } -} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java deleted file mode 100644 index c3b8abbc2ea88..0000000000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectBinding.java +++ /dev/null @@ -1,428 +0,0 @@ -/* - * 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.select; - -import java.io.IOException; -import java.util.Locale; - -import software.amazon.awssdk.services.s3.model.CSVInput; -import software.amazon.awssdk.services.s3.model.CSVOutput; -import software.amazon.awssdk.services.s3.model.ExpressionType; -import software.amazon.awssdk.services.s3.model.InputSerialization; -import software.amazon.awssdk.services.s3.model.OutputSerialization; -import software.amazon.awssdk.services.s3.model.QuoteFields; -import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; -import org.apache.hadoop.util.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.PathIOException; -import org.apache.hadoop.fs.s3a.Retries; -import org.apache.hadoop.fs.s3a.S3AReadOpContext; -import org.apache.hadoop.fs.s3a.S3ObjectAttributes; -import org.apache.hadoop.fs.s3a.WriteOperationHelper; - -import static org.apache.hadoop.util.Preconditions.checkNotNull; -import static org.apache.commons.lang3.StringUtils.isNotEmpty; -import static org.apache.hadoop.fs.s3a.select.SelectConstants.*; - -/** - * Class to do the S3 select binding and build a select request from the - * supplied arguments/configuration. - * - * This class is intended to be instantiated by the owning S3AFileSystem - * instance to handle the construction of requests: IO is still done exclusively - * in the filesystem. - * - */ -public class SelectBinding { - - static final Logger LOG = - LoggerFactory.getLogger(SelectBinding.class); - - /** Operations on the store. */ - private final WriteOperationHelper operations; - - /** Is S3 Select enabled? */ - private final boolean enabled; - private final boolean errorsIncludeSql; - - /** - * Constructor. - * @param operations callback to owner FS, with associated span. - */ - public SelectBinding(final WriteOperationHelper operations) { - this.operations = checkNotNull(operations); - Configuration conf = getConf(); - this.enabled = isSelectEnabled(conf); - this.errorsIncludeSql = conf.getBoolean(SELECT_ERRORS_INCLUDE_SQL, false); - } - - Configuration getConf() { - return operations.getConf(); - } - - /** - * Is the service supported? - * @return true iff select is enabled. - */ - public boolean isEnabled() { - return enabled; - } - - /** - * Static probe for select being enabled. - * @param conf configuration - * @return true iff select is enabled. - */ - public static boolean isSelectEnabled(Configuration conf) { - return conf.getBoolean(FS_S3A_SELECT_ENABLED, true); - } - - /** - * Build and execute a select request. - * @param readContext the read context, which includes the source path. - * @param expression the SQL expression. - * @param builderOptions query options - * @param objectAttributes object attributes from a HEAD request - * @return an FSDataInputStream whose wrapped stream is a SelectInputStream - * @throws IllegalArgumentException argument failure - * @throws IOException failure building, validating or executing the request. - * @throws PathIOException source path is a directory. - */ - @Retries.RetryTranslated - public FSDataInputStream select( - final S3AReadOpContext readContext, - final String expression, - final Configuration builderOptions, - final S3ObjectAttributes objectAttributes) throws IOException { - - return new FSDataInputStream( - executeSelect(readContext, - objectAttributes, - builderOptions, - buildSelectRequest( - readContext.getPath(), - expression, - builderOptions - ))); - } - - /** - * Build a select request. - * @param path source path. - * @param expression the SQL expression. - * @param builderOptions config to extract other query options from - * @return the request to serve - * @throws IllegalArgumentException argument failure - * @throws IOException problem building/validating the request - */ - public SelectObjectContentRequest buildSelectRequest( - final Path path, - final String expression, - final Configuration builderOptions) - throws IOException { - Preconditions.checkState(isEnabled(), - "S3 Select is not enabled for %s", path); - - SelectObjectContentRequest.Builder request = operations.newSelectRequestBuilder(path); - buildRequest(request, expression, builderOptions); - return request.build(); - } - - /** - * Execute the select request. - * @param readContext read context - * @param objectAttributes object attributes from a HEAD request - * @param builderOptions the options which came in from the openFile builder. - * @param request the built up select request. - * @return a SelectInputStream - * @throws IOException failure - * @throws PathIOException source path is a directory. - */ - @Retries.RetryTranslated - private SelectInputStream executeSelect( - final S3AReadOpContext readContext, - final S3ObjectAttributes objectAttributes, - final Configuration builderOptions, - final SelectObjectContentRequest request) throws IOException { - - Path path = readContext.getPath(); - if (readContext.getDstFileStatus().isDirectory()) { - throw new PathIOException(path.toString(), - "Can't select " + path - + " because it is a directory"); - } - boolean sqlInErrors = builderOptions.getBoolean(SELECT_ERRORS_INCLUDE_SQL, - errorsIncludeSql); - String expression = request.expression(); - final String errorText = sqlInErrors ? expression : "Select"; - if (sqlInErrors) { - LOG.info("Issuing SQL request {}", expression); - } - SelectEventStreamPublisher selectPublisher = operations.select(path, request, errorText); - return new SelectInputStream(readContext, - objectAttributes, selectPublisher); - } - - /** - * Build the select request from the configuration built up - * in {@code S3AFileSystem.openFile(Path)} and the default - * options in the cluster configuration. - * - * Options are picked up in the following order. - *
      - *
    1. Options in {@code openFileOptions}.
    2. - *
    3. Options in the owning filesystem configuration.
    4. - *
    5. The default values in {@link SelectConstants}
    6. - *
    - * - * @param requestBuilder request to build up - * @param expression SQL expression - * @param builderOptions the options which came in from the openFile builder. - * @throws IllegalArgumentException if an option is somehow invalid. - * @throws IOException if an option is somehow invalid. - */ - void buildRequest( - final SelectObjectContentRequest.Builder requestBuilder, - final String expression, - final Configuration builderOptions) - throws IllegalArgumentException, IOException { - Preconditions.checkArgument(StringUtils.isNotEmpty(expression), - "No expression provided in parameter " + SELECT_SQL); - - final Configuration ownerConf = operations.getConf(); - - String inputFormat = builderOptions.get(SELECT_INPUT_FORMAT, - SELECT_FORMAT_CSV).toLowerCase(Locale.ENGLISH); - Preconditions.checkArgument(SELECT_FORMAT_CSV.equals(inputFormat), - "Unsupported input format %s", inputFormat); - String outputFormat = builderOptions.get(SELECT_OUTPUT_FORMAT, - SELECT_FORMAT_CSV) - .toLowerCase(Locale.ENGLISH); - Preconditions.checkArgument(SELECT_FORMAT_CSV.equals(outputFormat), - "Unsupported output format %s", outputFormat); - - requestBuilder.expressionType(ExpressionType.SQL); - requestBuilder.expression(expandBackslashChars(expression)); - - requestBuilder.inputSerialization( - buildCsvInput(ownerConf, builderOptions)); - requestBuilder.outputSerialization( - buildCSVOutput(ownerConf, builderOptions)); - } - - /** - * Build the CSV input format for a request. - * @param ownerConf FS owner configuration - * @param builderOptions options on the specific request - * @return the input format - * @throws IllegalArgumentException argument failure - * @throws IOException validation failure - */ - public InputSerialization buildCsvInput( - final Configuration ownerConf, - final Configuration builderOptions) - throws IllegalArgumentException, IOException { - - String headerInfo = opt(builderOptions, - ownerConf, - CSV_INPUT_HEADER, - CSV_INPUT_HEADER_OPT_DEFAULT, - true).toUpperCase(Locale.ENGLISH); - String commentMarker = xopt(builderOptions, - ownerConf, - CSV_INPUT_COMMENT_MARKER, - CSV_INPUT_COMMENT_MARKER_DEFAULT); - String fieldDelimiter = xopt(builderOptions, - ownerConf, - CSV_INPUT_INPUT_FIELD_DELIMITER, - CSV_INPUT_FIELD_DELIMITER_DEFAULT); - String recordDelimiter = xopt(builderOptions, - ownerConf, - CSV_INPUT_RECORD_DELIMITER, - CSV_INPUT_RECORD_DELIMITER_DEFAULT); - String quoteCharacter = xopt(builderOptions, - ownerConf, - CSV_INPUT_QUOTE_CHARACTER, - CSV_INPUT_QUOTE_CHARACTER_DEFAULT); - String quoteEscapeCharacter = xopt(builderOptions, - ownerConf, - CSV_INPUT_QUOTE_ESCAPE_CHARACTER, - CSV_INPUT_QUOTE_ESCAPE_CHARACTER_DEFAULT); - - // CSV input - CSVInput.Builder csvBuilder = CSVInput.builder() - .fieldDelimiter(fieldDelimiter) - .recordDelimiter(recordDelimiter) - .comments(commentMarker) - .quoteCharacter(quoteCharacter); - if (StringUtils.isNotEmpty(quoteEscapeCharacter)) { - csvBuilder.quoteEscapeCharacter(quoteEscapeCharacter); - } - csvBuilder.fileHeaderInfo(headerInfo); - - InputSerialization.Builder inputSerialization = - InputSerialization.builder() - .csv(csvBuilder.build()); - String compression = opt(builderOptions, - ownerConf, - SELECT_INPUT_COMPRESSION, - COMPRESSION_OPT_NONE, - true).toUpperCase(Locale.ENGLISH); - if (isNotEmpty(compression)) { - inputSerialization.compressionType(compression); - } - return inputSerialization.build(); - } - - /** - * Build CSV output format for a request. - * @param ownerConf FS owner configuration - * @param builderOptions options on the specific request - * @return the output format - * @throws IllegalArgumentException argument failure - * @throws IOException validation failure - */ - public OutputSerialization buildCSVOutput( - final Configuration ownerConf, - final Configuration builderOptions) - throws IllegalArgumentException, IOException { - String fieldDelimiter = xopt(builderOptions, - ownerConf, - CSV_OUTPUT_FIELD_DELIMITER, - CSV_OUTPUT_FIELD_DELIMITER_DEFAULT); - String recordDelimiter = xopt(builderOptions, - ownerConf, - CSV_OUTPUT_RECORD_DELIMITER, - CSV_OUTPUT_RECORD_DELIMITER_DEFAULT); - String quoteCharacter = xopt(builderOptions, - ownerConf, - CSV_OUTPUT_QUOTE_CHARACTER, - CSV_OUTPUT_QUOTE_CHARACTER_DEFAULT); - String quoteEscapeCharacter = xopt(builderOptions, - ownerConf, - CSV_OUTPUT_QUOTE_ESCAPE_CHARACTER, - CSV_OUTPUT_QUOTE_ESCAPE_CHARACTER_DEFAULT); - String quoteFields = xopt(builderOptions, - ownerConf, - CSV_OUTPUT_QUOTE_FIELDS, - CSV_OUTPUT_QUOTE_FIELDS_ALWAYS).toUpperCase(Locale.ENGLISH); - - CSVOutput.Builder csvOutputBuilder = CSVOutput.builder() - .quoteCharacter(quoteCharacter) - .quoteFields(QuoteFields.fromValue(quoteFields)) - .fieldDelimiter(fieldDelimiter) - .recordDelimiter(recordDelimiter); - if (!quoteEscapeCharacter.isEmpty()) { - csvOutputBuilder.quoteEscapeCharacter(quoteEscapeCharacter); - } - - // output is CSV, always - return OutputSerialization.builder() - .csv(csvOutputBuilder.build()) - .build(); - } - - /** - * Stringify the given SelectObjectContentRequest, as its - * toString() operator doesn't. - * @param request request to convert to a string - * @return a string to print. Does not contain secrets. - */ - public static String toString(final SelectObjectContentRequest request) { - StringBuilder sb = new StringBuilder(); - sb.append("SelectObjectContentRequest{") - .append("bucket name=").append(request.bucket()) - .append("; key=").append(request.key()) - .append("; expressionType=").append(request.expressionType()) - .append("; expression=").append(request.expression()); - InputSerialization input = request.inputSerialization(); - if (input != null) { - sb.append("; Input") - .append(input.toString()); - } else { - sb.append("; Input Serialization: none"); - } - OutputSerialization out = request.outputSerialization(); - if (out != null) { - sb.append("; Output") - .append(out.toString()); - } else { - sb.append("; Output Serialization: none"); - } - return sb.append("}").toString(); - } - - /** - * Resolve an option. - * @param builderOptions the options which came in from the openFile builder. - * @param fsConf configuration of the owning FS. - * @param base base option (no s3a: prefix) - * @param defVal default value. Must not be null. - * @param trim should the result be trimmed. - * @return the possibly trimmed value. - */ - static String opt(Configuration builderOptions, - Configuration fsConf, - String base, - String defVal, - boolean trim) { - String r = builderOptions.get(base, fsConf.get(base, defVal)); - return trim ? r.trim() : r; - } - - /** - * Get an option with backslash arguments transformed. - * These are not trimmed, so whitespace is significant. - * @param selectOpts options in the select call - * @param fsConf filesystem conf - * @param base base option name - * @param defVal default value - * @return the transformed value - */ - static String xopt(Configuration selectOpts, - Configuration fsConf, - String base, - String defVal) { - return expandBackslashChars( - opt(selectOpts, fsConf, base, defVal, false)); - } - - /** - * Perform escaping. - * @param src source string. - * @return the replaced value - */ - static String expandBackslashChars(String src) { - return src.replace("\\n", "\n") - .replace("\\\"", "\"") - .replace("\\t", "\t") - .replace("\\r", "\r") - .replace("\\\"", "\"") - // backslash substitution must come last - .replace("\\\\", "\\"); - } - - -} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectConstants.java index 0e2bf914f83c5..97771368769fd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectConstants.java @@ -25,13 +25,18 @@ * Options related to S3 Select. * * These options are set for the entire filesystem unless overridden - * as an option in the URI + * as an option in the URI. + * + * The S3 Select API is no longer supported -however this class is retained + * so that any application which imports the dependencies will still link. */ @InterfaceAudience.Public @InterfaceStability.Unstable public final class SelectConstants { - public static final String SELECT_UNSUPPORTED = "S3 Select is not supported"; + public static final String SELECT_UNSUPPORTED = "S3 Select is no longer supported"; + + public static final String NAME = "select"; private SelectConstants() { } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectEventStreamPublisher.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectEventStreamPublisher.java deleted file mode 100644 index c71ea5f1623a1..0000000000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectEventStreamPublisher.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * 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.select; - -import java.io.ByteArrayInputStream; -import java.io.InputStream; -import java.io.SequenceInputStream; -import java.util.concurrent.CompletableFuture; -import java.util.function.Consumer; - -import org.reactivestreams.Subscriber; - -import software.amazon.awssdk.core.async.SdkPublisher; -import software.amazon.awssdk.http.AbortableInputStream; -import software.amazon.awssdk.services.s3.model.EndEvent; -import software.amazon.awssdk.services.s3.model.RecordsEvent; -import software.amazon.awssdk.services.s3.model.SelectObjectContentEventStream; -import software.amazon.awssdk.services.s3.model.SelectObjectContentResponse; -import software.amazon.awssdk.utils.ToString; - -/** - * Async publisher of {@link SelectObjectContentEventStream}s returned - * from a SelectObjectContent call. - */ -public final class SelectEventStreamPublisher implements - SdkPublisher { - - private final CompletableFuture selectOperationFuture; - private final SelectObjectContentResponse response; - private final SdkPublisher publisher; - - /** - * Create the publisher. - * @param selectOperationFuture SelectObjectContent future - * @param response SelectObjectContent response - * @param publisher SelectObjectContentEventStream publisher to wrap - */ - public SelectEventStreamPublisher( - CompletableFuture selectOperationFuture, - SelectObjectContentResponse response, - SdkPublisher publisher) { - this.selectOperationFuture = selectOperationFuture; - this.response = response; - this.publisher = publisher; - } - - /** - * Retrieve an input stream to the subset of the S3 object that matched the select query. - * This is equivalent to loading the content of all RecordsEvents into an InputStream. - * This will lazily-load the content from S3, minimizing the amount of memory used. - * @param onEndEvent callback on the end event - * @return the input stream - */ - public AbortableInputStream toRecordsInputStream(Consumer onEndEvent) { - SdkPublisher recordInputStreams = this.publisher - .filter(e -> { - if (e instanceof RecordsEvent) { - return true; - } else if (e instanceof EndEvent) { - onEndEvent.accept((EndEvent) e); - } - return false; - }) - .map(e -> ((RecordsEvent) e).payload().asInputStream()); - - // Subscribe to the async publisher using an enumeration that will - // buffer a single chunk (RecordsEvent's payload) at a time and - // block until it is consumed. - // Also inject an empty stream as the first element that - // SequenceInputStream will request on construction. - BlockingEnumeration enumeration = - new BlockingEnumeration(recordInputStreams, 1, EMPTY_STREAM); - return AbortableInputStream.create( - new SequenceInputStream(enumeration), - this::cancel); - } - - /** - * The response from the SelectObjectContent call. - * @return the response object - */ - public SelectObjectContentResponse response() { - return response; - } - - @Override - public void subscribe(Subscriber subscriber) { - publisher.subscribe(subscriber); - } - - /** - * Cancel the operation. - */ - public void cancel() { - selectOperationFuture.cancel(true); - } - - @Override - public String toString() { - return ToString.builder("SelectObjectContentEventStream") - .add("response", response) - .add("publisher", publisher) - .build(); - } - - private static final InputStream EMPTY_STREAM = - new ByteArrayInputStream(new byte[0]); -} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java deleted file mode 100644 index 3586d83a0a434..0000000000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectInputStream.java +++ /dev/null @@ -1,455 +0,0 @@ -/* - * 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.select; - -import java.io.EOFException; -import java.io.IOException; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; - -import software.amazon.awssdk.core.exception.AbortedException; -import software.amazon.awssdk.http.AbortableInputStream; -import org.apache.hadoop.util.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.CanSetReadahead; -import org.apache.hadoop.fs.FSExceptionMessages; -import org.apache.hadoop.fs.FSInputStream; -import org.apache.hadoop.fs.PathIOException; -import org.apache.hadoop.fs.s3a.Retries; -import org.apache.hadoop.fs.s3a.S3AReadOpContext; -import org.apache.hadoop.fs.s3a.S3ObjectAttributes; -import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; -import org.apache.hadoop.io.IOUtils; - - -import static org.apache.hadoop.util.Preconditions.checkNotNull; -import static org.apache.commons.lang3.StringUtils.isNotEmpty; -import static org.apache.hadoop.fs.s3a.Invoker.once; -import static org.apache.hadoop.fs.s3a.S3AInputStream.validateReadahead; - -/** - * An input stream for S3 Select return values. - * This is simply an end-to-end GET request, without any - * form of seek or recovery from connectivity failures. - * - * Currently only seek and positioned read operations on the current - * location are supported. - * - * The normal S3 input counters are updated by this stream. - */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -public class SelectInputStream extends FSInputStream implements - CanSetReadahead { - - private static final Logger LOG = - LoggerFactory.getLogger(SelectInputStream.class); - - public static final String SEEK_UNSUPPORTED = "seek()"; - - /** - * Same set of arguments as for an S3AInputStream. - */ - private final S3ObjectAttributes objectAttributes; - - /** - * Tracks the current position. - */ - private AtomicLong pos = new AtomicLong(0); - - /** - * Closed flag. - */ - private final AtomicBoolean closed = new AtomicBoolean(false); - - /** - * Did the read complete successfully? - */ - private final AtomicBoolean completedSuccessfully = new AtomicBoolean(false); - - /** - * Abortable response stream. - * This is guaranteed to never be null. - */ - private final AbortableInputStream wrappedStream; - - private final String bucket; - - private final String key; - - private final String uri; - - private final S3AReadOpContext readContext; - - private final S3AInputStreamStatistics streamStatistics; - - private long readahead; - - /** - * Create the stream. - * The read attempt is initiated immediately. - * @param readContext read context - * @param objectAttributes object attributes from a HEAD request - * @param selectPublisher event stream publisher from the already executed call - * @throws IOException failure - */ - @Retries.OnceTranslated - public SelectInputStream( - final S3AReadOpContext readContext, - final S3ObjectAttributes objectAttributes, - final SelectEventStreamPublisher selectPublisher) throws IOException { - Preconditions.checkArgument(isNotEmpty(objectAttributes.getBucket()), - "No Bucket"); - Preconditions.checkArgument(isNotEmpty(objectAttributes.getKey()), - "No Key"); - this.objectAttributes = objectAttributes; - this.bucket = objectAttributes.getBucket(); - this.key = objectAttributes.getKey(); - this.uri = "s3a://" + this.bucket + "/" + this.key; - this.readContext = readContext; - this.readahead = readContext.getReadahead(); - this.streamStatistics = readContext.getS3AStatisticsContext() - .newInputStreamStatistics(); - - AbortableInputStream stream = once( - "S3 Select", - uri, - () -> { - return selectPublisher.toRecordsInputStream(e -> { - LOG.debug("Completed successful S3 select read from {}", uri); - completedSuccessfully.set(true); - }); - }); - - this.wrappedStream = checkNotNull(stream); - // this stream is already opened, so mark as such in the statistics. - streamStatistics.streamOpened(); - } - - @Override - public void close() throws IOException { - long skipped = 0; - boolean aborted = false; - if (!closed.getAndSet(true)) { - try { - // set up for aborts. - // if we know the available amount > readahead. Abort. - // - boolean shouldAbort = wrappedStream.available() > readahead; - if (!shouldAbort) { - // read our readahead range worth of data - skipped = wrappedStream.skip(readahead); - shouldAbort = wrappedStream.read() >= 0; - } - // now, either there is data left or not. - if (shouldAbort) { - // yes, more data. Abort and add this fact to the stream stats - aborted = true; - wrappedStream.abort(); - } - } catch (IOException | AbortedException e) { - LOG.debug("While closing stream", e); - } finally { - IOUtils.cleanupWithLogger(LOG, wrappedStream); - streamStatistics.streamClose(aborted, skipped); - streamStatistics.close(); - super.close(); - } - } - } - - /** - * Verify that the input stream is open. Non blocking; this gives - * the last state of the atomic {@link #closed} field. - * @throws PathIOException if the connection is closed. - */ - private void checkNotClosed() throws IOException { - if (closed.get()) { - throw new PathIOException(uri, FSExceptionMessages.STREAM_IS_CLOSED); - } - } - - @Override - public int available() throws IOException { - checkNotClosed(); - return wrappedStream.available(); - } - - @Override - @Retries.OnceTranslated - public synchronized long skip(final long n) throws IOException { - checkNotClosed(); - long skipped = once("skip", uri, () -> wrappedStream.skip(n)); - pos.addAndGet(skipped); - // treat as a forward skip for stats - streamStatistics.seekForwards(skipped, skipped); - return skipped; - } - - @Override - public long getPos() { - return pos.get(); - } - - /** - * Set the readahead. - * @param readahead The readahead to use. null means to use the default. - */ - @Override - public void setReadahead(Long readahead) { - this.readahead = validateReadahead(readahead); - } - - /** - * Get the current readahead value. - * @return the readahead - */ - public long getReadahead() { - return readahead; - } - - /** - * Read a byte. There's no attempt to recover, but AWS-SDK exceptions - * such as {@code SelectObjectContentEventException} are translated into - * IOExceptions. - * @return a byte read or -1 for an end of file. - * @throws IOException failure. - */ - @Override - @Retries.OnceTranslated - public synchronized int read() throws IOException { - checkNotClosed(); - int byteRead; - try { - byteRead = once("read()", uri, () -> wrappedStream.read()); - } catch (EOFException e) { - // this could be one of: end of file, some IO failure - if (completedSuccessfully.get()) { - // read was successful - return -1; - } else { - // the stream closed prematurely - LOG.info("Reading of S3 Select data from {} failed before all results " - + " were generated.", uri); - streamStatistics.readException(); - throw new PathIOException(uri, - "Read of S3 Select data did not complete"); - } - } - - if (byteRead >= 0) { - incrementBytesRead(1); - } - return byteRead; - } - - @SuppressWarnings("NullableProblems") - @Override - @Retries.OnceTranslated - public synchronized int read(final byte[] buf, final int off, final int len) - throws IOException { - checkNotClosed(); - validatePositionedReadArgs(pos.get(), buf, off, len); - if (len == 0) { - return 0; - } - - int bytesRead; - try { - streamStatistics.readOperationStarted(pos.get(), len); - bytesRead = wrappedStream.read(buf, off, len); - } catch (EOFException e) { - streamStatistics.readException(); - // the base implementation swallows EOFs. - return -1; - } - - incrementBytesRead(bytesRead); - streamStatistics.readOperationCompleted(len, bytesRead); - return bytesRead; - } - - /** - * Forward seeks are supported, but not backwards ones. - * Forward seeks are implemented using read, so - * means that long-distance seeks will be (literally) expensive. - * - * @param newPos new seek position. - * @throws PathIOException Backwards seek attempted. - * @throws EOFException attempt to seek past the end of the stream. - * @throws IOException IO failure while skipping bytes - */ - @Override - @Retries.OnceTranslated - public synchronized void seek(long newPos) throws IOException { - long current = getPos(); - long distance = newPos - current; - if (distance < 0) { - throw unsupported(SEEK_UNSUPPORTED - + " backwards from " + current + " to " + newPos); - } - if (distance == 0) { - LOG.debug("ignoring seek to current position."); - } else { - // the complicated one: Forward seeking. Useful for split files. - LOG.debug("Forward seek by reading {} bytes", distance); - long bytesSkipped = 0; - // read byte-by-byte, hoping that buffering will compensate for this. - // doing it this way ensures that the seek stops at exactly the right - // place. skip(len) can return a smaller value, at which point - // it's not clear what to do. - while(distance > 0) { - int r = read(); - if (r == -1) { - // reached an EOF too early - throw new EOFException("Seek to " + newPos - + " reached End of File at offset " + getPos()); - } - distance--; - bytesSkipped++; - } - // read has finished. - streamStatistics.seekForwards(bytesSkipped, bytesSkipped); - } - } - - /** - * Build an exception to raise when an operation is not supported here. - * @param action action which is Unsupported. - * @return an exception to throw. - */ - protected PathIOException unsupported(final String action) { - return new PathIOException( - String.format("s3a://%s/%s", bucket, key), - action + " not supported"); - } - - @Override - public boolean seekToNewSource(long targetPos) throws IOException { - return false; - } - - // Not supported. - @Override - public boolean markSupported() { - return false; - } - - @SuppressWarnings("NonSynchronizedMethodOverridesSynchronizedMethod") - @Override - public void mark(int readLimit) { - // Do nothing - } - - @SuppressWarnings("NonSynchronizedMethodOverridesSynchronizedMethod") - @Override - public void reset() throws IOException { - throw unsupported("Mark"); - } - - /** - * Aborts the IO. - */ - public void abort() { - if (!closed.get()) { - LOG.debug("Aborting"); - wrappedStream.abort(); - } - } - - /** - * Read at a specific position. - * Reads at a position earlier than the current {@link #getPos()} position - * will fail with a {@link PathIOException}. See {@link #seek(long)}. - * Unlike the base implementation And the requirements of the filesystem - * specification, this updates the stream position as returned in - * {@link #getPos()}. - * @param position offset in the stream. - * @param buffer buffer to read in to. - * @param offset offset within the buffer - * @param length amount of data to read. - * @return the result. - * @throws PathIOException Backwards seek attempted. - * @throws EOFException attempt to seek past the end of the stream. - * @throws IOException IO failure while seeking in the stream or reading data. - */ - @Override - public int read(final long position, - final byte[] buffer, - final int offset, - final int length) - throws IOException { - // maybe seek forwards to the position. - seek(position); - return read(buffer, offset, length); - } - - /** - * Increment the bytes read counter if there is a stats instance - * and the number of bytes read is more than zero. - * This also updates the {@link #pos} marker by the same value. - * @param bytesRead number of bytes read - */ - private void incrementBytesRead(long bytesRead) { - if (bytesRead > 0) { - pos.addAndGet(bytesRead); - } - streamStatistics.bytesRead(bytesRead); - if (readContext.getStats() != null && bytesRead > 0) { - readContext.getStats().incrementBytesRead(bytesRead); - } - } - - /** - * Get the Stream statistics. - * @return the statistics for this stream. - */ - @InterfaceAudience.Private - @InterfaceStability.Unstable - public S3AInputStreamStatistics getS3AStreamStatistics() { - return streamStatistics; - } - - /** - * String value includes statistics as well as stream state. - * Important: there are no guarantees as to the stability - * of this value. - * @return a string value for printing in logs/diagnostics - */ - @Override - @InterfaceStability.Unstable - public String toString() { - String s = streamStatistics.toString(); - synchronized (this) { - final StringBuilder sb = new StringBuilder( - "SelectInputStream{"); - sb.append(uri); - sb.append("; state ").append(!closed.get() ? "open" : "closed"); - sb.append("; pos=").append(getPos()); - sb.append("; readahead=").append(readahead); - sb.append('\n').append(s); - sb.append('}'); - return sb.toString(); - } - } -} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectObjectContentHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectObjectContentHelper.java deleted file mode 100644 index 8233e67eea0a5..0000000000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectObjectContentHelper.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * 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.select; - -import java.io.IOException; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; - -import software.amazon.awssdk.core.async.SdkPublisher; -import software.amazon.awssdk.core.exception.SdkException; -import software.amazon.awssdk.services.s3.model.SelectObjectContentEventStream; -import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; -import software.amazon.awssdk.services.s3.model.SelectObjectContentResponse; -import software.amazon.awssdk.services.s3.model.SelectObjectContentResponseHandler; - -import org.apache.commons.lang3.tuple.Pair; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.s3a.S3AUtils; - -import static org.apache.hadoop.fs.s3a.WriteOperationHelper.WriteOperationHelperCallbacks; - -/** - * Helper for SelectObjectContent queries against an S3 Bucket. - */ -public final class SelectObjectContentHelper { - - private SelectObjectContentHelper() { - } - - /** - * Execute an S3 Select operation. - * @param writeOperationHelperCallbacks helper callbacks - * @param source source for selection - * @param request Select request to issue. - * @param action the action for use in exception creation - * @return the select response event stream publisher - * @throws IOException on failure - */ - public static SelectEventStreamPublisher select( - WriteOperationHelperCallbacks writeOperationHelperCallbacks, - Path source, - SelectObjectContentRequest request, - String action) - throws IOException { - try { - Handler handler = new Handler(); - CompletableFuture selectOperationFuture = - writeOperationHelperCallbacks.selectObjectContent(request, handler); - return handler.eventPublisher(selectOperationFuture).join(); - } catch (Throwable e) { - if (e instanceof CompletionException) { - e = e.getCause(); - } - IOException translated; - if (e instanceof SdkException) { - translated = S3AUtils.translateException(action, source, - (SdkException)e); - } else { - translated = new IOException(e); - } - throw translated; - } - } - - private static class Handler implements SelectObjectContentResponseHandler { - private volatile CompletableFuture>> responseAndPublisherFuture = - new CompletableFuture<>(); - - private volatile SelectObjectContentResponse response; - - public CompletableFuture eventPublisher( - CompletableFuture selectOperationFuture) { - return responseAndPublisherFuture.thenApply(p -> - new SelectEventStreamPublisher(selectOperationFuture, - p.getLeft(), p.getRight())); - } - - @Override - public void responseReceived(SelectObjectContentResponse selectObjectContentResponse) { - this.response = selectObjectContentResponse; - } - - @Override - public void onEventStream(SdkPublisher publisher) { - responseAndPublisherFuture.complete(Pair.of(response, publisher)); - } - - @Override - public void exceptionOccurred(Throwable error) { - responseAndPublisherFuture.completeExceptionally(error); - } - - @Override - public void complete() { - } - } -} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectTool.java deleted file mode 100644 index 7a6c1afdc1fc3..0000000000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectTool.java +++ /dev/null @@ -1,347 +0,0 @@ -/* - * 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.select; - -import java.io.BufferedReader; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InputStreamReader; -import java.io.OutputStream; -import java.io.PrintStream; -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.Locale; -import java.util.Optional; -import java.util.Scanner; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FutureDataInputStreamBuilder; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; -import org.apache.hadoop.fs.shell.CommandFormat; -import org.apache.hadoop.util.DurationInfo; -import org.apache.hadoop.util.ExitUtil; -import org.apache.hadoop.util.OperationDuration; -import org.apache.hadoop.util.functional.FutureIO; - -import static org.apache.commons.lang3.StringUtils.isNotEmpty; -import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; -import static org.apache.hadoop.service.launcher.LauncherExitCodes.*; -import static org.apache.hadoop.fs.s3a.select.SelectConstants.*; - -/** - * This is a CLI tool for the select operation, which is available - * through the S3Guard command. - * - * Usage: - *
    - *   hadoop s3guard select [options] Path Statement
    - * 
    - */ -public class SelectTool extends S3GuardTool { - - private static final Logger LOG = - LoggerFactory.getLogger(SelectTool.class); - - public static final String NAME = "select"; - - public static final String PURPOSE = "make an S3 Select call"; - - private static final String USAGE = NAME - + " [OPTIONS]" - + " [-limit rows]" - + " [-header (use|none|ignore)]" - + " [-out path]" - + " [-expected rows]" - + " [-compression (gzip|bzip2|none)]" - + " [-inputformat csv]" - + " [-outputformat csv]" - + " -``` - -The output is printed, followed by some summary statistics, unless the `-out` -option is used to declare a destination file. In this mode -status will be logged to the console, but the output of the query will be -saved directly to the output file. - -### Example 1 - -Read the first 100 rows of the landsat dataset where cloud cover is zero: - -```bash -hadoop s3guard select -header use -compression gzip -limit 100 \ - s3a://landsat-pds/scene_list.gz \ - "SELECT * FROM S3OBJECT s WHERE s.cloudCover = '0.0'" -``` - -### Example 2 - -Return the `entityId` column for all rows in the dataset where the cloud -cover was "0.0", and save it to the file `output.csv`: - -```bash -hadoop s3guard select -header use -out s3a://mybucket/output.csv \ - -compression gzip \ - s3a://landsat-pds/scene_list.gz \ - "SELECT s.entityId from S3OBJECT s WHERE s.cloudCover = '0.0'" -``` - -This file will: - -1. Be UTF-8 encoded. -1. Have quotes on all columns returned. -1. Use commas as a separator. -1. Not have any header. - -The output can be saved to a file with the `-out` option. Note also that -`-D key=value` settings can be used to control the operation, if placed after -the `s3guard` command and before `select` - - -```bash -hadoop s3guard \ - -D s.s3a.select.output.csv.quote.fields=asneeded \ - select \ - -header use \ - -compression gzip \ - -limit 500 \ - -inputformat csv \ - -outputformat csv \ - -out s3a://hwdev-steve-new/output.csv \ - s3a://landsat-pds/scene_list.gz \ - "SELECT s.entityId from S3OBJECT s WHERE s.cloudCover = '0.0'" -``` - - -## Use in MR/Analytics queries: Partially Supported - -S3 Select support in analytics queries is only partially supported. -It does not work reliably with large source files where the work is split up, -and as the various query engines all assume that .csv and .json formats are splittable, -things go very wrong, fast. - -As a proof of concept *only*, S3 Select queries can be made through -MapReduce jobs which use any Hadoop `RecordReader` -class which uses the new `openFile()` API. - -Currently this consists of the following MRv2 readers. - -``` -org.apache.hadoop.mapreduce.lib.input.LineRecordReader -org.apache.hadoop.mapreduce.lib.input.FixedLengthRecordReader -``` - -And a limited number of the MRv1 record readers: - -``` -org.apache.hadoop.mapred.LineRecordReader -``` - -All of these readers use the new API and can be have its optional/mandatory -options set via the `JobConf` used when creating/configuring the reader. - -These readers are instantiated within input formats; the following -formats therefore support S3 Select. - -``` -org.apache.hadoop.mapreduce.lib.input.FixedLengthInputFormat -org.apache.hadoop.mapreduce.lib.input.KeyValueTextInputFormat -org.apache.hadoop.mapreduce.lib.input.NLineInputFormat -org.apache.hadoop.mapreduce.lib.input.TextInputFormat -org.apache.hadoop.mapred.KeyValueTextInputFormat -org.apache.hadoop.mapred.TextInputFormat -org.apache.hadoop.mapred.lib.NLineInputFormat -``` - -All `JobConf` options which begin with the prefix `mapreduce.job.input.file.option.` -will have that prefix stripped and the remainder used as the name for an option -when opening the file. - -All `JobConf` options which being with the prefix `mapreduce.job.input.file.must.` -will be converted into mandatory options. - -To use an S3 Select call, set the following options - -``` -mapreduce.job.input.file.must.fs.s3a.select.sql = -mapreduce.job.input.file.must.fs.s3a.select.input.format = CSV -mapreduce.job.input.file.must.fs.s3a.select.output.format = CSV -``` - -Further options may be set to tune the behaviour, for example: - -```java -jobConf.set("mapreduce.job.input.file.must.fs.s3a.select.input.csv.header", "use"); -``` - -*Note* How to tell if a reader has migrated to the new `openFile()` builder -API: - -Set a mandatory option which is not known; if the job does not fail then -an old reader is being used. - -```java -jobConf.set("mapreduce.job.input.file.must.unknown.option", "anything"); -``` - - -### Querying Compressed objects - -S3 Select queries can be made against gzipped source files; the S3A input -stream receives the output in text format, rather than as a (re)compressed -stream. - -To read a gzip file, set `fs.s3a.select.input.compression` to `gzip`. - -```java -jobConf.set("mapreduce.job.input.file.must.fs.s3a.select.input.compression", - "gzip"); -``` - - -Most of the Hadoop RecordReader classes automatically choose a decompressor -based on the extension of the source file. This causes problems when -reading `.gz` files, because S3 Select is automatically decompressing and -returning csv-formatted text. - -By default, a query across gzipped files will fail with the error -"IOException: not a gzip file" - -To avoid this problem, declare that the job should switch to the -"Passthrough Codec" for all files with a ".gz" extension: - -```java -jobConf.set("io.compression.codecs", - "org.apache.hadoop.io.compress.PassthroughCodec"); -jobConf.set("io.compress.passthrough.extension", ".gz"); -``` - -Obviously, this breaks normal `.gz` decompression: only set it on S3 Select -jobs. - -## S3 Select configuration options. - -Consult the javadocs for `org.apache.hadoop.fs.s3a.select.SelectConstants`. - -The listed options can be set in `core-site.xml`, supported by S3A per-bucket -configuration, and can be set programmatically on the `Configuration` object -use to configure a new filesystem instance. - -Any of these options can be set in the builder returned by the `openFile()` call -—simply set them through a chain of `builder.must()` operations. - -```xml - - fs.s3a.select.input.format - csv - Input format - - - - fs.s3a.select.output.format - csv - Output format - - - - fs.s3a.select.input.csv.comment.marker - # - In S3 Select queries: the marker for comment lines in CSV files - - - - fs.s3a.select.input.csv.record.delimiter - \n - In S3 Select queries over CSV files: the record delimiter. - \t is remapped to the TAB character, \r to CR \n to newline. \\ to \ - and \" to " - - - - - fs.s3a.select.input.csv.field.delimiter - , - In S3 Select queries over CSV files: the field delimiter. - \t is remapped to the TAB character, \r to CR \n to newline. \\ to \ - and \" to " - - - - - fs.s3a.select.input.csv.quote.character - " - In S3 Select queries over CSV files: quote character. - \t is remapped to the TAB character, \r to CR \n to newline. \\ to \ - and \" to " - - - - - fs.s3a.select.input.csv.quote.escape.character - \\ - In S3 Select queries over CSV files: quote escape character. - \t is remapped to the TAB character, \r to CR \n to newline. \\ to \ - and \" to " - - - - - fs.s3a.select.input.csv.header - none - In S3 Select queries over CSV files: what is the role of the header? One of "none", "ignore" and "use" - - - - fs.s3a.select.input.compression - none - In S3 Select queries, the source compression - algorithm. One of: "none" and "gzip" - - - - fs.s3a.select.output.csv.quote.fields - always - - In S3 Select queries: should fields in generated CSV Files be quoted? - One of: "always", "asneeded". - - - - - fs.s3a.select.output.csv.quote.character - " - - In S3 Select queries: the quote character for generated CSV Files. - - - - - fs.s3a.select.output.csv.quote.escape.character - \\ - - In S3 Select queries: the quote escape character for generated CSV Files. - - - - - fs.s3a.select.output.csv.record.delimiter - \n - - In S3 Select queries: the record delimiter for generated CSV Files. - - - - - fs.s3a.select.output.csv.field.delimiter - , - - In S3 Select queries: the field delimiter for generated CSV Files. - - - - - fs.s3a.select.errors.include.sql - false - - Include the SQL statement in errors: this is useful for development but - may leak security and Personally Identifying Information in production, - so must be disabled there. - - -``` - -## Security and Privacy - -SQL Injection attacks are the classic attack on data. -Because S3 Select is a read-only API, the classic ["Bobby Tables"](https://xkcd.com/327/) -attack to gain write access isn't going to work. Even so: sanitize your inputs. - -CSV does have security issues of its own, specifically: - -*Excel and other spreadsheets may interpret some fields beginning with special -characters as formula, and execute them* - -S3 Select does not appear vulnerable to this, but in workflows where untrusted -data eventually ends up in a spreadsheet (including Google Document spreadsheets), -the data should be sanitized/audited first. There is no support for -such sanitization in S3 Select or in the S3A connector. - -Logging Select statements may expose secrets if they are in the statement. -Even if they are just logged, this may potentially leak Personally Identifying -Information as covered in the EU GDPR legislation and equivalents. - -For both privacy and security reasons, SQL statements are not included -in exception strings by default, nor logged at INFO level. - -To enable them, set `fs.s3a.select.errors.include.sql` to `true`, either in the -site/application configuration, or as an option in the builder for a -single request. When set, the request will also be logged at -the INFO level of the log `org.apache.hadoop.fs.s3a.select.SelectBinding`. - -Personal Identifiable Information is not printed in the AWS S3 logs. -Those logs contain only the SQL keywords from the query planner. -All column names and literals are masked. Following is a sample log example: - -*Query:* - -```sql -SELECT * FROM S3OBJECT s; -``` - -*Log:* - -```sql -select (project (list (project_all))) (from (as str0 (id str1 case_insensitive))) -``` - -Note also that: - -1. Debug-level Hadoop logs for the module `org.apache.hadoop.fs.s3a` and other -components's debug logs may also log the SQL statements (e.g. aws-sdk HTTP logs). - -The best practise here is: only enable SQL in exceptions while developing -SQL queries, especially in an application/notebook where the exception -text is a lot easier to see than the application logs. - -In production: don't log or report. If you do, all logs and output must be -considered sensitive from security and privacy perspectives. - -The `hadoop s3guard select` command does enable the logging, so -can be used as an initial place to experiment with the SQL syntax. -Rationale: if you are constructing SQL queries on the command line, -your shell history is already tainted with the query. - -### Links - -* [CVE-2014-3524](https://cve.mitre.org/cgi-bin/cvename.cgi?name=CVE-2014-3524). -* [The Absurdly Underestimated Dangers of CSV Injection](http://georgemauer.net/2017/10/07/csv-injection.html). -* [Comma Separated Vulnerabilities](https://www.contextis.com/blog/comma-separated-vulnerabilities). - -### SQL Syntax - -The SQL Syntax directly supported by the AWS S3 Select API is [documented by -Amazon](https://docs.aws.amazon.com/AmazonS3/latest/dev/s3-glacier-select-sql-reference.html). - -* Use single quotes for all constants, not double quotes. -* All CSV column values are strings unless cast to a type -* Simple `SELECT` calls, no `JOIN`. - -### CSV formats - -"CSV" is less a format, more "a term meaning the data is in some nonstandard -line-by-line" text file, and there are even "multiline CSV files". - -S3 Select only supports a subset of the loose "CSV" concept, as covered in -the AWS documentation. There are also limits on how many columns and how -large a single line may be. - -The specific quotation character, field and record delimiters, comments and escape -characters can be configured in the Hadoop configuration. - -### Consistency, Concurrency and Error handling - -**Consistency** - -Since November 2020, AWS S3 has been fully consistent. -This also applies to S3 Select. -We do not know what happens if an object is overwritten while a query is active. - - -**Concurrency** - -The outcome of what happens when source file is overwritten while the result of -a select call is overwritten is undefined. - -The input stream returned by the operation is *NOT THREAD SAFE*. - -**Error Handling** - -If an attempt to issue an S3 select call fails, the S3A connector will -reissue the request if-and-only-if it believes a retry may succeed. -That is: it considers the operation to be idempotent and if the failure is -considered to be a recoverable connectivity problem or a server-side rejection -which can be retried (500, 503). - -If an attempt to read data from an S3 select stream (`org.apache.hadoop.fs.s3a.select.SelectInputStream)` fails partway through the read, *no attempt is made to retry the operation* - -In contrast, the normal S3A input stream tries to recover from (possibly transient) -failures by attempting to reopen the file. - - -## Performance - -The select operation is best when the least amount of data is returned by -the query, as this reduces the amount of data downloaded. - -* Limit the number of columns projected to only those needed. -* Use `LIMIT` to set an upper limit on the rows read, rather than implementing -a row counter in application code and closing the stream when reached. -This avoids having to abort the HTTPS connection and negotiate a new one -on the next S3 request. - -The select call itself can be slow, especially when the source is a multi-MB -compressed file with aggressive filtering in the `WHERE` clause. -Assumption: the select query starts at row 1 and scans through each row, -and does not return data until it has matched one or more rows. - -If the asynchronous nature of the `openFile().build().get()` sequence -can be taken advantage of, by performing other work before or in parallel -to the `get()` call: do it. - -## Troubleshooting - -### `NoClassDefFoundError: software/amazon/eventstream/MessageDecoder` - -Select operation failing with a missing eventstream class. - -``` -java.io.IOException: java.lang.NoClassDefFoundError: software/amazon/eventstream/MessageDecoder -at org.apache.hadoop.fs.s3a.select.SelectObjectContentHelper.select(SelectObjectContentHelper.java:75) -at org.apache.hadoop.fs.s3a.WriteOperationHelper.lambda$select$10(WriteOperationHelper.java:660) -at org.apache.hadoop.fs.store.audit.AuditingFunctions.lambda$withinAuditSpan$0(AuditingFunctions.java:62) -at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:122) -``` - -The eventstream JAR is not on the classpath/not in sync with the version of the full "bundle.jar" JDK - -Fix: get a compatible version of the JAR on the classpath. - -### SQL errors - -Getting S3 Select code to work is hard, though those knowledgeable in SQL -will find it easier. - -Problems can be split into: - -1. Basic configuration of the client to issue the query. -1. Bad SQL select syntax and grammar. -1. Datatype casting issues -1. Bad records/data in source files. -1. Failure to configure MR jobs to work correctly. - -The exceptions here are all based on the experience during writing tests; -more may surface with broader use. - -All failures other than network errors on request initialization are considered -unrecoverable and will not be reattempted. - -As parse-time errors always state the line and column of an error, you can -simplify debugging by breaking a SQL statement across lines, e.g. - -```java -String sql = "SELECT\n" - + "s.entityId \n" - + "FROM " + "S3OBJECT s WHERE\n" - + "s.\"cloudCover\" = '100.0'\n" - + " LIMIT 100"; -``` -Now if the error is declared as "line 4", it will be on the select conditions; -the column offset will begin from the first character on that row. - -The SQL Statements issued are only included in exceptions if `fs.s3a.select.errors.include.sql` -is explicitly set to true. This can be done in an application during development, -or in a `openFile()` option parameter. This should only be done during development, -to reduce the risk of logging security or privacy information. - - -### "mid-query" failures on large datasets - -S3 Select returns paged results; the source file is _not_ filtered in -one go in the initial request. - -This means that errors related to the content of the data (type casting, etc) -may only surface partway through the read. The errors reported in such a -case may be different than those raised on reading the first page of data, -where it will happen earlier on in the read process. - -### External Resources on for troubleshooting - -See: - -* [SELECT Command Reference](https://docs.aws.amazon.com/AmazonS3/latest/dev/s3-glacier-select-sql-reference-select.html) -* [SELECT Object Content](https://docs.aws.amazon.com/AmazonS3/latest/API/RESTObjectSELECTContent.html) - -### IOException: "not a gzip file" - -This surfaces when trying to read in data from a `.gz` source file through an MR -or other analytics query, and the gzip codec has tried to parse it. - -``` -java.io.IOException: not a gzip file -at org.apache.hadoop.io.compress.zlib.BuiltInGzipDecompressor.processBasicHeader(BuiltInGzipDecompressor.java:496) -at org.apache.hadoop.io.compress.zlib.BuiltInGzipDecompressor.executeHeaderState(BuiltInGzipDecompressor.java:257) -at org.apache.hadoop.io.compress.zlib.BuiltInGzipDecompressor.decompress(BuiltInGzipDecompressor.java:186) -at org.apache.hadoop.io.compress.DecompressorStream.decompress(DecompressorStream.java:111) -at org.apache.hadoop.io.compress.DecompressorStream.read(DecompressorStream.java:105) -at java.io.InputStream.read(InputStream.java:101) -at org.apache.hadoop.util.LineReader.fillBuffer(LineReader.java:182) -at org.apache.hadoop.util.LineReader.readCustomLine(LineReader.java:306) -at org.apache.hadoop.util.LineReader.readLine(LineReader.java:174) -at org.apache.hadoop.mapreduce.lib.input.LineRecordReader.skipUtfByteOrderMark(LineRecordReader.java:158) -at org.apache.hadoop.mapreduce.lib.input.LineRecordReader.nextKeyValue(LineRecordReader.java:198) -``` - -The underlying problem is that the gzip decompressor is automatically enabled -when the source file ends with the ".gz" extension. Because S3 Select -returns decompressed data, the codec fails. - -The workaround here is to declare that the job should add the "Passthrough Codec" -to its list of known decompressors, and that this codec should declare the -file format it supports to be ".gz". - -``` -io.compression.codecs = org.apache.hadoop.io.compress.PassthroughCodec -io.compress.passthrough.extension = .gz -``` - -### AWSBadRequestException `InvalidColumnIndex` - - -Your SQL is wrong and the element at fault is considered an unknown column -name. - -``` -org.apache.hadoop.fs.s3a.AWSBadRequestException: - Select: SELECT * FROM S3OBJECT WHERE odd = true on test/testSelectOddLines.csv: - com.amazonaws.services.s3.model.AmazonS3Exception: - The column index at line 1, column 30 is invalid. - Please check the service documentation and try again. - (Service: Amazon S3; Status Code: 400; Error Code: InvalidColumnIndex; -``` - -Here it's the first line of the query, column 30. Paste the query -into an editor and position yourself on the line and column at fault. - -```sql -SELECT * FROM S3OBJECT WHERE odd = true - ^ HERE -``` - -Another example: - -``` -org.apache.hadoop.fs.s3a.AWSBadRequestException: Select: -SELECT * FROM S3OBJECT s WHERE s._1 = "true" on test/testSelectOddLines.csv: - com.amazonaws.services.s3.model.AmazonS3Exception: - The column index at line 1, column 39 is invalid. - Please check the service documentation and try again. - (Service: Amazon S3; Status Code: 400; - Error Code: InvalidColumnIndex; -``` - -Here it is because strings must be single quoted, not double quoted. - -```sql -SELECT * FROM S3OBJECT s WHERE s._1 = "true" - ^ HERE -``` - -S3 select uses double quotes to wrap column names, interprets the string -as column "true", and fails with a non-intuitive message. - -*Tip*: look for the element at fault and treat the `InvalidColumnIndex` -message as a parse-time message, rather than the definitive root -cause of the problem. - -### AWSBadRequestException `ParseInvalidPathComponent` - -Your SQL is wrong. - -``` -org.apache.hadoop.fs.s3a.AWSBadRequestException: -Select: SELECT * FROM S3OBJECT s WHERE s.'odd' is "true" on test/testSelectOddLines.csv -: com.amazonaws.services.s3.model.AmazonS3Exception: Invalid Path component, - expecting either an IDENTIFIER or STAR, got: LITERAL,at line 1, column 34. - (Service: Amazon S3; Status Code: 400; Error Code: ParseInvalidPathComponent; - -``` - -``` -SELECT * FROM S3OBJECT s WHERE s.'odd' is "true" on test/testSelectOddLines.csv - ^ HERE -``` - - -### AWSBadRequestException `ParseExpectedTypeName` - -Your SQL is still wrong. - -``` - -org.apache.hadoop.fs.s3a.AWSBadRequestException: - Select: SELECT * FROM S3OBJECT s WHERE s.odd = "true" -on test/testSelectOddLines.csv: -com.amazonaws.services.s3.model.AmazonS3Exception -: Expected type name, found QUOTED_IDENTIFIER:'true' at line 1, column 41. -(Service: Amazon S3; Status Code: 400; Error Code: ParseExpectedTypeName; -``` - -### `ParseUnexpectedToken` - -Your SQL is broken. - -``` -org.apache.hadoop.fs.s3a.AWSBadRequestException: -Select: SELECT * FROM S3OBJECT s WHERE s.5 = `true` on test/testSelectOddLines.csv: -com.amazonaws.services.s3.model.AmazonS3Exception: -Unexpected token found LITERAL:5d-1 at line 1, column 33. -(Service: Amazon S3; Status Code: 400; Error Code: ParseUnexpectedToken; -``` -### `ParseUnexpectedOperator` - -Your SQL is broken. - -``` -com.amazonaws.services.s3.model.AmazonS3Exception: Unexpected operator OPERATOR:'%' at line 1, column 45. -(Service: Amazon S3; Status Code: 400; -Error Code: ParseUnexpectedOperator; Request ID: E87F30C57436B459; -S3 Extended Request ID: UBFOIgkQxBBL+bcBFPaZaPBsjdnd8NRz3NFWAgcctqm3n6f7ib9FMOpR+Eu1Cy6cNMYHCpJbYEY - =:ParseUnexpectedOperator: Unexpected operator OPERATOR:'%' at line 1, column 45. -at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357) -at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895) -``` - -### `MissingHeaders` - -``` -org.apache.hadoop.fs.s3a.AWSBadRequestException: -Select: SELECT * FROM S3OBJECT s WHERE s."odd" = `true` on test/testSelectOddLines.csv: -com.amazonaws.services.s3.model.AmazonS3Exception: -Some headers in the query are missing from the file. -Please check the file and try again. -(Service: Amazon S3; Status Code: 400; Error Code: MissingHeaders; -``` - -1. There's a header used in the query which doesn't match any in the document -itself. -1. The header option for the select query is set to "none" or "ignore", and -you are trying to use a header named there. - -This can happen if you are trying to use double quotes for constants in the -SQL expression. - -``` -SELECT * FROM S3OBJECT s WHERE s."odd" = "true" on test/testSelectOddLines.csv: - ^ HERE -``` - -Double quotes (") may only be used when naming columns; for constants -single quotes are required. - -### Method not allowed - -``` -org.apache.hadoop.fs.s3a.AWSS3IOException: Select on test/testSelectWholeFile: -com.amazonaws.services.s3.model.AmazonS3Exception: The specified method is not -allowed against this resource. (Service: Amazon S3; Status Code: 405; -Error Code: MethodNotAllowed; -``` - -You are trying to use S3 Select to read data which for some reason -you are not allowed to. - -### AWSBadRequestException `InvalidTextEncoding` - -The file couldn't be parsed. This can happen if you try to read a `.gz` file -and forget to set the compression in the select request. - -That can be done through the `fs.s3a.select.compression` option. - -``` -org.apache.hadoop.fs.s3a.AWSBadRequestException: - Select: '" SELECT * FROM S3OBJECT s WHERE endstation_name = 'Bayswater Road: Hyde Park' " - on s3a://example/dataset.csv.gz: - com.amazonaws.services.s3.model.AmazonS3Exception: - UTF-8 encoding is required. The text encoding error was found near byte 8,192. - (Service: Amazon S3; Status Code: 400; Error Code: InvalidTextEncoding -``` - -### AWSBadRequestException `InvalidCompressionFormat` "GZIP is not applicable to the queried object" - -A SELECT call has been made using a compression which doesn't match that of the -source object, such as it being a plain text file. - -``` -org.apache.hadoop.fs.s3a.AWSBadRequestException: Select: - '" SELECT * FROM S3OBJECT s WHERE endstation_name = 'Bayswater Road: Hyde Park' " - on s3a://example/dataset.csv: - com.amazonaws.services.s3.model.AmazonS3Exception: - GZIP is not applicable to the queried object. Please correct the request and try again. - (Service: Amazon S3; Status Code: 400; Error Code: InvalidCompressionFormat; - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:212) - at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111) -... -Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: GZIP is not applicable to the queried object. - Please correct the request and try again. - Service: Amazon S3; Status Code: 400; Error Code: InvalidCompressionFormat; - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse - ... -``` - - -### AWSBadRequestException `UnsupportedStorageClass` - -S3 Select doesn't work with some storage classes like Glacier or Reduced Redundancy. -Make sure you've set `fs.s3a.create.storage.class` to a supported storage class for S3 Select. - -``` -org.apache.hadoop.fs.s3a.AWSBadRequestException: - Select on s3a://example/dataset.csv.gz: - com.amazonaws.services.s3.model.AmazonS3Exception: - We do not support REDUCED_REDUNDANCY storage class. - Please check the service documentation and try again. - (Service: Amazon S3; Status Code: 400; Error Code: UnsupportedStorageClass -``` - -### `PathIOException`: "seek() not supported" - -The input stream returned by the select call does not support seeking -backwards in the stream. - -Similarly, `PositionedReadable` operations will fail when used to read -data any offset other than that of `getPos()`. - -``` -org.apache.hadoop.fs.PathIOException: `s3a://landsat-pds/landsat.csv.gz': seek() not supported - - at org.apache.hadoop.fs.s3a.select.SelectInputStream.unsupported(SelectInputStream.java:254) - at org.apache.hadoop.fs.s3a.select.SelectInputStream.seek(SelectInputStream.java:243) - at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:66) -``` - -There is no fix for this. You can move forward in a file using `skip(offset)`; -bear in mind that the return value indicates what offset was skipped -it -may be less than expected. - -### `IllegalArgumentException`: "Unknown mandatory key "fs.s3a.select.sql" - -The filesystem is not an S3A filesystem, and the s3a select option is not recognized. - -``` -java.lang.IllegalArgumentException: Unknown mandatory key "fs.s3a.select.sql" -at com.google.common.base.Preconditions.checkArgument(Preconditions.java:88) -at org.apache.hadoop.fs.AbstractFSBuilder.lambda$rejectUnknownMandatoryKeys$0(AbstractFSBuilder.java:331) -at java.lang.Iterable.forEach(Iterable.java:75) -at java.util.Collections$UnmodifiableCollection.forEach(Collections.java:1080) -at org.apache.hadoop.fs.AbstractFSBuilder.rejectUnknownMandatoryKeys(AbstractFSBuilder.java:330) -at org.apache.hadoop.fs.filesystem.openFileWithOptions(FileSystem.java:3541) -at org.apache.hadoop.fs.FileSystem$FSDataInputStreamBuilder.build(FileSystem.java:4442) -``` - -* Verify that the URL has an "s3a:" prefix. -* If it does, there may be a non-standard S3A implementation, or some -a filtering/relaying class has been placed in front of the S3AFilesystem. - -### `IllegalArgumentException`: "Unknown mandatory key in non-select file I/O" - -The file options to tune an S3 select call are only valid when a SQL expression -is set in the `fs.s3a.select.sql` option. If not, any such option added as a `must()` value -will fail. - -``` -java.lang.IllegalArgumentException: Unknown mandatory key for s3a://example/test/testSelectOptionsOnlyOnSelectCalls.csv in non-select file I/O "fs.s3a.select.input.csv.header" - - at com.google.common.base.Preconditions.checkArgument(Preconditions.java:115) - at org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.lambda$rejectUnknownMandatoryKeys$0(AbstractFSBuilderImpl.java:352) - at java.lang.Iterable.forEach(Iterable.java:75) - at java.util.Collections$UnmodifiableCollection.forEach(Collections.java:1080) - at org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(AbstractFSBuilderImpl.java:351) - at org.apache.hadoop.fs.s3a.S3AFileSystem.openFileWithOptions(S3AFileSystem.java:3736) - at org.apache.hadoop.fs.FileSystem$FSDataInputStreamBuilder.build(FileSystem.java:4471) -``` - -Requiring these options without providing a SQL query is invariably an error. -Fix: add the SQL statement, or use `opt()` calls to set the option. - -If the `fs.s3a.select.sql` option is set, and still a key is rejected, then -either the spelling of the key is wrong, it has leading or trailing spaces, -or it is an option not supported in that specific release of Hadoop. - - -### PathIOException : "seek() backwards from not supported" - -Backwards seeks in an S3 Select `SelectInputStream` are not supported. - -``` -org.apache.hadoop.fs.PathIOException: `s3a://landsat-pds/scene_list.gz': - seek() backwards from 16387 to 0 not supported - - at org.apache.hadoop.fs.s3a.select.SelectInputStream.unsupported(SelectInputStream.java:288) - at org.apache.hadoop.fs.s3a.select.SelectInputStream.seek(SelectInputStream.java:253) - at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:66) -``` - -### InvalidTableAlias - -The SELECT refers to the name of a column which is not recognized - -* the name of a column is wrong, here `s.oddf`. -* headers are not enabled for the CSV source file. Fix: enable. -* a generated alias is used e.g `s._1`, but headers have been enabled. -Fix. disable, or use the header name. - -``` -org.apache.hadoop.fs.s3a.AWSBadRequestException: - SELECT * FROM S3OBJECT WHERE s."oddf" = 'true' - on s3a://example/test/testParseBrokenCSVFile: - com.amazonaws.services.s3.model.AmazonS3Exception: - Invalid table alias is specified at line 1, column 30. - Please check the file and try again. (Service: Amazon S3; Status Code: 400; Error Code: InvalidTableAlias; - Invalid table alias is specified at line 1, column 30. Please check the file and try again. - (Service: Amazon S3; Status Code: 400; - Error Code: InvalidTableAlias; - Request ID: 8693B86A52CFB91C; - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:225) - at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111) - at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:265) - ... -Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: - Invalid table alias is specified at line 1, column 30. - Please check the file and try again. - (Service: Amazon S3; Status Code: 400; Error Code: InvalidTableAlias; Request ID: 8693B86A52CFB91C; - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1640) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1304) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1058) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699) - at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667) - at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649) -``` - -### `AWSBadRequestException` "Attempt to convert from one data type to another failed: cast from STRING to TIMESTAMP." - -A string field could not be converted to a timestamp because one or more of its entries were not parseable -with the given timestamp. - -Example, from a spreadsheet where "timestamp" is normally a well-formatted timestamp field, -but in one column it is just "Tuesday" - -```sql -SELECT CAST(s.date AS TIMESTAMP) FROM S3OBJECT s -``` - -``` -org.apache.hadoop.fs.s3a.AWSBadRequestException: Select on s3a://example/test/testParseBrokenCSVFile: -com.amazonaws.services.s3.model.AmazonS3Exception: -Attempt to convert from one data type to another failed: cast from STRING to TIMESTAMP. -(Service: Amazon S3; Status Code: 400; Error Code: CastFailed; -Request ID: E2158FE45AF2049A; S3 Extended Request ID: iM40fzGuaPt6mQo0QxDDX+AY1bAgSVD1sKErFq6Y4GDJYHIAnmc00i0EvGGnH+0MFCFhKIivIrQ=), -S3 Extended Request ID: iM40fzGuaPt6mQo0QxDDX+AY1bAgSVD1sKErFq6Y4GDJYHIAnmc00i0EvGGnH+0MFCFhKIivIrQ=:CastFailed: -Attempt to convert from one data type to another failed: cast from STRING to TIMESTAMP. -(Service: Amazon S3; Status Code: 400; Error Code: CastFailed; Request ID: E2158FE45AF2049A; S3 Extended Request ID: iM40fzGuaPt6mQo0QxDDX+AY1bAgSVD1sKErFq6Y4GDJYHIAnmc00i0EvGGnH+0MFCFhKIivIrQ=) - at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:225) - at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111) - at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:265) -Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: - Attempt to convert from one data type to another failed: cast from STRING to TIMESTAMP. - (Service: Amazon S3; Status Code: 400; Error Code: CastFailed;) - -``` +It is no longer supported in Hadoop releases. -There's no way to recover from a bad record here; no option to skip invalid -rows. +Any Hadoop release built on the [AWS V2 SDK](./aws_sdk_upgrade.html) +will reject calls to open files using the select APIs. -*Note:* This is an example stack trace *without* the SQL being printed. +If a build of Hadoop with S3 Select is desired, the relevant +classes can be found in hadoop trunk commit `8bf72346a59c`. \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md index 62d449daeea56..cb435535b7cd9 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md @@ -572,10 +572,6 @@ on third party stores. test.fs.s3a.create.storage.class.enabled false - - fs.s3a.select.enabled - false - test.fs.s3a.sts.enabled false @@ -643,23 +639,10 @@ the `fs.s3a.scale.test.csvfile` option set to its path. (yes, the space is necessary. The Hadoop `Configuration` class treats an empty value as "do not override the default"). -### Turning off S3 Select - -The S3 select tests are skipped when the S3 endpoint doesn't support S3 Select. - -```xml - - fs.s3a.select.enabled - false - -``` - -If your endpoint doesn't support that feature, this option should be in -your `core-site.xml` file, so that trying to use S3 select fails fast with -a meaningful error ("S3 Select not supported") rather than a generic Bad Request -exception. +### Enabling prefetch for all tests -### Disabling V1 List API tests +The tests are run with prefetch if the `prefetch` property is set in the +maven build. This can be combined with the scale tests as well. If `ITestS3AContractGetFileStatusV1List` fails with any error about unsupported API. @@ -1296,19 +1279,6 @@ bin/hadoop fs -du -h -s $BUCKET/ mkdir tmp time bin/hadoop fs -copyToLocal -t 10 $BUCKET/\*aws\* tmp -# --------------------------------------------------- -# S3 Select on Landsat -# this will fail with a ClassNotFoundException unless -# eventstore JAR is added to the classpath -# --------------------------------------------------- - -export LANDSATGZ=s3a://landsat-pds/scene_list.gz - - -bin/hadoop s3guard select -header use -compression gzip $LANDSATGZ \ - "SELECT s.entityId,s.cloudCover FROM S3OBJECT s WHERE s.cloudCover < '0.0' LIMIT 100" - - # --------------------------------------------------- # Cloudstore # check out and build https://github.com/steveloughran/cloudstore diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md index 1aa6e83b110e2..de3ea79f4fb1d 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/third_party_stores.md @@ -428,11 +428,6 @@ this makes renaming and deleting significantly slower. false - - fs.s3a.bucket.gcs-container.select.enabled - false - - fs.s3a.bucket.gcs-container.path.style.access true diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md index be359336463ad..c1b499e3da95d 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md @@ -132,9 +132,7 @@ was built. This can also be caused by having more than one version of an AWS SDK JAR on the classpath. If the full `bundle.jar` JAR is on the -classpath, do not add any of the `aws-sdk-` JARs *except* for -`aws-crt.jar` (which is required) and -`eventstream.jar` which is required when using S3 Select. +classpath, do not add any of the `aws-sdk-` JARs. ### `java.lang.NoSuchMethodError` referencing an `org.apache.hadoop` class diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java index 1fb576a55514c..f5e91fae2a33e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java @@ -178,7 +178,6 @@ private void createFactoryObjects(RequestFactory factory) throws IOException { PutObjectOptions.keepingDirs(), -1, true)); a(factory.newPutObjectRequestBuilder(path, PutObjectOptions.deletingDirs(), 1024, false)); - a(factory.newSelectRequestBuilder(path)); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/AbstractS3SelectTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/AbstractS3SelectTest.java deleted file mode 100644 index a3d41116182e5..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/AbstractS3SelectTest.java +++ /dev/null @@ -1,756 +0,0 @@ -/* - * 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.select; - -import java.io.BufferedReader; -import java.io.EOFException; -import java.io.IOException; -import java.io.InputStreamReader; -import java.time.Duration; -import java.time.ZonedDateTime; -import java.time.format.DateTimeFormatter; -import java.util.ArrayList; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.Optional; -import java.util.Scanner; -import java.util.function.Consumer; - -import org.junit.Assume; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileContext; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FutureDataInputStreamBuilder; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.StreamCapabilities; -import org.apache.hadoop.fs.s3a.AWSServiceIOException; -import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; -import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.commit.AbstractCommitITest; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.compress.PassthroughCodec; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapreduce.MRJobConfig; -import org.apache.hadoop.mapreduce.RecordReader; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.mapreduce.TaskAttemptID; -import org.apache.hadoop.mapreduce.lib.input.FileSplit; -import org.apache.hadoop.mapreduce.lib.input.LineRecordReader; -import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; -import org.apache.hadoop.util.DurationInfo; - -import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.getLandsatCSVPath; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; -import static org.apache.hadoop.fs.s3a.select.CsvFile.ALL_QUOTES; -import static org.apache.hadoop.fs.s3a.select.SelectConstants.*; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; -import static org.apache.hadoop.util.functional.FutureIO.awaitFuture; - -/** - * Superclass for S3 Select tests. - * A lot of the work here goes into creating and querying a simple CSV test - * format, with various datatypes which can be used in type-casting queries. - *
    - * 1  "ID": index of the row
    - * 2  "date": date as ISO 8601
    - * 3  "timestamp": timestamp in seconds of epoch
    - * 4  "name", entry-$row
    - * 5  "odd", odd/even as boolean. True means odd,
    - * 6  "oddint", odd/even as int : 1 for odd, 0 for even
    - * 7  "oddrange": odd/even as 1 for odd, -1 for even
    - * 
    - */ -public abstract class AbstractS3SelectTest extends AbstractS3ATestBase { - - /** - * Number of columns in the CSV file: {@value}. - */ - public static final int CSV_COLUMN_COUNT = 7; - - protected static final String TRUE = q("TRUE"); - - protected static final String FALSE = q("FALSE"); - - public static final String SELECT_EVERYTHING = "SELECT * FROM S3OBJECT s"; - - public static final String SELECT_EVEN_ROWS_NO_HEADER = - "SELECT * FROM S3OBJECT s WHERE s._5 = " + TRUE; - public static final String SELECT_ODD_ROWS - = "SELECT s.name FROM S3OBJECT s WHERE s.odd = " + TRUE; - - public static final String SELECT_ODD_ENTRIES - = "SELECT * FROM S3OBJECT s WHERE s.odd = `TRUE`"; - - public static final String SELECT_ODD_ENTRIES_BOOL - = "SELECT * FROM S3OBJECT s WHERE CAST(s.odd AS BOOL) = TRUE"; - - public static final String SELECT_ODD_ENTRIES_INT - = "SELECT * FROM S3OBJECT s WHERE CAST(s.\"oddint\" AS INT) = 1"; - - public static final String SELECT_ODD_ENTRIES_DECIMAL - = "SELECT * FROM S3OBJECT s WHERE CAST(s.\"oddint\" AS DECIMAL) = 1"; - - /** - * Playing with timestamps: {@value}. - */ - public static final String SELECT_TO_DATE - = "SELECT\n" - + "CAST(s.\"date\" AS TIMESTAMP)\n" - + "FROM S3OBJECT s"; - - - /** - * How many rows are being generated. - */ - protected static final int ALL_ROWS_COUNT = 10; - - /** - * Row count of all rows + header. - */ - protected static final int ALL_ROWS_COUNT_WITH_HEADER = ALL_ROWS_COUNT + 1; - - /** - * Number of odd rows expected: {@value}. - */ - protected static final int ODD_ROWS_COUNT = ALL_ROWS_COUNT / 2; - - /** - * Number of even rows expected: {@value}. - * This is the same as the odd row count; it's separate just to - * be consistent on tests which select even results. - */ - protected static final int EVEN_ROWS_COUNT = ODD_ROWS_COUNT; - - protected static final String ENTRY_0001 = "\"entry-0001\""; - - protected static final String ENTRY_0002 = "\"entry-0002\""; - - /** - * Path to the landsat csv.gz file. - */ - private Path landsatGZ; - - /** - * The filesystem with the landsat data. - */ - private S3AFileSystem landsatFS; - - - // A random task attempt id for testing. - private String attempt0; - - private TaskAttemptID taskAttempt0; - - private String jobId; - - /** - * Base CSV file is headers. - *
    -   * 1  "ID": index of the row
    -   * 2  "date": date as Date.toString
    -   * 3  "timestamp": timestamp in seconds of epoch
    -   * 4  "name", entry-$row
    -   * 5  "odd", odd/even as boolean
    -   * 6  "oddint", odd/even as int : 1 for odd, 0 for even
    -   * 7  "oddrange": odd/even as 1 for odd, -1 for even
    -   * 
    - * @param fs filesystem - * @param path path to write - * @param header should the standard header be printed? - * @param quoteHeaderPolicy what the header quote policy is. - * @param quoteRowPolicy what the row quote policy is. - * @param rows number of rows - * @param separator column separator - * @param eol end of line characters - * @param quote quote char - * @param footer callback to run after the main CSV file is written - * @throws IOException IO failure. - */ - public static void createStandardCsvFile( - final FileSystem fs, - final Path path, - final boolean header, - final long quoteHeaderPolicy, - final long quoteRowPolicy, - final int rows, - final String separator, - final String eol, - final String quote, - final Consumer footer) throws IOException { - try (CsvFile csv = new CsvFile(fs, - path, - true, - separator, - eol, - quote)) { - - if (header) { - writeStandardHeader(csv, quoteHeaderPolicy); - } - DateTimeFormatter formatter - = DateTimeFormatter.ISO_OFFSET_DATE_TIME; - ZonedDateTime timestamp = ZonedDateTime.now(); - Duration duration = Duration.ofHours(20); - // loop is at 1 for use in counters and flags - for (int i = 1; i <= rows; i++) { - // flip the odd flags - boolean odd = (i & 1) == 1; - // and move the timestamp back - timestamp = timestamp.minus(duration); - csv.row(quoteRowPolicy, - i, - timestamp.format(formatter), - timestamp.toEpochSecond(), - String.format("entry-%04d", i), - odd ? "TRUE" : "FALSE", - odd ? 1 : 0, - odd ? 1 : -1 - ); - } - // write the footer - footer.accept(csv); - } - } - - /** - * Write out the standard header to a CSV file. - * @param csv CSV file to use. - * @param quoteHeaderPolicy quote policy. - * @return the input file. - * @throws IOException failure to write. - */ - private static CsvFile writeStandardHeader(final CsvFile csv, - final long quoteHeaderPolicy) throws IOException { - return csv.row(quoteHeaderPolicy, - "id", - "date", - "timestamp", - "name", - "odd", - "oddint", - "oddrange"); - } - - /** - * Verify that an exception has a specific error code. - * if not: an assertion is raised containing the original value. - * @param code expected code. - * @param ex exception caught - * @throws AssertionError on a mismatch - */ - protected static AWSServiceIOException verifyErrorCode(final String code, - final AWSServiceIOException ex) { - logIntercepted(ex); - if (!code.equals(ex.awsErrorDetails().errorCode())) { - throw new AssertionError("Expected Error code" + code - + " actual " + ex.awsErrorDetails().errorCode(), - ex); - } - return ex; - } - - /** - * Probe for a filesystem instance supporting S3 Select. - * @param filesystem filesystem - * @return true iff the filesystem supports S3 Select. - */ - boolean isSelectAvailable(final FileSystem filesystem) { - return filesystem instanceof StreamCapabilities - && ((StreamCapabilities) filesystem) - .hasCapability(S3_SELECT_CAPABILITY); - } - - @Override - protected Configuration createConfiguration() { - Configuration conf = super.createConfiguration(); - removeBaseAndBucketOverrides(conf, STORAGE_CLASS); - - return conf; - } - - /** - * Setup: requires select to be available. - */ - @Override - public void setup() throws Exception { - super.setup(); - Assume.assumeTrue("S3 Select is not enabled on " - + getFileSystem().getUri(), - isSelectAvailable(getFileSystem())); - Configuration conf = getConfiguration(); - landsatGZ = getLandsatCSVPath(conf); - landsatFS = (S3AFileSystem) landsatGZ.getFileSystem(conf); - Assume.assumeTrue("S3 Select is not enabled on " + landsatFS.getUri(), - isSelectAvailable(landsatFS)); - // create some job info - jobId = AbstractCommitITest.randomJobId(); - attempt0 = "attempt_" + jobId + "_m_000000_0"; - taskAttempt0 = TaskAttemptID.forName(attempt0); - } - - /** - * Build the SQL statement, using String.Format rules. - * @param template template - * @param args arguments for the template - * @return the template to use - */ - protected static String sql( - final String template, - final Object... args) { - return args.length > 0 ? String.format(template, args) : template; - } - - /** - * Quote a constant with the SQL quote logic. - * @param c constant - * @return quoted constant - */ - protected static String q(String c) { - return '\'' + c + '\''; - } - - /** - * Select from a source file. - * @param fileSystem FS. - * @param source source file. - * @param conf config for the select call. - * @param sql template for a formatted SQL request. - * @param args arguments for the formatted request. - * @return the input stream. - * @throws IOException failure - */ - protected FSDataInputStream select( - final FileSystem fileSystem, - final Path source, - final Configuration conf, - final String sql, - final Object... args) - throws IOException { - String expression = sql(sql, args); - describe("Execution Select call: %s", expression); - FutureDataInputStreamBuilder builder = - fileSystem.openFile(source) - .must(SELECT_SQL, expression); - // propagate all known options - for (String key : InternalSelectConstants.SELECT_OPTIONS) { - String value = conf.get(key); - if (value != null) { - builder.must(key, value); - } - } - return awaitFuture(builder.build()); - } - - /** - * Select from a source file via the file context API. - * @param fc file context - * @param source source file. - * @param conf config for the select call. - * @param sql template for a formatted SQL request. - * @param args arguments for the formatted request. - * @return the input stream. - * @throws IOException failure - */ - protected FSDataInputStream select( - final FileContext fc, - final Path source, - final Configuration conf, - final String sql, - final Object... args) - throws IOException { - String expression = sql(sql, args); - describe("Execution Select call: %s", expression); - FutureDataInputStreamBuilder builder = fc.openFile(source) - .must(SELECT_SQL, expression); - // propagate all known options - InternalSelectConstants.SELECT_OPTIONS.forEach((key) -> - Optional.ofNullable(conf.get(key)) - .map((v) -> builder.must(key, v))); - return awaitFuture(builder.build()); - } - - /** - * Parse a selection to lines; log at info. - * @param selection selection input - * @return a list of lines. - * @throws IOException if raised during the read. - */ - protected List parseToLines(final FSDataInputStream selection) - throws IOException { - return parseToLines(selection, getMaxLines()); - } - - /** - * Enable the passthrough codec for a job, with the given extension. - * @param conf configuration to update - * @param extension extension to use - */ - protected void enablePassthroughCodec(final Configuration conf, - final String extension) { - conf.set(CommonConfigurationKeys.IO_COMPRESSION_CODECS_KEY, - PassthroughCodec.CLASSNAME); - conf.set(PassthroughCodec.OPT_EXTENSION, extension); - } - - /** - * Override if a test suite is likely to ever return more lines. - * @return the max number for parseToLines/1 - */ - protected int getMaxLines() { - return 100; - } - - /** - * Parse a selection to lines; log at info. - * @param selection selection input - * @param maxLines maximum number of lines. - * @return a list of lines. - * @throws IOException if raised during the read. - */ - protected List parseToLines(final FSDataInputStream selection, - int maxLines) - throws IOException { - List result = new ArrayList<>(); - String stats; - // the scanner assumes that any IOE => EOF; we don't want - // that and so will check afterwards. - try (Scanner scanner = new Scanner( - new BufferedReader(new InputStreamReader(selection)))) { - scanner.useDelimiter(CSV_INPUT_RECORD_DELIMITER_DEFAULT); - while (maxLines > 0) { - try { - String l = scanner.nextLine(); - LOG.info("{}", l); - result.add(l); - maxLines--; - } catch (NoSuchElementException e) { - // EOL or an error - break; - } - } - stats = selection.toString(); - describe("Result line count: %s\nStatistics\n%s", - result.size(), stats); - // look for any raised error. - IOException ioe = scanner.ioException(); - if (ioe != null && !(ioe instanceof EOFException)) { - throw ioe; - } - } - return result; - } - - /** - * Verify the selection count; return the original list. - * If there's a mismatch, the whole list is logged at error, then - * an assertion raised. - * @param expected expected value. - * @param expression expression -for error messages. - * @param selection selected result. - * @return the input list. - */ - protected List verifySelectionCount( - final int expected, - final String expression, - final List selection) { - return verifySelectionCount(expected, expected, expression, selection); - } - - /** - * Verify the selection count is within a given range; - * return the original list. - * If there's a mismatch, the whole list is logged at error, then - * an assertion raised. - * @param min min value (exclusive). - * @param max max value (exclusive). If -1: no maximum. - * @param expression expression -for error messages. - * @param selection selected result. - * @return the input list. - */ - protected List verifySelectionCount( - final int min, - final int max, - final String expression, - final List selection) { - int size = selection.size(); - if (size < min || (max > -1 && size > max)) { - // mismatch: log and then fail - String listing = prepareToPrint(selection); - LOG.error("\n{} => \n{}", expression, listing); - fail("row count from select call " + expression - + " is out of range " + min + " to " + max - + ": " + size - + " \n" + listing); - } - return selection; - } - - /** - * Do whatever is needed to prepare a string for logging. - * @param selection selection - * @return something printable. - */ - protected String prepareToPrint(final List selection) { - return String.join("\n", selection); - } - - /** - * Create "the standard" CSV file with the default row count. - * @param fs filesystem - * @param path path to write - * @param quoteRowPolicy what the row quote policy is. - * @throws IOException IO failure. - */ - protected void createStandardCsvFile( - final FileSystem fs, - final Path path, - final long quoteRowPolicy) - throws IOException { - createStandardCsvFile( - fs, path, - true, - ALL_QUOTES, - quoteRowPolicy, - ALL_ROWS_COUNT, - ",", - "\n", - "\"", - c -> {}); - } - - /** - * Set an MR Job input option. - * @param conf configuration - * @param key key to set - * @param val value - */ - void inputOpt(Configuration conf, String key, String val) { - conf.set(MRJobConfig.INPUT_FILE_OPTION_PREFIX + key, val); - } - - /** - * Set a mandatory MR Job input option. - * @param conf configuration - * @param key key to set - * @param val value - */ - void inputMust(Configuration conf, String key, String val) { - conf.set(MRJobConfig.INPUT_FILE_MANDATORY_PREFIX + key, - val); - } - - /** - * Reads lines through a v2 RecordReader, as if it were part of a - * MRv2 job. - * @param conf job conf - * @param path path to query - * @param sql sql to add to the configuration. - * @param initialCapacity capacity of the read - * @param reader reader: this is closed after the read - * @return the selected lines. - * @throws Exception failure - */ - protected List readRecords(JobConf conf, - Path path, - String sql, - RecordReader reader, - int initialCapacity) throws Exception { - - inputMust(conf, SELECT_SQL, sql); - List lines = new ArrayList<>(initialCapacity); - try { - reader.initialize( - createSplit(conf, path), - createTaskAttemptContext(conf)); - while (reader.nextKeyValue()) { - lines.add(reader.getCurrentValue().toString()); - } - } finally { - reader.close(); - } - return lines; - } - /** - * Reads lines through a v1 RecordReader, as if it were part of a - * MRv1 job. - * @param conf job conf - * @param reader reader: this is closed after the read - * @param initialCapacity capacity of the read - * @return the selected lines. - * @throws Exception failure - */ - protected List readRecordsV1(JobConf conf, - org.apache.hadoop.mapred.RecordReader reader, - K key, - V value, - int initialCapacity) throws Exception { - List lines = new ArrayList<>(initialCapacity); - try { - while (reader.next(key, value)) { - lines.add(value.toString()); - } - } finally { - reader.close(); - } - return lines; - } - - /** - * Create a task attempt context for a job, creating a random JobID to - * do this. - * @param conf job configuration. - * @return a new task attempt context containing the job conf - * @throws Exception failure. - */ - protected TaskAttemptContext createTaskAttemptContext(final JobConf conf) - throws Exception { - String id = AbstractCommitITest.randomJobId(); - return new TaskAttemptContextImpl(conf, - TaskAttemptID.forName("attempt_" + id + "_m_000000_0")); - } - - /** - * Create an MRv2 file input split. - * @param conf job configuration - * @param path path to file - * @return the split - * @throws IOException problems reading the file. - */ - protected FileSplit createSplit(final JobConf conf, final Path path) - throws IOException { - FileSystem fs = path.getFileSystem(conf); - FileStatus status = fs.getFileStatus(path); - return new FileSplit(path, 0, status.getLen(), - new String[]{"localhost"}); - } - - /** - * Create an MRv1 file input split. - * @param conf job configuration - * @param path path to file - * @return the split - * @throws IOException problems reading the file. - */ - protected org.apache.hadoop.mapred.FileSplit - createSplitV1(final JobConf conf, final Path path) - throws IOException { - FileSystem fs = path.getFileSystem(conf); - FileStatus status = fs.getFileStatus(path); - return new org.apache.hadoop.mapred.FileSplit(path, 0, status.getLen(), - new String[]{"localhost"}); - } - - /** - * Create a v2 line record reader expecting newlines as the EOL marker. - * @return a reader - */ - protected RecordReader createLineRecordReader() { - return new LineRecordReader(new byte[]{'\n'}); - } - - /** - * Create a v1 line record reader. - * @return a reader - */ - protected org.apache.hadoop.mapred.RecordReader - createLineRecordReaderV1( - final JobConf conf, - final Path path) throws IOException { - return new org.apache.hadoop.mapred.LineRecordReader( - conf, createSplitV1(conf, path)); - } - - /** - * Get the path to the landsat file. - * @return the landsat CSV.GZ path. - */ - protected Path getLandsatGZ() { - return landsatGZ; - } - - /** - * Get the filesystem for the landsat file. - * @return the landsat FS. - */ - protected S3AFileSystem getLandsatFS() { - return landsatFS; - } - - /** - * Perform a seek: log duration of the operation. - * @param stream stream to seek. - * @param target target position. - * @throws IOException on an error - */ - protected void seek(final FSDataInputStream stream, final long target) - throws IOException { - try(DurationInfo ignored = - new DurationInfo(LOG, "Seek to %d", target)) { - stream.seek(target); - } - } - - /** - * Execute a seek so far past the EOF that it will be rejected. - * If the seek did not fail, the exception raised includes the toString() - * value of the stream. - * @param seekStream stream to seek in. - * @param newpos new position - * @return the EOF Exception raised. - * @throws Exception any other exception. - */ - protected EOFException expectSeekEOF(final FSDataInputStream seekStream, - final int newpos) throws Exception { - return intercept(EOFException.class, - () -> { - seek(seekStream, newpos); - // return this for the test failure reports. - return "Stream after seek to " + newpos + ": " + seekStream; - }); - } - - public String getAttempt0() { - return attempt0; - } - - public TaskAttemptID getTaskAttempt0() { - return taskAttempt0; - } - - public String getJobId() { - return jobId; - } - - /** - * Logs intercepted exceptions. - * This generates the stack traces for the documentation. - * @param ex exception - * @return the exception passed in (for chaining) - */ - protected static T logIntercepted(T ex) { - LOG.info("Intercepted Exception is ", ex); - return ex; - } -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java deleted file mode 100644 index cdb28c81b17d2..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3Select.java +++ /dev/null @@ -1,981 +0,0 @@ -/* - * 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.select; - -import java.io.BufferedReader; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InputStreamReader; -import java.util.List; -import java.util.concurrent.CompletableFuture; - -import org.junit.Assume; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSExceptionMessages; -import org.apache.hadoop.fs.FileContext; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FutureDataInputStreamBuilder; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.PathIOException; -import org.apache.hadoop.fs.Seekable; -import org.apache.hadoop.fs.contract.ContractTestUtils; -import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl; -import org.apache.hadoop.fs.s3a.AWSBadRequestException; -import org.apache.hadoop.fs.s3a.AWSServiceIOException; -import org.apache.hadoop.fs.s3a.Constants; -import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.S3AInputStream; -import org.apache.hadoop.fs.s3a.S3ATestUtils; -import org.apache.hadoop.fs.s3a.Statistic; -import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.JobContext; -import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import org.apache.hadoop.mapreduce.lib.input.LineRecordReader; -import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; -import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; -import org.apache.hadoop.mapreduce.task.JobContextImpl; -import org.apache.hadoop.util.DurationInfo; - -import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY; -import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_DEFAULT; -import static org.apache.hadoop.fs.s3a.Constants.READAHEAD_RANGE; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfS3ExpressBucket; -import static org.apache.hadoop.fs.s3a.select.CsvFile.ALL_QUOTES; -import static org.apache.hadoop.fs.s3a.select.SelectBinding.expandBackslashChars; -import static org.apache.hadoop.fs.s3a.select.SelectConstants.*; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; -import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture; -import static org.hamcrest.CoreMatchers.hasItem; -import static org.hamcrest.CoreMatchers.not; -import static org.hamcrest.collection.IsCollectionWithSize.hasSize; - -/** - * Test the S3 Select feature with some basic SQL Commands. - * Executed if the destination store declares its support for the feature. - */ -public class ITestS3Select extends AbstractS3SelectTest { - - private static final Logger LOG = - LoggerFactory.getLogger(ITestS3Select.class); - - public static final String E_CAST_FAILED = "CastFailed"; - - public static final String E_PARSE_INVALID_PATH_COMPONENT - = "ParseInvalidPathComponent"; - - public static final String E_INVALID_TABLE_ALIAS = "InvalidTableAlias"; - - private Configuration selectConf; - - /** well formed CSV. */ - private Path csvPath; - - /** CSV file with fewer columns than expected, all fields parse badly. */ - private Path brokenCSV; - - @Override - protected Configuration createConfiguration() { - final Configuration conf = super.createConfiguration(); - skipIfS3ExpressBucket(conf); - return conf; - } - - @Override - public void setup() throws Exception { - super.setup(); - csvPath = path(getMethodName() + ".csv"); - Assume.assumeTrue("S3 Select is not enabled", - getFileSystem().hasPathCapability(csvPath, S3_SELECT_CAPABILITY)); - selectConf = new Configuration(false); - selectConf.setBoolean(SELECT_ERRORS_INCLUDE_SQL, true); - createStandardCsvFile(getFileSystem(), csvPath, ALL_QUOTES); - // create the broken CSV file. - brokenCSV = path("testParseBrokenCSVFile"); - createStandardCsvFile( - getFileSystem(), brokenCSV, - true, - ALL_QUOTES, - ALL_ROWS_COUNT, - ALL_ROWS_COUNT, - ",", - "\n", - "\"", - csv -> csv - .line("# comment") - .row(ALL_QUOTES, "bad", "Tuesday", 0, "entry-bad", "yes", false)); - } - - @Override - public void teardown() throws Exception { - describe("teardown"); - try { - if (csvPath != null) { - getFileSystem().delete(csvPath, false); - } - if (brokenCSV != null) { - getFileSystem().delete(brokenCSV, false); - } - } finally { - super.teardown(); - } - } - - @Test - public void testCapabilityProbe() throws Throwable { - - // this should always hold true if we get past test setup - assertTrue("Select is not available on " + getFileSystem(), - isSelectAvailable(getFileSystem())); - } - - @SuppressWarnings("NestedAssignment") - @Test - public void testReadWholeFileClassicAPI() throws Throwable { - describe("create and read the whole file. Verifies setup working"); - int lines; - try (BufferedReader reader = new BufferedReader( - new InputStreamReader( - getFileSystem().open(csvPath)))) { - lines = 0; - // seek to 0, which is what some input formats do - String line; - while ((line = reader.readLine()) != null) { - lines++; - LOG.info("{}", line); - } - } - assertEquals("line count", ALL_ROWS_COUNT_WITH_HEADER, lines); - } - - @Test - public void testSelectWholeFileNoHeader() throws Throwable { - describe("Select the entire file, expect all rows but the header"); - expectSelected( - ALL_ROWS_COUNT, - selectConf, - CSV_HEADER_OPT_USE, - "SELECT * FROM S3OBJECT"); - } - - @Test - public void testSelectFirstColumnNoHeader() throws Throwable { - describe("Select the entire file, expect all rows but the header"); - expectSelected( - ALL_ROWS_COUNT_WITH_HEADER, - selectConf, - CSV_HEADER_OPT_NONE, - "SELECT s._1 FROM S3OBJECT s"); - } - - @Test - public void testSelectSelfNoHeader() throws Throwable { - describe("Select the entire file, expect all rows but the header"); - expectSelected( - ALL_ROWS_COUNT_WITH_HEADER, - selectConf, - CSV_HEADER_OPT_NONE, - "SELECT s._1 FROM S3OBJECT s WHERE s._1 = s._1"); - } - - @Test - public void testSelectSelfUseHeader() throws Throwable { - describe("Select the entire file, expect all rows including the header"); - expectSelected( - ALL_ROWS_COUNT, - selectConf, - CSV_HEADER_OPT_USE, - "SELECT s.id FROM S3OBJECT s WHERE s.id = s.id"); - } - - @Test - public void testSelectID2UseHeader() throws Throwable { - describe("Select where ID=2; use the header"); - expectSelected( - 1, - selectConf, - CSV_HEADER_OPT_USE, - "SELECT s.id FROM S3OBJECT s WHERE s.id = '2'"); - } - - @Test - public void testSelectNoMatchingID() throws Throwable { - describe("Select where there is no match; expect nothing back"); - expectSelected( - 0, - selectConf, - CSV_HEADER_OPT_USE, - "SELECT s.id FROM S3OBJECT s WHERE s.id = '0x8000'"); - } - - @Test - public void testSelectId1() throws Throwable { - describe("Select the first element in the file"); - expectSelected( - 1, - selectConf, - CSV_HEADER_OPT_NONE, - "SELECT * FROM S3OBJECT s WHERE s._1 = '1'", - TRUE); - } - - @Test - public void testSelectEmptySQL() throws Throwable { - describe("An empty SQL statement fails fast"); - FutureDataInputStreamBuilder builder = getFileSystem().openFile( - csvPath) - .must(SELECT_SQL, ""); - interceptFuture(IllegalArgumentException.class, - SELECT_SQL, - builder.build()); - } - - @Test - public void testSelectEmptyFile() throws Throwable { - describe("Select everything from an empty file"); - Path path = path("testSelectEmptyFile"); - S3AFileSystem fs = getFileSystem(); - ContractTestUtils.touch(fs, path); - parseToLines(fs.openFile(path) - .must(SELECT_SQL, SELECT_EVERYTHING) - .withFileStatus(fs.getFileStatus(path)) - .build() - .get(), - 0); - } - - @Test - public void testSelectEmptyFileWithConditions() throws Throwable { - describe("Select everything from an empty file with a more complex SQL"); - Path path = path("testSelectEmptyFileWithConditions"); - S3AFileSystem fs = getFileSystem(); - ContractTestUtils.touch(fs, path); - String sql = "SELECT * FROM S3OBJECT s WHERE s._1 = `TRUE`"; - CompletableFuture future = fs.openFile(path) - .must(SELECT_SQL, sql).build(); - assertEquals("Not at the end of the file", -1, future.get().read()); - } - - @Test - public void testSelectSeek() throws Throwable { - describe("Verify forward seeks work, not others"); - - // start: read in the full data through the initial select - // this makes asserting that contents match possible - Path path = csvPath; - S3AFileSystem fs = getFileSystem(); - int len = (int) fs.getFileStatus(path).getLen(); - byte[] fullData = new byte[len]; - int actualLen; - try (DurationInfo ignored = - new DurationInfo(LOG, "Initial read of %s", path); - FSDataInputStream sourceStream = - select(fs, path, - selectConf, - SELECT_EVERYTHING)) { - // read it in - actualLen = IOUtils.read(sourceStream, fullData); - } - int seekRange = 20; - - try (FSDataInputStream seekStream = - select(fs, path, - selectConf, - SELECT_EVERYTHING)) { - SelectInputStream sis - = (SelectInputStream) seekStream.getWrappedStream(); - S3AInputStreamStatistics streamStats = - sis.getS3AStreamStatistics(); - // lazy seek doesn't raise a problem here - seekStream.seek(0); - assertEquals("first byte read", fullData[0], seekStream.read()); - - // and now the pos has moved, again, seek will be OK - seekStream.seek(1); - seekStream.seek(1); - // but trying to seek elsewhere now fails - PathIOException ex = intercept(PathIOException.class, - SelectInputStream.SEEK_UNSUPPORTED, - () -> seekStream.seek(0)); - LOG.info("Seek error is as expected", ex); - // positioned reads from the current location work. - byte[] buffer = new byte[1]; - long pos = seekStream.getPos(); - seekStream.readFully(pos, buffer); - // but positioned backwards fail. - intercept(PathIOException.class, - SelectInputStream.SEEK_UNSUPPORTED, - () -> seekStream.readFully(0, buffer)); - // the position has now moved on. - assertPosition(seekStream, pos + 1); - // so a seek to the old pos will fail - intercept(PathIOException.class, - SelectInputStream.SEEK_UNSUPPORTED, - () -> seekStream.readFully(pos, buffer)); - - // set the readahead to the default. - // This verifies it reverts to the default. - seekStream.setReadahead(null); - assertEquals("Readahead in ", - Constants.DEFAULT_READAHEAD_RANGE, sis.getReadahead()); - // forward seeks are implemented as 1+ skip - long target = seekStream.getPos() + seekRange; - seek(seekStream, target); - assertPosition(seekStream, target); - // now do a read and compare values - assertEquals("byte at seek position", - fullData[(int)seekStream.getPos()], seekStream.read()); - assertEquals("Seek bytes skipped in " + streamStats, - seekRange, streamStats.getBytesSkippedOnSeek()); - - // try an invalid readahead range - intercept(IllegalArgumentException.class, - S3AInputStream.E_NEGATIVE_READAHEAD_VALUE, - () -> seekStream.setReadahead(-1L)); - - // do a slightly forward offset read - int read = seekStream.read(seekStream.getPos() + 2, buffer, 0, 1); - assertEquals(1, read); - - // final fun: seek way past the EOF - logIntercepted(expectSeekEOF(seekStream, actualLen * 2)); - assertPosition(seekStream, actualLen); - assertEquals(-1, seekStream.read()); - LOG.info("Seek statistics {}", streamStats); - // this will return no, but not fail - assertFalse("Failed to seek to new source in " + seekStream, - seekStream.seekToNewSource(0)); - // and set the readahead to 0 to see that close path works - seekStream.setReadahead(0L); - // then do a manual close even though there's one in the try resource. - // which will verify that a double close is harmless - seekStream.close(); - LOG.info("Final stream state {}", sis); - } - } - - /** - * Assert that a stream is in a specific position. - * @param stream stream or other seekable. - * @param pos expected position. - * @throws IOException failure of the getPos() call. - * @throws AssertionError mismatch between expected and actual. - */ - private void assertPosition(Seekable stream, long pos) - throws IOException { - assertEquals("Wrong stream position in " + stream, - pos, stream.getPos()); - } - - @Test - public void testSelectOddLinesNoHeader() throws Throwable { - describe("Select odd lines, ignoring the header"); - expectSelected( - ODD_ROWS_COUNT, - selectConf, - CSV_HEADER_OPT_IGNORE, - "SELECT * FROM S3OBJECT s WHERE s._5 = `TRUE`"); - // and do a quick check on the instrumentation - long bytesRead = getFileSystem().getInstrumentation() - .getCounterValue(Statistic.STREAM_READ_BYTES); - assertNotEquals("No bytes read count in filesystem instrumentation counter", - 0, bytesRead); - } - - @Test - public void testSelectOddLinesHeader() throws Throwable { - describe("Select the odd values"); - List selected = expectSelected( - ODD_ROWS_COUNT, - selectConf, - CSV_HEADER_OPT_USE, - SELECT_ODD_ROWS); - // the list includes odd values - assertThat(selected, hasItem(ENTRY_0001)); - // but not the evens - assertThat(selected, not(hasItem(ENTRY_0002))); - } - - @Test - public void testSelectOddLinesHeaderTSVOutput() throws Throwable { - describe("Select the odd values with tab spaced output"); - selectConf.set(CSV_OUTPUT_FIELD_DELIMITER, "\t"); - selectConf.set(CSV_OUTPUT_QUOTE_CHARACTER, "'"); - selectConf.set(CSV_OUTPUT_QUOTE_FIELDS, - CSV_OUTPUT_QUOTE_FIELDS_AS_NEEEDED); - selectConf.set(CSV_OUTPUT_RECORD_DELIMITER, "\r"); - List selected = expectSelected( - ODD_ROWS_COUNT, - selectConf, - CSV_HEADER_OPT_USE, - SELECT_ODD_ENTRIES_BOOL); - // the list includes odd values - String row1 = selected.get(0); - - // split that first line into columns: This is why TSV is better for code - // to work with than CSV - String[] columns = row1.split("\t", -1); - assertEquals("Wrong column count from tab split line <" + row1 + ">", - CSV_COLUMN_COUNT, columns.length); - assertEquals("Wrong column value from tab split line <" + row1 + ">", - "entry-0001", columns[3]); - } - - @Test - public void testSelectNotOperationHeader() throws Throwable { - describe("Select the even values with a NOT call; quote the header name"); - List selected = expectSelected( - EVEN_ROWS_COUNT, - selectConf, - CSV_HEADER_OPT_USE, - "SELECT s.name FROM S3OBJECT s WHERE NOT s.\"odd\" = %s", - TRUE); - // the list includes no odd values - assertThat(selected, not(hasItem(ENTRY_0001))); - // but has the evens - assertThat(selected, hasItem(ENTRY_0002)); - } - - @Test - public void testBackslashExpansion() throws Throwable { - assertEquals("\t\r\n", expandBackslashChars("\t\r\n")); - assertEquals("\t", expandBackslashChars("\\t")); - assertEquals("\r", expandBackslashChars("\\r")); - assertEquals("\r \n", expandBackslashChars("\\r \\n")); - assertEquals("\\", expandBackslashChars("\\\\")); - } - - /** - * This is an expanded example for the documentation. - * Also helps catch out unplanned changes to the configuration strings. - */ - @Test - public void testSelectFileExample() throws Throwable { - describe("Select the entire file, expect all rows but the header"); - int len = (int) getFileSystem().getFileStatus(csvPath).getLen(); - FutureDataInputStreamBuilder builder = - getFileSystem().openFile(csvPath) - .must("fs.s3a.select.sql", - SELECT_ODD_ENTRIES) - .must("fs.s3a.select.input.format", "CSV") - .must("fs.s3a.select.input.compression", "NONE") - .must("fs.s3a.select.input.csv.header", "use") - .must("fs.s3a.select.output.format", "CSV"); - - CompletableFuture future = builder.build(); - try (FSDataInputStream select = future.get()) { - // process the output - byte[] bytes = new byte[len]; - int actual = select.read(bytes); - LOG.info("file length is {}; length of selected data is {}", - len, actual); - } - } - - /** - * This is an expanded example for the documentation. - * Also helps catch out unplanned changes to the configuration strings. - */ - @Test - public void testSelectUnsupportedInputFormat() throws Throwable { - describe("Request an Unsupported input format"); - FutureDataInputStreamBuilder builder = getFileSystem().openFile(csvPath) - .must(SELECT_SQL, SELECT_ODD_ENTRIES) - .must(SELECT_INPUT_FORMAT, "pptx"); - interceptFuture(IllegalArgumentException.class, - "pptx", - builder.build()); - } - - /** - * Ask for an invalid output format. - */ - @Test - public void testSelectUnsupportedOutputFormat() throws Throwable { - describe("Request a (currently) Unsupported output format"); - FutureDataInputStreamBuilder builder = getFileSystem().openFile(csvPath) - .must(SELECT_SQL, SELECT_ODD_ENTRIES) - .must(SELECT_INPUT_FORMAT, "csv") - .must(SELECT_OUTPUT_FORMAT, "json"); - interceptFuture(IllegalArgumentException.class, - "json", - builder.build()); - } - - /** - * Missing files fail lazy. - */ - @Test - public void testSelectMissingFile() throws Throwable { - - describe("Select a missing file, expect it to surface in the future"); - - Path missing = path("missing"); - - FutureDataInputStreamBuilder builder = - getFileSystem().openFile(missing) - .must(SELECT_SQL, SELECT_ODD_ENTRIES); - - interceptFuture(FileNotFoundException.class, - "", builder.build()); - } - - @Test - public void testSelectDirectoryFails() throws Throwable { - describe("Verify that secondary select options are only valid on select" - + " queries"); - S3AFileSystem fs = getFileSystem(); - Path dir = path("dir"); - // this will be an empty dir marker - fs.mkdirs(dir); - - FutureDataInputStreamBuilder builder = - getFileSystem().openFile(dir) - .must(SELECT_SQL, SELECT_ODD_ENTRIES); - interceptFuture(FileNotFoundException.class, - "", builder.build()); - - // try the parent - builder = getFileSystem().openFile(dir.getParent()) - .must(SELECT_SQL, - SELECT_ODD_ENTRIES); - interceptFuture(FileNotFoundException.class, - "", builder.build()); - } - - @Test - public void testSelectRootFails() throws Throwable { - describe("verify root dir selection is rejected"); - FutureDataInputStreamBuilder builder = - getFileSystem().openFile(path("/")) - .must(SELECT_SQL, SELECT_ODD_ENTRIES); - interceptFuture(IOException.class, - "", builder.build()); - } - - /** - * Validate the abort logic. - */ - @Test - public void testCloseWithAbort() throws Throwable { - describe("Close the stream with the readahead outstanding"); - S3ATestUtils.MetricDiff readOps = new S3ATestUtils.MetricDiff( - getFileSystem(), - Statistic.STREAM_READ_OPERATIONS_INCOMPLETE); - selectConf.setInt(READAHEAD_RANGE, 2); - - FSDataInputStream stream = select(getFileSystem(), csvPath, selectConf, - "SELECT * FROM S3OBJECT s"); - SelectInputStream sis = (SelectInputStream) stream.getWrappedStream(); - assertEquals("Readahead on " + sis, 2, sis.getReadahead()); - stream.setReadahead(1L); - assertEquals("Readahead on " + sis, 1, sis.getReadahead()); - stream.read(); - S3AInputStreamStatistics stats - = (S3AInputStreamStatistics) - sis.getS3AStreamStatistics(); - assertEquals("Read count in " + sis, - 1, stats.getBytesRead()); - stream.close(); - assertEquals("Abort count in " + sis, - 1, stats.getAborted()); - readOps.assertDiffEquals("Read operations are still considered active", - 0); - intercept(PathIOException.class, FSExceptionMessages.STREAM_IS_CLOSED, - () -> stream.read()); - } - - @Test - public void testCloseWithNoAbort() throws Throwable { - describe("Close the stream with the readahead outstandingV"); - FSDataInputStream stream = select(getFileSystem(), csvPath, selectConf, - "SELECT * FROM S3OBJECT s"); - stream.setReadahead(0x1000L); - SelectInputStream sis = (SelectInputStream) stream.getWrappedStream(); - S3AInputStreamStatistics stats - = (S3AInputStreamStatistics) - sis.getS3AStreamStatistics(); - stream.close(); - assertEquals("Close count in " + sis, 1, stats.getClosed()); - assertEquals("Abort count in " + sis, 0, stats.getAborted()); - assertTrue("No bytes read in close of " + sis, - stats.getBytesReadInClose() > 0); - } - - @Test - public void testFileContextIntegration() throws Throwable { - describe("Test that select works through FileContext"); - FileContext fc = S3ATestUtils.createTestFileContext(getConfiguration()); - selectConf.set(CSV_INPUT_HEADER, CSV_HEADER_OPT_USE); - - List selected = - verifySelectionCount(ODD_ROWS_COUNT, SELECT_ODD_ENTRIES_INT, - parseToLines( - select(fc, csvPath, selectConf, SELECT_ODD_ROWS))); - // the list includes odd values - assertThat(selected, hasItem(ENTRY_0001)); - // but not the evens - assertThat(selected, not(hasItem(ENTRY_0002))); - } - - @Test - public void testSelectOptionsOnlyOnSelectCalls() throws Throwable { - describe("Secondary select options are only valid on select" - + " queries"); - String key = CSV_INPUT_HEADER; - intercept(IllegalArgumentException.class, key, - () -> getFileSystem().openFile(csvPath) - .must(key, CSV_HEADER_OPT_USE).build()); - } - - @Test - public void testSelectMustBeEnabled() throws Throwable { - describe("Verify that the FS must have S3 select enabled."); - Configuration conf = new Configuration(getFileSystem().getConf()); - conf.setBoolean(FS_S3A_SELECT_ENABLED, false); - try (FileSystem fs2 = FileSystem.newInstance(csvPath.toUri(), conf)) { - intercept(UnsupportedOperationException.class, - SELECT_UNSUPPORTED, - () -> { - assertFalse("S3 Select Capability must be disabled on " + fs2, - isSelectAvailable(fs2)); - return fs2.openFile(csvPath) - .must(SELECT_SQL, SELECT_ODD_ROWS) - .build(); - }); - } - } - - @Test - public void testSelectOptionsRejectedOnNormalOpen() throws Throwable { - describe("Verify that a normal open fails on select must() options"); - intercept(IllegalArgumentException.class, - AbstractFSBuilderImpl.UNKNOWN_MANDATORY_KEY, - () -> getFileSystem().openFile(csvPath) - .must(CSV_INPUT_HEADER, CSV_HEADER_OPT_USE) - .build()); - } - - @Test - public void testSelectOddRecordsWithHeader() - throws Throwable { - describe("work through a record reader"); - JobConf conf = createJobConf(); - inputMust(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_USE); - expectRecordsRead(ODD_ROWS_COUNT, conf, SELECT_ODD_ENTRIES_DECIMAL); - } - - @Test - public void testSelectDatestampsConverted() - throws Throwable { - describe("timestamp conversion in record IIO"); - JobConf conf = createJobConf(); - inputMust(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_USE); - inputMust(conf, CSV_OUTPUT_QUOTE_FIELDS, - CSV_OUTPUT_QUOTE_FIELDS_AS_NEEEDED); - String sql = SELECT_TO_DATE; - List records = expectRecordsRead(ALL_ROWS_COUNT, conf, sql); - LOG.info("Result of {}\n{}", sql, prepareToPrint(records)); - } - - @Test - public void testSelectNoMatch() - throws Throwable { - describe("when there's no match to a query, 0 records are returned,"); - JobConf conf = createJobConf(); - inputMust(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_USE); - expectRecordsRead(0, conf, - "SELECT * FROM S3OBJECT s WHERE s.odd = " + q("maybe")); - } - - @Test - public void testSelectOddRecordsIgnoreHeader() - throws Throwable { - describe("work through a record reader"); - JobConf conf = createJobConf(); - inputOpt(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_NONE); - inputMust(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_IGNORE); - expectRecordsRead(EVEN_ROWS_COUNT, conf, - SELECT_EVEN_ROWS_NO_HEADER); - } - - @Test - public void testSelectRecordsUnknownMustOpt() - throws Throwable { - describe("verify reader key validation is remapped"); - JobConf conf = createJobConf(); - inputOpt(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_NONE); - inputMust(conf, CSV_INPUT_HEADER + ".something", CSV_HEADER_OPT_IGNORE); - intercept(IllegalArgumentException.class, - AbstractFSBuilderImpl.UNKNOWN_MANDATORY_KEY, - () -> readRecords(conf, SELECT_EVEN_ROWS_NO_HEADER)); - } - - @Test - public void testSelectOddRecordsWithHeaderV1() - throws Throwable { - describe("work through a V1 record reader"); - JobConf conf = createJobConf(); - inputMust(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_USE); - // using a double backslash here makes the string "\t" which will then - // be parsed in the SelectBinding code as it if had come in on from an XML - // entry - inputMust(conf, CSV_OUTPUT_FIELD_DELIMITER, "\\t"); - inputMust(conf, CSV_OUTPUT_QUOTE_CHARACTER, "'"); - inputMust(conf, CSV_OUTPUT_QUOTE_FIELDS, - CSV_OUTPUT_QUOTE_FIELDS_AS_NEEEDED); - inputMust(conf, CSV_OUTPUT_RECORD_DELIMITER, "\n"); - verifySelectionCount(ODD_ROWS_COUNT, - SELECT_ODD_ROWS, - readRecordsV1(conf, SELECT_ODD_ROWS)); - } - - /** - * Create a job conf for line reader tests. - * This patches the job with the passthrough codec for - * CSV files. - * @return a job configuration - */ - private JobConf createJobConf() { - JobConf conf = new JobConf(getConfiguration()); - enablePassthroughCodec(conf, ".csv"); - return conf; - } - - @Test - public void testSelectOddRecordsIgnoreHeaderV1() - throws Throwable { - describe("work through a V1 record reader"); - JobConf conf = createJobConf(); - inputOpt(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_NONE); - inputMust(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_IGNORE); - inputMust(conf, FS_OPTION_OPENFILE_READ_POLICY, - FS_OPTION_OPENFILE_READ_POLICY_DEFAULT); - inputMust(conf, SELECT_ERRORS_INCLUDE_SQL, "true"); - verifySelectionCount(EVEN_ROWS_COUNT, - SELECT_EVEN_ROWS_NO_HEADER, - readRecordsV1(conf, SELECT_EVEN_ROWS_NO_HEADER)); - } - - protected List expectRecordsRead(final int expected, - final JobConf conf, - final String sql) throws Exception { - return verifySelectionCount(expected, sql, readRecords(conf, sql)); - } - - /** - * Reads lines through {@link LineRecordReader}, as if it were an MR - * job. - * @param conf jpb conf - * @param sql sql to add to the configuration. - * @return the selected lines. - * @throws Exception failure - */ - private List readRecords(JobConf conf, String sql) throws Exception { - return readRecords(conf, - csvPath, - sql, - createLineRecordReader(), - ALL_ROWS_COUNT_WITH_HEADER); - } - - /** - * Reads lines through a v1 LineRecordReader}. - * @param conf jpb conf - * @param sql sql to add to the configuration. - * @return the selected lines. - * @throws Exception failure - */ - private List readRecordsV1(JobConf conf, String sql) - throws Exception { - inputMust(conf, SELECT_SQL, sql); - return super.readRecordsV1(conf, - createLineRecordReaderV1(conf, csvPath), - new LongWritable(), - new Text(), - ALL_ROWS_COUNT_WITH_HEADER); - } - - /** - * Issue a select call, expect the specific number of rows back. - * Error text will include the SQL. - * @param expected expected row count. - * @param conf config for the select call. - * @param header header option - * @param sql template for a formatted SQL request. - * @param args arguments for the formatted request. - * @return the lines selected - * @throws IOException failure - */ - private List expectSelected( - final int expected, - final Configuration conf, - final String header, - final String sql, - final Object...args) throws Exception { - conf.set(CSV_INPUT_HEADER, header); - return verifySelectionCount(expected, sql(sql, args), - selectCsvFile(conf, sql, args)); - } - - /** - * Select from the CSV file. - * @param conf config for the select call. - * @param sql template for a formatted SQL request. - * @param args arguments for the formatted request. - * @return the lines selected - * @throws IOException failure - */ - private List selectCsvFile( - final Configuration conf, - final String sql, - final Object...args) - throws Exception { - - return parseToLines( - select(getFileSystem(), csvPath, conf, sql, args)); - } - - @Test - public void testCommentsSkipped() throws Throwable { - describe("Verify that comments are skipped"); - selectConf.set(CSV_INPUT_HEADER, CSV_HEADER_OPT_USE); - - List lines = verifySelectionCount( - ALL_ROWS_COUNT_WITH_HEADER, - "select s.id", - parseToLines( - select(getFileSystem(), brokenCSV, selectConf, - "SELECT * FROM S3OBJECT s"))); - LOG.info("\n{}", prepareToPrint(lines)); - } - - @Test - public void testEmptyColumnsRegenerated() throws Throwable { - describe("if you ask for a column but your row doesn't have it," - + " an empty column is inserted"); - selectConf.set(CSV_INPUT_HEADER, CSV_HEADER_OPT_USE); - - List lines = verifySelectionCount( - ALL_ROWS_COUNT_WITH_HEADER, "select s.oddrange", - parseToLines( - select(getFileSystem(), brokenCSV, selectConf, - "SELECT s.oddrange FROM S3OBJECT s"))); - LOG.info("\n{}", prepareToPrint(lines)); - assertEquals("Final oddrange column is not regenerated empty", - "\"\"", lines.get(lines.size() - 1)); - } - - @Test - public void testIntCastFailure() throws Throwable { - describe("Verify that int casts fail"); - expectSelectFailure(E_CAST_FAILED, SELECT_ODD_ENTRIES_INT); - - } - - @Test - public void testSelectToDateParseFailure() throws Throwable { - describe("Verify date parsing failure"); - expectSelectFailure(E_CAST_FAILED, SELECT_TO_DATE); - } - - @Test - public void testParseInvalidPathComponent() throws Throwable { - describe("Verify bad SQL parseing"); - expectSelectFailure(E_PARSE_INVALID_PATH_COMPONENT, - "SELECT * FROM S3OBJECT WHERE s.'oddf' = true"); - } - - @Test - public void testSelectInvalidTableAlias() throws Throwable { - describe("select with unknown column name"); - expectSelectFailure(E_INVALID_TABLE_ALIAS, - "SELECT * FROM S3OBJECT WHERE s.\"oddf\" = 'true'"); - } - - @Test - public void testSelectGeneratedAliases() throws Throwable { - describe("select with a ._2 column when headers are enabled"); - expectSelectFailure(E_INVALID_TABLE_ALIAS, - "SELECT * FROM S3OBJECT WHERE s._2 = 'true'"); - } - - /** - * Expect select against the broken CSV file to fail with a specific - * AWS exception error code. - * If the is no failure, the results are included in the assertion raised. - * @param expectedErrorCode error code in getErrorCode() - * @param sql SQL to invoke - * @return the exception, if it is as expected. - * @throws Exception any other failure - * @throws AssertionError when an exception is raised, but its error code - * is different, or when no exception was raised. - */ - protected AWSServiceIOException expectSelectFailure( - String expectedErrorCode, - String sql) - throws Exception { - selectConf.set(CSV_INPUT_HEADER, CSV_HEADER_OPT_USE); - return verifyErrorCode(expectedErrorCode, - intercept(AWSBadRequestException.class, - () -> - prepareToPrint( - parseToLines( - select(getFileSystem(), brokenCSV, selectConf, sql) - )))); - - } - - - @Test - public void testInputSplit() - throws Throwable { - describe("Verify that only a single file is used for splits"); - JobConf conf = new JobConf(getConfiguration()); - - - inputMust(conf, CSV_INPUT_HEADER, CSV_HEADER_OPT_USE); - final Path input = csvPath; - S3AFileSystem fs = getFileSystem(); - final Path output = path("testLandsatSelect") - .makeQualified(fs.getUri(), fs.getWorkingDirectory()); - conf.set(FileInputFormat.INPUT_DIR, input.toString()); - conf.set(FileOutputFormat.OUTDIR, output.toString()); - - final Job job = Job.getInstance(conf, "testInputSplit"); - JobContext jobCtx = new JobContextImpl(job.getConfiguration(), - getTaskAttempt0().getJobID()); - - TextInputFormat tif = new TextInputFormat(); - List splits = tif.getSplits(jobCtx); - assertThat("split count wrong", splits, hasSize(1)); - - } - -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectCLI.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectCLI.java deleted file mode 100644 index 2dd813bf40a91..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectCLI.java +++ /dev/null @@ -1,357 +0,0 @@ -/* - * 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.select; - -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.List; - -import org.junit.Assume; -import org.junit.Test; - -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.S3ATestUtils; -import org.apache.hadoop.fs.s3a.Statistic; -import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; -import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.Source; -import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; -import org.apache.hadoop.util.ExitUtil; -import org.apache.hadoop.util.OperationDuration; -import org.apache.hadoop.util.ToolRunner; - -import static org.apache.hadoop.util.Preconditions.checkNotNull; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; -import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.exec; -import static org.apache.hadoop.fs.s3a.select.ITestS3SelectLandsat.SELECT_NOTHING; -import static org.apache.hadoop.fs.s3a.select.ITestS3SelectLandsat.SELECT_SUNNY_ROWS_NO_LIMIT; -import static org.apache.hadoop.fs.s3a.select.SelectConstants.*; -import static org.apache.hadoop.fs.s3a.select.SelectTool.*; -import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_COMMAND_ARGUMENT_ERROR; -import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_NOT_FOUND; -import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_SERVICE_UNAVAILABLE; -import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_SUCCESS; -import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_USAGE; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; - -/** - * Test the S3 Select CLI through some operations against landsat - * and files generated from it. - */ -public class ITestS3SelectCLI extends AbstractS3SelectTest { - - public static final int LINE_COUNT = 100; - - public static final String SELECT_EVERYTHING = "SELECT * FROM S3OBJECT s"; - - private SelectTool selectTool; - - private Configuration selectConf; - - public static final String D = "-D"; - - private File localFile; - - private String landsatSrc; - - @Override - public void setup() throws Exception { - super.setup(); - selectTool = new SelectTool(getConfiguration()); - selectConf = new Configuration(getConfiguration()); - localFile = getTempFilename(); - landsatSrc = getLandsatGZ().toString(); - final S3AFileSystem landsatFS = getLandsatFS(); - ChangeDetectionPolicy changeDetectionPolicy = - landsatFS.getChangeDetectionPolicy(); - Assume.assumeFalse("the standard landsat bucket doesn't have versioning", - changeDetectionPolicy.getSource() == Source.VersionId - && changeDetectionPolicy.isRequireVersion()); - Assume.assumeTrue("S3 Select is not enabled", - landsatFS.hasPathCapability(new Path("/"), S3_SELECT_CAPABILITY)); - } - - @Override - public void teardown() throws Exception { - super.teardown(); - if (localFile != null) { - localFile.delete(); - } - } - - /** - * Expect a command to succeed. - * @param message any extra text to include in the assertion error message - * @param tool tool to run - * @param args arguments to the command - * @return the output of any successful run - * @throws Exception failure - */ - protected static String expectSuccess( - String message, - S3GuardTool tool, - String... args) throws Exception { - ByteArrayOutputStream buf = new ByteArrayOutputStream(); - exec(EXIT_SUCCESS, message, tool, buf, args); - return buf.toString(); - } - - /** - * Run a S3GuardTool command from a varags list and the - * configuration returned by {@code getConfiguration()}. - * @param conf config to use - * @param args argument list - * @return the return code - * @throws Exception any exception - */ - protected int run(Configuration conf, S3GuardTool tool, - String... args) throws Exception { - return ToolRunner.run(conf, tool, args); - } - - /** - * Run a S3GuardTool command from a varags list, catch any raised - * ExitException and verify the status code matches that expected. - * @param status expected status code of the exception - * @param conf config to use - * @param args argument list - * @throws Exception any exception - */ - protected void runToFailure(int status, Configuration conf, - String message, - S3GuardTool tool, String... args) - throws Exception { - final ExitUtil.ExitException ex = - intercept(ExitUtil.ExitException.class, message, - () -> ToolRunner.run(conf, tool, args)); - if (ex.status != status) { - throw ex; - } - - } - - @Test - public void testLandsatToFile() throws Throwable { - describe("select part of the landsat to a file"); - int lineCount = LINE_COUNT; - S3AFileSystem landsatFS = - (S3AFileSystem) getLandsatGZ().getFileSystem(getConfiguration()); - S3ATestUtils.MetricDiff selectCount = new S3ATestUtils.MetricDiff(landsatFS, - Statistic.OBJECT_SELECT_REQUESTS); - - run(selectConf, selectTool, - D, v(CSV_OUTPUT_QUOTE_CHARACTER, "'"), - D, v(CSV_OUTPUT_QUOTE_FIELDS, CSV_OUTPUT_QUOTE_FIELDS_AS_NEEEDED), - "select", - o(OPT_HEADER), CSV_HEADER_OPT_USE, - o(OPT_COMPRESSION), COMPRESSION_OPT_GZIP, - o(OPT_LIMIT), Integer.toString(lineCount), - o(OPT_OUTPUT), localFile.toString(), - landsatSrc, - SELECT_SUNNY_ROWS_NO_LIMIT); - List lines = IOUtils.readLines(new FileInputStream(localFile), StandardCharsets.UTF_8); - LOG.info("Result from select:\n{}", lines.get(0)); - assertEquals(lineCount, lines.size()); - selectCount.assertDiffEquals("select count", 1); - OperationDuration duration = selectTool.getSelectDuration(); - assertTrue("Select duration was not measured", - duration.value() > 0); - } - - private File getTempFilename() throws IOException { - File dest = File.createTempFile("landat", ".csv"); - dest.delete(); - return dest; - } - - @Test - public void testLandsatToConsole() throws Throwable { - describe("select part of the landsat to the console"); - // this verifies the input stream was actually closed - S3ATestUtils.MetricDiff readOps = new S3ATestUtils.MetricDiff( - getFileSystem(), - Statistic.STREAM_READ_OPERATIONS_INCOMPLETE); - run(selectConf, selectTool, - D, v(CSV_OUTPUT_QUOTE_CHARACTER, "'"), - D, v(CSV_OUTPUT_QUOTE_FIELDS, CSV_OUTPUT_QUOTE_FIELDS_ALWAYS), - "select", - o(OPT_HEADER), CSV_HEADER_OPT_USE, - o(OPT_COMPRESSION), COMPRESSION_OPT_GZIP, - o(OPT_LIMIT), Integer.toString(LINE_COUNT), - landsatSrc, - SELECT_SUNNY_ROWS_NO_LIMIT); - assertEquals("Lines read and printed to console", - LINE_COUNT, selectTool.getLinesRead()); - readOps.assertDiffEquals("Read operations are still considered active", - 0); } - - @Test - public void testSelectNothing() throws Throwable { - describe("an empty select is not an error"); - run(selectConf, selectTool, - "select", - o(OPT_HEADER), CSV_HEADER_OPT_USE, - o(OPT_COMPRESSION), COMPRESSION_OPT_GZIP, - o(OPT_INPUTFORMAT), "csv", - o(OPT_OUTPUTFORMAT), "csv", - o(OPT_EXPECTED), "0", - o(OPT_LIMIT), Integer.toString(LINE_COUNT), - landsatSrc, - SELECT_NOTHING); - assertEquals("Lines read and printed to console", - 0, selectTool.getLinesRead()); - } - - @Test - public void testLandsatToRemoteFile() throws Throwable { - describe("select part of the landsat to a file"); - Path dest = path("testLandsatToRemoteFile.csv"); - run(selectConf, selectTool, - D, v(CSV_OUTPUT_QUOTE_CHARACTER, "'"), - D, v(CSV_OUTPUT_QUOTE_FIELDS, CSV_OUTPUT_QUOTE_FIELDS_ALWAYS), - "select", - o(OPT_HEADER), CSV_HEADER_OPT_USE, - o(OPT_COMPRESSION), COMPRESSION_OPT_GZIP, - o(OPT_LIMIT), Integer.toString(LINE_COUNT), - o(OPT_OUTPUT), dest.toString(), - landsatSrc, - SELECT_SUNNY_ROWS_NO_LIMIT); - FileStatus status = getFileSystem().getFileStatus(dest); - assertEquals( - "Mismatch between bytes selected and file len in " + status, - selectTool.getBytesRead(), status.getLen()); - assertIsFile(dest); - - // now select on that - Configuration conf = getConfiguration(); - SelectTool tool2 = new SelectTool(conf); - run(conf, tool2, - "select", - o(OPT_HEADER), CSV_HEADER_OPT_NONE, - dest.toString(), - SELECT_EVERYTHING); - } - - @Test - public void testUsage() throws Throwable { - runToFailure(EXIT_USAGE, getConfiguration(), TOO_FEW_ARGUMENTS, - selectTool, "select"); - } - - @Test - public void testRejectionOfNonS3FS() throws Throwable { - File dest = getTempFilename(); - runToFailure(EXIT_SERVICE_UNAVAILABLE, - getConfiguration(), - WRONG_FILESYSTEM, - selectTool, "select", dest.toString(), - SELECT_EVERYTHING); - } - - @Test - public void testFailMissingFile() throws Throwable { - Path dest = path("testFailMissingFile.csv"); - runToFailure(EXIT_NOT_FOUND, - getConfiguration(), - "", - selectTool, "select", dest.toString(), - SELECT_EVERYTHING); - } - - @Test - public void testS3SelectDisabled() throws Throwable { - Configuration conf = getConfiguration(); - conf.setBoolean(FS_S3A_SELECT_ENABLED, false); - disableFilesystemCaching(conf); - runToFailure(EXIT_SERVICE_UNAVAILABLE, - conf, - SELECT_IS_DISABLED, - selectTool, "select", - o(OPT_HEADER), CSV_HEADER_OPT_USE, - o(OPT_COMPRESSION), COMPRESSION_OPT_GZIP, - o(OPT_LIMIT), Integer.toString(LINE_COUNT), - landsatSrc, - SELECT_SUNNY_ROWS_NO_LIMIT); - } - - @Test - public void testSelectBadLimit() throws Throwable { - runToFailure(EXIT_USAGE, - getConfiguration(), - "", - selectTool, "select", - o(OPT_HEADER), CSV_HEADER_OPT_USE, - o(OPT_COMPRESSION), COMPRESSION_OPT_GZIP, - o(OPT_LIMIT), "-1", - landsatSrc, - SELECT_NOTHING); - } - - @Test - public void testSelectBadInputFormat() throws Throwable { - runToFailure(EXIT_COMMAND_ARGUMENT_ERROR, - getConfiguration(), - "", - selectTool, "select", - o(OPT_HEADER), CSV_HEADER_OPT_USE, - o(OPT_INPUTFORMAT), "pptx", - o(OPT_COMPRESSION), COMPRESSION_OPT_GZIP, - landsatSrc, - SELECT_NOTHING); - } - - @Test - public void testSelectBadOutputFormat() throws Throwable { - runToFailure(EXIT_COMMAND_ARGUMENT_ERROR, - getConfiguration(), - "", - selectTool, "select", - o(OPT_HEADER), CSV_HEADER_OPT_USE, - o(OPT_OUTPUTFORMAT), "pptx", - o(OPT_COMPRESSION), COMPRESSION_OPT_GZIP, - landsatSrc, - SELECT_NOTHING); - } - - /** - * Take an option and add the "-" prefix. - * @param in input option - * @return value for the tool args list. - */ - private static String o(String in) { - return "-" + in; - } - - /** - * Create the key=value bit of the -D key=value pair. - * @param key key to set - * @param value value to use - * @return a string for the tool args list. - */ - private static String v(String key, String value) { - return checkNotNull(key) + "=" + checkNotNull(value); - } - -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java deleted file mode 100644 index 4d4af822ee50b..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectLandsat.java +++ /dev/null @@ -1,435 +0,0 @@ -/* - * 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.select; - -import java.io.IOException; -import java.util.List; - -import org.junit.Assume; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FileContext; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.PathIOException; -import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; -import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.Source; -import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; -import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.S3ATestUtils; -import org.apache.hadoop.fs.s3a.Statistic; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.util.DurationInfo; - -import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume; -import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestPropertyBool; -import static org.apache.hadoop.fs.s3a.scale.S3AScaleTestBase._1KB; -import static org.apache.hadoop.fs.s3a.scale.S3AScaleTestBase._1MB; -import static org.apache.hadoop.fs.s3a.select.SelectConstants.*; -import static org.apache.hadoop.test.LambdaTestUtils.intercept; -import static org.hamcrest.CoreMatchers.containsString; -import static org.hamcrest.CoreMatchers.not; - -/** - * Test the S3 Select feature with the Landsat dataset. - * - * This helps explore larger datasets, compression and the like. - * - * This suite is only executed if the destination store declares its support for - * the feature and the test CSV file configuration option points to the - * standard landsat GZip file. That's because these tests require the specific - * format of the landsat file. - * - * Normally working with the landsat file is a scale test. - * Here, because of the select operations, there's a lot less data - * to download. - * For this to work: write aggressive select calls: filtering, using LIMIT - * and projecting down to a few columns. - * - * For the structure, see - * Landsat on AWS - * - * - * entityId: String LC80101172015002LGN00 - * acquisitionDate: String 2015-01-02 15:49:05.571384 - * cloudCover: Float (possibly -ve) 80.81 - * processingLevel: String L1GT - * path: Int 10 - * row: Int 117 - * min_lat: Float -79.09923 - * min_lon: Float -139.66082 - * max_lat: Float -77.7544 - * max_lon: Float 125.09297 - * download_url: HTTPS URL https://s3-us-west-2.amazonaws.com/landsat-pds/L8/010/117/LC80101172015002LGN00/index.html - * - * Ranges - *
      - *
    1. Latitude should range in -180 <= lat <= 180
    2. - *
    3. Longitude in 0 <= lon <= 360
    4. - *
    5. Standard Greenwich Meridian (not the french one which still surfaces)
    6. - *
    7. Cloud cover Should be 0-100, but there are some negative ones.
    8. - *
    - * - * Head of the file: - * - entityId,acquisitionDate,cloudCover,processingLevel,path,row,min_lat,min_lon,max_lat,max_lon,download_url - * LC80101172015002LGN00,2015-01-02 15:49:05.571384,80.81,L1GT,10,117,-79.09923,-139.66082,-77.7544,-125.09297,https://s3-us-west-2.amazonaws.com/landsat-pds/L8/010/117/LC80101172015002LGN00/index.html - * LC80260392015002LGN00,2015-01-02 16:56:51.399666,90.84,L1GT,26,39,29.23106,-97.48576,31.36421,-95.16029,https://s3-us-west-2.amazonaws.com/landsat-pds/L8/026/039/LC80260392015002LGN00/index.html - * LC82270742015002LGN00,2015-01-02 13:53:02.047000,83.44,L1GT,227,74,-21.28598,-59.27736,-19.17398,-57.07423,https://s3-us-west-2.amazonaws.com/landsat-pds/L8/227/074/LC82270742015002LGN00/index.html - * LC82270732015002LGN00,2015-01-02 13:52:38.110317,52.29,L1T,227,73,-19.84365,-58.93258,-17.73324,-56.74692,https://s3-us-west-2.amazonaws.com/landsat-pds/L8/227/073/LC82270732015002LGN00/index.html - * - * - * For the Curious this is the Scala/Spark declaration of the schema. - * - * def addLandsatColumns(csv: DataFrame): DataFrame = { - * csv - * .withColumnRenamed("entityId", "id") - * .withColumn("acquisitionDate", - * csv.col("acquisitionDate").cast(TimestampType)) - * .withColumn("cloudCover", csv.col("cloudCover").cast(DoubleType)) - * .withColumn("path", csv.col("path").cast(IntegerType)) - * .withColumn("row", csv.col("row").cast(IntegerType)) - * .withColumn("min_lat", csv.col("min_lat").cast(DoubleType)) - * .withColumn("min_lon", csv.col("min_lon").cast(DoubleType)) - * .withColumn("max_lat", csv.col("max_lat").cast(DoubleType)) - * .withColumn("max_lon", csv.col("max_lon").cast(DoubleType)) - * .withColumn("year", - * year(col("acquisitionDate"))) - * .withColumn("month", - * month(col("acquisitionDate"))) - * .withColumn("day", - * month(col("acquisitionDate"))) - * } - * - */ -public class ITestS3SelectLandsat extends AbstractS3SelectTest { - - private static final Logger LOG = - LoggerFactory.getLogger(ITestS3SelectLandsat.class); - - private JobConf selectConf; - - /** - * Normal limit for select operations. - * Value: {@value}. - */ - public static final int SELECT_LIMIT = 250; - - /** - * And that select limit as a limit string. - */ - public static final String LIMITED = " LIMIT " + SELECT_LIMIT; - - /** - * Select days with 100% cloud cover, limited to {@link #SELECT_LIMIT}. - * Value: {@value}. - */ - public static final String SELECT_ENTITY_ID_ALL_CLOUDS = - "SELECT\n" - + "s.entityId from\n" - + "S3OBJECT s WHERE\n" - + "s.\"cloudCover\" = '100.0'\n" - + LIMITED; - - /** - * Select sunny days. There's no limit on the returned values, so - * set one except for a scale test. - * Value: {@value}. - */ - public static final String SELECT_SUNNY_ROWS_NO_LIMIT - = "SELECT * FROM S3OBJECT s WHERE s.cloudCover = '0.0'"; - - /** - * A Select call which returns nothing, always. - * Value: {@value}. - */ - public static final String SELECT_NOTHING - = "SELECT * FROM S3OBJECT s WHERE s.cloudCover = 'sunny'"; - - /** - * Select the processing level; no limit. - * Value: {@value}. - */ - public static final String SELECT_PROCESSING_LEVEL_NO_LIMIT = - "SELECT\n" - + "s.processingLevel from\n" - + "S3OBJECT s"; - - @Override - public void setup() throws Exception { - super.setup(); - - selectConf = new JobConf(false); - // file is compressed. - selectConf.set(SELECT_INPUT_COMPRESSION, COMPRESSION_OPT_GZIP); - // and has a header - selectConf.set(CSV_INPUT_HEADER, CSV_HEADER_OPT_USE); - selectConf.setBoolean(SELECT_ERRORS_INCLUDE_SQL, true); - inputMust(selectConf, CSV_INPUT_HEADER, CSV_HEADER_OPT_USE); - inputMust(selectConf, SELECT_INPUT_FORMAT, SELECT_FORMAT_CSV); - inputMust(selectConf, SELECT_OUTPUT_FORMAT, SELECT_FORMAT_CSV); - inputMust(selectConf, SELECT_INPUT_COMPRESSION, COMPRESSION_OPT_GZIP); - // disable the gzip codec, so that the record readers do not - // get confused - enablePassthroughCodec(selectConf, ".gz"); - ChangeDetectionPolicy changeDetectionPolicy = - getLandsatFS().getChangeDetectionPolicy(); - Assume.assumeFalse("the standard landsat bucket doesn't have versioning", - changeDetectionPolicy.getSource() == Source.VersionId - && changeDetectionPolicy.isRequireVersion()); - } - - protected int getMaxLines() { - return SELECT_LIMIT * 2; - } - - @Test - public void testSelectCloudcoverIgnoreHeader() throws Throwable { - describe("select ignoring the header"); - selectConf.set(CSV_INPUT_HEADER, CSV_HEADER_OPT_IGNORE); - String sql = "SELECT\n" - + "* from\n" - + "S3OBJECT s WHERE\n" - + "s._3 = '0.0'\n" - + LIMITED; - List list = selectLandsatFile(selectConf, sql); - LOG.info("Line count: {}", list.size()); - verifySelectionCount(1, SELECT_LIMIT, sql, list); - } - - @Test - public void testSelectCloudcoverUseHeader() throws Throwable { - describe("select 100% cover using the header, " - + "+ verify projection and incrementing select statistics"); - S3ATestUtils.MetricDiff selectCount = new S3ATestUtils.MetricDiff( - getLandsatFS(), - Statistic.OBJECT_SELECT_REQUESTS); - - List list = selectLandsatFile(selectConf, - SELECT_ENTITY_ID_ALL_CLOUDS); - LOG.info("Line count: {}", list.size()); - verifySelectionCount(1, SELECT_LIMIT, SELECT_ENTITY_ID_ALL_CLOUDS, list); - String line1 = list.get(0); - assertThat("no column filtering from " + SELECT_ENTITY_ID_ALL_CLOUDS, - line1, not(containsString("100.0"))); - selectCount.assertDiffEquals("select count", 1); - } - - @Test - public void testFileContextIntegration() throws Throwable { - describe("Test that select works through FileContext"); - FileContext fc = S3ATestUtils.createTestFileContext(getConfiguration()); - - // there's a limit on the number of rows to read; this is larger - // than the SELECT_LIMIT call to catch any failure where more than - // that is returned, newline parsing fails, etc etc. - List list = parseToLines( - select(fc, getLandsatGZ(), selectConf, SELECT_ENTITY_ID_ALL_CLOUDS), - SELECT_LIMIT * 2); - LOG.info("Line count: {}", list.size()); - verifySelectionCount(1, SELECT_LIMIT, SELECT_ENTITY_ID_ALL_CLOUDS, list); - } - - @Test - public void testReadLandsatRecords() throws Throwable { - describe("Use a record reader to read the records"); - inputMust(selectConf, CSV_OUTPUT_FIELD_DELIMITER, "\\t"); - inputMust(selectConf, CSV_OUTPUT_QUOTE_CHARACTER, "'"); - inputMust(selectConf, CSV_OUTPUT_QUOTE_FIELDS, - CSV_OUTPUT_QUOTE_FIELDS_AS_NEEEDED); - inputMust(selectConf, CSV_OUTPUT_RECORD_DELIMITER, "\n"); - List records = readRecords( - selectConf, - getLandsatGZ(), - SELECT_ENTITY_ID_ALL_CLOUDS, - createLineRecordReader(), - SELECT_LIMIT); - verifySelectionCount(1, SELECT_LIMIT, SELECT_ENTITY_ID_ALL_CLOUDS, records); - } - - @Test - public void testReadLandsatRecordsNoMatch() throws Throwable { - describe("Verify the v2 record reader does not fail" - + " when there are no results"); - verifySelectionCount(0, 0, SELECT_NOTHING, - readRecords( - selectConf, - getLandsatGZ(), - SELECT_NOTHING, - createLineRecordReader(), - SELECT_LIMIT)); - } - - @Test - public void testReadLandsatRecordsGZipEnabled() throws Throwable { - describe("Verify that by default, the gzip codec is connected to .gz" - + " files, and so fails"); - // implicitly re-enable the gzip codec. - selectConf.unset(CommonConfigurationKeys.IO_COMPRESSION_CODECS_KEY); - intercept(IOException.class, "gzip", - () -> readRecords( - selectConf, - getLandsatGZ(), - SELECT_ENTITY_ID_ALL_CLOUDS, - createLineRecordReader(), - SELECT_LIMIT)); - } - - @Test - public void testReadLandsatRecordsV1() throws Throwable { - describe("Use a record reader to read the records"); - - verifySelectionCount(1, SELECT_LIMIT, SELECT_ENTITY_ID_ALL_CLOUDS, - readRecords( - selectConf, - getLandsatGZ(), - SELECT_ENTITY_ID_ALL_CLOUDS, - createLineRecordReader(), - SELECT_LIMIT)); - } - - @Test - public void testReadLandsatRecordsV1NoResults() throws Throwable { - describe("verify that a select with no results is not an error"); - - verifySelectionCount(0, 0, SELECT_NOTHING, - readRecords( - selectConf, - getLandsatGZ(), - SELECT_NOTHING, - createLineRecordReader(), - SELECT_LIMIT)); - } - - /** - * Select from the landsat file. - * @param conf config for the select call. - * @param sql template for a formatted SQL request. - * @param args arguments for the formatted request. - * @return the lines selected - * @throws IOException failure - */ - private List selectLandsatFile( - final Configuration conf, - final String sql, - final Object... args) - throws Exception { - - // there's a limit on the number of rows to read; this is larger - // than the SELECT_LIMIT call to catch any failure where more than - // that is returned, newline parsing fails, etc etc. - return parseToLines( - select(getLandsatFS(), getLandsatGZ(), conf, sql, args)); - } - - /** - * This is a larger-scale version of {@link ITestS3Select#testSelectSeek()}. - */ - @Test - public void testSelectSeekFullLandsat() throws Throwable { - describe("Verify forward seeks work, not others"); - - boolean enabled = getTestPropertyBool( - getConfiguration(), - KEY_SCALE_TESTS_ENABLED, - DEFAULT_SCALE_TESTS_ENABLED); - assume("Scale test disabled", enabled); - - // start: read in the full data through the initial select - // this makes asserting that contents match possible - final Path path = getLandsatGZ(); - S3AFileSystem fs = getLandsatFS(); - - int len = (int) fs.getFileStatus(path).getLen(); - byte[] dataset = new byte[4 * _1MB]; - int actualLen; - try (DurationInfo ignored = - new DurationInfo(LOG, "Initial read of %s", path); - FSDataInputStream sourceStream = - select(fs, path, - selectConf, - SELECT_EVERYTHING)) { - // read it in - actualLen = IOUtils.read(sourceStream, dataset); - } - int seekRange = 16 * _1KB; - - try (FSDataInputStream seekStream = - select(fs, path, - selectConf, - SELECT_EVERYTHING)) { - SelectInputStream sis - = (SelectInputStream) seekStream.getWrappedStream(); - S3AInputStreamStatistics streamStats - = sis.getS3AStreamStatistics(); - // lazy seek doesn't raise a problem here - seekStream.seek(0); - assertEquals("first byte read", dataset[0], seekStream.read()); - - // and now the pos has moved, again, seek will be OK - seekStream.seek(1); - seekStream.seek(1); - // but trying to seek elsewhere now fails - intercept(PathIOException.class, - SelectInputStream.SEEK_UNSUPPORTED, - () -> seekStream.seek(0)); - // positioned reads from the current location work. - byte[] buffer = new byte[1]; - seekStream.readFully(seekStream.getPos(), buffer); - // but positioned backwards fail. - intercept(PathIOException.class, - SelectInputStream.SEEK_UNSUPPORTED, - () -> seekStream.readFully(0, buffer)); - // forward seeks are implemented as 1+ skip - long target = seekStream.getPos() + seekRange; - seek(seekStream, target); - assertEquals("Seek position in " + seekStream, - target, seekStream.getPos()); - // now do a read and compare values - assertEquals("byte at seek position", - dataset[(int) seekStream.getPos()], seekStream.read()); - assertEquals("Seek bytes skipped in " + streamStats, - seekRange, streamStats.getBytesSkippedOnSeek()); - long offset; - long increment = 64 * _1KB; - - // seek forward, comparing bytes - for(offset = 32 * _1KB; offset < 256 * _1KB; offset += increment) { - seek(seekStream, offset); - assertEquals("Seek position in " + seekStream, - offset, seekStream.getPos()); - // now do a read and compare values - assertEquals("byte at seek position", - dataset[(int) seekStream.getPos()], seekStream.read()); - } - // there's no knowledge of how much data is left, but with Gzip - // involved there can be a lot. To keep the test duration down, - // this test, unlike the simpler one, doesn't try to read past the - // EOF. Know this: it will be slow. - - LOG.info("Seek statistics {}", streamStats); - } - } - -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java deleted file mode 100644 index c1c7b89dce83d..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestS3SelectMRJob.java +++ /dev/null @@ -1,216 +0,0 @@ -/* - * 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.select; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; -import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy.Source; -import org.junit.Assume; -import org.junit.Test; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.examples.WordCount; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.S3ATestUtils; -import org.apache.hadoop.fs.s3a.S3AUtils; -import org.apache.hadoop.fs.s3a.commit.files.SuccessData; -import org.apache.hadoop.fs.s3a.commit.staging.StagingCommitter; -import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; -import org.apache.hadoop.util.DurationInfo; -import org.apache.hadoop.util.functional.FutureIO; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.MiniYARNCluster; - -import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_NAME; -import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_UNIQUE_FILENAMES; -import static org.apache.hadoop.fs.s3a.select.SelectConstants.*; - -/** - * Run an MR job with a select query. - * This is the effective end-to-end test which verifies: - *
      - *
    1. Passing of select parameters through an MR job conf.
    2. - *
    3. Automatic pick-up of these parameter through TextInputFormat's use - * of the mapreduce.lib.input.LineRecordReaderLineRecordReader.
    4. - *
    5. Issuing of S3 Select queries in mapper processes.
    6. - *
    7. Projection of columns in a select.
    8. - *
    9. Ability to switch to the Passthrough decompressor in an MR job.
    10. - *
    11. Saving of results through the S3A Staging committer.
    12. - *
    13. Basic validation of results.
    14. - *
    - * This makes it the most complex of the MR jobs in the hadoop-aws test suite. - * - * The query used is - * {@link ITestS3SelectLandsat#SELECT_PROCESSING_LEVEL_NO_LIMIT}, - * which lists the processing level of all records in the source file, - * and counts the number in each one by way of the normal word-count - * routines. - * This works because the SQL is projecting only the processing level. - * - * The result becomes something like (with tabs between fields): - *
    - * L1GT   370231
    - * L1T    689526
    - * 
    - */ -public class ITestS3SelectMRJob extends AbstractS3SelectTest { - - private final Configuration conf = new YarnConfiguration(); - - private S3AFileSystem fs; - - private MiniYARNCluster yarnCluster; - - private Path rootPath; - - @Override - public void setup() throws Exception { - super.setup(); - fs = S3ATestUtils.createTestFileSystem(conf); - - ChangeDetectionPolicy changeDetectionPolicy = - getLandsatFS().getChangeDetectionPolicy(); - Assume.assumeFalse("the standard landsat bucket doesn't have versioning", - changeDetectionPolicy.getSource() == Source.VersionId - && changeDetectionPolicy.isRequireVersion()); - - rootPath = path("ITestS3SelectMRJob"); - Path workingDir = path("working"); - fs.setWorkingDirectory(workingDir); - fs.mkdirs(new Path(rootPath, "input/")); - - yarnCluster = new MiniYARNCluster("ITestS3SelectMRJob", // testName - 1, // number of node managers - 1, // number of local log dirs per node manager - 1); // number of hdfs dirs per node manager - yarnCluster.init(conf); - yarnCluster.start(); - } - - @Override - public void teardown() throws Exception { - if (yarnCluster != null) { - yarnCluster.stop(); - } - super.teardown(); - } - - @Test - public void testLandsatSelect() throws Exception { - final Path input = getLandsatGZ(); - final Path output = path("testLandsatSelect") - .makeQualified(fs.getUri(), fs.getWorkingDirectory()); - - final Job job = Job.getInstance(conf, "process level count"); - job.setJarByClass(WordCount.class); - job.setMapperClass(WordCount.TokenizerMapper.class); - job.setCombinerClass(WordCount.IntSumReducer.class); - job.setReducerClass(WordCount.IntSumReducer.class); - job.setOutputKeyClass(Text.class); - job.setOutputValueClass(IntWritable.class); - FileInputFormat.addInputPath(job, input); - FileOutputFormat.setOutputPath(job, output); - - // job with use the staging committer - final JobConf jobConf = (JobConf) job.getConfiguration(); - jobConf.set(FS_S3A_COMMITTER_NAME, StagingCommitter.NAME); - jobConf.setBoolean(FS_S3A_COMMITTER_STAGING_UNIQUE_FILENAMES, - false); - - final String query - = ITestS3SelectLandsat.SELECT_PROCESSING_LEVEL_NO_LIMIT; - inputMust(jobConf, SELECT_SQL, - query); - inputMust(jobConf, SELECT_INPUT_COMPRESSION, COMPRESSION_OPT_GZIP); - - // input settings - inputMust(jobConf, SELECT_INPUT_FORMAT, SELECT_FORMAT_CSV); - inputMust(jobConf, CSV_INPUT_HEADER, CSV_HEADER_OPT_USE); - - // output - inputMust(jobConf, SELECT_OUTPUT_FORMAT, SELECT_FORMAT_CSV); - inputMust(jobConf, CSV_OUTPUT_QUOTE_FIELDS, - CSV_OUTPUT_QUOTE_FIELDS_AS_NEEEDED); - - // disable the gzip codec, so that the record readers do not - // get confused - enablePassthroughCodec(jobConf, ".gz"); - - try (DurationInfo ignored = new DurationInfo(LOG, "SQL " + query)) { - int exitCode = job.waitForCompletion(true) ? 0 : 1; - assertEquals("Returned error code.", 0, exitCode); - } - - // log the success info - Path successPath = new Path(output, "_SUCCESS"); - SuccessData success = SuccessData.load(fs, successPath); - LOG.info("Job _SUCCESS\n{}", success); - - // process the results by ver - // - LOG.info("Results for query \n{}", query); - final AtomicLong parts = new AtomicLong(0); - S3AUtils.applyLocatedFiles(fs.listFiles(output, false), - (status) -> { - Path path = status.getPath(); - // ignore _SUCCESS, any temp files in subdirectories... - if (path.getName().startsWith("part-")) { - parts.incrementAndGet(); - String result = readStringFromFile(path); - LOG.info("{}\n{}", path, result); - String[] lines = result.split("\n", -1); - int l = lines.length; - // add a bit of slack here in case some new processing - // option was added. - assertTrue("Wrong number of lines (" + l + ") in " + result, - l > 0 && l < 15); - } - }); - assertEquals("More part files created than expected", 1, parts.get()); - } - - /** - * Read a file; using Async IO for completeness and to see how - * well the async IO works in practice. - * Summary: checked exceptions cripple Async operations. - */ - private String readStringFromFile(Path path) throws IOException { - int bytesLen = (int)fs.getFileStatus(path).getLen(); - byte[] buffer = new byte[bytesLen]; - return FutureIO.awaitFuture( - fs.openFile(path).build().thenApply(in -> { - try { - IOUtils.readFully(in, buffer, 0, bytesLen); - return new String(buffer); - } catch (IOException ex) { - throw new UncheckedIOException(ex); - } - })); - } -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestSelectUnsupported.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestSelectUnsupported.java new file mode 100644 index 0000000000000..c522db9c5a541 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/ITestSelectUnsupported.java @@ -0,0 +1,100 @@ +/* + * 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.select; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; +import org.apache.hadoop.util.ExitUtil; + +import static org.apache.hadoop.fs.s3a.select.SelectConstants.SELECT_SQL; +import static org.apache.hadoop.fs.s3a.select.SelectConstants.SELECT_UNSUPPORTED; +import static org.apache.hadoop.service.launcher.LauncherExitCodes.EXIT_UNSUPPORTED_VERSION; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.apache.hadoop.util.ExitUtil.disableSystemExit; + +/** + * Verify that s3 select is unsupported. + */ +public class ITestSelectUnsupported extends AbstractS3ATestBase { + + /** + * S3 Select SQL statement. + */ + private static final String STATEMENT = "SELECT *" + + " FROM S3Object s" + + " WHERE s._1 = 'foo'"; + + /** + * A {@code .must(SELECT_SQL, _)} option MSUT not fail the build. + */ + @Test + public void testSelectOpenFileMust() throws Throwable { + + intercept(UnsupportedOperationException.class, SELECT_UNSUPPORTED, () -> + getFileSystem().openFile(methodPath()) + .must(SELECT_SQL, STATEMENT) + .build() + /**/.get()); + } + + /** + * A {@code .opt(SELECT_SQL, _)} option does not fail the build. + */ + @Test + public void testSelectOpenFileMay() throws Throwable { + + final Path path = methodPath(); + final S3AFileSystem fs = getFileSystem(); + ContractTestUtils.touch(fs, path); + fs.openFile(path) + .opt(SELECT_SQL, STATEMENT) + .build() + .get() + .close(); + } + + @Test + public void testPathCapabilityNotAvailable() throws Throwable { + describe("verify that the FS lacks the path capability"); + Assertions.assertThat(getFileSystem().hasPathCapability(methodPath(), SELECT_SQL)) + .describedAs("S3 Select reported as present") + .isFalse(); + } + + @Test + public void testS3GuardToolFails() throws Throwable { + + // ensure that the command doesn't actually exit the VM. + disableSystemExit(); + final ExitUtil.ExitException ex = + intercept(ExitUtil.ExitException.class, SELECT_UNSUPPORTED, + () -> S3GuardTool.main(new String[]{ + "select", "-sql", STATEMENT + })); + Assertions.assertThat(ex.getExitCode()) + .describedAs("exit code of exception") + .isEqualTo(EXIT_UNSUPPORTED_VERSION); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/StreamPublisher.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/StreamPublisher.java deleted file mode 100644 index 461aef726876c..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/StreamPublisher.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * 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.select; - -import java.util.Iterator; -import java.util.concurrent.Executor; -import java.util.stream.Stream; - -import org.reactivestreams.Subscriber; -import org.reactivestreams.Subscription; -import software.amazon.awssdk.core.async.SdkPublisher; - -/** - * Publisher used to test the handling of asynchronous responses. - * @param The type of published elements. - */ -final class StreamPublisher implements SdkPublisher { - private final Executor executor; - private final Iterator iterator; - private Boolean done = false; - - StreamPublisher(Stream data, Executor executor) { - this.iterator = data.iterator(); - this.executor = executor; - } - - StreamPublisher(Stream data) { - this(data, Runnable::run); - } - - @Override - public void subscribe(Subscriber subscriber) { - subscriber.onSubscribe(new Subscription() { - @Override - public void request(long n) { - if (done) { - return; - } - - if (n < 1) { - done = true; - executor.execute(() -> subscriber.onError(new IllegalArgumentException())); - return; - } - - for (long i = 0; i < n; i++) { - final T value; - try { - synchronized (iterator) { - value = iterator.hasNext() ? iterator.next() : null; - } - } catch (Throwable e) { - executor.execute(() -> subscriber.onError(e)); - break; - } - - if (value == null) { - done = true; - executor.execute(subscriber::onComplete); - break; - } else { - executor.execute(() -> subscriber.onNext(value)); - } - } - } - - @Override - public void cancel() { - done = true; - } - }); - } -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/TestBlockingEnumeration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/TestBlockingEnumeration.java deleted file mode 100644 index 43bdcb062f0a3..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/TestBlockingEnumeration.java +++ /dev/null @@ -1,200 +0,0 @@ -/* - * 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.select; - -import java.util.concurrent.Executor; -import java.util.concurrent.Executors; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.IntStream; -import java.util.stream.Stream; - -import org.junit.Assert; -import org.junit.Test; -import software.amazon.awssdk.core.async.SdkPublisher; -import software.amazon.awssdk.core.exception.SdkException; - -/** - * Unit tests for {@link BlockingEnumeration}. - */ -public final class TestBlockingEnumeration extends Assert { - - @Test - public void containsElement() { - SdkPublisher publisher = new StreamPublisher<>(Stream.of("foo")); - - BlockingEnumeration enumeration = - new BlockingEnumeration<>(publisher, 1); - - assertTrue(enumeration.hasMoreElements()); - assertEquals("foo", enumeration.nextElement()); - assertFalse(enumeration.hasMoreElements()); - } - - @Test - public void containsInjectedElement() { - SdkPublisher publisher = new StreamPublisher<>(Stream.of("foo")); - - BlockingEnumeration enumeration = - new BlockingEnumeration<>(publisher, 1, "bar"); - - assertTrue(enumeration.hasMoreElements()); - assertEquals("bar", enumeration.nextElement()); - assertTrue(enumeration.hasMoreElements()); - assertEquals("foo", enumeration.nextElement()); - assertFalse(enumeration.hasMoreElements()); - } - - @Test - public void throwsExceptionOnFirstElement() { - SdkPublisher publisher = new StreamPublisher<>( - Stream.of(0, 1) - .map(i -> { - throw SdkException.create("error!", null); - }), - Executors.newSingleThreadExecutor()); - - BlockingEnumeration enumeration = - new BlockingEnumeration<>(publisher, 1); - assertThrows(SdkException.class, enumeration::hasMoreElements); - } - - @Test - public void throwsExceptionAfterInjectedElement() { - SdkPublisher publisher = new StreamPublisher<>( - Stream.of(0, 1) - .peek(i -> { - throw SdkException.create("error!", null); - }), - Executors.newSingleThreadExecutor()); - - BlockingEnumeration enumeration = - new BlockingEnumeration<>(publisher, 1, 99); - assertTrue(enumeration.hasMoreElements()); - assertEquals(99, enumeration.nextElement().intValue()); - assertThrows(SdkException.class, enumeration::hasMoreElements); - } - - @Test - public void throwsNonSdkException() { - SdkPublisher publisher = new StreamPublisher<>( - Stream.of(0, 1) - .peek(i -> { - throw new RuntimeException("error!", null); - }), - Executors.newSingleThreadExecutor()); - - BlockingEnumeration enumeration = - new BlockingEnumeration<>(publisher, 1); - SdkException exception = Assert.assertThrows(SdkException.class, enumeration::hasMoreElements); - assertEquals(RuntimeException.class, exception.getCause().getClass()); - } - - @Test - public void throwsError() { - SdkPublisher publisher = new StreamPublisher<>( - Stream.of(0, 1) - .peek(i -> { - throw new Error("error!", null); - }), - Executors.newSingleThreadExecutor()); - - BlockingEnumeration enumeration = - new BlockingEnumeration<>(publisher, 1); - assertThrows(Error.class, enumeration::hasMoreElements); - } - - @Test - public void throwsExceptionOnSecondElement() { - SdkPublisher publisher = new StreamPublisher<>( - Stream.of(0, 1) - .peek(i -> { - if (i == 1) { - throw SdkException.create("error!", null); - } - }), - Executors.newSingleThreadExecutor()); - - BlockingEnumeration enumeration = - new BlockingEnumeration<>(publisher, 1); - assertTrue(enumeration.hasMoreElements()); - assertEquals(0, enumeration.nextElement().intValue()); - assertThrows(SdkException.class, enumeration::hasMoreElements); - } - - @Test - public void noMoreElementsAfterThrow() { - SdkPublisher publisher = new StreamPublisher<>( - Stream.of(0, 1) - .map(i -> { - throw SdkException.create("error!", null); - }), - Executors.newSingleThreadExecutor()); - - BlockingEnumeration enumeration = - new BlockingEnumeration<>(publisher, 1); - assertThrows(SdkException.class, enumeration::hasMoreElements); - assertFalse(enumeration.hasMoreElements()); - } - - @Test - public void buffersOnSameThread() { - verifyBuffering(10, 3, Runnable::run); - } - - @Test - public void publisherOnDifferentThread() { - verifyBuffering(5, 1, Executors.newSingleThreadExecutor()); - } - - @Test - public void publisherOnDifferentThreadWithBuffer() { - verifyBuffering(30, 10, Executors.newSingleThreadExecutor()); - } - - private static void verifyBuffering(int length, int bufferSize, Executor executor) { - AtomicInteger emitted = new AtomicInteger(); - SdkPublisher publisher = new StreamPublisher<>( - IntStream.range(0, length).boxed().peek(i -> emitted.incrementAndGet()), - executor); - - BlockingEnumeration enumeration = - new BlockingEnumeration<>(publisher, bufferSize); - - int pulled = 0; - while (true) { - try { - int expected = Math.min(length, pulled + bufferSize); - if (expected != emitted.get()) { - Thread.sleep(10); - } - assertEquals(expected, emitted.get()); - } catch (InterruptedException e) { - fail("Interrupted: " + e); - } - - if (!enumeration.hasMoreElements()) { - break; - } - - int i = enumeration.nextElement(); - assertEquals(pulled, i); - pulled++; - } - } -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/TestSelectEventStreamPublisher.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/TestSelectEventStreamPublisher.java deleted file mode 100644 index fdf3b5b725376..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/TestSelectEventStreamPublisher.java +++ /dev/null @@ -1,190 +0,0 @@ -/* - * 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.select; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.util.Arrays; -import java.util.Collection; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executor; -import java.util.concurrent.Executors; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Stream; - -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import software.amazon.awssdk.core.SdkBytes; -import software.amazon.awssdk.core.async.SdkPublisher; -import software.amazon.awssdk.core.exception.SdkException; -import software.amazon.awssdk.http.AbortableInputStream; -import software.amazon.awssdk.services.s3.model.SelectObjectContentEventStream; -import software.amazon.awssdk.services.s3.model.SelectObjectContentResponse; - -/** - * Unit tests for {@link SelectEventStreamPublisher}. - */ -@RunWith(Parameterized.class) -public final class TestSelectEventStreamPublisher extends Assert { - - @Parameterized.Parameters(name = "threading-{0}") - public static Collection params() { - return Arrays.asList(new Object[][]{ - {"main"}, - {"background"} - }); - } - - private final String threading; - - public TestSelectEventStreamPublisher(String threading) { - this.threading = threading; - } - - private Executor createExecutor() { - if (threading.equals("main")) { - return Runnable::run; - } else if (threading.equals("background")) { - return Executors.newSingleThreadExecutor(); - } else { - throw new IllegalArgumentException("Unknown: " + threading); - } - } - - @Test - public void emptyRecordsInputStream() throws IOException { - SelectEventStreamPublisher selectEventStreamPublisher = - createSelectPublisher(Stream.of( - SelectObjectContentEventStream.recordsBuilder() - .payload(SdkBytes.fromByteArray(new byte[0])) - .build())); - - try (AbortableInputStream inputStream = - selectEventStreamPublisher.toRecordsInputStream(e -> {})) { - assertEquals(-1, inputStream.read()); - } - } - - @Test - public void multipleRecords() throws IOException { - SelectEventStreamPublisher selectEventStreamPublisher = - createSelectPublisher(Stream.of( - SelectObjectContentEventStream.recordsBuilder() - .payload(SdkBytes.fromUtf8String("foo")) - .build(), - SelectObjectContentEventStream.recordsBuilder() - .payload(SdkBytes.fromUtf8String("bar")) - .build())); - - try (AbortableInputStream inputStream = - selectEventStreamPublisher.toRecordsInputStream(e -> {})) { - String result = readAll(inputStream); - assertEquals("foobar", result); - } - } - - @Test - public void skipsOtherEvents() throws IOException { - SelectEventStreamPublisher selectEventStreamPublisher = - createSelectPublisher(Stream.of( - SelectObjectContentEventStream.recordsBuilder() - .payload(SdkBytes.fromUtf8String("foo")) - .build(), - SelectObjectContentEventStream.progressBuilder() - .build(), - SelectObjectContentEventStream.statsBuilder() - .build(), - SelectObjectContentEventStream.recordsBuilder() - .payload(SdkBytes.fromUtf8String("bar")) - .build(), - SelectObjectContentEventStream.endBuilder() - .build())); - - try (AbortableInputStream inputStream = - selectEventStreamPublisher.toRecordsInputStream(e -> {})) { - String result = readAll(inputStream); - assertEquals("foobar", result); - } - } - - @Test - public void callsOnEndEvent() throws IOException { - SelectEventStreamPublisher selectEventStreamPublisher = - createSelectPublisher(Stream.of( - SelectObjectContentEventStream.recordsBuilder() - .payload(SdkBytes.fromUtf8String("foo")) - .build(), - SelectObjectContentEventStream.endBuilder() - .build())); - - AtomicBoolean endEvent = new AtomicBoolean(false); - try (AbortableInputStream inputStream = - selectEventStreamPublisher.toRecordsInputStream(e -> endEvent.set(true))) { - String result = readAll(inputStream); - assertEquals("foo", result); - } - - assertTrue(endEvent.get()); - } - - @Test - public void handlesErrors() throws IOException { - SelectEventStreamPublisher selectEventStreamPublisher = - createSelectPublisher(Stream.of( - SelectObjectContentEventStream.recordsBuilder() - .payload(SdkBytes.fromUtf8String("foo")) - .build(), - SelectObjectContentEventStream.recordsBuilder() - .payload(SdkBytes.fromUtf8String("bar")) - .build()) - .map(e -> { - throw SdkException.create("error!", null); - })); - - try (AbortableInputStream inputStream = - selectEventStreamPublisher.toRecordsInputStream(e -> {})) { - assertThrows(SdkException.class, () -> readAll(inputStream)); - } - } - - private SelectEventStreamPublisher createSelectPublisher( - Stream stream) { - SdkPublisher sdkPublisher = - new StreamPublisher<>(stream, createExecutor()); - CompletableFuture future = - CompletableFuture.completedFuture(null); - SelectObjectContentResponse response = - SelectObjectContentResponse.builder().build(); - return new SelectEventStreamPublisher(future, response, sdkPublisher); - } - - private static String readAll(InputStream inputStream) throws IOException { - try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream()) { - byte[] buffer = new byte[8096]; - int read; - while ((read = inputStream.read(buffer, 0, buffer.length)) != -1) { - outputStream.write(buffer, 0, read); - } - return outputStream.toString(); - } - } -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalWriteOperationHelperCallbacks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalWriteOperationHelperCallbacks.java index eee0c71950566..6838129bb30d1 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalWriteOperationHelperCallbacks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/MinimalWriteOperationHelperCallbacks.java @@ -18,12 +18,8 @@ package org.apache.hadoop.fs.s3a.test; -import java.util.concurrent.CompletableFuture; - import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadResponse; -import software.amazon.awssdk.services.s3.model.SelectObjectContentRequest; -import software.amazon.awssdk.services.s3.model.SelectObjectContentResponseHandler; import org.apache.hadoop.fs.s3a.WriteOperationHelper; @@ -33,18 +29,11 @@ public class MinimalWriteOperationHelperCallbacks implements WriteOperationHelper.WriteOperationHelperCallbacks { - @Override - public CompletableFuture selectObjectContent( - SelectObjectContentRequest request, - SelectObjectContentResponseHandler th) { - return null; - } - @Override public CompleteMultipartUploadResponse completeMultipartUpload( CompleteMultipartUploadRequest request) { return null; } -}; +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/CsvFile.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/CsvFile.java similarity index 98% rename from hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/CsvFile.java rename to hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/CsvFile.java index 17574372de429..a786f974e0c40 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/select/CsvFile.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/CsvFile.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.select; +package org.apache.hadoop.fs.s3a.tools; import java.io.Closeable; import java.io.IOException; From 736a3ccf194bb6a3082b65e85f121d1b9302f991 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Mon, 29 Jan 2024 14:56:39 +0000 Subject: [PATCH 2/3] HADOOP-18830. S3 select. final review comments +update doc with explicit consequences of removal, including the "hadoop s3guard select" command Change-Id: Ica3ffb33d803f8a03e0d733447e4f026d547f8c1 --- .../hadoop/fs/s3a/select/SelectConstants.java | 12 +++-- .../markdown/tools/hadoop-aws/s3_select.md | 50 ++++++++++++++++++- .../fs/s3a/select/ITestSelectUnsupported.java | 10 ++-- 3 files changed, 63 insertions(+), 9 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectConstants.java index 97771368769fd..d1c977f92824d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/select/SelectConstants.java @@ -31,7 +31,8 @@ * so that any application which imports the dependencies will still link. */ @InterfaceAudience.Public -@InterfaceStability.Unstable +@InterfaceStability.Stable +@Deprecated public final class SelectConstants { public static final String SELECT_UNSUPPORTED = "S3 Select is no longer supported"; @@ -46,13 +47,18 @@ private SelectConstants() { /** * This is the big SQL expression: {@value}. - * When used in an open() call, switch to a select operation. - * This is only used in the open call, never in a filesystem configuration. + * When used in an open() call: + *
      + *
    1. if the option is set in a {@code .may()} clause: warn and continue
    2. + *
    3. if the option is set in a {@code .must()} clause: + * {@code UnsupportedOperationException}.
    4. + *
    */ public static final String SELECT_SQL = FS_S3A_SELECT + "sql"; /** * Does the FS Support S3 Select? + * This is false everywhere. * Value: {@value}. */ public static final String S3_SELECT_CAPABILITY = "fs.s3a.capability.select.sql"; diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3_select.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3_select.md index 57771da8b273a..197a7cd4084d5 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3_select.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3_select.md @@ -24,4 +24,52 @@ Any Hadoop release built on the [AWS V2 SDK](./aws_sdk_upgrade.html) will reject calls to open files using the select APIs. If a build of Hadoop with S3 Select is desired, the relevant -classes can be found in hadoop trunk commit `8bf72346a59c`. \ No newline at end of file +classes can be found in hadoop trunk commit `8bf72346a59c`. + +## Consequences of the removal + +The path capabilities probe `fs.s3a.capability.select.sql` returns "false" for any and all +`s3a://` paths. + +Any `openFile()` call where a SQL query is passed in as a `must()` clause +SHALL raise `UnsupportedOperationException`: +```java +// fails +openFile("s3a://bucket/path") + .must("fs.s3a.select.sql", "SELECT ...") + .get(); +``` + +Any `openFile()` call to an S3A Path where a SQL query is passed in as a `may()` +clause SHALL be logged at WARN level the first time it is invoked, then ignored. +```java +// ignores the option after printing a warning. +openFile("s3a://bucket/path") + .may("fs.s3a.select.sql", "SELECT ...") + .get(); +``` + +The `hadoop s3guard select` command is no longer supported. + +Previously, the command would either generate an S3 select or a error (with exit code 42 being +the one for not enough arguments): + +``` +hadoop s3guard select +select [OPTIONS] [-limit rows] [-header (use|none|ignore)] [-out path] [-expected rows] + [-compression (gzip|bzip2|none)] [-inputformat csv] [-outputformat csv]