Experimental Feature
S3 Select is a feature for Amazon S3 introduced in April 2018. It allows for SQL-like SELECT expressions to be applied to files in some structured formats, including CSV and JSON.
By performing the SELECT operation in the S3 storage infrastructure, the bandwidth requirements between S3 and the hosts making the request can be reduced. Along with latency, this bandwidth is often the limiting factor in processing data from S3, especially with larger CSV and JSON datasets.
Apache Hadoop’s S3A Client has experimental support for this feature, with the following warnings:
S3 Select is enabled by default:
<property> <name>fs.s3a.select.enabled</name> <value>true</value> <description>Is S3 Select enabled?</description> </property>
To disable it, set the option fs.s3a.select.enabled to false.
To probe to see if a FileSystem instance implements it, StreamCapabilities.hasCapability("s3a:fs.s3a.select.sql") will be true on an instance of the S3AFileSystem class if the version of Hadoop supports S3 Select, and it is enabled for that specific instance.
If this call returns false, then S3 Select calls will fail.
Rather than cast a filesystem to the S3AFileSystem class, cast it to org.apache.hadoop.fs.StreamCapabilities; a class which was added in Hadoop 2.9. This should result in less brittle code -and there is no need to have the hadoop-aws JAR on the classpath at compile time.
/** * Probe for a filesystem instance supporting S3 Select. * @param fs filesystem * @return true if the filesystem supports S3 Select. */ public static boolean hasS3SelectCapability(final FileSystem fs) { return (fs instanceof StreamCapabilities) && ((StreamCapabilities)fs).hasCapability("s3a:fs.s3a.select.sql"); }
Applications can issue select queries through the Hadoop FileSystem/FileContext APIs via the asynchronous openFile() call added in Hadoop 3.3.
Use the FileSystem.openFile(path) or FileContext.openFile(path) methods command to get a builder class for the open operations, then set the mandatory s3 select options though multiple must() parameters.
FileSystem.FSDataInputStreamBuilder builder = filesystem.openFile("s3a://bucket/path-to-file.csv") .must("fs.s3a.select.sql", "SELECT * FROM S3OBJECT s WHERE s.\"odd\" = `TRUE`") .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") .must("fs.s3a.select.output.csv.field.delimiter", "\t") .must("fs.s3a.select.output.csv.quote.character", "\"") .must("fs.s3a.select.output.csv.quote.fields", "asneeded") ; CompletableFuture<FSDataInputStream> future = builder.build(); try (FSDataInputStream select = future.get()) { // process the output byte[] bytes = new byte[8192]; int actual = select.read(bytes); }
When the Builder’s build() call is made, if the FileSystem/FileContext instance does not recognize any of the mandatory options it will fail. The S3A connector does recognize them, and, if S3 Select has not been disabled, will issue the Select query against the object store.
If the S3A connector has S3 Select disabled, it will fail with an UnsupportedOperationException.
The build() call returns a CompletableFuture<FSDataInputStream>. This future retrieves the result of the select call, which is executed asynchronously in the S3A FileSystem instance’s executor pool.
Errors in the SQL, missing file, permission failures and suchlike will surface when the future is evaluated, not the build call.
In the returned stream, seeking and positioned reads do not work as usual, because there are no absolute positions in the file to seek to.
This is all done as a best-effort attempt to support existing code which often uses seek() to move forward in a split file after opening, or does a series of positioned read calls.
The returned stream, of type org.apache.hadoop.fs.s3a.select.SelectInputStream, only supports forward seek() operations.
A zero-byte seek operation is always valid:
stream.seek(stream.getPos());
A negative seek operation will always fail:
stream.seek(stream.getPos() - offset);
A forward seek operation will work, provided the final position is less than the total length of the stream:
stream.seek(stream.getPos() + offset);
If it is past the end of the file, an EOFException is raised.
Important Forward seeks are implemented by reading and discarding the contents of the stream. The bigger the forward seek, the more data is thrown away, the longer the operation takes. And, if the data is being paid for over a long-haul S3 connection. the more the seek costs.
Calling seek() on a SelectInputStream should only be used with care.
The feature has been implemented for splittable queries across Selected data, where the initial read starts with a seek() to the offset. However, for reasons covered below, a codec should be used to declare the input unsplittable.
Third party object stores do not, at the time of writing, support S3 Select. S3 Select operations against such stores will fail, presumably with a store-specific error code.
To avoid problems, disable S3 Select entirely:
<property> <name>fs.s3a.select.enabled</name> <value>false</value> </property>
This guarantees that the hasCapability() check will fail immediately, rather than delaying the failure until an SQL query is attempted.
The s3guard select command allows direct select statements to be made of a path.
Usage:
hadoop s3guard select [OPTIONS] \ [-limit rows] \ [-header (use|none|ignore)] \ [-out file] \ [-compression (gzip|none)] \ [-expected rows] [-inputformat csv] [-outputformat csv] <PATH> <SELECT QUERY>
The output is printed, followed by some summary statistics, unless the -out option is used to declare a destination file. In this mode status will be logged to the console, but the output of the query will be saved directly to the output file.
Read the first 100 rows of the landsat dataset where cloud cover is zero:
hadoop s3guard select -header use -compression gzip -limit 100 \ s3a://landsat-pds/scene_list.gz \ "SELECT * FROM S3OBJECT s WHERE s.cloudCover = '0.0'"
Return the entityId column for all rows in the dataset where the cloud cover was “0.0”, and save it to the file output.csv:
hadoop s3guard select -header use -out s3a://mybucket/output.csv \ -compression gzip \ s3a://landsat-pds/scene_list.gz \ "SELECT s.entityId from S3OBJECT s WHERE s.cloudCover = '0.0'"
This file will:
The output can be saved to a file with the -out option. Note also that -D key=value settings can be used to control the operation, if placed after the s3guard command and before select
hadoop s3guard \ -D s.s3a.select.output.csv.quote.fields=asneeded \ select \ -header use \ -compression gzip \ -limit 500 \ -inputformat csv \ -outputformat csv \ -out s3a://hwdev-steve-new/output.csv \ s3a://landsat-pds/scene_list.gz \ "SELECT s.entityId from S3OBJECT s WHERE s.cloudCover = '0.0'"
S3 Select support in analytics queries is a work in progress. It does not work reliably with large source files where the work is split up.
As a proof of concept only, S3 Select queries can be made through MapReduce jobs which use any Hadoop RecordReader class which uses the new openFile() API.
Currently this consists of the following MRv2 readers.
org.apache.hadoop.mapreduce.lib.input.LineRecordReader org.apache.hadoop.mapreduce.lib.input.FixedLengthRecordReader
And a limited number of the MRv1 record readers:
org.apache.hadoop.mapred.LineRecordReader
All of these readers use the new API and can be have its optional/mandatory options set via the JobConf used when creating/configuring the reader.
These readers are instantiated within input formats; the following formats therefore support S3 Select.
org.apache.hadoop.mapreduce.lib.input.FixedLengthInputFormat org.apache.hadoop.mapreduce.lib.input.KeyValueTextInputFormat org.apache.hadoop.mapreduce.lib.input.NLineInputFormat org.apache.hadoop.mapreduce.lib.input.TextInputFormat org.apache.hadoop.mapred.KeyValueTextInputFormat org.apache.hadoop.mapred.TextInputFormat org.apache.hadoop.mapred.lib.NLineInputFormat
All JobConf options which begin with the prefix mapreduce.job.input.file.option. will have that prefix stripped and the remainder used as the name for an option when opening the file.
All JobConf options which being with the prefix mapreduce.job.input.file.must. will be converted into mandatory options.
To use an S3 Select call, set the following options
mapreduce.job.input.file.must.fs.s3a.select.sql = <SQL STATEMENT> mapreduce.job.input.file.must.fs.s3a.select.input.format = CSV mapreduce.job.input.file.must.fs.s3a.select.output.format = CSV
Further options may be set to tune the behaviour, for example:
jobConf.set("mapreduce.job.input.file.must.fs.s3a.select.input.csv.header", "use");
Note How to tell if a reader has migrated to the new openFile() builder API:
Set a mandatory option which is not known; if the job does not fail then an old reader is being used.
jobConf.set("mapreduce.job.input.file.must.unknown.option", "anything");
S3 Select queries can be made against gzipped source files; the S3A input stream receives the output in text format, rather than as a (re)compressed stream.
To read a gzip file, set fs.s3a.select.input.compression to gzip.
jobConf.set("mapreduce.job.input.file.must.fs.s3a.select.input.compression", "gzip");
Most of the Hadoop RecordReader classes automatically choose a decompressor based on the extension of the source file. This causes problems when reading .gz files, because S3 Select is automatically decompressing and returning csv-formatted text.
By default, a query across gzipped files will fail with the error “IOException: not a gzip file”
To avoid this problem, declare that the job should switch to the “Passthrough Codec” for all files with a “.gz” extension:
jobConf.set("io.compression.codecs", "org.apache.hadoop.io.compress.PassthroughCodec"); jobConf.set("io.compress.passthrough.extension", ".gz");
Obviously, this breaks normal .gz decompression: only set it on S3 Select jobs.
Consult the javadocs for org.apache.hadoop.fs.s3a.select.SelectConstants.
The listed options can be set in core-site.xml, supported by S3A per-bucket configuration, and can be set programmatically on the Configuration object use to configure a new filesystem instance.
Any of these options can be set in the builder returned by the openFile() call —simply set them through a chain of builder.must() operations.
<property> <name>fs.s3a.select.input.format</name> <value>csv</value> <description>Input format</description> </property> <property> <name>fs.s3a.select.output.format</name> <value>csv</value> <description>Output format</description> </property> <property> <name>fs.s3a.select.input.csv.comment.marker</name> <value>#</value> <description>In S3 Select queries: the marker for comment lines in CSV files</description> </property> <property> <name>fs.s3a.select.input.csv.record.delimiter</name> <value>\n</value> <description>In S3 Select queries over CSV files: the record delimiter. \t is remapped to the TAB character, \r to CR \n to newline. \\ to \ and \" to " </description> </property> <property> <name>fs.s3a.select.input.csv.field.delimiter</name> <value>,</value> <description>In S3 Select queries over CSV files: the field delimiter. \t is remapped to the TAB character, \r to CR \n to newline. \\ to \ and \" to " </description> </property> <property> <name>fs.s3a.select.input.csv.quote.character</name> <value>"</value> <description>In S3 Select queries over CSV files: quote character. \t is remapped to the TAB character, \r to CR \n to newline. \\ to \ and \" to " </description> </property> <property> <name>fs.s3a.select.input.csv.quote.escape.character</name> <value>\\</value> <description>In S3 Select queries over CSV files: quote escape character. \t is remapped to the TAB character, \r to CR \n to newline. \\ to \ and \" to " </description> </property> <property> <name>fs.s3a.select.input.csv.header</name> <value>none</value> <description>In S3 Select queries over CSV files: what is the role of the header? One of "none", "ignore" and "use"</description> </property> <property> <name>fs.s3a.select.input.compression</name> <value>none</value> <description>In S3 Select queries, the source compression algorithm. One of: "none" and "gzip"</description> </property> <property> <name>fs.s3a.select.output.csv.quote.fields</name> <value>always</value> <description> In S3 Select queries: should fields in generated CSV Files be quoted? One of: "always", "asneeded". </description> </property> <property> <name>fs.s3a.select.output.csv.quote.character</name> <value>"</value> <description> In S3 Select queries: the quote character for generated CSV Files. </description> </property> <property> <name>fs.s3a.select.output.csv.quote.escape.character</name> <value>\\</value> <description> In S3 Select queries: the quote escape character for generated CSV Files. </description> </property> <property> <name>fs.s3a.select.output.csv.record.delimiter</name> <value>\n</value> <description> In S3 Select queries: the record delimiter for generated CSV Files. </description> </property> <property> <name>fs.s3a.select.output.csv.field.delimiter</name> <value>,</value> <description> In S3 Select queries: the field delimiter for generated CSV Files. </description> </property> <property> <name>fs.s3a.select.errors.include.sql</name> <value>false</value> <description> Include the SQL statement in errors: this is useful for development but may leak security and Personally Identifying Information in production, so must be disabled there. </description> </property>
SQL Injection attacks are the classic attack on data. Because S3 Select is a read-only API, the classic “Bobby Tables” attack to gain write access isn’t going to work. Even so: sanitize your inputs.
CSV does have security issues of its own, specifically:
Excel and other spreadsheets may interpret some fields beginning with special characters as formula, and execute them
S3 Select does not appear vulnerable to this, but in workflows where untrusted data eventually ends up in a spreadsheet (including Google Document spreadsheets), the data should be sanitized/audited first. There is no support for such sanitization in S3 Select or in the S3A connector.
Logging Select statements may expose secrets if they are in the statement. Even if they are just logged, this may potentially leak Personally Identifying Information as covered in the EU GDPR legislation and equivalents.
For both privacy and security reasons, SQL statements are not included in exception strings by default, nor logged at INFO level.
To enable them, set fs.s3a.select.errors.include.sql to true, either in the site/application configuration, or as an option in the builder for a single request. When set, the request will also be logged at the INFO level of the log org.apache.hadoop.fs.s3a.select.SelectBinding.
Personal Identifiable Information is not printed in the AWS S3 logs. Those logs contain only the SQL keywords from the query planner. All column names and literals are masked. Following is a sample log example:
Query:
SELECT * FROM S3OBJECT s;
Log:
select (project (list (project_all))) (from (as str0 (id str1 case_insensitive)))
Note also that:
The best practise here is: only enable SQL in exceptions while developing SQL queries, especially in an application/notebook where the exception text is a lot easier to see than the application logs.
In production: don’t log or report. If you do, all logs and output must be considered sensitive from security and privacy perspectives.
The hadoop s3guard select command does enable the logging, so can be used as an initial place to experiment with the SQL syntax. Rationale: if you are constructing SQL queries on the command line, your shell history is already tainted with the query.
The SQL Syntax directly supported by the AWS S3 Select API is documented by Amazon.
“CSV” is less a format, more “a term meaning the data is in some nonstandard line-by-line” text file, and there are even “multiline CSV files”.
S3 Select only supports a subset of the loose “CSV” concept, as covered in the AWS documentation. There are also limits on how many columns and how large a single line may be.
The specific quotation character, field and record delimiters, comments and escape characters can be configured in the Hadoop configuration.
Consistency
Assume the usual S3 consistency model applies.
When enabled, S3Guard’s DynamoDB table will declare whether or not a newly deleted file is visible: if it is marked as deleted, the select request will be rejected with a FileNotFoundException.
When an existing S3-hosted object is changed, the S3 select operation may return the results of a SELECT call as applied to either the old or new version.
We don’t know whether you can get partially consistent reads, or whether an extended read ever picks up a later value.
The AWS S3 load balancers can briefly cache 404/Not-Found entries from a failed HEAD/GET request against a nonexistent file; this cached entry can briefly create create inconsistency, despite the AWS “Create is consistent” model. There is no attempt to detect or recover from this.
Concurrency
The outcome of what happens when source file is overwritten while the result of a select call is overwritten is undefined.
The input stream returned by the operation is NOT THREAD SAFE.
Error Handling
If an attempt to issue an S3 select call fails, the S3A connector will reissue the request if-and-only-if it believes a retry may succeed. That is: it considers the operation to be idempotent and if the failure is considered to be a recoverable connectivity problem or a server-side rejection which can be retried (500, 503).
If an attempt to read data from an S3 select stream (org.apache.hadoop.fs.s3a.select.SelectInputStream) fails partway through the read, no attempt is made to retry the operation
In contrast, the normal S3A input stream tries to recover from (possibly transient) failures by attempting to reopen the file.
The select operation is best when the least amount of data is returned by the query, as this reduces the amount of data downloaded.
The select call itself can be slow, especially when the source is a multi-MB compressed file with aggressive filtering in the WHERE clause. Assumption: the select query starts at row 1 and scans through each row, and does not return data until it has matched one or more rows.
If the asynchronous nature of the openFile().build().get() sequence can be taken advantage of, by performing other work before or in parallel to the get() call: do it.
Getting S3 Select code to work is hard, though those knowledgeable in SQL will find it easier.
Problems can be split into:
The exceptions here are all based on the experience during writing tests; more may surface with broader use.
All failures other than network errors on request initialization are considered unrecoverable and will not be reattempted.
As parse-time errors always state the line and column of an error, you can simplify debugging by breaking a SQL statement across lines, e.g.
String sql = "SELECT\n" + "s.entityId \n" + "FROM " + "S3OBJECT s WHERE\n" + "s.\"cloudCover\" = '100.0'\n" + " LIMIT 100";
Now if the error is declared as “line 4”, it will be on the select conditions; the column offset will begin from the first character on that row.
The SQL Statements issued are only included in exceptions if fs.s3a.select.errors.include.sql is explicitly set to true. This can be done in an application during development, or in a openFile() option parameter. This should only be done during development, to reduce the risk of logging security or privacy information.
S3 Select returns paged results; the source file is not filtered in one go in the initial request.
This means that errors related to the content of the data (type casting, etc) may only surface partway through the read. The errors reported in such a case may be different than those raised on reading the first page of data, where it will happen earlier on in the read process.
This surfaces when trying to read in data from a .gz source file through an MR or other analytics query, and the gzip codec has tried to parse it.
java.io.IOException: not a gzip file at org.apache.hadoop.io.compress.zlib.BuiltInGzipDecompressor.processBasicHeader(BuiltInGzipDecompressor.java:496) at org.apache.hadoop.io.compress.zlib.BuiltInGzipDecompressor.executeHeaderState(BuiltInGzipDecompressor.java:257) at org.apache.hadoop.io.compress.zlib.BuiltInGzipDecompressor.decompress(BuiltInGzipDecompressor.java:186) at org.apache.hadoop.io.compress.DecompressorStream.decompress(DecompressorStream.java:111) at org.apache.hadoop.io.compress.DecompressorStream.read(DecompressorStream.java:105) at java.io.InputStream.read(InputStream.java:101) at org.apache.hadoop.util.LineReader.fillBuffer(LineReader.java:182) at org.apache.hadoop.util.LineReader.readCustomLine(LineReader.java:306) at org.apache.hadoop.util.LineReader.readLine(LineReader.java:174) at org.apache.hadoop.mapreduce.lib.input.LineRecordReader.skipUtfByteOrderMark(LineRecordReader.java:158) at org.apache.hadoop.mapreduce.lib.input.LineRecordReader.nextKeyValue(LineRecordReader.java:198)
The underlying problem is that the gzip decompressor is automatically enabled when the the source file ends with the “.gz” extension. Because S3 Select returns decompressed data, the codec fails.
The workaround here is to declare that the job should add the “Passthrough Codec” to its list of known decompressors, and that this codec should declare the file format it supports to be “.gz”.
io.compression.codecs = org.apache.hadoop.io.compress.PassthroughCodec io.compress.passthrough.extension = .gz
Your SQL is wrong and the element at fault is considered an unknown column name.
org.apache.hadoop.fs.s3a.AWSBadRequestException: Select: SELECT * FROM S3OBJECT WHERE odd = true on test/testSelectOddLines.csv: com.amazonaws.services.s3.model.AmazonS3Exception: The column index at line 1, column 30 is invalid. Please check the service documentation and try again. (Service: Amazon S3; Status Code: 400; Error Code: InvalidColumnIndex;
Here it’s the first line of the query, column 30. Paste the query into an editor and position yourself on the line and column at fault.
SELECT * FROM S3OBJECT WHERE odd = true ^ HERE
Another example:
org.apache.hadoop.fs.s3a.AWSBadRequestException: Select: SELECT * FROM S3OBJECT s WHERE s._1 = "true" on test/testSelectOddLines.csv: com.amazonaws.services.s3.model.AmazonS3Exception: The column index at line 1, column 39 is invalid. Please check the service documentation and try again. (Service: Amazon S3; Status Code: 400; Error Code: InvalidColumnIndex;
Here it is because strings must be single quoted, not double quoted.
SELECT * FROM S3OBJECT s WHERE s._1 = "true" ^ HERE
S3 select uses double quotes to wrap column names, interprets the string as column “true”, and fails with a non-intuitive message.
Tip: look for the element at fault and treat the InvalidColumnIndex message as a parse-time message, rather than the definitive root cause of the problem.
Your SQL is wrong.
org.apache.hadoop.fs.s3a.AWSBadRequestException: Select: SELECT * FROM S3OBJECT s WHERE s.'odd' is "true" on test/testSelectOddLines.csv : com.amazonaws.services.s3.model.AmazonS3Exception: Invalid Path component, expecting either an IDENTIFIER or STAR, got: LITERAL,at line 1, column 34. (Service: Amazon S3; Status Code: 400; Error Code: ParseInvalidPathComponent;
SELECT * FROM S3OBJECT s WHERE s.'odd' is "true" on test/testSelectOddLines.csv ^ HERE
Your SQL is still wrong.
org.apache.hadoop.fs.s3a.AWSBadRequestException: Select: SELECT * FROM S3OBJECT s WHERE s.odd = "true" on test/testSelectOddLines.csv: com.amazonaws.services.s3.model.AmazonS3Exception : Expected type name, found QUOTED_IDENTIFIER:'true' at line 1, column 41. (Service: Amazon S3; Status Code: 400; Error Code: ParseExpectedTypeName;
Your SQL is broken.
org.apache.hadoop.fs.s3a.AWSBadRequestException: Select: SELECT * FROM S3OBJECT s WHERE s.5 = `true` on test/testSelectOddLines.csv: com.amazonaws.services.s3.model.AmazonS3Exception: Unexpected token found LITERAL:5d-1 at line 1, column 33. (Service: Amazon S3; Status Code: 400; Error Code: ParseUnexpectedToken;
Your SQL is broken.
com.amazonaws.services.s3.model.AmazonS3Exception: Unexpected operator OPERATOR:'%' at line 1, column 45. (Service: Amazon S3; Status Code: 400; Error Code: ParseUnexpectedOperator; Request ID: E87F30C57436B459; S3 Extended Request ID: UBFOIgkQxBBL+bcBFPaZaPBsjdnd8NRz3NFWAgcctqm3n6f7ib9FMOpR+Eu1Cy6cNMYHCpJbYEY =:ParseUnexpectedOperator: Unexpected operator OPERATOR:'%' at line 1, column 45. at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357) at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895)
org.apache.hadoop.fs.s3a.AWSBadRequestException: Select: SELECT * FROM S3OBJECT s WHERE s."odd" = `true` on test/testSelectOddLines.csv: com.amazonaws.services.s3.model.AmazonS3Exception: Some headers in the query are missing from the file. Please check the file and try again. (Service: Amazon S3; Status Code: 400; Error Code: MissingHeaders;
This can happen if you are trying to use double quotes for constants in the SQL expression.
SELECT * FROM S3OBJECT s WHERE s."odd" = "true" on test/testSelectOddLines.csv: ^ HERE
Double quotes (") may only be used when naming columns; for constants single quotes are required.
org.apache.hadoop.fs.s3a.AWSS3IOException: Select on test/testSelectWholeFile: com.amazonaws.services.s3.model.AmazonS3Exception: The specified method is not allowed against this resource. (Service: Amazon S3; Status Code: 405; Error Code: MethodNotAllowed;
You are trying to use S3 Select to read data which for some reason you are not allowed to.
The file couldn’t be parsed. This can happen if you try to read a .gz file and forget to set the compression in the select request.
That can be done through the fs.s3a.select.compression option.
org.apache.hadoop.fs.s3a.AWSBadRequestException: Select: '" SELECT * FROM S3OBJECT s WHERE endstation_name = 'Bayswater Road: Hyde Park' " on s3a://example/dataset.csv.gz: com.amazonaws.services.s3.model.AmazonS3Exception: UTF-8 encoding is required. The text encoding error was found near byte 8,192. (Service: Amazon S3; Status Code: 400; Error Code: InvalidTextEncoding
A SELECT call has been made using a compression which doesn’t match that of the source object, such as it being a plain text file.
org.apache.hadoop.fs.s3a.AWSBadRequestException: Select: '" SELECT * FROM S3OBJECT s WHERE endstation_name = 'Bayswater Road: Hyde Park' " on s3a://example/dataset.csv: com.amazonaws.services.s3.model.AmazonS3Exception: GZIP is not applicable to the queried object. Please correct the request and try again. (Service: Amazon S3; Status Code: 400; Error Code: InvalidCompressionFormat; at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:212) at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111) ... Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: GZIP is not applicable to the queried object. Please correct the request and try again. Service: Amazon S3; Status Code: 400; Error Code: InvalidCompressionFormat; at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse ...
The input stream returned by the select call does not support seeking backwards in the stream.
Similarly, PositionedReadable operations will fail when used to read data any offset other than that of getPos().
org.apache.hadoop.fs.PathIOException: `s3a://landsat-pds/landsat.csv.gz': seek() not supported at org.apache.hadoop.fs.s3a.select.SelectInputStream.unsupported(SelectInputStream.java:254) at org.apache.hadoop.fs.s3a.select.SelectInputStream.seek(SelectInputStream.java:243) at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:66)
There is no fix for this. You can move forward in a file using skip(offset); bear in mind that the return value indicates what offset was skipped -it may be less than expected.
The filesystem is not an S3A filesystem, and the s3a select option is not recognized.
java.lang.IllegalArgumentException: Unknown mandatory key "fs.s3a.select.sql" at com.google.common.base.Preconditions.checkArgument(Preconditions.java:88) at org.apache.hadoop.fs.AbstractFSBuilder.lambda$rejectUnknownMandatoryKeys$0(AbstractFSBuilder.java:331) at java.lang.Iterable.forEach(Iterable.java:75) at java.util.Collections$UnmodifiableCollection.forEach(Collections.java:1080) at org.apache.hadoop.fs.AbstractFSBuilder.rejectUnknownMandatoryKeys(AbstractFSBuilder.java:330) at org.apache.hadoop.fs.filesystem.openFileWithOptions(FileSystem.java:3541) at org.apache.hadoop.fs.FileSystem$FSDataInputStreamBuilder.build(FileSystem.java:4442)
The file options to tune an S3 select call are only valid when a SQL expression is set in the fs.s3a.select.sql option. If not, any such option added as a must() value will fail.
java.lang.IllegalArgumentException: Unknown mandatory key for s3a://example/test/testSelectOptionsOnlyOnSelectCalls.csv in non-select file I/O "fs.s3a.select.input.csv.header" at com.google.common.base.Preconditions.checkArgument(Preconditions.java:115) at org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.lambda$rejectUnknownMandatoryKeys$0(AbstractFSBuilderImpl.java:352) at java.lang.Iterable.forEach(Iterable.java:75) at java.util.Collections$UnmodifiableCollection.forEach(Collections.java:1080) at org.apache.hadoop.fs.impl.AbstractFSBuilderImpl.rejectUnknownMandatoryKeys(AbstractFSBuilderImpl.java:351) at org.apache.hadoop.fs.s3a.S3AFileSystem.openFileWithOptions(S3AFileSystem.java:3736) at org.apache.hadoop.fs.FileSystem$FSDataInputStreamBuilder.build(FileSystem.java:4471)
Requiring these options without providing a SQL query is invariably an error. Fix: add the SQL statement, or use opt() calls to set the option.
If the fs.s3a.select.sql option is set, and still a key is rejected, then either the spelling of the key is wrong, it has leading or trailing spaces, or it is an option not supported in that specific release of Hadoop.
Backwards seeks in an S3 Select SelectInputStream are not supported.
org.apache.hadoop.fs.PathIOException: `s3a://landsat-pds/scene_list.gz': seek() backwards from 16387 to 0 not supported at org.apache.hadoop.fs.s3a.select.SelectInputStream.unsupported(SelectInputStream.java:288) at org.apache.hadoop.fs.s3a.select.SelectInputStream.seek(SelectInputStream.java:253) at org.apache.hadoop.fs.FSDataInputStream.seek(FSDataInputStream.java:66)
The SELECT refers to the name of a column which is not recognized
org.apache.hadoop.fs.s3a.AWSBadRequestException: SELECT * FROM S3OBJECT WHERE s."oddf" = 'true' on s3a://example/test/testParseBrokenCSVFile: com.amazonaws.services.s3.model.AmazonS3Exception: Invalid table alias is specified at line 1, column 30. Please check the file and try again. (Service: Amazon S3; Status Code: 400; Error Code: InvalidTableAlias; Invalid table alias is specified at line 1, column 30. Please check the file and try again. (Service: Amazon S3; Status Code: 400; Error Code: InvalidTableAlias; Request ID: 8693B86A52CFB91C; at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:225) at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111) at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:265) ... Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Invalid table alias is specified at line 1, column 30. Please check the file and try again. (Service: Amazon S3; Status Code: 400; Error Code: InvalidTableAlias; Request ID: 8693B86A52CFB91C; at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1640) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1304) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1058) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:743) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:717) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:699) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:667) at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:649)
A string field could not be converted to a timestamp because one or more of its entries were not parseable with the given timestamp.
Example, from a spreadsheet where “timestamp” is normally a well-formatted timestamp field, but in one column it is just “Tuesday”
SELECT CAST(s.date AS TIMESTAMP) FROM S3OBJECT s
org.apache.hadoop.fs.s3a.AWSBadRequestException: Select on s3a://example/test/testParseBrokenCSVFile: com.amazonaws.services.s3.model.AmazonS3Exception: Attempt to convert from one data type to another failed: cast from STRING to TIMESTAMP. (Service: Amazon S3; Status Code: 400; Error Code: CastFailed; Request ID: E2158FE45AF2049A; S3 Extended Request ID: iM40fzGuaPt6mQo0QxDDX+AY1bAgSVD1sKErFq6Y4GDJYHIAnmc00i0EvGGnH+0MFCFhKIivIrQ=), S3 Extended Request ID: iM40fzGuaPt6mQo0QxDDX+AY1bAgSVD1sKErFq6Y4GDJYHIAnmc00i0EvGGnH+0MFCFhKIivIrQ=:CastFailed: Attempt to convert from one data type to another failed: cast from STRING to TIMESTAMP. (Service: Amazon S3; Status Code: 400; Error Code: CastFailed; Request ID: E2158FE45AF2049A; S3 Extended Request ID: iM40fzGuaPt6mQo0QxDDX+AY1bAgSVD1sKErFq6Y4GDJYHIAnmc00i0EvGGnH+0MFCFhKIivIrQ=) at org.apache.hadoop.fs.s3a.S3AUtils.translateException(S3AUtils.java:225) at org.apache.hadoop.fs.s3a.Invoker.once(Invoker.java:111) at org.apache.hadoop.fs.s3a.Invoker.lambda$retry$3(Invoker.java:265) Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Attempt to convert from one data type to another failed: cast from STRING to TIMESTAMP. (Service: Amazon S3; Status Code: 400; Error Code: CastFailed;)
There’s no way to recover from a bad record here; no option to skip invalid rows.
Note: This is an example stack trace without the SQL being printed.