Re: Null checking in Beam

2021-03-02 Thread Kyle Weaver
Can you try adding the generated classes to generatedClassPatterns in the
JavaNatureConfiguration?

https://github.com/apache/beam/blob/03b883b415d27244ddabb17a0fb5bab147b86f89/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy#L92


On Tue, Mar 2, 2021 at 12:05 AM Reuven Lax  wrote:

> I'm running into a problem with this check. I added a protocol-buffer file
> to a module (google-cloud-platform) that previously did have any protobuf
> files in it. The generated files contain lines that violate this null
> checker, so they won't compile. I can't annotate the files, because they
> are codegen files. I tried adding the package to spotbugs-filter.xml, but
> that didn't help.
>
> Any suggestions?
>
> Reuven
>
> On Fri, Jan 22, 2021 at 10:38 AM Brian Hulette 
> wrote:
>
>>
>>
>> On Fri, Jan 22, 2021 at 1:18 AM Jan Lukavský  wrote:
>>
>>> Hi,
>>>
>>> I'll give my two cents here.
>>>
>>> I'm not 100% sure that the 1-5% of bugs are as severe as other types of
>>> bugs. Yes, throwing NPEs at user is not very polite. On the other hand,
>>> many of these actually boil down to user errors. Then we might ask what a
>>> correct solution would be. If we manage to figure out what the actual
>>> problem is and tell user what specifically is missing or going wrong, that
>>> would be just awesome. On the other hand, if a tool used for avoiding
>>> "unexpected" NPEs forces us to code
>>>
>>>Object value = Objects.requireNonNull(myNullableObject); // or
>>> similar using Preconditions
>>>value.doStuff();
>>>
>>> instead of just
>>>
>>>   myNullableObject.doStuff()
>>>
>>> what we actually did, is a) made a framework happy, and b) changed a
>>> line at which NPE is thrown by 1 (and yes, internally prevented JVM from
>>> thrown SIGSEGV at itself, but that is deeply technical thing). Nothing
>>> changed semantically, from user perspective.
>>>
>> I'd argue there's value in asking Beam developers to make that change. It
>> makes us acknowledge that there's a possibility myNullableObject is null.
>> If myNullableObject being null is something relevant to the user we would
>> likely want to wrap it in some other exception or provide a more useful
>> message than just NPE(!!).
>>
>>>
>>> Now, given that the framework significantly rises compile time (due to
>>> all the checks), causes many "bugs" being reported by static code analysis
>>> tools (because the framework adds @Nonnull default annotations everywhere,
>>> even when Beam's code actually counts with nullability of a field), and
>>> given how much we currently suppress these checks ($ git grep BEAM-10402 |
>>> wc -l -> 1981), I'd say this deserves a deeper discussion.
>>>
>> The reason there are so many suppressions is that fixing all the errors
>> is a monumental task. Rather than addressing them all, Kenn
>> programmatically added suppressions for classes that failed the checks (
>> https://github.com/apache/beam/pull/13261). This allowed us to start
>> running the checker on the code that passes it while the errors are fixed.
>>
>>>  Jan
>>>
>>>
>>> On 1/20/21 10:48 PM, Kenneth Knowles wrote:
>>>
>>> Yes, completely sound nullability checking has been added to the project
>>> via checkerframework, based on a large number of NPE bugs (1-5% depending
>>> on how you search, but many other bugs likely attributable to
>>> nullness-based design errors) which are extra embarrassing because NPEs
>>> have were essentially solved, even in practice for Java, well before Beam
>>> existed.
>>>
>>> Checker framework is a pluggable type system analysis with some amount
>>> of control flow sensitivity. Every value has a type that is either nullable
>>> or not, and certain control structures (like checking for null) can alter
>>> the type inside a scope. The best way to think about it is to consider
>>> every value in the program as either nullable or not, much like you think
>>> of every value as either a string or not, and to view method calls as
>>> inherently stateful/nondetermistic. This can be challenging in esoteric
>>> cases, but usually makes the overall code health better anyhow.
>>>
>>> Your example illustrates how problematic the design of the Java language
>>> is: the analysis cannot assume that `getDescription` is a pure function,
>>> and neither should you. Even if it is aware of boolean-short-circuit it
>>> would not be sound to accept this code. There is an annotation for this in
>>> the cases where it is true (like proto-generate getters):
>>> https://checkerframework.org/api/org/checkerframework/dataflow/qual/Pure.html
>>>
>>> The refactor for cases like this is trivial so there isn't a lot of
>>> value to thinking too hard about it.
>>>
>>> if (statusCode.equals(Code.INVALID_ARGUMENT)
>>>   @Nullable String desc = statusCode.toStatus().getDescription();
>>>   if (desc != null && desc.contains("finalized")) {
>>> return false;
>>>   }
>>> }
>>>
>>> To a casual eye, this may look like a noop change. To the analysis it
>>> makes all the 

Re: Random outputs for ARRAY_CONCAT_AGG fn zetasql

2021-03-02 Thread Kyle Weaver
As you can see from existing tests, Beam doesn't materialize the output
array directly. Instead you must use the PAssert API. I agree with Tyson's
suggestion to use `satisfies`, which lets you do arbitrary assertions on
the output data.

On Tue, Mar 2, 2021 at 3:57 AM Sonam Ramchand <
sonam.ramch...@venturedive.com> wrote:

> Is there any way I can access the output array resulting from the sql
> query? Then maybe I can sort and compare both *output array* and *expected
> output array *for the test to pass.
>
> On Tue, Mar 2, 2021 at 12:24 AM Kenneth Knowles  wrote:
>
>> Yea, the reason is that SQL relations are not ordered. So any ordering of
>> [1, 2, 3, 4] and [5, 6] and [7, 8, 9] is possible and correct.
>>
>> Kenn
>>
>> On Mon, Mar 1, 2021 at 11:01 AM Tyson Hamilton 
>> wrote:
>>
>>> I didn't find anything like that after a brief look. What you could do
>>> instead is something like:
>>>
>>> PAssert.thatSingleton(stream).satisfies( row -> assertThat("array_field
>>> containsInAnyOrder", row.getArray("array_field"),
>>> containsInAnyOrder(Arrays.asList(...)));
>>>
>>> using junit/hamcrest matchers. I didn't verify this works myself but it
>>> should give you an idea for some next steps.
>>>
>>>
>>> On Mon, Mar 1, 2021 at 12:37 AM Sonam Ramchand <
>>> sonam.ramch...@venturedive.com> wrote:
>>>
 Hi Devs,
 I have implemented the ARRAY_CONCAT_AGG function for zetasql dialect. I
 am trying to validate the test as:

 @Test
 public void testArrayConcatAggZetasql() {
   String sql =
   "SELECT ARRAY_CONCAT_AGG(x) AS array_concat_agg FROM (SELECT [1, 2, 
 3, 4] AS x UNION ALL SELECT [5, 6] UNION ALL SELECT [7, 8, 9])";

   ZetaSQLQueryPlanner zetaSQLQueryPlanner = new 
 ZetaSQLQueryPlanner(config);
   BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
   PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, 
 beamRelNode);

   Schema schema = Schema.builder().addArrayField("array_field", 
 FieldType.INT64).build();
   PAssert.that(stream)
   .containsInAnyOrder(
   Row.withSchema(schema).addArray(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 
 9L).build());
   
 pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
 }

 Expected Output is: 1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L.
 But I am getting randomly different outputs:
 1. 1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L
 2. 5L, 6L, 7L, 8L, 9L, 1L, 2L, 3L, 4L
 3. 7L, 8L, 9L, 5L, 6L, 1L, 2L, 3L, 4L

 As per my understanding, it is because of containsInAnyOrder function.
 Is there anything Like:

