The PR has been submitted. I think now it'd be nice to add test coverage to all IOs using this utility. It should be pretty trivial, I wonder how many bugs we'll find.
On Mon, Aug 28, 2017 at 12:06 PM Eugene Kirpichov <kirpic...@google.com> wrote: > I sent a PR for review with something that I think is a still better > option: > https://github.com/apache/beam/pull/3753 +Ben Chambers > <bchamb...@google.com> > > Example usage: > p.apply("Read", AvroIO.read(GenericClass.class) > .from(*p.newProvider*(outputFile.getAbsolutePath())))) > > TestPipeline.newProvider() returns a ValueProvider that is inaccessible at > construction time, but will have the given value at runtime. > > The advantage of this approach is that it makes it very easy to have > testing on inaccessible ValueProvider's be the default, with no boilerplate > required to write such tests: this is even fewer code than writing > StaticValueProvider.of(). > > On Wed, Jul 19, 2017 at 12:46 PM Eugene Kirpichov <kirpic...@google.com> > wrote: > >> +1, 3 is a good option. >> >> On Wed, Jul 19, 2017, 12:40 PM Kenneth Knowles <k...@google.com.invalid> >> wrote: >> >>> I think (3) sounds good for BEAM-2644. I think keep them both open since >>> one is to develop the capability and the other is to use it. >>> >>> On Wed, Jul 19, 2017 at 12:32 PM, Ben Chambers >>> <bchamb...@google.com.invalid >>> > wrote: >>> >>> > I also reported something similar to this as >>> > https://issues.apache.org/jira/browse/BEAM-2577. That issue was >>> reported >>> > because we don't have any tests that use a runner and attempt to pass >>> > ValueProviders in. This means that we've found bugs such as >>> > NestedValueProviders used with non-serializable anonymous functions. >>> > >>> > One solution seems be to use that pattern: >>> > >>> > 1. Create an extension of PipelineOptions with some ValueProviders >>> > 2. Use that in your test pipeline >>> > 3. allow TestPipeline.run() to take additional arguments to provide at >>> > template execution time to populate those value providers >>> > >>> > On Wed, Jul 19, 2017 at 12:03 PM Eugene Kirpichov >>> > <kirpic...@google.com.invalid> wrote: >>> > >>> > > Hi, >>> > > >>> > > Just filed JIRA https://issues.apache.org/jira/browse/BEAM-2644 >>> > > >>> > > Many transforms that take ValueProvider's have different codepaths >>> for >>> > when >>> > > the provider is accessible or not. However, as far as I can tell, >>> there >>> > is >>> > > no good way to construct a pipeline with PipelineOptions containing >>> an >>> > > inaccessible ValueProvider, and then test how it would run as a >>> template >>> > > invocation with an actual value supplied. >>> > > >>> > > The only way I could come up with is mimicking >>> > > >>> > > https://github.com/apache/beam/blob/master/sdks/java/ >>> > >>> core/src/test/java/org/apache/beam/sdk/options/ValueProviderTest.java#L202 >>> > > , which is very ugly. >>> > > >>> > > Am I missing something? Is there already a good way to do this? >>> > > >>> > >>> >>