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. 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. 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." 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. 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. 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? 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. 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 > >> > > >