It looks like ReadableFile#open does currently decompress the stream, but it seems like we could add a ReadableFile#openRaw(...) or something like that which didn't implicitly decompress. Then libraries such as Tika which want the *actual* file content could use that method. Would that address your concerns?
https://github.com/apache/beam/blob/393e5631054a81ae1fdcd304f81cc68cf53d3422/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java#L131 On Wed, Oct 4, 2017 at 2:42 PM Sergey Beryozkin <sberyoz...@gmail.com> wrote: > Wait, but what about Tika doing checks like Zip bombs, etc ? Tika is > expected to decompress itself, while ReadableFile has the content > decompressed. > > The other point is that Tika reports the names of the zipped files too, > in the content, as you can see from TikaIOTest#readZippedPdfFile. > > Can we assume that if Metadata does not point to the local file then it > can be opened as a URL stream ? The same issue affects TikaConfig, so > I'd rather have a solution which will work for MatchResult.Metadata and > TikaConfig > > Thanks, Sergey > On 04/10/17 22:02, Sergey Beryozkin wrote: > > Good point... > > > > Sergey > > > > On 04/10/17 18:24, Eugene Kirpichov wrote: > >> Can TikaInputStream consume a regular InputStream? If so, you can > >> apply it > >> to Channels.newInputStream(channel). If not, applying it to the filename > >> extracted from Metadata won't work either because it can point to a file > >> that's not on the local disk. > >> > >> On Wed, Oct 4, 2017, 10:08 AM Sergey Beryozkin <sberyoz...@gmail.com> > >> wrote: > >> > >>> I'm starting moving toward > >>> > >>> class TikaIO { > >>> public static ParseAllToString parseAllToString() {..} > >>> class ParseAllToString extends > PTransform<PCollection<ReadableFile>, > >>> PCollection<ParseResult>> { > >>> ...configuration properties... > >>> expand { > >>> return input.apply(ParDo.of(new ParseToStringFn)) > >>> } > >>> class ParseToStringFn extends DoFn<...> {...} > >>> } > >>> } > >>> > >>> as suggested by Eugene > >>> > >>> The initial migration seems to work fine, except that ReadableFile and > >>> in particular, ReadableByteChannel can not be consumed by > >>> TikaInputStream yet (I'll open an enhancement request), besides, it's > >>> better let Tika to unzip if needed given that a lot of effort went in > >>> Tika into detecting zip security issues... > >>> > >>> So I'm typing it as > >>> > >>> class ParseAllToString extends > >>> PTransform<PCollection<MatchResult.Metadata>, PCollection<ParseResult>> > >>> > >>> Cheers, Sergey > >>> > >>> On 02/10/17 12:03, Sergey Beryozkin wrote: > >>>> Thanks for the review, please see the last comment: > >>>> > >>>> https://github.com/apache/beam/pull/3835#issuecomment-333502388 > >>>> > >>>> (sorry for the possible duplication - but I'm not sure that GitHub > will > >>>> propagate it - as I can not see a comment there that I left on > >>>> Saturday). > >>>> > >>>> Cheers, Sergey > >>>> On 29/09/17 10:21, Sergey Beryozkin wrote: > >>>>> Hi > >>>>> On 28/09/17 17:09, Eugene Kirpichov wrote: > >>>>>> Hi! Glad the refactoring is happening, thanks! > >>>>> > >>>>> Thanks for getting me focused on having TikaIO supporting the simpler > >>>>> (and practical) cases first :-) > >>>>>> It was auto-assigned to Reuven as formal owner of the component. I > >>>>>> reassigned it to you. > >>>>> OK, thanks... > >>>>>> > >>>>>> On Thu, Sep 28, 2017 at 7:57 AM Sergey Beryozkin > >>>>>> <sberyoz...@gmail.com > >>>> > >>>>>> wrote: > >>>>>> > >>>>>>> Hi > >>>>>>> > >>>>>>> I started looking at > >>>>>>> https://issues.apache.org/jira/browse/BEAM-2994 > >>>>>>> > >>>>>>> and pushed some initial code to my tikaio branch introducing > >>>>>>> ParseResult > >>>>>>> and updating the tests but keeping the BounderSource/Reader, > >>>>>>> dropping > >>>>>>> the asynchronous parsing code, and few other bits. > >>>>>>> > >>>>>>> Just noticed it is assigned to Reuven - does it mean Reuven is > >>>>>>> looking > >>>>>>> into it too or was it auto-assigned ? > >>>>>>> > >>>>>>> I don't mind, would it make sense for me to do an 'interim' PR on > >>>>>>> what've done so far before completely removing BoundedSource/Reader > >>>>>>> based code ? > >>>>>>> > >>>>>> Yes :) > >>>>>> > >>>>> I did commit yesterday to my branch, and it made its way to the > >>>>> pending PR (which I forgot about) where I only tweaked a couple of > doc > >>>>> typos, so I renamed that PR: > >>>>> > >>>>> https://github.com/apache/beam/pull/3835 > >>>>> > >>>>> (The build failures are apparently due to the build timeouts) > >>>>> > >>>>> As I mentioned, in this PR I updated the existing TikaIO test to work > >>>>> with ParseResult, at the moment a file location as its property. Only > >>>>> a file name can easily be saved, I thought it might be important > where > >>>>> on the network the file is - may be copy it afterwards if needed, > etc. > >>>>> I'd also have no problems with having it typed as a K key, was only > >>>>> trying to make it a bit simpler at the start. > >>>>> > >>>>> I'll deal with the new configurations after a switch. TikaConfig > would > >>>>> most likely still need to be supported but I recall you mentioned the > >>>>> way it's done now will make it work only with the direct runner. I > >>>>> guess I can load it as a URL resource... The other bits like > providing > >>>>> custom content handlers, parsers, input metadata, may be setting the > >>>>> max size of the files, etc, can all be added after a switch. > >>>>> > >>>>> Note I haven't dealt with a number of your comments to the original > >>>>> code which can still be dealt with in the current code - given that > >>>>> most of that code will go with the next PR anyway. > >>>>> > >>>>> Please review or merge if it looks like it is a step in the right > >>>>> direction... > >>>>> > >>>>>> > >>>>>>> > >>>>>>> I have another question anyway, > >>>>>>> > >>>>>>> > >>>>>>>> E.g. TikaIO could: > >>>>>>>> - take as input a PCollection<ReadableFile> > >>>>>>>> - return a PCollection<KV<String, TikaIO.ParseResult>>, where > >>>>>>>> ParseResult > >>>>>>>> is a class with properties { String content, Metadata metadata } > >>>>>>>> - be configured by: a Parser (it implements Serializable so can be > >>>>>>>> specified at pipeline construction time) and a ContentHandler > whose > >>>>>>>> toString() will go into "content". ContentHandler does not > >>>>>>>> implement > >>>>>>>> Serializable, so you can not specify it at construction time - > >>>>>>>> however, > >>>>>>> you > >>>>>>>> can let the user specify either its class (if it's a simple > handler > >>>>>>>> like > >>>>>>> a > >>>>>>>> BodyContentHandler) or specify a lambda for creating the handler > >>>>>>>> (SerializableFunction<Void, ContentHandler>), and potentially > >>>>>>>> you can > >>>>>>> have > >>>>>>>> a simpler facade for Tika.parseAsString() - e.g. call it > >>>>>>>> TikaIO.parseAllAsStrings(). > >>>>>>>> > >>>>>>>> Example usage would look like: > >>>>>>>> > >>>>>>>> PCollection<KV<String, ParseResult>> parseResults = > >>>>>>>> p.apply(FileIO.match().filepattern(...)) > >>>>>>>> .apply(FileIO.readMatches()) > >>>>>>>> .apply(TikaIO.parseAllAsStrings()) > >>>>>>>> > >>>>>>>> or: > >>>>>>>> > >>>>>>>> .apply(TikaIO.parseAll() > >>>>>>>> .withParser(new AutoDetectParser()) > >>>>>>>> .withContentHandler(() -> new BodyContentHandler(new > >>>>>>>> ToXMLContentHandler()))) > >>>>>>>> > >>>>>>>> You could also have shorthands for letting the user avoid using > >>> FileIO > >>>>>>>> directly in simple cases, for example: > >>>>>>>> p.apply(TikaIO.parseAsStrings().from(filepattern)) > >>>>>>>> > >>>>>>>> This would of course be implemented as a ParDo or even > MapElements, > >>>>>>>> and > >>>>>>>> you'll be able to share the code between parseAll and regular > >>>>>>>> parse. > >>>>>>>> > >>>>>>> I'd like to understand how to do > >>>>>>> > >>>>>>> TikaIO.parse().from(filepattern) > >>>>>>> > >>>>>>> Right now I have TikaIO.Read extending > >>>>>>> PTransform<PBegin, PCollection<ParseResult> > >>>>>>> > >>>>>>> and then the boilerplate code which builds Read when I do something > >>>>>>> like > >>>>>>> > >>>>>>> TikaIO.read().from(filepattern). > >>>>>>> > >>>>>>> What is the convention for supporting something like > >>>>>>> TikaIO.parse().from(filepattern) to be implemented as a ParDo, can > I > >>>>>>> see > >>>>>>> some example ? > >>>>>>> > >>>>>> There are a number of IOs that don't use Source - e.g. DatastoreIO > >>>>>> and > >>>>>> JdbcIO. TextIO.readMatches() might be an even better transform to > >>> mimic. > >>>>>> Note that in TikaIO you probably won't need a fusion break after the > >>>>>> ParDo > >>>>>> since there's 1 result per input file. > >>>>>> > >>>>> > >>>>> OK, I'll have a look > >>>>> > >>>>> Cheers, Sergey > >>>>> > >>>>>> > >>>>>>> > >>>>>>> Many thanks, Sergey > >>>>>>> > >>>>>> > >>> > >> >