Hi, @Sergey: - I already marked TikaIO @Experimental, so we can make changes. - Yes, the String in KV<String, ParseResult> is the filename. I guess we could alternatively put it into ParseResult - don't have a strong opinion.
@Chris: unorderedness of Metadata would have helped if we extracted each Metadata item into a separate PCollection element, but that's not what we want to do (we want to have an element per document instead). @Timothy: can you tell more about this RecursiveParserWrapper? Is this something that the user can configure by specifying the Parser on TikaIO if they so wish? On Thu, Sep 21, 2017 at 2:23 PM Allison, Timothy B. <talli...@mitre.org> wrote: > Like Sergey, it’ll take me some time to understand your recommendations. > Thank you! > > > > On one small point: > > >return a PCollection<KV<String, TikaIO.ParseResult>>, where ParseResult > is a class with properties { String content, Metadata metadata } > > > > For this option, I’d strongly encourage using the Json output from the > RecursiveParserWrapper that contains metadata and content, and captures > metadata even from embedded documents. > > > > > However, since TikaIO can be applied to very large files, this could > produce very large elements, which is a bad idea > > Large documents are a problem, no doubt about it… > > > > *From:* Eugene Kirpichov [mailto:kirpic...@google.com] > *Sent:* Thursday, September 21, 2017 4:41 PM > *To:* Allison, Timothy B. <talli...@mitre.org>; dev@beam.apache.org > *Cc:* d...@tika.apache.org > *Subject:* Re: TikaIO concerns > > > > Thanks all for the discussion. It seems we have consensus that both > within-document order and association with the original filename are > necessary, but currently absent from TikaIO. > > > > *Association with original file:* > > Sergey - Beam does not *automatically* provide a way to associate an > element with the file it originated from: automatically tracking data > provenance is a known very hard research problem on which many papers have > been written, and obvious solutions are very easy to break. See related > discussion at > https://lists.apache.org/thread.html/32aab699db3901d9f0191ac7dbc0091b31cb8be85eee6349deaee671@%3Cuser.beam.apache.org%3E > . > > > > If you want the elements of your PCollection to contain additional > information, you need the elements themselves to contain this information: > the elements are self-contained and have no metadata associated with them > (beyond the timestamp and windows, universal to the whole Beam model). > > > > *Order within a file:* > > The only way to have any kind of order within a PCollection is to have the > elements of the PCollection contain something ordered, e.g. have a > PCollection<List<Something>>, where each List is for one file [I'm assuming > Tika, at a low level, works on a per-file basis?]. However, since TikaIO > can be applied to very large files, this could produce very large elements, > which is a bad idea. Because of this, I don't think the result of applying > Tika to a single file can be encoded as a PCollection element. > > > > Given both of these, I think that it's not possible to create a > *general-purpose* TikaIO transform that will be better than manual > invocation of Tika as a DoFn on the result of FileIO.readMatches(). > > > > However, looking at the examples at > https://tika.apache.org/1.16/examples.html - almost all of the examples > involve extracting a single String from each document. This use case, with > the assumption that individual documents are small enough, can certainly be > simplified and TikaIO could be a facade for doing just this. > > > > 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. > > > > On Thu, Sep 21, 2017 at 7:38 AM Sergey Beryozkin <sberyoz...@gmail.com> > wrote: > > Hi Tim > On 21/09/17 14:33, Allison, Timothy B. wrote: > > Thank you, Sergey. > > > > My knowledge of Apache Beam is limited -- I saw Davor and > Jean-Baptiste's talk at ApacheCon in Miami, and I was and am totally > impressed, but I haven't had a chance to work with it yet. > > > > From my perspective, if I understand this thread (and I may not!), > getting unordered text from _a given file_ is a non-starter for most > applications. The implementation needs to guarantee order per file, and > the user has to be able to link the "extract" back to a unique identifier > for the document. If the current implementation doesn't do those things, > we need to change it, IMHO. > > > Right now Tika-related reader does not associate a given text fragment > with the file name, so a function looking at some text and trying to > find where it came from won't be able to do so. > > So I asked how to do it in Beam, how to attach some context to the given > piece of data. I hope it can be done and if not - then perhaps some > improvement can be applied. > > Re the unordered text - yes - this is what we currently have with Beam + > TikaIO :-). > > The use-case I referred to earlier in this thread (upload PDFs - save > the possibly unordered text to Lucene with the file name 'attached', let > users search for the files containing some words - phrases, this works > OK given that I can see PDF parser for ex reporting the lines) can be > supported OK with the current TikaIO (provided we find a way to 'attach' > a file name to the flow). > > I see though supporting the total ordering can be a big deal in other > cases. Eugene, can you please explain how it can be done, is it > achievable in principle, without the users having to do some custom > coding ? > > > To the question of -- why is this in Beam at all; why don't we let users > call it if they want it?... > > > > No matter how much we do to Tika, it will behave badly sometimes -- > permanent hangs requiring kill -9 and OOMs to name a few. I imagine folks > using Beam -- folks likely with large batches of unruly/noisy documents -- > are more likely to run into these problems than your average > couple-of-thousand-docs-from-our-own-company user. So, if there are things > we can do in Beam to prevent developers around the world from having to > reinvent the wheel for defenses against these problems, then I'd be > enormously grateful if we could put Tika into Beam. That means: > > > > 1) a process-level timeout (because you can't actually kill a thread in > Java) > > 2) a process-level restart on OOM > > 3) avoid trying to reprocess a badly behaving document > > > > If Beam automatically handles those problems, then I'd say, y, let users > write their own code. If there is so much as a single configuration knob > (and it sounds like Beam is against complex configuration...yay!) to get > that working in Beam, then I'd say, please integrate Tika into Beam. From > a safety perspective, it is critical to keep the extraction process > entirely separate (jvm, vm, m, rack, data center!) from the > transformation+loading steps. IMHO, very few devs realize this because > Tika works well lots of the time...which is why it is critical for us to > make it easy for people to get it right all of the time. > > > > Even in my desktop (gah, y, desktop!) search app, I run Tika in batch > mode first in one jvm, and then I kick off another process to do > transform/loading into Lucene/Solr from the .json files that Tika generates > for each input file. If I were to scale up, I'd want to maintain this > complete separation of steps. > > > > Apologies if I've derailed the conversation or misunderstood this thread. > > > Major thanks for your input :-) > > Cheers, Sergey > > > Cheers, > > > > Tim > > > > -----Original Message----- > > From: Sergey Beryozkin [mailto:sberyoz...@gmail.com] > > Sent: Thursday, September 21, 2017 9:07 AM > > To: dev@beam.apache.org > > Cc: Allison, Timothy B. <talli...@mitre.org> > > Subject: Re: TikaIO concerns > > > > Hi All > > > > Please welcome Tim, one of Apache Tika leads and practitioners. > > > > Tim, thanks for joining in :-). If you have some great Apache Tika > stories to share (preferably involving the cases where it did not really > matter the ordering in which Tika-produced data were dealt with by the > > consumers) then please do so :-). > > > > At the moment, even though Tika ContentHandler will emit the ordered > data, the Beam runtime will have no guarantees that the downstream pipeline > components will see the data coming in the right order. > > > > (FYI, I understand from the earlier comments that the total ordering is > also achievable but would require the extra API support) > > > > Other comments would be welcome too > > > > Thanks, Sergey > > > > On 21/09/17 10:55, Sergey Beryozkin wrote: > >> I noticed that the PDF and ODT parsers actually split by lines, not > >> individual words and nearly 100% sure I saw Tika reporting individual > >> lines when it was parsing the text files. The 'min text length' > >> feature can help with reporting several lines at a time, etc... > >> > >> I'm working with this PDF all the time: > >> https://rwc.iacr.org/2017/Slides/nguyen.quan.pdf > >> > >> try it too if you get a chance. > >> > >> (and I can imagine not all PDFs/etc representing the 'story' but can > >> be for ex a log-like content too) > >> > >> That said, I don't know how a parser for the format N will behave, it > >> depends on the individual parsers. > >> > >> IMHO it's an equal candidate alongside Text-based bounded IOs... > >> > >> I'd like to know though how to make a file name available to the > >> pipeline which is working with the current text fragment ? > >> > >> Going to try and do some measurements and compare the sync vs async > >> parsing modes... > >> > >> Asked the Tika team to support with some more examples... > >> > >> Cheers, Sergey > >> On 20/09/17 22:17, Sergey Beryozkin wrote: > >>> Hi, > >>> > >>> thanks for the explanations, > >>> > >>> On 20/09/17 16:41, Eugene Kirpichov wrote: > >>>> Hi! > >>>> > >>>> TextIO returns an unordered soup of lines contained in all files you > >>>> ask it to read. People usually use TextIO for reading files where 1 > >>>> line corresponds to 1 independent data element, e.g. a log entry, or > >>>> a row of a CSV file - so discarding order is ok. > >>> Just a side note, I'd probably want that be ordered, though I guess > >>> it depends... > >>>> However, there is a number of cases where TextIO is a poor fit: > >>>> - Cases where discarding order is not ok - e.g. if you're doing > >>>> natural language processing and the text files contain actual prose, > >>>> where you need to process a file as a whole. TextIO can't do that. > >>>> - Cases where you need to remember which file each element came > >>>> from, e.g. > >>>> if you're creating a search index for the files: TextIO can't do > >>>> this either. > >>>> > >>>> Both of these issues have been raised in the past against TextIO; > >>>> however it seems that the overwhelming majority of users of TextIO > >>>> use it for logs or CSV files or alike, so solving these issues has > >>>> not been a priority. > >>>> Currently they are solved in a general form via FileIO.read() which > >>>> gives you access to reading a full file yourself - people who want > >>>> more flexibility will be able to use standard Java text-parsing > >>>> utilities on a ReadableFile, without involving TextIO. > >>>> > >>>> Same applies for XmlIO: it is specifically designed for the narrow > >>>> use case where the files contain independent data entries, so > >>>> returning an unordered soup of them, with no association to the > >>>> original file, is the user's intention. XmlIO will not work for > >>>> processing more complex XML files that are not simply a sequence of > >>>> entries with the same tag, and it also does not remember the > >>>> original filename. > >>>> > >>> > >>> OK... > >>> > >>>> However, if my understanding of Tika use cases is correct, it is > >>>> mainly used for extracting content from complex file formats - for > >>>> example, extracting text and images from PDF files or Word > >>>> documents. I believe this is the main difference between it and > >>>> TextIO - people usually use Tika for complex use cases where the > >>>> "unordered soup of stuff" abstraction is not useful. > >>>> > >>>> My suspicion about this is confirmed by the fact that the crux of > >>>> the Tika API is ContentHandler > >>>> http://docs.oracle.com/javase/6/docs/api/org/xml/sax/ContentHandler. > >>>> html?is-external=true > >>>> > >>>> whose > >>>> documentation says "The order of events in this interface is very > >>>> important, and mirrors the order of information in the document > itself." > >>> All that says is that a (Tika) ContentHandler will be a true SAX > >>> ContentHandler... > >>>> > >>>> Let me give a few examples of what I think is possible with the raw > >>>> Tika API, but I think is not currently possible with TikaIO - please > >>>> correct me where I'm wrong, because I'm not particularly familiar > >>>> with Tika and am judging just based on what I read about it. > >>>> - User has 100,000 Word documents and wants to convert each of them > >>>> to text files for future natural language processing. > >>>> - User has 100,000 PDF files with financial statements, each > >>>> containing a bunch of unrelated text and - the main content - a list > >>>> of transactions in PDF tables. User wants to extract each > >>>> transaction as a PCollection element, discarding the unrelated text. > >>>> - User has 100,000 PDF files with scientific papers, and wants to > >>>> extract text from them, somehow parse author and affiliation from > >>>> the text, and compute statistics of topics and terminology usage by > >>>> author name and affiliation. > >>>> - User has 100,000 photos in JPEG made by a set of automatic cameras > >>>> observing a location over time: they want to extract metadata from > >>>> each image using Tika, analyze the images themselves using some > >>>> other library, and detect anomalies in the overall appearance of the > >>>> location over time as seen from multiple cameras. > >>>> I believe all of these cases can not be solved with TikaIO because > >>>> the resulting PCollection<String> contains no information about > >>>> which String comes from which document and about the order in which > >>>> they appear in the document. > >>> These are good use cases, thanks... I thought what you were talking > >>> about the unordered soup of data produced by TikaIO (and its friends > >>> TextIO and alike :-)). > >>> Putting the ordered vs unordered question aside for a sec, why > >>> exactly a Tika Reader can not make the name of the file it's > >>> currently reading from available to the pipeline, as some Beam > pipeline metadata piece ? > >>> Surely it can be possible with Beam ? If not then I would be > surprised... > >>> > >>>> > >>>> I am, honestly, struggling to think of a case where I would want to > >>>> use Tika, but where I *would* be ok with getting an unordered soup > >>>> of strings. > >>>> So some examples would be very helpful. > >>>> > >>> Yes. I'll ask Tika developers to help with some examples, but I'll > >>> give one example where it did not matter to us in what order > >>> Tika-produced data were available to the downstream layer. > >>> > >>> It's a demo the Apache CXF colleague of mine showed at one of Apache > >>> Con NAs, and we had a happy audience: > >>> > >>> https://github.com/apache/cxf/tree/master/distribution/src/main/relea > >>> se/samples/jax_rs/search > >>> > >>> > >>> PDF or ODT files uploaded, Tika parses them, and all of that is put > >>> into Lucene. We associate a file name with the indexed content and > >>> then let users find a list of PDF files which contain a given word or > >>> few words, details are here > >>> https://github.com/apache/cxf/blob/master/distribution/src/main/relea > >>> se/samples/jax_rs/search/src/main/java/demo/jaxrs/search/server/Catal > >>> og.java#L131 > >>> > >>> > >>> I'd say even more involved search engines would not mind supporting a > >>> case like that :-) > >>> > >>> Now there we process one file at a time, and I understand now that > >>> with TikaIO and N files it's all over the place really as far as the > >>> ordering is concerned, which file it's coming from. etc. That's why > >>> TikaReader must be able to associate the file name with a given piece > >>> of text it's making available to the pipeline. > >>> > >>> I'd be happy to support the ParDo way of linking Tika with Beam. > >>> If it makes things simpler then it would be good, I've just no idea > >>> at the moment how to start the pipeline without using a > >>> Source/Reader, but I'll learn :-). Re the sync issue I mentioned > >>> earlier - how can one avoid it with ParDo when implementing a 'min > >>> len chunk' feature, where the ParDo would have to concatenate several > >>> SAX data pieces first before making a single composite piece to the > pipeline ? > >>> > >>> > >>>> Another way to state it: currently, if I wanted to solve all of the > >>>> use cases above, I'd just use FileIO.readMatches() and use the Tika > >>>> API myself on the resulting ReadableFile. How can we make TikaIO > >>>> provide a usability improvement over such usage? > >>>> > >>> > >>> > >>> If you are actually asking, does it really make sense for Beam to > >>> ship Tika related code, given that users can just do it themselves, > >>> I'm not sure. > >>> > >>> IMHO it always works better if users have to provide just few config > >>> options to an integral part of the framework and see things happening. > >>> It will bring more users. > >>> > >>> Whether the current Tika code (refactored or not) stays with Beam or > >>> not - I'll let you and the team decide; believe it or not I was > >>> seriously contemplating at the last moment to make it all part of the > >>> Tika project itself and have a bit more flexibility over there with > >>> tweaking things, but now that it is in the Beam snapshot - I don't > >>> know - it's no my decision... > >>> > >>>> I am confused by your other comment - "Does the ordering matter ? > >>>> Perhaps > >>>> for some cases it does, and for some it does not. May be it makes > >>>> sense to support running TikaIO as both the bounded reader/source > >>>> and ParDo, with getting the common code reused." - because using > >>>> BoundedReader or ParDo is not related to the ordering issue, only to > >>>> the issue of asynchronous reading and complexity of implementation. > >>>> The resulting PCollection will be unordered either way - this needs > >>>> to be solved separately by providing a different API. > >>> Right I see now, so ParDo is not about making Tika reported data > >>> available to the downstream pipeline components ordered, only about > >>> the simpler implementation. > >>> Association with the file should be possible I hope, but I understand > >>> it would be possible to optionally make the data coming out in the > >>> ordered way as well... > >>> > >>> Assuming TikaIO stays, and before trying to re-implement as ParDo, > >>> let me double check: should we still give some thought to the > >>> possible performance benefit of the current approach ? As I said, I > >>> can easily get rid of all that polling code, use a simple Blocking > queue. > >>> > >>> Cheers, Sergey > >>>> > >>>> Thanks. > >>>> > >>>> On Wed, Sep 20, 2017 at 1:51 AM Sergey Beryozkin > >>>> <sberyoz...@gmail.com> > >>>> wrote: > >>>> > >>>>> Hi > >>>>> > >>>>> Glad TikaIO getting some serious attention :-), I believe one thing > >>>>> we both agree upon is that Tika can help Beam in its own unique way. > >>>>> > >>>>> Before trying to reply online, I'd like to state that my main > >>>>> assumption is that TikaIO (as far as the read side is concerned) is > >>>>> no different to Text, XML or similar bounded reader components. > >>>>> > >>>>> I have to admit I don't understand your questions about TikaIO > >>>>> usecases. > >>>>> > >>>>> What are the Text Input or XML input use-cases ? These use cases > >>>>> are TikaInput cases as well, the only difference is Tika can not > >>>>> split the individual file into a sequence of sources/etc, > >>>>> > >>>>> TextIO can read from the plain text files (possibly zipped), XML - > >>>>> optimized around reading from the XML files, and I thought I made > >>>>> it clear (and it is a known fact anyway) Tika was about reading > >>>>> basically from any file format. > >>>>> > >>>>> Where is the difference (apart from what I've already mentioned) ? > >>>>> > >>>>> Sergey > >>>>> > >>>>> > >>>>> > >>>>> On 19/09/17 23:29, Eugene Kirpichov wrote: > >>>>>> Hi, > >>>>>> > >>>>>> Replies inline. > >>>>>> > >>>>>> On Tue, Sep 19, 2017 at 3:41 AM Sergey Beryozkin > >>>>>> <sberyoz...@gmail.com> > >>>>>> wrote: > >>>>>> > >>>>>>> Hi All > >>>>>>> > >>>>>>> This is my first post the the dev list, I work for Talend, I'm a > >>>>>>> Beam novice, Apache Tika fan, and thought it would be really > >>>>>>> great to try and link both projects together, which led me to > >>>>>>> opening [1] where I typed some early thoughts, followed by PR > >>>>>>> [2]. > >>>>>>> > >>>>>>> I noticed yesterday I had the robust :-) (but useful and helpful) > >>>>>>> newer review comments from Eugene pending, so I'd like to > >>>>>>> summarize a bit why I did TikaIO (reader) the way I did, and then > >>>>>>> decide, based on the feedback from the experts, what to do next. > >>>>>>> > >>>>>>> Apache Tika Parsers report the text content in chunks, via > >>>>>>> SaxParser events. It's not possible with Tika to take a file and > >>>>>>> read it bit by bit at the 'initiative' of the Beam Reader, line > >>>>>>> by line, the only way is to handle the SAXParser callbacks which > >>>>>>> report the data chunks. > >>>>>>> Some > >>>>>>> parsers may report the complete lines, some individual words, > >>>>>>> with some being able report the data only after the completely > >>>>>>> parse the document. > >>>>>>> All depends on the data format. > >>>>>>> > >>>>>>> At the moment TikaIO's TikaReader does not use the Beam threads > >>>>>>> to parse the files, Beam threads will only collect the data from > >>>>>>> the internal queue where the internal TikaReader's thread will > >>>>>>> put the data into (note the data chunks are ordered even though > >>>>>>> the tests might suggest otherwise). > >>>>>>> > >>>>>> I agree that your implementation of reader returns records in > >>>>>> order > >>>>>> - but > >>>>>> Beam PCollection's are not ordered. Nothing in Beam cares about > >>>>>> the order in which records are produced by a BoundedReader - the > >>>>>> order produced by your reader is ignored, and when applying any > >>>>>> transforms to the > >>>>> PCollection > >>>>>> produced by TikaIO, it is impossible to recover the order in which > >>>>>> your reader returned the records. > >>>>>> > >>>>>> With that in mind, is PCollection<String>, containing individual > >>>>>> Tika-detected items, still the right API for representing the > >>>>>> result of parsing a large number of documents with Tika? > >>>>>> > >>>>>> > >>>>>>> > >>>>>>> The reason I did it was because I thought > >>>>>>> > >>>>>>> 1) it would make the individual data chunks available faster to > >>>>>>> the pipeline - the parser will continue working via the > >>>>>>> binary/video etc file while the data will already start flowing - > >>>>>>> I agree there should be some tests data available confirming it - > >>>>>>> but I'm positive at the moment this approach might yield some > >>>>>>> performance gains with the large sets. If the file is large, if > >>>>>>> it has the embedded attachments/videos to deal with, then it may > >>>>>>> be more effective not to get the Beam thread deal with it... > >>>>>>> > >>>>>>> As I said on the PR, this description contains unfounded and > >>>>>>> potentially > >>>>>> incorrect assumptions about how Beam runners execute (or may > >>>>>> execute in > >>>>> the > >>>>>> future) a ParDo or a BoundedReader. For example, if I understand > >>>>> correctly, > >>>>>> you might be assuming that: > >>>>>> - Beam runners wait for a full @ProcessElement call of a ParDo to > >>>>> complete > >>>>>> before processing its outputs with downstream transforms > >>>>>> - Beam runners can not run a @ProcessElement call of a ParDo > >>>>> *concurrently* > >>>>>> with downstream processing of its results > >>>>>> - Passing an element from one thread to another using a > >>>>>> BlockingQueue is free in terms of performance All of these are > >>>>>> false at least in some runners, and I'm almost certain that in > >>>>>> reality, performance of this approach is worse than a ParDo in > >>>>> most > >>>>>> production runners. > >>>>>> > >>>>>> There are other disadvantages to this approach: > >>>>>> - Doing the bulk of the processing in a separate thread makes it > >>>>> invisible > >>>>>> to Beam's instrumentation. If a Beam runner provided per-transform > >>>>>> profiling capabilities, or the ability to get the current stack > >>>>>> trace for stuck elements, this approach would make the real > >>>>>> processing invisible to all of these capabilities, and a user > >>>>>> would only see that the bulk of the time is spent waiting for the > >>>>>> next element, but not *why* the next > >>>>> element > >>>>>> is taking long to compute. > >>>>>> - Likewise, offloading all the CPU and IO to a separate thread, > >>>>>> invisible to Beam, will make it harder for runners to do > >>>>>> autoscaling, binpacking > >>>>> and > >>>>>> other resource management magic (how much of this runners actually > >>>>>> do is > >>>>> a > >>>>>> separate issue), because the runner will have no way of knowing > >>>>>> how much CPU/IO this particular transform is actually using - all > >>>>>> the processing happens in a thread about which the runner is > >>>>>> unaware. > >>>>>> - As far as I can tell, the code also hides exceptions that happen > >>>>>> in the Tika thread > >>>>>> - Adding the thread management makes the code much more complex, > >>>>>> easier > >>>>> to > >>>>>> introduce bugs, and harder for others to contribute > >>>>>> > >>>>>> > >>>>>>> 2) As I commented at the end of [2], having an option to > >>>>>>> concatenate the data chunks first before making them available to > >>>>>>> the pipeline is useful, and I guess doing the same in ParDo would > >>>>>>> introduce some synchronization issues (though not exactly sure > >>>>>>> yet) > >>>>>>> > >>>>>> What are these issues? > >>>>>> > >>>>>> > >>>>>>> > >>>>>>> One of valid concerns there is that the reader is polling the > >>>>>>> internal queue so, in theory at least, and perhaps in some rare > >>>>>>> cases too, we may have a case where the max polling time has been > >>>>>>> reached, the parser is still busy, and TikaIO fails to report all > >>>>>>> the file data. I think that it can be solved by either 2a) > >>>>>>> configuring the max polling time to a very large number which > >>>>>>> will never be reached for a practical case, or > >>>>>>> 2b) simply use a blocking queue without the time limits - in the > >>>>>>> worst case, if TikaParser spins and fails to report the end of > >>>>>>> the document, then, Bean can heal itself if the pipeline blocks. > >>>>>>> I propose to follow 2b). > >>>>>>> > >>>>>> I agree that there should be no way to unintentionally configure > >>>>>> the transform in a way that will produce silent data loss. Another > >>>>>> reason for not having these tuning knobs is that it goes against > >>>>>> Beam's "no knobs" > >>>>>> philosophy, and that in most cases users have no way of figuring > >>>>>> out a > >>>>> good > >>>>>> value for tuning knobs except for manual experimentation, which is > >>>>>> extremely brittle and typically gets immediately obsoleted by > >>>>>> running on > >>>>> a > >>>>>> new dataset or updating a version of some of the involved > >>>>>> dependencies > >>>>> etc. > >>>>>> > >>>>>> > >>>>>>> > >>>>>>> > >>>>>>> Please let me know what you think. > >>>>>>> My plan so far is: > >>>>>>> 1) start addressing most of Eugene's comments which would require > >>>>>>> some minor TikaIO updates > >>>>>>> 2) work on removing the TikaSource internal code dealing with > >>>>>>> File patterns which I copied from TextIO at the next stage > >>>>>>> 3) If needed - mark TikaIO Experimental to give Tika and Beam > >>>>>>> users some time to try it with some real complex files and also > >>>>>>> decide if TikaIO can continue implemented as a > >>>>>>> BoundedSource/Reader or not > >>>>>>> > >>>>>>> Eugene, all, will it work if I start with 1) ? > >>>>>>> > >>>>>> Yes, but I think we should start by discussing the anticipated use > >>>>>> cases > >>>>> of > >>>>>> TikaIO and designing an API for it based on those use cases; and > >>>>>> then see what's the best implementation for that particular API > >>>>>> and set of anticipated use cases. > >>>>>> > >>>>>> > >>>>>>> > >>>>>>> Thanks, Sergey > >>>>>>> > >>>>>>> [1] https://issues.apache.org/jira/browse/BEAM-2328 > >>>>>>> [2] https://github.com/apache/beam/pull/3378 > >>>>>>> > >>>>>> > >>>>> > >>>> > >>> > >