PAssert.that(stream)
 .containsAnyOfThem(
 Row.withSchema(schema).addArray(1L, 2L, 3L, 4L, 5L, 6L, 7L, 
 8L, 9L).build(),
 Row.withSchema(schema).addArray(5L, 6L, 7L, 8L, 9L, 1L, 
 2L, 3L, 4L).build(),
 Row.withSchema(schema).addArray(7L, 8L, 9L, 5L, 6L, 1L, 
 2L, 3L, 4L).build());

 I would really appreciate if anyone can help me in knowing how to
 handle such scenario in Beam.

 Thanks!
 --
 Regards,
 *Sonam*
 Software Engineer
 Mobile: +92 3088337296 <+92%20308%208337296>

 

>>>
>
> --
>
> Regards,
> *Sonam*
> Software Engineer
> Mobile: +92 3088337296 <+92%20308%208337296>
>
> 
>


Re: DoFn @Setup with PipelineOptions

2021-03-02 Thread Xinyu Liu
I created a ticket to track this:
https://issues.apache.org/jira/browse/BEAM-11914. Thanks everyone for the
comments!

Thanks,
Xinyu

On Mon, Mar 1, 2021 at 4:45 PM Xinyu Liu  wrote:

> The reason for not passing it in directly is that we have a large amount
> of configs here at LinkedIn so we use an internal config management
> framework to hide the actual detailed configs needed to construct these
> resources. Internally we put a config map inside the PipelineOptions and
> then during @Setup, we would like to init the config framework with the
> configs inside PipelineOptions. The user does not need to be aware of how
> the configus are populated. They can use something like
>
>   ConfigFramework.create(PipelineOptions).getInstance(SomeResourceFactory)
>
> to create a resource instance they need.
>
> On the other hand, even without this kind of use case, it seems still
> simpler for the users to use parameters in PipelineOptions if we provide it
> directly with @setup.
>
> Thanks,
> Xinyu
>
> On Mon, Mar 1, 2021 at 4:14 PM Kenneth Knowles  wrote:
>
>> Why not just pass in the arguments to the DoFn constructor or as a
>> variable in the containing scope? Do you only know the option after the
>> pipeline is completely constructed so you need to make the switch at
>> runtime? Makes sense. I think passing options to @Setup is useful and
>> harmless.
>>
>> Kenn
>>
>> On Mon, Mar 1, 2021 at 3:42 PM Xinyu Liu  wrote:
>>
>>> Hi, all,
>>>
>>> Currently the @Setup method signature in DoFn does not support any
>>> arguments. This is a bit cumbersome to use for use cases such as creating a
>>> db connection, rest client or fetch some resources, where we would like to
>>> read the configs from the PipelineOptions during setup. Shall we support
>>> adding a DoFn SetupContext that can let the users specify the
>>> PipelineOptions in the arguments, similar to @StartBundle? Seems the
>>> PipelineOptions should always be available when the DoFnRunner is created.
>>> Anyone seeing the downside of it?
>>>
>>> Thanks,
>>> Xinyu
>>>
>>


