In terms of performance, it would likely be minimal overhead if (as is
likely) the step consuming the filename gets fused with the read. There's
still overhead constructing this composite, object, etc. but that's (again
likely) smaller than the cost of doing the read itself.

On Sun, Feb 10, 2019 at 7:03 AM Reuven Lax <re...@google.com> wrote:

> I think we could definitely add an option to FileIO to add the filename to
> every record. It would come at a (performance) cost - often the filename is
> much larger than the actual record..
>
> On Thu, Feb 7, 2019 at 6:29 AM Kenneth Knowles <k...@apache.org> wrote:
>
>> This comes up a lot, wanting file names alongside the data that came from
>> the file. It is a historical quirk that none of our connectors used to have
>> the file names. What is the change needed for FileIO + parse Avro to be
>> really easy to use?
>>
>> Kenn
>>
>> On Thu, Feb 7, 2019 at 6:18 AM Jeff Klukas <jklu...@mozilla.com> wrote:
>>
>>> I haven't needed to do this with Beam before, but I've definitely had
>>> similar needs in the past. Spark, for example, provides an input_file_name
>>> function that can be applied to a dataframe to add the input file as an
>>> additional column. It's not clear to me how that's implemented, though.
>>>
>>> Perhaps others have suggestions, but I'm not aware of a way to do this
>>> conveniently in Beam today. To my knowledge, today you would have to use
>>> FileIO.match() and FileIO.readMatches() to get a collection of
>>> ReadableFile. You'd then have to FlatMapElements to pull out the metadata
>>> and the bytes of the file, and you'd be responsible for parsing those bytes
>>> into avro records. You'd  be able to output something like a KV<String, T>
>>> that groups the file name together with the parsed avro record.
>>>
>>> Seems like something worth providing better support for in Beam itself
>>> if this indeed doesn't already exist.
>>>
>>> On Thu, Feb 7, 2019 at 7:29 AM Chaim Turkel <ch...@behalf.com> wrote:
>>>
>>>> Hi,
>>>>   I am working on a pipeline that listens to a topic on pubsub to get
>>>> files that have changes in the storage. Then i read avro files, and
>>>> would like to write them to bigquery based on the file name (to
>>>> different tables).
>>>>   My problem is that the transformer that reads the avro does not give
>>>> me back the files name (like a tuple or something like that). I seem
>>>> to have this pattern come back a lot.
>>>> Can you think of any solutions?
>>>>
>>>> Chaim
>>>>
>>>> --
>>>>
>>>>
>>>> Loans are funded by
>>>> FinWise Bank, a Utah-chartered bank located in Sandy,
>>>> Utah, member FDIC, Equal
>>>> Opportunity Lender. Merchant Cash Advances are
>>>> made by Behalf. For more
>>>> information on ECOA, click here
>>>> <https://www.behalf.com/legal/ecoa/>. For important information about
>>>> opening a new
>>>> account, review Patriot Act procedures here
>>>> <https://www.behalf.com/legal/patriot/>.
>>>> Visit Legal
>>>> <https://www.behalf.com/legal/> to
>>>> review our comprehensive program terms,
>>>> conditions, and disclosures.
>>>>
>>>

Reply via email to