HADOOP-18830. Cut S3 Select (#6144)
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 Contributed by Steve Loughran
This commit is contained in:
parent
6da1a19a83
commit
8261229daa
@ -1121,11 +1121,6 @@
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>software.amazon.eventstream</groupId>
|
||||
<artifactId>eventstream</artifactId>
|
||||
<version>${aws.eventstream.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.mina</groupId>
|
||||
<artifactId>mina-core</artifactId>
|
||||
|
@ -508,11 +508,6 @@
|
||||
<artifactId>bundle</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>software.amazon.eventstream</groupId>
|
||||
<artifactId>eventstream</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.assertj</groupId>
|
||||
<artifactId>assertj-core</artifactId>
|
||||
|
@ -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 <T> CompletableFuture<T> submit(final CallableRaisingIOE<T> operation) {
|
||||
private final class WriteOperationHelperCallbacksImpl
|
||||
implements WriteOperationHelper.WriteOperationHelperCallbacks {
|
||||
|
||||
@Override
|
||||
public CompletableFuture<Void> 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<FSDataInputStream> openFileWithOptions(
|
||||
parameters,
|
||||
getDefaultBlockSize());
|
||||
CompletableFuture<FSDataInputStream> 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;
|
||||
}
|
||||
|
||||
|
@ -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.
|
||||
|
@ -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",
|
||||
|
@ -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 @@
|
||||
* <li>Other low-level access to S3 functions, for private use.</li>
|
||||
* <li>Failure handling, including converting exceptions to IOEs.</li>
|
||||
* <li>Integration with instrumentation.</li>
|
||||
* <li>Evolution to add more low-level operations, such as S3 select.</li>
|
||||
* </ul>
|
||||
*
|
||||
* 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<Void> selectObjectContent(SelectObjectContentRequest request,
|
||||
SelectObjectContentResponseHandler t);
|
||||
|
||||
/**
|
||||
* Initiates a complete multi-part upload request.
|
||||
* @param request Complete multi-part upload request
|
||||
|
@ -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.
|
||||
|
@ -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
|
||||
|
@ -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,
|
||||
|
@ -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
|
||||
|
@ -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<String> S3A_OPENFILE_KEYS;
|
||||
|
@ -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<String> 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
|
||||
|
@ -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);
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
@ -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 <T> the type of element.
|
||||
*/
|
||||
public final class BlockingEnumeration<T> implements Enumeration<T> {
|
||||
private static final class Signal<T> {
|
||||
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<T> endSignal = new Signal<>((Throwable)null);
|
||||
private final CompletableFuture<Subscription> subscription = new CompletableFuture<>();
|
||||
private final BlockingQueue<Signal<T>> signalQueue;
|
||||
private final int bufferSize;
|
||||
private Signal<T> 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<T> 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<T> 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<T> {
|
||||
|
||||
@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);
|
||||
}
|
||||
}
|
||||
}
|
@ -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<String> 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<String> 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);
|
||||
}
|
||||
}
|
@ -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.
|
||||
* <ol>
|
||||
* <li> Options in {@code openFileOptions}.</li>
|
||||
* <li> Options in the owning filesystem configuration.</li>
|
||||
* <li>The default values in {@link SelectConstants}</li>
|
||||
* </ol>
|
||||
*
|
||||
* @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("\\\\", "\\");
|
||||
}
|
||||
|
||||
|
||||
}
|
@ -25,13 +25,19 @@
|
||||
* 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
|
||||
@InterfaceStability.Stable
|
||||
@Deprecated
|
||||
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() {
|
||||
}
|
||||
@ -41,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:
|
||||
* <ol>
|
||||
* <li>if the option is set in a {@code .may()} clause: warn and continue</li>
|
||||
* <li>if the option is set in a {@code .must()} clause:
|
||||
* {@code UnsupportedOperationException}.</li>
|
||||
* </ol>
|
||||
*/
|
||||
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";
|
||||
|
@ -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<SelectObjectContentEventStream> {
|
||||
|
||||
private final CompletableFuture<Void> selectOperationFuture;
|
||||
private final SelectObjectContentResponse response;
|
||||
private final SdkPublisher<SelectObjectContentEventStream> publisher;
|
||||
|
||||
/**
|
||||
* Create the publisher.
|
||||
* @param selectOperationFuture SelectObjectContent future
|
||||
* @param response SelectObjectContent response
|
||||
* @param publisher SelectObjectContentEventStream publisher to wrap
|
||||
*/
|
||||
public SelectEventStreamPublisher(
|
||||
CompletableFuture<Void> selectOperationFuture,
|
||||
SelectObjectContentResponse response,
|
||||
SdkPublisher<SelectObjectContentEventStream> 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<EndEvent> onEndEvent) {
|
||||
SdkPublisher<InputStream> 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<? super SelectObjectContentEventStream> 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]);
|
||||
}
|
@ -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 <i>And the requirements of the filesystem
|
||||
* specification, this updates the stream position as returned in
|
||||
* {@link #getPos()}.</i>
|
||||
* @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.
|
||||
* <b>Important: there are no guarantees as to the stability
|
||||
* of this value.</b>
|
||||
* @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();
|
||||
}
|
||||
}
|
||||
}
|
@ -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<Void> 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<Pair<SelectObjectContentResponse,
|
||||
SdkPublisher<SelectObjectContentEventStream>>> responseAndPublisherFuture =
|
||||
new CompletableFuture<>();
|
||||
|
||||
private volatile SelectObjectContentResponse response;
|
||||
|
||||
public CompletableFuture<SelectEventStreamPublisher> eventPublisher(
|
||||
CompletableFuture<Void> 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<SelectObjectContentEventStream> publisher) {
|
||||
responseAndPublisherFuture.complete(Pair.of(response, publisher));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void exceptionOccurred(Throwable error) {
|
||||
responseAndPublisherFuture.completeExceptionally(error);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void complete() {
|
||||
}
|
||||
}
|
||||
}
|
@ -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:
|
||||
* <pre>
|
||||
* hadoop s3guard select [options] Path Statement
|
||||
* </pre>
|
||||
*/
|
||||
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]"
|
||||
+ " <PATH> <SELECT QUERY>\n"
|
||||
+ "\t" + PURPOSE + "\n\n";
|
||||
|
||||
public static final String OPT_COMPRESSION = "compression";
|
||||
|
||||
public static final String OPT_EXPECTED = "expected";
|
||||
|
||||
public static final String OPT_HEADER = "header";
|
||||
|
||||
public static final String OPT_INPUTFORMAT = "inputformat";
|
||||
|
||||
public static final String OPT_LIMIT = "limit";
|
||||
|
||||
public static final String OPT_OUTPUT = "out";
|
||||
|
||||
public static final String OPT_OUTPUTFORMAT = "inputformat";
|
||||
|
||||
static final String TOO_FEW_ARGUMENTS = "Too few arguments";
|
||||
|
||||
static final String SELECT_IS_DISABLED = "S3 Select is disabled";
|
||||
|
||||
private OperationDuration selectDuration;
|
||||
|
||||
private long bytesRead;
|
||||
|
||||
private long linesRead;
|
||||
|
||||
public SelectTool(Configuration conf) {
|
||||
super(conf);
|
||||
// read capacity.
|
||||
getCommandFormat().addOptionWithValue(OPT_COMPRESSION);
|
||||
getCommandFormat().addOptionWithValue(OPT_EXPECTED);
|
||||
getCommandFormat().addOptionWithValue(OPT_HEADER);
|
||||
getCommandFormat().addOptionWithValue(OPT_INPUTFORMAT);
|
||||
getCommandFormat().addOptionWithValue(OPT_LIMIT);
|
||||
getCommandFormat().addOptionWithValue(OPT_OUTPUT);
|
||||
getCommandFormat().addOptionWithValue(OPT_OUTPUTFORMAT);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return NAME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getUsage() {
|
||||
return USAGE;
|
||||
}
|
||||
|
||||
public OperationDuration getSelectDuration() {
|
||||
return selectDuration;
|
||||
}
|
||||
|
||||
public long getBytesRead() {
|
||||
return bytesRead;
|
||||
}
|
||||
|
||||
/**
|
||||
* Number of lines read, when printing to the console.
|
||||
* @return line count. 0 if writing direct to a file.
|
||||
*/
|
||||
public long getLinesRead() {
|
||||
return linesRead;
|
||||
}
|
||||
|
||||
private int parseNaturalInt(String option, String value) {
|
||||
try {
|
||||
int r = Integer.parseInt(value);
|
||||
if (r < 0) {
|
||||
throw invalidArgs("Negative value for option %s : %s", option, value);
|
||||
}
|
||||
return r;
|
||||
} catch (NumberFormatException e) {
|
||||
throw invalidArgs("Invalid number for option %s : %s", option, value);
|
||||
}
|
||||
}
|
||||
|
||||
private Optional<String> getOptValue(String key) {
|
||||
String value = getCommandFormat().getOptValue(key);
|
||||
return isNotEmpty(value) ? Optional.of(value): Optional.empty();
|
||||
}
|
||||
|
||||
private Optional<Integer> getIntValue(String key) {
|
||||
Optional<String> v = getOptValue(key);
|
||||
return v.map(i -> parseNaturalInt(key, i));
|
||||
}
|
||||
|
||||
/**
|
||||
* Execute the select operation.
|
||||
* @param args argument list
|
||||
* @param out output stream
|
||||
* @return an exit code
|
||||
* @throws IOException IO failure
|
||||
* @throws ExitUtil.ExitException managed failure
|
||||
*/
|
||||
public int run(String[] args, PrintStream out)
|
||||
throws IOException, ExitUtil.ExitException {
|
||||
final List<String> parsedArgs;
|
||||
try {
|
||||
parsedArgs = parseArgs(args);
|
||||
} catch (CommandFormat.UnknownOptionException e) {
|
||||
errorln(getUsage());
|
||||
throw new ExitUtil.ExitException(EXIT_USAGE, e.getMessage(), e);
|
||||
}
|
||||
if (parsedArgs.size() < 2) {
|
||||
errorln(getUsage());
|
||||
throw new ExitUtil.ExitException(EXIT_USAGE, TOO_FEW_ARGUMENTS);
|
||||
}
|
||||
|
||||
// read mandatory arguments
|
||||
final String file = parsedArgs.get(0);
|
||||
final Path path = new Path(file);
|
||||
|
||||
String expression = parsedArgs.get(1);
|
||||
|
||||
println(out, "selecting file %s with query %s",
|
||||
path, expression);
|
||||
|
||||
// and the optional arguments to adjust the configuration.
|
||||
final Optional<String> header = getOptValue(OPT_HEADER);
|
||||
header.ifPresent(h -> println(out, "Using header option %s", h));
|
||||
|
||||
Path destPath = getOptValue(OPT_OUTPUT).map(
|
||||
output -> {
|
||||
println(out, "Saving output to %s", output);
|
||||
return new Path(output);
|
||||
}).orElse(null);
|
||||
final boolean toConsole = destPath == null;
|
||||
|
||||
// expected lines are only checked if empty
|
||||
final Optional<Integer> expectedLines = toConsole
|
||||
? getIntValue(OPT_EXPECTED)
|
||||
: Optional.empty();
|
||||
|
||||
final Optional<Integer> limit = getIntValue(OPT_LIMIT);
|
||||
if (limit.isPresent()) {
|
||||
final int l = limit.get();
|
||||
println(out, "Using line limit %s", l);
|
||||
if (expression.toLowerCase(Locale.ENGLISH).contains(" limit ")) {
|
||||
println(out, "line limit already specified in SELECT expression");
|
||||
} else {
|
||||
expression = expression + " LIMIT " + l;
|
||||
}
|
||||
}
|
||||
|
||||
// now bind to the filesystem.
|
||||
FileSystem fs = bindFilesystem(path.getFileSystem(getConf()));
|
||||
|
||||
if (!fs.hasPathCapability(path, S3_SELECT_CAPABILITY)) {
|
||||
// capability disabled
|
||||
throw new ExitUtil.ExitException(EXIT_SERVICE_UNAVAILABLE,
|
||||
SELECT_IS_DISABLED + " for " + file);
|
||||
}
|
||||
linesRead = 0;
|
||||
|
||||
selectDuration = new OperationDuration();
|
||||
|
||||
// open and scan the stream.
|
||||
final FutureDataInputStreamBuilder builder = fs.openFile(path)
|
||||
.must(SELECT_SQL, expression);
|
||||
|
||||
header.ifPresent(h -> builder.must(CSV_INPUT_HEADER, h));
|
||||
|
||||
getOptValue(OPT_COMPRESSION).ifPresent(compression ->
|
||||
builder.must(SELECT_INPUT_COMPRESSION,
|
||||
compression.toUpperCase(Locale.ENGLISH)));
|
||||
|
||||
getOptValue(OPT_INPUTFORMAT).ifPresent(opt -> {
|
||||
if (!"csv".equalsIgnoreCase(opt)) {
|
||||
throw invalidArgs("Unsupported input format %s", opt);
|
||||
}
|
||||
});
|
||||
getOptValue(OPT_OUTPUTFORMAT).ifPresent(opt -> {
|
||||
if (!"csv".equalsIgnoreCase(opt)) {
|
||||
throw invalidArgs("Unsupported output format %s", opt);
|
||||
}
|
||||
});
|
||||
// turn on SQL error reporting.
|
||||
builder.opt(SELECT_ERRORS_INCLUDE_SQL, true);
|
||||
|
||||
FSDataInputStream stream;
|
||||
try(DurationInfo ignored =
|
||||
new DurationInfo(LOG, "Selecting stream")) {
|
||||
stream = FutureIO.awaitFuture(builder.build());
|
||||
} catch (FileNotFoundException e) {
|
||||
// the source file is missing.
|
||||
throw notFound(e);
|
||||
}
|
||||
try {
|
||||
if (toConsole) {
|
||||
// logging to console
|
||||
bytesRead = 0;
|
||||
@SuppressWarnings("IOResourceOpenedButNotSafelyClosed")
|
||||
Scanner scanner =
|
||||
new Scanner(
|
||||
new BufferedReader(
|
||||
new InputStreamReader(stream, StandardCharsets.UTF_8)));
|
||||
scanner.useDelimiter("\n");
|
||||
while (scanner.hasNextLine()) {
|
||||
linesRead++;
|
||||
String l = scanner.nextLine();
|
||||
bytesRead += l.length() + 1;
|
||||
println(out, "%s", l);
|
||||
}
|
||||
} else {
|
||||
// straight dump of whole file; no line counting
|
||||
FileSystem destFS = destPath.getFileSystem(getConf());
|
||||
try(DurationInfo ignored =
|
||||
new DurationInfo(LOG, "Copying File");
|
||||
OutputStream destStream = destFS.createFile(destPath)
|
||||
.overwrite(true)
|
||||
.build()) {
|
||||
bytesRead = IOUtils.copy(stream, destStream);
|
||||
}
|
||||
}
|
||||
|
||||
// close the stream.
|
||||
// this will take time if there's a lot of data remaining
|
||||
try (DurationInfo ignored =
|
||||
new DurationInfo(LOG, "Closing stream")) {
|
||||
stream.close();
|
||||
}
|
||||
|
||||
// generate a meaningful result depending on the operation
|
||||
String result = toConsole
|
||||
? String.format("%s lines", linesRead)
|
||||
: String.format("%s bytes", bytesRead);
|
||||
|
||||
// print some statistics
|
||||
selectDuration.finished();
|
||||
println(out, "Read %s in time %s",
|
||||
result, selectDuration.getDurationString());
|
||||
|
||||
println(out, "Bytes Read: %,d bytes", bytesRead);
|
||||
|
||||
println(out, "Bandwidth: %,.1f MiB/s",
|
||||
bandwidthMBs(bytesRead, selectDuration.value()));
|
||||
|
||||
} finally {
|
||||
cleanupWithLogger(LOG, stream);
|
||||
}
|
||||
|
||||
LOG.debug("Statistics {}", stream);
|
||||
|
||||
expectedLines.ifPresent(l -> {
|
||||
if (l != linesRead) {
|
||||
throw exitException(EXIT_FAIL,
|
||||
"Expected %d rows but the operation returned %d",
|
||||
l, linesRead);
|
||||
}
|
||||
});
|
||||
out.flush();
|
||||
return EXIT_SUCCESS;
|
||||
}
|
||||
|
||||
/**
|
||||
* Work out the bandwidth in MB/s.
|
||||
* @param bytes bytes
|
||||
* @param durationMillisNS duration in nanos
|
||||
* @return the number of megabytes/second of the recorded operation
|
||||
*/
|
||||
public static double bandwidthMBs(long bytes, long durationMillisNS) {
|
||||
return durationMillisNS > 0
|
||||
? (bytes / 1048576.0 * 1000 / durationMillisNS)
|
||||
: 0;
|
||||
}
|
||||
}
|
@ -17,7 +17,12 @@
|
||||
*/
|
||||
|
||||
/**
|
||||
* Support for S3 Select.
|
||||
* Was the location for support for S3 Select.
|
||||
* Now removed apart from some constants.f
|
||||
* There was a class {@code BlockingEnumeration} which
|
||||
* mapped SdkPublisher to an Enumeration.
|
||||
* This may be of use elsewhere; it can be retrieved from
|
||||
* hadoop commit 8bf72346a59c.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
|
@ -74,7 +74,8 @@ There are three core settings to connect to an S3 store, endpoint, region and wh
|
||||
<name>fs.s3a.endpoint</name>
|
||||
<description>AWS S3 endpoint to connect to. An up-to-date list is
|
||||
provided in the AWS Documentation: regions and endpoints. Without this
|
||||
property, the standard region (s3.amazonaws.com) is assumed.
|
||||
property, the endpoint/hostname of the S3 Store is inferred from
|
||||
the value of fs.s3a.endpoint.region, fs.s3a.endpoint.fips and more.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
@ -230,8 +231,9 @@ S3 endpoint, documented [by Amazon](http://docs.aws.amazon.com/general/latest/gr
|
||||
use local buckets and local copies of data, wherever possible.
|
||||
2. With the V4 signing protocol, AWS requires the explicit region endpoint
|
||||
to be used —hence S3A must be configured to use the specific endpoint. This
|
||||
is done in the configuration option `fs.s3a.endpoint`.
|
||||
3. All endpoints other than the default endpoint only support interaction
|
||||
is done by setting the regon in the configuration option `fs.s3a.endpoint.region`,
|
||||
or by explicitly setting `fs.s3a.endpoint` and `fs.s3a.endpoint.region`.
|
||||
3. All endpoints other than the default region only support interaction
|
||||
with buckets local to that S3 instance.
|
||||
4. Standard S3 buckets support "cross-region" access where use of the original `us-east-1`
|
||||
endpoint allows access to the data, but newer storage types, particularly S3 Express are
|
||||
@ -248,25 +250,12 @@ The up to date list of regions is [Available online](https://docs.aws.amazon.com
|
||||
This list can be used to specify the endpoint of individual buckets, for example
|
||||
for buckets in the central and EU/Ireland endpoints.
|
||||
|
||||
```xml
|
||||
<property>
|
||||
<name>fs.s3a.bucket.landsat-pds.endpoint</name>
|
||||
<value>s3-us-west-2.amazonaws.com</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.s3a.bucket.eu-dataset.endpoint</name>
|
||||
<value>s3.eu-west-1.amazonaws.com</value>
|
||||
</property>
|
||||
```
|
||||
|
||||
Declaring the region for the data is simpler, as it avoid having to look up the full URL and having to worry about historical quirks of regional endpoint hostnames.
|
||||
|
||||
```xml
|
||||
<property>
|
||||
<name>fs.s3a.bucket.landsat-pds.endpoint.region</name>
|
||||
<value>us-west-2</value>
|
||||
<description>The endpoint for s3a://landsat-pds URLs</description>
|
||||
<description>The region for s3a://landsat-pds URLs</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
@ -421,7 +410,6 @@ bucket by bucket basis i.e. `fs.s3a.bucket.{YOUR-BUCKET}.accesspoint.required`.
|
||||
```
|
||||
|
||||
Before using Access Points make sure you're not impacted by the following:
|
||||
- `ListObjectsV1` is not supported, this is also deprecated on AWS S3 for performance reasons;
|
||||
- The endpoint for S3 requests will automatically change to use
|
||||
`s3-accesspoint.REGION.amazonaws.{com | com.cn}` depending on the Access Point ARN. While
|
||||
considering endpoints, if you have any custom signers that use the host endpoint property make
|
||||
|
@ -686,7 +686,6 @@ clients where S3-CSE has not been enabled.
|
||||
- Writing files may be slower, as only a single block can be encrypted and
|
||||
uploaded at a time.
|
||||
- Multipart Uploader API disabled.
|
||||
- S3 Select is not supported.
|
||||
- Multipart uploads would be serial, and partSize must be a multiple of 16
|
||||
bytes.
|
||||
- maximum message size in bytes that can be encrypted under this mode is
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -22,9 +22,6 @@ connection to S3 to interact with a bucket. Unit test suites follow the naming
|
||||
convention `Test*.java`. Integration tests follow the naming convention
|
||||
`ITest*.java`.
|
||||
|
||||
Due to eventual consistency, integration tests may fail without reason.
|
||||
Transient failures, which no longer occur upon rerunning the test, should thus
|
||||
be ignored.
|
||||
|
||||
## <a name="policy"></a> Policy for submitting patches which affect the `hadoop-aws` module.
|
||||
|
||||
@ -56,7 +53,6 @@ make for a slow iterative development.
|
||||
Please: run the tests. And if you don't, we are sorry for declining your
|
||||
patch, but we have to.
|
||||
|
||||
|
||||
### What if there's an intermittent failure of a test?
|
||||
|
||||
Some of the tests do fail intermittently, especially in parallel runs.
|
||||
@ -147,7 +143,7 @@ Example:
|
||||
</configuration>
|
||||
```
|
||||
|
||||
### <a name="encryption"></a> Configuring S3a Encryption
|
||||
## <a name="encryption"></a> Configuring S3a Encryption
|
||||
|
||||
For S3a encryption tests to run correctly, the
|
||||
`fs.s3a.encryption.key` must be configured in the s3a contract xml
|
||||
@ -175,6 +171,21 @@ on the AWS side. Some S3AFileSystem tests are skipped when default encryption is
|
||||
enabled due to unpredictability in how [ETags](https://docs.aws.amazon.com/AmazonS3/latest/API/RESTCommonResponseHeaders.html)
|
||||
are generated.
|
||||
|
||||
### Disabling the encryption tests
|
||||
|
||||
If the S3 store/storage class doesn't support server-side-encryption, these will fail. They
|
||||
can be turned off.
|
||||
|
||||
```xml
|
||||
<property>
|
||||
<name>test.fs.s3a.encryption.enabled</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
```
|
||||
|
||||
Encryption is only used for those specific test suites with `Encryption` in
|
||||
their classname.
|
||||
|
||||
## <a name="running"></a> Running the Tests
|
||||
|
||||
After completing the configuration, execute the test run through Maven.
|
||||
@ -241,23 +252,11 @@ define the target region in `auth-keys.xml`.
|
||||
|
||||
```xml
|
||||
<property>
|
||||
<name>fs.s3a.endpoint</name>
|
||||
<value>s3.eu-central-1.amazonaws.com</value>
|
||||
<name>fs.s3a.endpoint.region</name>
|
||||
<value>eu-central-1</value>
|
||||
</property>
|
||||
```
|
||||
|
||||
Alternatively you can use endpoints defined in [core-site.xml](../../../../test/resources/core-site.xml).
|
||||
|
||||
```xml
|
||||
<property>
|
||||
<name>fs.s3a.endpoint</name>
|
||||
<value>${frankfurt.endpoint}</value>
|
||||
</property>
|
||||
```
|
||||
|
||||
This is used for all tests expect for scale tests using a Public CSV.gz file
|
||||
(see below)
|
||||
|
||||
### <a name="csv"></a> CSV Data Tests
|
||||
|
||||
The `TestS3AInputStreamPerformance` tests require read access to a multi-MB
|
||||
@ -265,6 +264,12 @@ text file. The default file for these tests is one published by amazon,
|
||||
[s3a://landsat-pds.s3.amazonaws.com/scene_list.gz](http://landsat-pds.s3.amazonaws.com/scene_list.gz).
|
||||
This is a gzipped CSV index of other files which amazon serves for open use.
|
||||
|
||||
Historically it was required to be a `csv.gz` file to validate S3 Select
|
||||
support. Now that S3 Select support has been removed, other large files
|
||||
may be used instead.
|
||||
However, future versions may want to read a CSV file again, so testers
|
||||
should still reference one.
|
||||
|
||||
The path to this object is set in the option `fs.s3a.scale.test.csvfile`,
|
||||
|
||||
```xml
|
||||
@ -284,19 +289,21 @@ and "sufficiently" large.
|
||||
(the reason the space or newline is needed is to add "an empty entry"; an empty
|
||||
`<value/>` would be considered undefined and pick up the default)
|
||||
|
||||
Of using a test file in an S3 region requiring a different endpoint value
|
||||
set in `fs.s3a.endpoint`, a bucket-specific endpoint must be defined.
|
||||
|
||||
If using a test file in a different AWS S3 region then
|
||||
a bucket-specific region must be defined.
|
||||
For the default test dataset, hosted in the `landsat-pds` bucket, this is:
|
||||
|
||||
```xml
|
||||
<property>
|
||||
<name>fs.s3a.bucket.landsat-pds.endpoint</name>
|
||||
<value>s3.amazonaws.com</value>
|
||||
<description>The endpoint for s3a://landsat-pds URLs</description>
|
||||
<name>fs.s3a.bucket.landsat-pds.endpoint.region</name>
|
||||
<value>us-west-2</value>
|
||||
<description>The region for s3a://landsat-pds</description>
|
||||
</property>
|
||||
```
|
||||
|
||||
### <a name="csv"></a> Testing Access Point Integration
|
||||
### <a name="access"></a> Testing Access Point Integration
|
||||
|
||||
S3a supports using Access Point ARNs to access data in S3. If you think your changes affect VPC
|
||||
integration, request signing, ARN manipulation, or any code path that deals with the actual
|
||||
sending and retrieving of data to/from S3, make sure you run the entire integration test suite with
|
||||
@ -551,9 +558,9 @@ They do not run automatically: they must be explicitly run from the command line
|
||||
|
||||
Look in the source for these and reads the Javadocs before executing.
|
||||
|
||||
## <a name="alternate_s3"></a> Testing against non AWS S3 endpoints.
|
||||
## <a name="alternate_s3"></a> Testing against non-AWS S3 Stores.
|
||||
|
||||
The S3A filesystem is designed to work with storage endpoints which implement
|
||||
The S3A filesystem is designed to work with S3 stores which implement
|
||||
the S3 protocols to the extent that the amazon S3 SDK is capable of talking
|
||||
to it. We encourage testing against other filesystems and submissions of patches
|
||||
which address issues. In particular, we encourage testing of Hadoop release
|
||||
@ -572,10 +579,6 @@ on third party stores.
|
||||
<name>test.fs.s3a.create.storage.class.enabled</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.s3a.select.enabled</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>test.fs.s3a.sts.enabled</name>
|
||||
<value>false</value>
|
||||
@ -583,9 +586,11 @@ on third party stores.
|
||||
<property>
|
||||
<name>test.fs.s3a.create.create.acl.enabled</name>
|
||||
<value>false</value>
|
||||
< /property>
|
||||
</property>
|
||||
```
|
||||
|
||||
See [Third Party Stores](third_party_stores.html) for more on this topic.
|
||||
|
||||
### Public datasets used in tests
|
||||
|
||||
Some tests rely on the presence of existing public datasets available on Amazon S3.
|
||||
@ -599,20 +604,6 @@ store that supports these tests.
|
||||
An example of this might be the MarkerTools tests which require a bucket with a large number of
|
||||
objects or the requester pays tests that require requester pays to be enabled for the bucket.
|
||||
|
||||
### Disabling the encryption tests
|
||||
|
||||
If the endpoint doesn't support server-side-encryption, these will fail. They
|
||||
can be turned off.
|
||||
|
||||
```xml
|
||||
<property>
|
||||
<name>test.fs.s3a.encryption.enabled</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
```
|
||||
|
||||
Encryption is only used for those specific test suites with `Encryption` in
|
||||
their classname.
|
||||
|
||||
### Disabling the storage class tests
|
||||
|
||||
@ -643,23 +634,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
|
||||
### <a name="enabling-prefetch"></a> Enabling prefetch for all tests
|
||||
|
||||
The S3 select tests are skipped when the S3 endpoint doesn't support S3 Select.
|
||||
|
||||
```xml
|
||||
<property>
|
||||
<name>fs.s3a.select.enabled</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
```
|
||||
|
||||
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.
|
||||
|
||||
### 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.
|
||||
@ -671,7 +649,7 @@ If `ITestS3AContractGetFileStatusV1List` fails with any error about unsupported
|
||||
```
|
||||
|
||||
Note: there's no equivalent for turning off v2 listing API, which all stores are now
|
||||
expected to support.
|
||||
required to support.
|
||||
|
||||
|
||||
### Testing Requester Pays
|
||||
@ -762,12 +740,8 @@ after setting this rerun the tests
|
||||
log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
|
||||
```
|
||||
|
||||
There are also some logging options for debug logging of the AWS client
|
||||
```properties
|
||||
log4j.logger.com.amazonaws=DEBUG
|
||||
log4j.logger.com.amazonaws.http.conn.ssl=INFO
|
||||
log4j.logger.com.amazonaws.internal=INFO
|
||||
```
|
||||
There are also some logging options for debug logging of the AWS client;
|
||||
consult the file.
|
||||
|
||||
There is also the option of enabling logging on a bucket; this could perhaps
|
||||
be used to diagnose problems from that end. This isn't something actively
|
||||
@ -889,13 +863,13 @@ against other regions, or with third party S3 implementations. Thus the
|
||||
URL can be overridden for testing elsewhere.
|
||||
|
||||
|
||||
### Works With Other S3 Endpoints
|
||||
### Works With Other S3 Stored
|
||||
|
||||
Don't assume AWS S3 US-East only, do allow for working with external S3 implementations.
|
||||
Those may be behind the latest S3 API features, not support encryption, session
|
||||
APIs, etc.
|
||||
|
||||
They won't have the same CSV test files as some of the input tests rely on.
|
||||
They won't have the same CSV/large test files as some of the input tests rely on.
|
||||
Look at `ITestS3AInputStreamPerformance` to see how tests can be written
|
||||
to support the declaration of a specific large test file on alternate filesystems.
|
||||
|
||||
@ -952,6 +926,8 @@ modifying the config. As an example from `AbstractTestS3AEncryption`:
|
||||
protected Configuration createConfiguration() {
|
||||
Configuration conf = super.createConfiguration();
|
||||
S3ATestUtils.disableFilesystemCaching(conf);
|
||||
removeBaseAndBucketOverrides(conf,
|
||||
SERVER_SIDE_ENCRYPTION_ALGORITHM);
|
||||
conf.set(Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM,
|
||||
getSSEAlgorithm().getMethod());
|
||||
return conf;
|
||||
@ -1008,9 +984,8 @@ than on the maven command line:
|
||||
|
||||
### Keeping AWS Costs down
|
||||
|
||||
Most of the base S3 tests are designed to use public AWS data
|
||||
(the landsat-pds bucket) for read IO, so you don't have to pay for bytes
|
||||
downloaded or long term storage costs. The scale tests do work with more data
|
||||
Most of the base S3 tests are designed delete files after test runs,
|
||||
so you don't have to pay for storage costs. The scale tests do work with more data
|
||||
so will cost more as well as generally take more time to execute.
|
||||
|
||||
You are however billed for
|
||||
@ -1119,7 +1094,7 @@ The usual credentials needed to log in to the bucket will be used, but now
|
||||
the credentials used to interact with S3 will be temporary
|
||||
role credentials, rather than the full credentials.
|
||||
|
||||
## <a name="qualifiying_sdk_updates"></a> Qualifying an AWS SDK Update
|
||||
## <a name="qualifying_sdk_updates"></a> Qualifying an AWS SDK Update
|
||||
|
||||
Updating the AWS SDK is something which does need to be done regularly,
|
||||
but is rarely without complications, major or minor.
|
||||
@ -1296,19 +1271,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
|
||||
|
@ -428,11 +428,6 @@ this makes renaming and deleting significantly slower.
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.s3a.bucket.gcs-container.select.enabled</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.s3a.bucket.gcs-container.path.style.access</name>
|
||||
<value>true</value>
|
||||
|
@ -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
|
||||
|
@ -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));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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.
|
||||
* <pre>
|
||||
* 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
|
||||
* </pre>
|
||||
*/
|
||||
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.
|
||||
* <pre>
|
||||
* 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
|
||||
* </pre>
|
||||
* @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<CsvFile> 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<String> 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<String> parseToLines(final FSDataInputStream selection,
|
||||
int maxLines)
|
||||
throws IOException {
|
||||
List<String> 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<String> verifySelectionCount(
|
||||
final int expected,
|
||||
final String expression,
|
||||
final List<String> 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<String> verifySelectionCount(
|
||||
final int min,
|
||||
final int max,
|
||||
final String expression,
|
||||
final List<String> 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<String> 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<String> readRecords(JobConf conf,
|
||||
Path path,
|
||||
String sql,
|
||||
RecordReader<?, ?> reader,
|
||||
int initialCapacity) throws Exception {
|
||||
|
||||
inputMust(conf, SELECT_SQL, sql);
|
||||
List<String> 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 <K, V> List<String> readRecordsV1(JobConf conf,
|
||||
org.apache.hadoop.mapred.RecordReader<K, V> reader,
|
||||
K key,
|
||||
V value,
|
||||
int initialCapacity) throws Exception {
|
||||
List<String> 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<LongWritable, Text> createLineRecordReader() {
|
||||
return new LineRecordReader(new byte[]{'\n'});
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a v1 line record reader.
|
||||
* @return a reader
|
||||
*/
|
||||
protected org.apache.hadoop.mapred.RecordReader<LongWritable, Text>
|
||||
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 extends Exception> T logIntercepted(T ex) {
|
||||
LOG.info("Intercepted Exception is ", ex);
|
||||
return ex;
|
||||
}
|
||||
}
|
@ -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<FSDataInputStream> 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<String> 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<String> 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<String> 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<FSDataInputStream> 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<String> 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<String> 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<String> 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<String> 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<String> 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<String> 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<String> 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<String> 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<String> 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<InputSplit> splits = tif.getSplits(jobCtx);
|
||||
assertThat("split count wrong", splits, hasSize(1));
|
||||
|
||||
}
|
||||
|
||||
}
|
@ -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<String> 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);
|
||||
}
|
||||
|
||||
}
|
@ -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
|
||||
* <a href="https://docs.opendata.aws/landsat-pds/readme.html">Landsat on AWS</a>
|
||||
*
|
||||
* <code>
|
||||
* 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
|
||||
* </code>
|
||||
* Ranges
|
||||
* <ol>
|
||||
* <li>Latitude should range in -180 <= lat <= 180</li>
|
||||
* <li>Longitude in 0 <= lon <= 360</li>
|
||||
* <li>Standard Greenwich Meridian (not the french one which still surfaces)</li>
|
||||
* <li>Cloud cover <i>Should</i> be 0-100, but there are some negative ones.</li>
|
||||
* </ol>
|
||||
*
|
||||
* Head of the file:
|
||||
* <code>
|
||||
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
|
||||
* </code>
|
||||
*
|
||||
* For the Curious this is the Scala/Spark declaration of the schema.
|
||||
* <code>
|
||||
* 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")))
|
||||
* }
|
||||
* </code>
|
||||
*/
|
||||
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<String> 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<String> 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<String> 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<String> 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<String> 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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -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:
|
||||
* <ol>
|
||||
* <li>Passing of select parameters through an MR job conf.</li>
|
||||
* <li>Automatic pick-up of these parameter through TextInputFormat's use
|
||||
* of the mapreduce.lib.input.LineRecordReaderLineRecordReader.</li>
|
||||
* <li>Issuing of S3 Select queries in mapper processes.</li>
|
||||
* <li>Projection of columns in a select.</li>
|
||||
* <li>Ability to switch to the Passthrough decompressor in an MR job.</li>
|
||||
* <li>Saving of results through the S3A Staging committer.</li>
|
||||
* <li>Basic validation of results.</li>
|
||||
* </ol>
|
||||
* 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):
|
||||
* <pre>
|
||||
* L1GT 370231
|
||||
* L1T 689526
|
||||
* </pre>
|
||||
*/
|
||||
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);
|
||||
}
|
||||
}));
|
||||
}
|
||||
}
|
@ -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 MUST raise {@code UnsupportedOperationException}.
|
||||
*/
|
||||
@Test
|
||||
public void testSelectOpenFileMustFailure() throws Throwable {
|
||||
|
||||
intercept(UnsupportedOperationException.class, SELECT_UNSUPPORTED, () ->
|
||||
getFileSystem().openFile(methodPath())
|
||||
.must(SELECT_SQL, STATEMENT)
|
||||
.build()
|
||||
.get());
|
||||
}
|
||||
|
||||
/**
|
||||
* A {@code .opt(SELECT_SQL, _)} option is ignored..
|
||||
*/
|
||||
@Test
|
||||
public void testSelectOpenFileMayIsIgnored() 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);
|
||||
}
|
||||
}
|
@ -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 <T> The type of published elements.
|
||||
*/
|
||||
final class StreamPublisher<T> implements SdkPublisher<T> {
|
||||
private final Executor executor;
|
||||
private final Iterator<T> iterator;
|
||||
private Boolean done = false;
|
||||
|
||||
StreamPublisher(Stream<T> data, Executor executor) {
|
||||
this.iterator = data.iterator();
|
||||
this.executor = executor;
|
||||
}
|
||||
|
||||
StreamPublisher(Stream<T> data) {
|
||||
this(data, Runnable::run);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void subscribe(Subscriber<? super T> 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;
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
@ -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<String> publisher = new StreamPublisher<>(Stream.of("foo"));
|
||||
|
||||
BlockingEnumeration<String> enumeration =
|
||||
new BlockingEnumeration<>(publisher, 1);
|
||||
|
||||
assertTrue(enumeration.hasMoreElements());
|
||||
assertEquals("foo", enumeration.nextElement());
|
||||
assertFalse(enumeration.hasMoreElements());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void containsInjectedElement() {
|
||||
SdkPublisher<String> publisher = new StreamPublisher<>(Stream.of("foo"));
|
||||
|
||||
BlockingEnumeration<String> 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<Integer> publisher = new StreamPublisher<>(
|
||||
Stream.of(0, 1)
|
||||
.map(i -> {
|
||||
throw SdkException.create("error!", null);
|
||||
}),
|
||||
Executors.newSingleThreadExecutor());
|
||||
|
||||
BlockingEnumeration<Integer> enumeration =
|
||||
new BlockingEnumeration<>(publisher, 1);
|
||||
assertThrows(SdkException.class, enumeration::hasMoreElements);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void throwsExceptionAfterInjectedElement() {
|
||||
SdkPublisher<Integer> publisher = new StreamPublisher<>(
|
||||
Stream.of(0, 1)
|
||||
.peek(i -> {
|
||||
throw SdkException.create("error!", null);
|
||||
}),
|
||||
Executors.newSingleThreadExecutor());
|
||||
|
||||
BlockingEnumeration<Integer> enumeration =
|
||||
new BlockingEnumeration<>(publisher, 1, 99);
|
||||
assertTrue(enumeration.hasMoreElements());
|
||||
assertEquals(99, enumeration.nextElement().intValue());
|
||||
assertThrows(SdkException.class, enumeration::hasMoreElements);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void throwsNonSdkException() {
|
||||
SdkPublisher<Integer> publisher = new StreamPublisher<>(
|
||||
Stream.of(0, 1)
|
||||
.peek(i -> {
|
||||
throw new RuntimeException("error!", null);
|
||||
}),
|
||||
Executors.newSingleThreadExecutor());
|
||||
|
||||
BlockingEnumeration<Integer> enumeration =
|
||||
new BlockingEnumeration<>(publisher, 1);
|
||||
SdkException exception = Assert.assertThrows(SdkException.class, enumeration::hasMoreElements);
|
||||
assertEquals(RuntimeException.class, exception.getCause().getClass());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void throwsError() {
|
||||
SdkPublisher<Integer> publisher = new StreamPublisher<>(
|
||||
Stream.of(0, 1)
|
||||
.peek(i -> {
|
||||
throw new Error("error!", null);
|
||||
}),
|
||||
Executors.newSingleThreadExecutor());
|
||||
|
||||
BlockingEnumeration<Integer> enumeration =
|
||||
new BlockingEnumeration<>(publisher, 1);
|
||||
assertThrows(Error.class, enumeration::hasMoreElements);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void throwsExceptionOnSecondElement() {
|
||||
SdkPublisher<Integer> publisher = new StreamPublisher<>(
|
||||
Stream.of(0, 1)
|
||||
.peek(i -> {
|
||||
if (i == 1) {
|
||||
throw SdkException.create("error!", null);
|
||||
}
|
||||
}),
|
||||
Executors.newSingleThreadExecutor());
|
||||
|
||||
BlockingEnumeration<Integer> enumeration =
|
||||
new BlockingEnumeration<>(publisher, 1);
|
||||
assertTrue(enumeration.hasMoreElements());
|
||||
assertEquals(0, enumeration.nextElement().intValue());
|
||||
assertThrows(SdkException.class, enumeration::hasMoreElements);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void noMoreElementsAfterThrow() {
|
||||
SdkPublisher<Integer> publisher = new StreamPublisher<>(
|
||||
Stream.of(0, 1)
|
||||
.map(i -> {
|
||||
throw SdkException.create("error!", null);
|
||||
}),
|
||||
Executors.newSingleThreadExecutor());
|
||||
|
||||
BlockingEnumeration<Integer> 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<Integer> publisher = new StreamPublisher<>(
|
||||
IntStream.range(0, length).boxed().peek(i -> emitted.incrementAndGet()),
|
||||
executor);
|
||||
|
||||
BlockingEnumeration<Integer> 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++;
|
||||
}
|
||||
}
|
||||
}
|
@ -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<Object[]> 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<SelectObjectContentEventStream> stream) {
|
||||
SdkPublisher<SelectObjectContentEventStream> sdkPublisher =
|
||||
new StreamPublisher<>(stream, createExecutor());
|
||||
CompletableFuture<Void> 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();
|
||||
}
|
||||
}
|
||||
}
|
@ -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<Void> selectObjectContent(
|
||||
SelectObjectContentRequest request,
|
||||
SelectObjectContentResponseHandler th) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public CompleteMultipartUploadResponse completeMultipartUpload(
|
||||
CompleteMultipartUploadRequest request) {
|
||||
return null;
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -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;
|
@ -31,6 +31,13 @@
|
||||
</property>
|
||||
|
||||
<!-- Per-bucket configurations: landsat-pds -->
|
||||
<!--
|
||||
A CSV file in this bucket was used for testing S3 select.
|
||||
Although this feature has been removed, (HADOOP-18830)
|
||||
it is still used in some tests as a large file to read
|
||||
in a bucket without write permissions.
|
||||
These tests do not need a CSV file.
|
||||
-->
|
||||
<property>
|
||||
<name>fs.s3a.bucket.landsat-pds.endpoint.region</name>
|
||||
<value>us-west-2</value>
|
||||
@ -56,13 +63,13 @@
|
||||
<description>Do not add the referrer header to landsat operations</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.s3a.bucket.landsat-pds.endpoint.fips</name>
|
||||
<value>true</value>
|
||||
<description>Use the fips endpoint</description>
|
||||
</property>
|
||||
|
||||
<!-- Per-bucket configurations: usgs-landsat -->
|
||||
<!--
|
||||
This is a requester-pays bucket (so validates that feature)
|
||||
and, because it has many files, is used to validate paged file
|
||||
listing without needing to create thousands of files.
|
||||
-->
|
||||
|
||||
<property>
|
||||
<name>fs.s3a.bucket.usgs-landsat.endpoint.region</name>
|
||||
<value>us-west-2</value>
|
||||
|
Loading…
Reference in New Issue
Block a user