Re: Random outputs for ARRAY_CONCAT_AGG fn zetasql

2021-03-02 Thread Sonam Ramchand
Is there any way I can access the output array resulting from the sql
query? Then maybe I can sort and compare both *output array* and *expected
output array *for the test to pass.

On Tue, Mar 2, 2021 at 12:24 AM Kenneth Knowles  wrote:

> Yea, the reason is that SQL relations are not ordered. So any ordering of
> [1, 2, 3, 4] and [5, 6] and [7, 8, 9] is possible and correct.
>
> Kenn
>
> On Mon, Mar 1, 2021 at 11:01 AM Tyson Hamilton  wrote:
>
>> I didn't find anything like that after a brief look. What you could do
>> instead is something like:
>>
>> PAssert.thatSingleton(stream).satisfies( row -> assertThat("array_field
>> containsInAnyOrder", row.getArray("array_field"),
>> containsInAnyOrder(Arrays.asList(...)));
>>
>> using junit/hamcrest matchers. I didn't verify this works myself but it
>> should give you an idea for some next steps.
>>
>>
>> On Mon, Mar 1, 2021 at 12:37 AM Sonam Ramchand <
>> sonam.ramch...@venturedive.com> wrote:
>>
>>> Hi Devs,
>>> I have implemented the ARRAY_CONCAT_AGG function for zetasql dialect. I
>>> am trying to validate the test as:
>>>
>>> @Test
>>> public void testArrayConcatAggZetasql() {
>>>   String sql =
>>>   "SELECT ARRAY_CONCAT_AGG(x) AS array_concat_agg FROM (SELECT [1, 2, 
>>> 3, 4] AS x UNION ALL SELECT [5, 6] UNION ALL SELECT [7, 8, 9])";
>>>
>>>   ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
>>>   BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
>>>   PCollection stream = BeamSqlRelUtils.toPCollection(pipeline, 
>>> beamRelNode);
>>>
>>>   Schema schema = Schema.builder().addArrayField("array_field", 
>>> FieldType.INT64).build();
>>>   PAssert.that(stream)
>>>   .containsInAnyOrder(
>>>   Row.withSchema(schema).addArray(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 
>>> 9L).build());
>>>   
>>> pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
>>> }
>>>
>>> Expected Output is: 1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L.
>>> But I am getting randomly different outputs:
>>> 1. 1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L
>>> 2. 5L, 6L, 7L, 8L, 9L, 1L, 2L, 3L, 4L
>>> 3. 7L, 8L, 9L, 5L, 6L, 1L, 2L, 3L, 4L
>>>
>>> As per my understanding, it is because of containsInAnyOrder function.
>>> Is there anything Like:
>>>
>>>PAssert.that(stream)
>>> .containsAnyOfThem(
>>> Row.withSchema(schema).addArray(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 
>>> 9L).build(),
>>> Row.withSchema(schema).addArray(5L, 6L, 7L, 8L, 9L, 1L, 2L, 
>>> 3L, 4L).build(),
>>> Row.withSchema(schema).addArray(7L, 8L, 9L, 5L, 6L, 1L, 2L, 
>>> 3L, 4L).build());
>>>
>>> I would really appreciate if anyone can help me in knowing how to handle
>>> such scenario in Beam.
>>>
>>> Thanks!
>>> --
>>> Regards,
>>> *Sonam*
>>> Software Engineer
>>> Mobile: +92 3088337296 <+92%20308%208337296>
>>>
>>> 
>>>
>>

