Re: "Dynamic" read / expand of PCollection element by IOs

2019-03-15 Thread Jozef Vilcek
So `PTransform` must be written in a way to involve underlying `Source[]` based on incoming element and PBegin is not it. Cool, make sense to me. I will give it a try. Thanks On Thu, Mar 14, 2019 at 5:55 PM Eugene Kirpichov wrote: > I would phrase it more optimistically :) > While there is no wa

Re: "Dynamic" read / expand of PCollection element by IOs

2019-03-14 Thread Eugene Kirpichov
I would phrase it more optimistically :) While there is no way to generically apply a PTransform elementwise, Beam does have a pattern / best practice for developing IO connectors that can be applied elementwise - it's called "readAll" and some IOs provide this, e.g. TextIO.readAll(), JdbcIO.readAl

Re: "Dynamic" read / expand of PCollection element by IOs

2019-03-14 Thread Lukasz Cwik
1) Your looking for SplittableDoFn[1]. It is still in development and a conversion of all the current IO connectors that exist today to be able to consume a PCollection of resources is yet to come. There is some limited usecases that exist already like FileIO.match[2] and if these fit your usecase

"Dynamic" read / expand of PCollection element by IOs

2019-03-14 Thread Jozef Vilcek
Hello, I wanted to write a Beam code which expands incoming `PCollection<>`, element wise, by use of existing IO components. Example could be to have a `PCollection` which will hold arbitrary paths to data and I want to load them via `HadoopFormatIO.Read` which is of `PTransform>`. Problem is, I