Thank you for details! I’ll try to share some of my thoughts on this.

Well, maybe I don’t understand something but what is a problem to have a 
"flattened” PCollection as an output of your ParDo that reads input zip 
file(s)? For example, if input element in a file is an email, then your DoFn 
can create several outputs depending on the structure of this email. Of course, 
the order of elements in output PCollection won’t be guaranteed but all needed 
later information can be saved in the structure of output record (like POJO or 
AvroRecord).

Also, if I understand correctly, then on every precessing step you need to 
reject some records depending on SHA from already known rejection list. So, if 
it’s possible to calculate this SHA on the “Read” step for every record, then 
you can use either SideInput or GroupByKey transform (where key is SHA) to 
filter the records.

Please, let me know if I missed something.

> On 19 Jan 2021, at 18:43, Tucker Barbour <[email protected]> wrote:
> 
> My initial thought is the latter -- the output PCollection would be a 
> PCollection<Record> where Record can be either an email or attachment. A 
> Record would still need to have an attribute referencing its "parent". For 
> example an email Record would have a unique identifier, e.g. ID, and any 
> attachment Record would have a reference to it's parent email, e.g. parentID. 
> However, outputting pairs might also work and may be a better option 
> considering the need to maintain the relationship between a parent and child. 
> We're basically building a tree. An additional wrinkle is that attachments 
> may themselves have embedded items which would also need to be represented in 
> the output PCollection as Records. For example, an email with an attachment 
> which itself is a zip of Word documents. The structure of this file hierarchy 
> is not known ahead of time.
> 
> The input is expected to be a PCollection of one or more (though usually in 
> the order of 10s not anything like millions) zip files or other archive file 
> formats. The output is expected to be a PCollection whose elements are nodes 
> in the file hierarchy. If a zip file where to have the following structure
> 
> - Top Level Zip File
> `-> Email 001
>   `-> Attachment A
>     `-> Embedded Document A01
>     `-> Embedded Document A02
>  `-> Attachment B
>    `-> Embedded Document B01
> `-> Email 002
>   `-> Attachment C
>     `-> Embedded Document C01
> 
> We'd expect an output PCollection whose elements are:
> 
> Email001, Attachment A, Embedded Document A01, Embedded Document A02, 
> Attachment B, Embedded Document B01, Email 002, Attachment C, Embedded 
> Document C01. 
> 
> We'd then perform further PTransforms on this "flattened" PCollection. 
> 
> 
> 
> 
> 
> On Tue, Jan 19, 2021 at 5:14 PM Alexey Romanenko <[email protected] 
> <mailto:[email protected]>> wrote:
> What is exactly an output PCollection in your example? Is it just a 
> PCollection of pairs (email and attachment) or it’s like PCollection<Record>, 
> where Record can be either email or attachment? Or it is something else?
> 
> Could you add a simple example with expected input/output of your pipeline?
> 
> > On 18 Jan 2021, at 12:26, Tucker Barbour <[email protected] 
> > <mailto:[email protected]>> wrote:
> > 
> > I have a use-case where I'm extracting embedded items from archive file 
> > formats which themselves have embedded items. For example a zip file with 
> > emails with attachments. The goal in this example would be to create a 
> > PCollection where each email is an element as well as each attachment being 
> > an element. (No need to create a tree structure here.) There are certain 
> > criteria which would prevent continuing embedded item extraction, such as 
> > an item SHA being present in a "rejection" list. The pipeline will perform 
> > a series of transformations on the items and then continue to extract 
> > embedded items. This type of problem lends itself to be solved with an 
> > iterative algorithm. My understanding is that BEAM does not support 
> > iterative algorithms to the same extent Spark does. In BEAM I would have to 
> > persist the results of each iteration and instantiate a new pipeline for 
> > each iteration. This _works_ though isn't ideal. The "rejection" list is a 
> > PCollection of a few million elements. Re-reading this "rejection" list on 
> > each iteration isn't ideal.
> > 
> > Is there a way to write such an iterative algorithm in BEAM without having 
> > to create a new pipeline on each iteration? Further, would something like 
> > SplitableDoFn be a potential solution? Looking through the user's guide and 
> > some existing implementations of SplitableDoFns I'm thinking not but I'm 
> > still trying to understand SplitableDoFns.
> 

Reply via email to