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
>

Reply via email to