-- 

Regards,
*Sonam*
Software Engineer
Mobile: +92 3088337296




Re: Null checking in Beam

2021-03-02 Thread Reuven Lax
I'm running into a problem with this check. I added a protocol-buffer file
to a module (google-cloud-platform) that previously did have any protobuf
files in it. The generated files contain lines that violate this null
checker, so they won't compile. I can't annotate the files, because they
are codegen files. I tried adding the package to spotbugs-filter.xml, but
that didn't help.

Any suggestions?

Reuven

On Fri, Jan 22, 2021 at 10:38 AM Brian Hulette  wrote:

>
>
> On Fri, Jan 22, 2021 at 1:18 AM Jan Lukavský  wrote:
>
>> Hi,
>>
>> I'll give my two cents here.
>>
>> I'm not 100% sure that the 1-5% of bugs are as severe as other types of
>> bugs. Yes, throwing NPEs at user is not very polite. On the other hand,
>> many of these actually boil down to user errors. Then we might ask what a
>> correct solution would be. If we manage to figure out what the actual
>> problem is and tell user what specifically is missing or going wrong, that
>> would be just awesome. On the other hand, if a tool used for avoiding
>> "unexpected" NPEs forces us to code
>>
>>Object value = Objects.requireNonNull(myNullableObject); // or similar
>> using Preconditions
>>value.doStuff();
>>
>> instead of just
>>
>>   myNullableObject.doStuff()
>>
>> what we actually did, is a) made a framework happy, and b) changed a line
>> at which NPE is thrown by 1 (and yes, internally prevented JVM from thrown
>> SIGSEGV at itself, but that is deeply technical thing). Nothing changed
>> semantically, from user perspective.
>>
> I'd argue there's value in asking Beam developers to make that change. It
> makes us acknowledge that there's a possibility myNullableObject is null.
> If myNullableObject being null is something relevant to the user we would
> likely want to wrap it in some other exception or provide a more useful
> message than just NPE(!!).
>
>>
>> Now, given that the framework significantly rises compile time (due to
>> all the checks), causes many "bugs" being reported by static code analysis
>> tools (because the framework adds @Nonnull default annotations everywhere,
>> even when Beam's code actually counts with nullability of a field), and
>> given how much we currently suppress these checks ($ git grep BEAM-10402 |
>> wc -l -> 1981), I'd say this deserves a deeper discussion.
>>
> The reason there are so many suppressions is that fixing all the errors is
> a monumental task. Rather than addressing them all, Kenn programmatically
> added suppressions for classes that failed the checks (
> https://github.com/apache/beam/pull/13261). This allowed us to start
> running the checker on the code that passes it while the errors are fixed.
>
>>  Jan
>>
>>
>> On 1/20/21 10:48 PM, Kenneth Knowles wrote:
>>
>> Yes, completely sound nullability checking has been added to the project
>> via checkerframework, based on a large number of NPE bugs (1-5% depending
>> on how you search, but many other bugs likely attributable to
>> nullness-based design errors) which are extra embarrassing because NPEs
>> have were essentially solved, even in practice for Java, well before Beam
>> existed.
>>
>> Checker framework is a pluggable type system analysis with some amount of
>> control flow sensitivity. Every value has a type that is either nullable or
>> not, and certain control structures (like checking for null) can alter the
>> type inside a scope. The best way to think about it is to consider every
>> value in the program as either nullable or not, much like you think of
>> every value as either a string or not, and to view method calls as
>> inherently stateful/nondetermistic. This can be challenging in esoteric
>> cases, but usually makes the overall code health better anyhow.
>>
>> Your example illustrates how problematic the design of the Java language
>> is: the analysis cannot assume that `getDescription` is a pure function,
>> and neither should you. Even if it is aware of boolean-short-circuit it
>> would not be sound to accept this code. There is an annotation for this in
>> the cases where it is true (like proto-generate getters):
>> https://checkerframework.org/api/org/checkerframework/dataflow/qual/Pure.html
>>
>> The refactor for cases like this is trivial so there isn't a lot of value
>> to thinking too hard about it.
>>
>> if (statusCode.equals(Code.INVALID_ARGUMENT)
>>   @Nullable String desc = statusCode.toStatus().getDescription();
>>   if (desc != null && desc.contains("finalized")) {
>> return false;
>>   }
>> }
>>
>> To a casual eye, this may look like a noop change. To the analysis it
>> makes all the difference. And IMO this difference is real. Humans may
>> assume it is a noop and humans would be wrong. So many times when you
>> think/expect/hope that `getXYZ()` is a trivial getter method you later
>> learn that it was tweaked for some odd reason. I believe this code change
>> makes the code better. Suppressing these errors should be exceptionally
>> rare, and never in normal code. It is far better to improve