hadoop-common-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Steve Loughran (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (HADOOP-15229) Add FileSystem builder-based openFile() API to match createFile()
Date Mon, 17 Dec 2018 19:25:00 GMT

     [ https://issues.apache.org/jira/browse/HADOOP-15229?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Steve Loughran updated HADOOP-15229:
------------------------------------
    Status: Patch Available  (was: Open)

HADOOP-15229 patch 011

* switch to simpler fs.s3a. optiiions everywhere, docs updated to recommend this (and mandate
either SCHEMA. or fs.SCHEMA. 
* slight cleanup of SelectConstants.
* s3a input streams also support fs.s3a.encryption.key for both select and open options (untested,
yet)
* input format (fs.s3a.select.input.format) and output format (fs.s3a.select.output.format)
can be specifed; currently only CSV is allowed. (validated in a test).
The default is still CSV though...it may make sense to mandate the spec
* setting tests up fo test the S3Guard CLI Tool, but nothing implemented there yet.
* new test of Line record reader against Landsat .gz. file

Tested: s3 ireland. 

Fun feature: the LineRecordReader it doesn't work, because the codecs automatically map .gz
filename to GZipDecompressor, which breaks on the CSV-formatted text coming in.

Next actions

# people need to review this. I am trying to define a new API for filesystem interaction,
the first async one: early feedback matters. 
# plan to add JSON and stop there, will force a rework of the currrent s3 binding code.
# and a couple of tests for the S3Guard tool, which will need to work with this too (add a
--inputformat option, etc)

I don't know what to do about the landsat gz failure. the logic for binding decompressors
is more than just a simple "edit the config" as the service loader mech is the main way compression
codecs are found. You'd need to implement a new dummy decompressor which registered support
for .gz files but really just passed the text through. That is a fairly major piece of work
which I don't intend to do. I think I'll give up at that point and say "you'll need a better
record reader for this world"

{code}
[ERROR] Tests run: 4, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 10.419 s <<<
FAILURE! - in org.apache.hadoop.fs.s3a.select.ITestS3SelectLandsat
[ERROR] testReadLandsatRecords(org.apache.hadoop.fs.s3a.select.ITestS3SelectLandsat)  Time
elapsed: 1.436 s  <<< ERROR!
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:179)
	at org.apache.hadoop.util.LineReader.readCustomLine(LineReader.java:303)
	at org.apache.hadoop.util.LineReader.readLine(LineReader.java:171)
	at org.apache.hadoop.mapreduce.lib.input.LineRecordReader.skipUtfByteOrderMark(LineRecordReader.java:158)
	at org.apache.hadoop.mapreduce.lib.input.LineRecordReader.nextKeyValue(LineRecordReader.java:198)
	at org.apache.hadoop.fs.s3a.select.AbstractS3SelectTest.readRecords(AbstractS3SelectTest.java:391)
	at org.apache.hadoop.fs.s3a.select.ITestS3SelectLandsat.testReadLandsatRecords(ITestS3SelectLandsat.java:193)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
	at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)
	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
	at org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.lang.Thread.run(Thread.java:745)

[INFO] Running org.apache.hadoop.fs.s3a.select.ITestS3Select
{code}

> Add FileSystem builder-based openFile() API to match createFile()
> -----------------------------------------------------------------
>
>                 Key: HADOOP-15229
>                 URL: https://issues.apache.org/jira/browse/HADOOP-15229
>             Project: Hadoop Common
>          Issue Type: New Feature
>          Components: fs, fs/azure, fs/s3
>    Affects Versions: 3.0.0
>            Reporter: Steve Loughran
>            Assignee: Steve Loughran
>            Priority: Major
>         Attachments: HADOOP-15229-001.patch, HADOOP-15229-002.patch, HADOOP-15229-003.patch,
HADOOP-15229-004.patch, HADOOP-15229-004.patch, HADOOP-15229-005.patch, HADOOP-15229-006.patch,
HADOOP-15229-007.patch, HADOOP-15229-009.patch, HADOOP-15229-010.patch, HADOOP-15229-011.patch
>
>
> Replicate HDFS-1170 and HADOOP-14365 with an API to open files.
> A key requirement of this is not HDFS, it's to put in the fadvise policy for working
with object stores, where getting the decision to do a full GET and TCP abort on seek vs smaller
GETs is fundamentally different: the wrong option can cost you minutes. S3A and Azure both
have adaptive policies now (first backward seek), but they still don't do it that well.
> Columnar formats (ORC, Parquet) should be able to say "fs.input.fadvise" "random" as
an option when they open files; I can imagine other options too.
> The Builder model of [~eddyxu] is the one to mimic, method for method. Ideally with as
much code reuse as possible



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


Mime
View raw message