Re: Primitive Read not working with Flink portable runner

2021-07-24 Thread Jan Lukavský

On 7/24/21 12:34 AM, Robert Bradshaw wrote:


  On Thu, Jul 22, 2021 at 10:20 AM Jan Lukavský  wrote:

Hi,

this was a ride. But I managed to get that working. I'd like to discuss two 
points, though:

  a) I had to push Java coders to ModelCoders for Java (which makes sense to 
me, but is that correct?). See [1]. It is needed so that the Read transform 
(executed directly in TaskManager) can correctly communicate with Java SDK 
harness using custom coders (which is tested here [2]).

I think the intent was that ModelCoders represent the set of
language-agnostic in the model, though I have to admit I've always
been a bit fuzzy on when a coder must or must not be in that list.
I think that this definition works as long, as runner does not itself 
interfere with the Pipeline. Once the runner starts (by itself, not via 
SdkHarnessClient) producing data, it starts to be part of the 
environment, and therefore it should understand its own Coders. I'd 
propose the definition of "model coders" to be Coders that the SDK is 
able to understand, which then works naturally for the ModelCoders 
located in "core-construction-java", that it should understand Javs SDK 
Coders.



  b) I'd strongly prefer if we moved the handling of use_deprecated_read from 
outside of the Read PTransform directly into expand method, see [3]. Though 
this is not needed for the Read on Flink to work, it seems cleaner.

WDYT?

The default value of use_deprecated_read should depend on the runner
(e.g. some runners don't work well with it, others require it). As
such should not be visible to the PTransform's expand.
I think we should know what is the expected outcome. If a runner does 
not support primitive Read (and therefore use_deprecated_read), what 
should we do, if we have such experiment set? Should the Pipeline fail, 
or should it be silently ignored? I think that we should fail, because 
user expects something that cannot be fulfilled. Therefore, we have two 
options - handling the experiment explicitly in runners that do not 
support it, or handle it explicitly in all cases (both supported and 
unsupported). The latter case is when we force runners to call explicit 
conversion method (convertPrimitiveRead). Every runner that does not 
support primitive Read must handle the experiment either way, because 
otherwise the experiment would be simply silently ignored, which is not 
exactly user-friendly.



  Jan

[1] 
https://github.com/apache/beam/pull/15181/commits/394ddc3fdbaacc805d8f7ce02ad2698953f34375

[2] 
https://github.com/apache/beam/pull/15181/files#diff-b1ec58edff6c096481ff336f6fc96e7ba5bcb740dff56c72606ff4f8f0bf85f3R201

[3] 
https://github.com/apache/beam/pull/15181/commits/f1d3fd0217e5513995a72e92f68fe3d1d665c5bb

On 7/18/21 6:29 PM, Jan Lukavský wrote:

Hi,

I was debugging the issue and it relates to pipeline fusion - it seems that the 
primitive Read transform gets fused and then is 'missing' as source. I'm a 
little lost in the code, but the most strange parts are that:

  a) I tried to reject fusion of primitive Read by adding 
GreedyPCollectionFusers::cannotFuse for 
PTransformTranslation.READ_TRANSFORM_URN to 
GreedyPCollectionFusers.URN_FUSIBILITY_CHECKERS, but that didn't change the 
exception

  b) I tried adding Reshuffle.viaRandomKey between Read and PAssert, but that 
didn't change it either

  c) when I run portable Pipeline with use_deprecated_read on Flink it actually 
runs (though it fails when it actually reads any data, but if the input is 
empty, the job runs), so it does not hit the same issue, which is a mystery to 
me

If anyone has any pointers that I can investigate, I'd be really grateful.

Thanks in advance,

  Jan



On 7/16/21 2:00 PM, Jan Lukavský wrote:

Hi,

I hit another issue with the portable Flink runner. Long story short - reading 
from Kafka is not working in portable Flink. After solving issues with 
expansion service configuration (ability to add use_deprecated_read) option, 
because flink portable runner has issues with SDF [1], [2]. After being able to 
inject the use_deprecated_read into expansion service I was able to get an 
execution DAG that has the UnboundedSource, but then more and more issues 
appeared (probably related to missing LengthPrefixCoder somewhere - maybe at 
the output from the primitive Read). I wanted to create a test for it and I 
found out, that there actually is ReadSourcePortableTest in FlinkRunner, but 
_it tests nothing_. The problem is that Read is transformed to SDF, so this 
test tests the SDF, not the Read transform. As a result, the Read transform 
does not work.

I tried using convertReadBasedSplittableDoFnsToPrimitiveReads so that I could 
make the test fail and debug that, but I got into

java.lang.IllegalArgumentException: PCollectionNodes 
[PCollectionNode{id=PAssert$0/GroupGlobally/ParDo(ToSingletonIterables)/ParMultiDo(ToSingletonIterables).output,
 PCollection=unique_name: 
"PAssert$0/GroupGlobally/P

Re: Primitive Read not working with Flink portable runner

2021-07-23 Thread Jan Lukavský
One more note - the PR has a bunch of failing tests, which I plan to 
fix, once there is a consensus, that the overall approach is the correct 
one.


Thanks,

 Jan

On 7/22/21 7:20 PM, Jan Lukavský wrote:


Hi,

this was a ride. But I managed to get that working. I'd like to 
discuss two points, though:


 a) I had to push Java coders to ModelCoders for Java (which makes 
sense to me, but is that correct?). See [1]. It is needed so that the 
Read transform (executed directly in TaskManager) can correctly 
communicate with Java SDK harness using custom coders (which is tested 
here [2]).


 b) I'd strongly prefer if we moved the handling of 
use_deprecated_read from outside of the Read PTransform directly into 
expand method, see [3]. Though this is not needed for the Read on 
Flink to work, it seems cleaner.


WDYT?

 Jan

[1] 
https://github.com/apache/beam/pull/15181/commits/394ddc3fdbaacc805d8f7ce02ad2698953f34375


[2] 
https://github.com/apache/beam/pull/15181/files#diff-b1ec58edff6c096481ff336f6fc96e7ba5bcb740dff56c72606ff4f8f0bf85f3R201


[3] 
https://github.com/apache/beam/pull/15181/commits/f1d3fd0217e5513995a72e92f68fe3d1d665c5bb


On 7/18/21 6:29 PM, Jan Lukavský wrote:


Hi,

I was debugging the issue and it relates to pipeline fusion - it 
seems that the primitive Read transform gets fused and then is 
'missing' as source. I'm a little lost in the code, but the most 
strange parts are that:


 a) I tried to reject fusion of primitive Read by adding 
GreedyPCollectionFusers::cannotFuse for 
PTransformTranslation.READ_TRANSFORM_URN to 
GreedyPCollectionFusers.URN_FUSIBILITY_CHECKERS, but that didn't 
change the exception


 b) I tried adding Reshuffle.viaRandomKey between Read and PAssert, 
but that didn't change it either


 c) when I run portable Pipeline with use_deprecated_read on Flink it 
actually runs (though it fails when it actually reads any data, but 
if the input is empty, the job runs), so it does not hit the same 
issue, which is a mystery to me


If anyone has any pointers that I can investigate, I'd be really 
grateful.


Thanks in advance,

 Jan


On 7/16/21 2:00 PM, Jan Lukavský wrote:


Hi,

I hit another issue with the portable Flink runner. Long story short 
- reading from Kafka is not working in portable Flink. After solving 
issues with expansion service configuration (ability to add 
use_deprecated_read) option, because flink portable runner has 
issues with SDF [1], [2]. After being able to inject the 
use_deprecated_read into expansion service I was able to get an 
execution DAG that has the UnboundedSource, but then more and more 
issues appeared (probably related to missing LengthPrefixCoder 
somewhere - maybe at the output from the primitive Read). I wanted 
to create a test for it and I found out, that there actually is 
ReadSourcePortableTest in FlinkRunner, but _it tests nothing_. The 
problem is that Read is transformed to SDF, so this test tests the 
SDF, not the Read transform. As a result, the Read transform does 
not work.


I tried using convertReadBasedSplittableDoFnsToPrimitiveReads so 
that I could make the test fail and debug that, but I got into


java.lang.IllegalArgumentException: PCollectionNodes 
[PCollectionNode{id=PAssert$0/GroupGlobally/ParDo(ToSingletonIterables)/ParMultiDo(ToSingletonIterables).output,
 PCollection=unique_name: 
"PAssert$0/GroupGlobally/ParDo(ToSingletonIterables)/ParMultiDo(ToSingletonIterables).output"
coder_id: "IterableCoder"
is_bounded: BOUNDED
windowing_strategy_id: "WindowingStrategy(GlobalWindows)"
}] were consumed but never produced


which gave me the last knock-out. :)

My current impression is that starting from Beam 2.25.0, portable 
FlinkRunner is not able to read from Kafka. Could someone give me a 
hint about what is wrong with using 
convertReadBasedSplittableDoFnsToPrimitiveReads in the test [3]?


 Jan

[1] https://issues.apache.org/jira/browse/BEAM-11991

[2] https://issues.apache.org/jira/browse/BEAM-11998

[3] https://github.com/apache/beam/pull/15181



Re: Primitive Read not working with Flink portable runner

2021-07-22 Thread Jan Lukavský

Hi,

this was a ride. But I managed to get that working. I'd like to discuss 
two points, though:


 a) I had to push Java coders to ModelCoders for Java (which makes 
sense to me, but is that correct?). See [1]. It is needed so that the 
Read transform (executed directly in TaskManager) can correctly 
communicate with Java SDK harness using custom coders (which is tested 
here [2]).


 b) I'd strongly prefer if we moved the handling of use_deprecated_read 
from outside of the Read PTransform directly into expand method, see 
[3]. Though this is not needed for the Read on Flink to work, it seems 
cleaner.


WDYT?

 Jan

[1] 
https://github.com/apache/beam/pull/15181/commits/394ddc3fdbaacc805d8f7ce02ad2698953f34375


[2] 
https://github.com/apache/beam/pull/15181/files#diff-b1ec58edff6c096481ff336f6fc96e7ba5bcb740dff56c72606ff4f8f0bf85f3R201


[3] 
https://github.com/apache/beam/pull/15181/commits/f1d3fd0217e5513995a72e92f68fe3d1d665c5bb


On 7/18/21 6:29 PM, Jan Lukavský wrote:


Hi,

I was debugging the issue and it relates to pipeline fusion - it seems 
that the primitive Read transform gets fused and then is 'missing' as 
source. I'm a little lost in the code, but the most strange parts are 
that:


 a) I tried to reject fusion of primitive Read by adding 
GreedyPCollectionFusers::cannotFuse for 
PTransformTranslation.READ_TRANSFORM_URN to 
GreedyPCollectionFusers.URN_FUSIBILITY_CHECKERS, but that didn't 
change the exception


 b) I tried adding Reshuffle.viaRandomKey between Read and PAssert, 
but that didn't change it either


 c) when I run portable Pipeline with use_deprecated_read on Flink it 
actually runs (though it fails when it actually reads any data, but if 
the input is empty, the job runs), so it does not hit the same issue, 
which is a mystery to me


If anyone has any pointers that I can investigate, I'd be really grateful.

Thanks in advance,

 Jan


On 7/16/21 2:00 PM, Jan Lukavský wrote:


Hi,

I hit another issue with the portable Flink runner. Long story short 
- reading from Kafka is not working in portable Flink. After solving 
issues with expansion service configuration (ability to add 
use_deprecated_read) option, because flink portable runner has issues 
with SDF [1], [2]. After being able to inject the use_deprecated_read 
into expansion service I was able to get an execution DAG that has 
the UnboundedSource, but then more and more issues appeared (probably 
related to missing LengthPrefixCoder somewhere - maybe at the output 
from the primitive Read). I wanted to create a test for it and I 
found out, that there actually is ReadSourcePortableTest in 
FlinkRunner, but _it tests nothing_. The problem is that Read is 
transformed to SDF, so this test tests the SDF, not the Read 
transform. As a result, the Read transform does not work.


I tried using convertReadBasedSplittableDoFnsToPrimitiveReads so that 
I could make the test fail and debug that, but I got into


java.lang.IllegalArgumentException: PCollectionNodes 
[PCollectionNode{id=PAssert$0/GroupGlobally/ParDo(ToSingletonIterables)/ParMultiDo(ToSingletonIterables).output,
 PCollection=unique_name: 
"PAssert$0/GroupGlobally/ParDo(ToSingletonIterables)/ParMultiDo(ToSingletonIterables).output"
coder_id: "IterableCoder"
is_bounded: BOUNDED
windowing_strategy_id: "WindowingStrategy(GlobalWindows)"
}] were consumed but never produced


which gave me the last knock-out. :)

My current impression is that starting from Beam 2.25.0, portable 
FlinkRunner is not able to read from Kafka. Could someone give me a 
hint about what is wrong with using 
convertReadBasedSplittableDoFnsToPrimitiveReads in the test [3]?


 Jan

[1] https://issues.apache.org/jira/browse/BEAM-11991

[2] https://issues.apache.org/jira/browse/BEAM-11998

[3] https://github.com/apache/beam/pull/15181



Re: Primitive Read not working with Flink portable runner

2021-07-18 Thread Jan Lukavský

Hi,

I was debugging the issue and it relates to pipeline fusion - it seems 
that the primitive Read transform gets fused and then is 'missing' as 
source. I'm a little lost in the code, but the most strange parts are that:


 a) I tried to reject fusion of primitive Read by adding 
GreedyPCollectionFusers::cannotFuse for 
PTransformTranslation.READ_TRANSFORM_URN to 
GreedyPCollectionFusers.URN_FUSIBILITY_CHECKERS, but that didn't change 
the exception


 b) I tried adding Reshuffle.viaRandomKey between Read and PAssert, but 
that didn't change it either


 c) when I run portable Pipeline with use_deprecated_read on Flink it 
actually runs (though it fails when it actually reads any data, but if 
the input is empty, the job runs), so it does not hit the same issue, 
which is a mystery to me


If anyone has any pointers that I can investigate, I'd be really grateful.

Thanks in advance,

 Jan


On 7/16/21 2:00 PM, Jan Lukavský wrote:


Hi,

I hit another issue with the portable Flink runner. Long story short - 
reading from Kafka is not working in portable Flink. After solving 
issues with expansion service configuration (ability to add 
use_deprecated_read) option, because flink portable runner has issues 
with SDF [1], [2]. After being able to inject the use_deprecated_read 
into expansion service I was able to get an execution DAG that has the 
UnboundedSource, but then more and more issues appeared (probably 
related to missing LengthPrefixCoder somewhere - maybe at the output 
from the primitive Read). I wanted to create a test for it and I found 
out, that there actually is ReadSourcePortableTest in FlinkRunner, but 
_it tests nothing_. The problem is that Read is transformed to SDF, so 
this test tests the SDF, not the Read transform. As a result, the Read 
transform does not work.


I tried using convertReadBasedSplittableDoFnsToPrimitiveReads so that 
I could make the test fail and debug that, but I got into


java.lang.IllegalArgumentException: PCollectionNodes 
[PCollectionNode{id=PAssert$0/GroupGlobally/ParDo(ToSingletonIterables)/ParMultiDo(ToSingletonIterables).output,
 PCollection=unique_name: 
"PAssert$0/GroupGlobally/ParDo(ToSingletonIterables)/ParMultiDo(ToSingletonIterables).output"
coder_id: "IterableCoder"
is_bounded: BOUNDED
windowing_strategy_id: "WindowingStrategy(GlobalWindows)"
}] were consumed but never produced


which gave me the last knock-out. :)

My current impression is that starting from Beam 2.25.0, portable 
FlinkRunner is not able to read from Kafka. Could someone give me a 
hint about what is wrong with using 
convertReadBasedSplittableDoFnsToPrimitiveReads in the test [3]?


 Jan

[1] https://issues.apache.org/jira/browse/BEAM-11991

[2] https://issues.apache.org/jira/browse/BEAM-11998

[3] https://github.com/apache/beam/pull/15181



Primitive Read not working with Flink portable runner

2021-07-16 Thread Jan Lukavský

Hi,

I hit another issue with the portable Flink runner. Long story short - 
reading from Kafka is not working in portable Flink. After solving 
issues with expansion service configuration (ability to add 
use_deprecated_read) option, because flink portable runner has issues 
with SDF [1], [2]. After being able to inject the use_deprecated_read 
into expansion service I was able to get an execution DAG that has the 
UnboundedSource, but then more and more issues appeared (probably 
related to missing LengthPrefixCoder somewhere - maybe at the output 
from the primitive Read). I wanted to create a test for it and I found 
out, that there actually is ReadSourcePortableTest in FlinkRunner, but 
_it tests nothing_. The problem is that Read is transformed to SDF, so 
this test tests the SDF, not the Read transform. As a result, the Read 
transform does not work.


I tried using convertReadBasedSplittableDoFnsToPrimitiveReads so that I 
could make the test fail and debug that, but I got into


java.lang.IllegalArgumentException: PCollectionNodes 
[PCollectionNode{id=PAssert$0/GroupGlobally/ParDo(ToSingletonIterables)/ParMultiDo(ToSingletonIterables).output,
 PCollection=unique_name: 
"PAssert$0/GroupGlobally/ParDo(ToSingletonIterables)/ParMultiDo(ToSingletonIterables).output"
coder_id: "IterableCoder"
is_bounded: BOUNDED
windowing_strategy_id: "WindowingStrategy(GlobalWindows)"
}] were consumed but never produced


which gave me the last knock-out. :)

My current impression is that starting from Beam 2.25.0, portable 
FlinkRunner is not able to read from Kafka. Could someone give me a hint 
about what is wrong with using 
convertReadBasedSplittableDoFnsToPrimitiveReads in the test [3]?


 Jan

[1] https://issues.apache.org/jira/browse/BEAM-11991

[2] https://issues.apache.org/jira/browse/BEAM-11998

[3] https://github.com/apache/beam/pull/15181



Re: Missing coder in python x-lang transform when using window Fn with AfterCount trigger

2021-07-15 Thread Jan Lukavský

Makes sense, thank you!

 Jan

On 7/15/21 9:43 PM, Chamikara Jayalath wrote:
The PCollection you feed into WriteToKafka (or any other 
cross-language transform) needs to use standard coders [1].
What's the type of the PCollection before WriteToKafka ? It's possible 
that you just need to provide a type hint to prevent Python from 
picking the default Pickel Coder that Java doesn't understand.
See https://issues.apache.org/jira/browse/BEAM-11938 
<https://issues.apache.org/jira/browse/BEAM-11938>.


Thanks,
Cham

[1] 
https://github.com/apache/beam/blob/3e933b55f3d2072fb0248050f9091850933f33c7/model/pipeline/src/main/proto/beam_runner_api.proto#L784 
<https://github.com/apache/beam/blob/3e933b55f3d2072fb0248050f9091850933f33c7/model/pipeline/src/main/proto/beam_runner_api.proto#L784>


On Thu, Jul 15, 2021 at 12:16 PM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


Sorry, my bad. I made two modifications of the source at once - the
cause of the exception is not the expansion of ReadFromKafka, but
WriteToKafka. If I remove that the Pipeline runs. So it seems I
have to
change to coder of the PCollection that is consumed by WriteToKafka.
Which makes sense.

    On 7/15/21 8:44 PM, Jan Lukavský wrote:
> Hi,
>
> I hit an issue when using x-lang python pipeline (with
ReadFromKafka)
> with subsequent WindowInto with
trigger=Repeatedly(AfterCount(1)). The
> Pipeline looks as follows:
>
>   (p | ReadFromKafka(
>   consumer_config={'bootstrap.servers': bootstrapServer},
>   topics=[inputTopic],
>   expansion_service=get_expansion_service())
>     | "Tokenize" >> beam.FlatMap(lambda line:
> re.findall(r'[A-Za-z\']+', line))
>     | beam.WindowInto(
>   window.GlobalWindows(),
> trigger=trigger.Repeatedly(trigger.AfterCount(1)),
> accumulation_mode=trigger.AccumulationMode.DISCARDING,
>   allowed_lateness=window.Duration.of(0))
>     ...
>
> The error is
>
> Caused by: java.lang.IllegalArgumentException: Unknown Coder URN
> beam:coder:pickled_python:v1. Known URNs:
[beam:coder:avro:generic:v1,
> beam:coder:bytes:v1, beam:coder:bool:v1, beam:coder:string_utf8:v1,
> beam:coder:kv:v1, beam:coder:varint:v1,
beam:coder:interval_window:v1,
> beam:coder:iterable:v1, beam:coder:timer:v1,
> beam:coder:length_prefix:v1, beam:coder:global_window:v1,
> beam:coder:windowed_value:v1, beam:coder:param_windowed_value:v1,
> beam:coder:double:v1, beam:coder:row:v1, beam:coder:sharded_key:v1]
>     at
>

org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:440)
>     at
>

org.apache.beam.runners.core.construction.CoderTranslation.fromKnownCoder(CoderTranslation.java:165)
>     at
>

org.apache.beam.runners.core.construction.CoderTranslation.fromProto(CoderTranslation.java:145)
>     at
>

org.apache.beam.runners.core.construction.RehydratedComponents$2.load(RehydratedComponents.java:87)
>     at
>

org.apache.beam.runners.core.construction.RehydratedComponents$2.load(RehydratedComponents.java:82)
>     at
>

org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3528)
>     at
>

org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2277)
>     at
>

org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2154)
>     at
>

org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2044)
>     ... 50 more
>
> Which seems like missing coder, but what I do not understand is
why is
> the coder passed to the ExpansionService in the first place. This
> happens even if I place a (map) transform between the
ReadFromKafka a
> WindowInto transform. I think the expansion service should not
need to
> know about what happens later in the Pipeline. With default trigger
> the Pipeline is able to run.
>
> Any ideas?
>
>  Jan
>



Re: Missing coder in python x-lang transform when using window Fn with AfterCount trigger

2021-07-15 Thread Jan Lukavský
Sorry, my bad. I made two modifications of the source at once - the 
cause of the exception is not the expansion of ReadFromKafka, but 
WriteToKafka. If I remove that the Pipeline runs. So it seems I have to 
change to coder of the PCollection that is consumed by WriteToKafka. 
Which makes sense.


On 7/15/21 8:44 PM, Jan Lukavský wrote:

Hi,

I hit an issue when using x-lang python pipeline (with ReadFromKafka) 
with subsequent WindowInto with trigger=Repeatedly(AfterCount(1)). The 
Pipeline looks as follows:


  (p | ReadFromKafka(
  consumer_config={'bootstrap.servers': bootstrapServer},
  topics=[inputTopic],
  expansion_service=get_expansion_service())
    | "Tokenize" >> beam.FlatMap(lambda line: 
re.findall(r'[A-Za-z\']+', line))

    | beam.WindowInto(
  window.GlobalWindows(),
  trigger=trigger.Repeatedly(trigger.AfterCount(1)),
  accumulation_mode=trigger.AccumulationMode.DISCARDING,
  allowed_lateness=window.Duration.of(0))
    ...

The error is

Caused by: java.lang.IllegalArgumentException: Unknown Coder URN 
beam:coder:pickled_python:v1. Known URNs: [beam:coder:avro:generic:v1, 
beam:coder:bytes:v1, beam:coder:bool:v1, beam:coder:string_utf8:v1, 
beam:coder:kv:v1, beam:coder:varint:v1, beam:coder:interval_window:v1, 
beam:coder:iterable:v1, beam:coder:timer:v1, 
beam:coder:length_prefix:v1, beam:coder:global_window:v1, 
beam:coder:windowed_value:v1, beam:coder:param_windowed_value:v1, 
beam:coder:double:v1, beam:coder:row:v1, beam:coder:sharded_key:v1]
    at 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:440)
    at 
org.apache.beam.runners.core.construction.CoderTranslation.fromKnownCoder(CoderTranslation.java:165)
    at 
org.apache.beam.runners.core.construction.CoderTranslation.fromProto(CoderTranslation.java:145)
    at 
org.apache.beam.runners.core.construction.RehydratedComponents$2.load(RehydratedComponents.java:87)
    at 
org.apache.beam.runners.core.construction.RehydratedComponents$2.load(RehydratedComponents.java:82)
    at 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3528)
    at 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2277)
    at 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2154)
    at 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2044)

    ... 50 more

Which seems like missing coder, but what I do not understand is why is 
the coder passed to the ExpansionService in the first place. This 
happens even if I place a (map) transform between the ReadFromKafka a 
WindowInto transform. I think the expansion service should not need to 
know about what happens later in the Pipeline. With default trigger 
the Pipeline is able to run.


Any ideas?

 Jan



Missing coder in python x-lang transform when using window Fn with AfterCount trigger

2021-07-15 Thread Jan Lukavský

Hi,

I hit an issue when using x-lang python pipeline (with ReadFromKafka) 
with subsequent WindowInto with trigger=Repeatedly(AfterCount(1)). The 
Pipeline looks as follows:


  (p | ReadFromKafka(
  consumer_config={'bootstrap.servers': bootstrapServer},
  topics=[inputTopic],
  expansion_service=get_expansion_service())
    | "Tokenize" >> beam.FlatMap(lambda line: 
re.findall(r'[A-Za-z\']+', line))

    | beam.WindowInto(
  window.GlobalWindows(),
  trigger=trigger.Repeatedly(trigger.AfterCount(1)),
  accumulation_mode=trigger.AccumulationMode.DISCARDING,
  allowed_lateness=window.Duration.of(0))
    ...

The error is

Caused by: java.lang.IllegalArgumentException: Unknown Coder URN 
beam:coder:pickled_python:v1. Known URNs: [beam:coder:avro:generic:v1, 
beam:coder:bytes:v1, beam:coder:bool:v1, beam:coder:string_utf8:v1, 
beam:coder:kv:v1, beam:coder:varint:v1, beam:coder:interval_window:v1, 
beam:coder:iterable:v1, beam:coder:timer:v1, 
beam:coder:length_prefix:v1, beam:coder:global_window:v1, 
beam:coder:windowed_value:v1, beam:coder:param_windowed_value:v1, 
beam:coder:double:v1, beam:coder:row:v1, beam:coder:sharded_key:v1]
    at 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:440)
    at 
org.apache.beam.runners.core.construction.CoderTranslation.fromKnownCoder(CoderTranslation.java:165)
    at 
org.apache.beam.runners.core.construction.CoderTranslation.fromProto(CoderTranslation.java:145)
    at 
org.apache.beam.runners.core.construction.RehydratedComponents$2.load(RehydratedComponents.java:87)
    at 
org.apache.beam.runners.core.construction.RehydratedComponents$2.load(RehydratedComponents.java:82)
    at 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3528)
    at 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2277)
    at 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2154)
    at 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2044)

    ... 50 more

Which seems like missing coder, but what I do not understand is why is 
the coder passed to the ExpansionService in the first place. This 
happens even if I place a (map) transform between the ReadFromKafka a 
WindowInto transform. I think the expansion service should not need to 
know about what happens later in the Pipeline. With default trigger the 
Pipeline is able to run.


Any ideas?

 Jan



Re: Dataflow dependencies require non-maven central dependencies (confluent kafka)

2021-07-12 Thread Jan Lukavský

Can you try to exclude only the "kafka-avro-serializer"?

On 7/12/21 6:28 PM, Alex Van Boxel wrote:
That's not the problem, the example Gradle build excludes it, the 
Dataflow runner fails with the error stated in the original post.


 _/
_/ Alex Van Boxel


On Mon, Jul 12, 2021 at 6:27 PM Alexey Romanenko 
mailto:aromanenko@gmail.com>> wrote:


I agree that we should make this optional. What would be the best
way to it with gradle?


On 11 Jul 2021, at 16:40, Jan Lukavský mailto:je...@seznam.cz>> wrote:

I'd be +1 to making it optional as well. Looks really like an
overhead for users not using avro.

On 7/11/21 10:36 AM, Alex Van Boxel wrote:

It worked before 2.30. It's fine for when you're using Confluent
Kafka, but feels like a hard dependency for non-Kafka users.
Certainly the requirement for including an extra repo. Certain
companies have to go through a lengthy process to include extra
repo's. Feels like a strange requirement, for nothing. Isn't
it a bug in the DataflowRunner?

dependencies {
compile("org.apache.beam:beam-sdks-java-core:$beamVersion")
compile("org.apache.beam:beam-runners-direct-java:$beamVersion")

compile("org.apache.beam:beam-runners-google-cloud-dataflow-java:$beamVersion")
// waiting for response on mailinglist (2.30 onwards), dataflow
runner fails
//    {
//        exclude module: 'beam-sdks-java-io-kafka'
//    }

compile("org.apache.beam:beam-sdks-java-io-elasticsearch:$beamVersion")
compile("org.apache.beam:beam-sdks-java-io-jdbc:$beamVersion")

compile("org.apache.beam:beam-sdks-java-extensions-protobuf:$beamVersion")

compile("org.apache.beam:beam-sdks-java-extensions-google-cloud-platform-core:$beamVersion")
compile("org.apache.beam:beam-sdks-java-extensions-sql:$beamVersion")
compile("org.apache.beam:beam-sdks-java-extensions-zetasketch:$beamVersion")

compile("org.apache.beam:beam-sdks-java-extensions-json-jackson:$beamVersion")


compile("org.apache.beam:beam-sdks-java-io-google-cloud-platform:$beamVersion")
compile("org.apache.beam:beam-sdks-java-io-common:$beamVersion")

// force to the JRE, as the android version is auto resolved
compile("com.google.guava:guava:30.1.1-jre")

compile("org.slf4j:slf4j-log4j12:1.7.30")
compile("commons-io:commons-io:2.8.0")

compile("io.opentelemetry:opentelemetry-proto:1.3.0-alpha")

compile("com.microsoft.sqlserver:mssql-jdbc:9.1.0.jre8-preview")

compile("io.swagger.parser.v3:swagger-parser:2.0.24")

testCompile("org.hamcrest:hamcrest-all:1.3")
testCompile("org.assertj:assertj-core:3.4.1")
testCompile("junit:junit:4.12")
}


 _/
_/ Alex Van Boxel


On Fri, Jul 9, 2021 at 1:55 PM Alexey Romanenko
mailto:aromanenko@gmail.com>> wrote:

Hi Alex,

Yes, starting from Beam 2.20.0,  "beam-sdks-java-io-kafka”
requires an additional dependency “kafka-avro-serializer”
from external repository
(https://packages.confluent.io/maven/
<https://packages.confluent.io/maven/>).

This is reflected in published POM file:

https://search.maven.org/artifact/org.apache.beam/beam-sdks-java-io-kafka/2.31.0/jar

<https://search.maven.org/artifact/org.apache.beam/beam-sdks-java-io-kafka/2.31.0/jar>

Did it work for you before version 2.30.0?
Could you share your build.gradle file?

—
Alexey




On 9 Jul 2021, at 11:23, Alex Van Boxel mailto:a...@vanboxel.be>> wrote:

Hi all,

I've been building for years via gradle. The dependency
management is probably a bit different from that of maven,
but it seems that dataflow now requires Confluent Kafka
dependencies. They are not available in Maven Central. This
feels wrong for an Apache project.

 -

file:/Users/alex.vanboxel/.m2/repository/io/confluent/kafka-avro-serializer/5.3.2/kafka-avro-serializer-5.3.2.pom
       -

https://repo.maven.apache.org/maven2/io/confluent/kafka-avro-serializer/5.3.2/kafka-avro-serializer-5.3.2.pom

<https://repo.maven.apache.org/maven2/io/confluent/kafka-avro-serializer/5.3.2/kafka-avro-serializer-5.3.2.pom>
        -

https://repository.apache.org/content/repositories/releases/io/confluent/kafka-avro-serializer/5.3.2/kafka-avro-serializer-5.3.2.pom

<https://repository.apache.org/content/repositories/releases/io/confluent/kafka-avro-serializer/5.3.2/kafka-avro-serializer-5.3.2.pom>

Excluding the dependencies "exclude module:

Re: Dataflow dependencies require non-maven central dependencies (confluent kafka)

2021-07-11 Thread Jan Lukavský
I'd be +1 to making it optional as well. Looks really like an overhead 
for users not using avro.


On 7/11/21 10:36 AM, Alex Van Boxel wrote:
It worked before 2.30. It's fine for when you're using Confluent 
Kafka, but feels like a hard dependency for non-Kafka users. Certainly 
the requirement for including an extra repo. Certain companies have to 
go through a lengthy process to include extra repo's. Feels like a 
strange requirement, for nothing. Isn't it a bug in the 
DataflowRunner?


dependencies {
compile("org.apache.beam:beam-sdks-java-core:$beamVersion")
compile("org.apache.beam:beam-runners-direct-java:$beamVersion")
compile("org.apache.beam:beam-runners-google-cloud-dataflow-java:$beamVersion")
// waiting for response on mailinglist (2.30 onwards), dataflow runner 
fails

//    {
//        exclude module: 'beam-sdks-java-io-kafka'
//    }

compile("org.apache.beam:beam-sdks-java-io-elasticsearch:$beamVersion")
compile("org.apache.beam:beam-sdks-java-io-jdbc:$beamVersion")

compile("org.apache.beam:beam-sdks-java-extensions-protobuf:$beamVersion")
compile("org.apache.beam:beam-sdks-java-extensions-google-cloud-platform-core:$beamVersion")
compile("org.apache.beam:beam-sdks-java-extensions-sql:$beamVersion")
compile("org.apache.beam:beam-sdks-java-extensions-zetasketch:$beamVersion")
compile("org.apache.beam:beam-sdks-java-extensions-json-jackson:$beamVersion")

compile("org.apache.beam:beam-sdks-java-io-google-cloud-platform:$beamVersion")
compile("org.apache.beam:beam-sdks-java-io-common:$beamVersion")

// force to the JRE, as the android version is auto resolved
compile("com.google.guava:guava:30.1.1-jre")

compile("org.slf4j:slf4j-log4j12:1.7.30")
compile("commons-io:commons-io:2.8.0")

compile("io.opentelemetry:opentelemetry-proto:1.3.0-alpha")

compile("com.microsoft.sqlserver:mssql-jdbc:9.1.0.jre8-preview")

compile("io.swagger.parser.v3:swagger-parser:2.0.24")

testCompile("org.hamcrest:hamcrest-all:1.3")
testCompile("org.assertj:assertj-core:3.4.1")
testCompile("junit:junit:4.12")
}


 _/
_/ Alex Van Boxel


On Fri, Jul 9, 2021 at 1:55 PM Alexey Romanenko 
mailto:aromanenko@gmail.com>> wrote:


Hi Alex,

Yes, starting from Beam 2.20.0,  "beam-sdks-java-io-kafka”
requires an additional dependency “kafka-avro-serializer” from
external repository (https://packages.confluent.io/maven/
).

This is reflected in published POM file:

https://search.maven.org/artifact/org.apache.beam/beam-sdks-java-io-kafka/2.31.0/jar



Did it work for you before version 2.30.0?
Could you share your build.gradle file?

—
Alexey




On 9 Jul 2021, at 11:23, Alex Van Boxel mailto:a...@vanboxel.be>> wrote:

Hi all,

I've been building for years via gradle. The dependency
management is probably a bit different from that of maven, but it
seems that dataflow now requires Confluent Kafka dependencies.
They are not available in Maven Central. This feels wrong for an
Apache project.

       -

file:/Users/alex.vanboxel/.m2/repository/io/confluent/kafka-avro-serializer/5.3.2/kafka-avro-serializer-5.3.2.pom
       -

https://repo.maven.apache.org/maven2/io/confluent/kafka-avro-serializer/5.3.2/kafka-avro-serializer-5.3.2.pom


        -

https://repository.apache.org/content/repositories/releases/io/confluent/kafka-avro-serializer/5.3.2/kafka-avro-serializer-5.3.2.pom



Excluding the dependencies "exclude module:
'*beam-sdks-java-io-kafka*'" doesn't work with:

Exception in thread "main" java.lang.NoClassDefFoundError:
org/apache/beam/sdk/io/kafka/KafkaIO$Read
at

org.apache.beam.runners.dataflow.DataflowRunner.getOverrides(DataflowRunner.java:522)
at

org.apache.beam.runners.dataflow.DataflowRunner.replaceV1Transforms(DataflowRunner.java:1337)
at
org.apache.beam.runners.dataflow.DataflowRunner.run(DataflowRunner.java:967)
at
org.apache.beam.runners.dataflow.DataflowRunner.run(DataflowRunner.java:196)

This happens from version 2.30 onwards. Is this intended?!
 _/
_/ Alex Van Boxel




Re: JavaPrecommit fails

2021-07-06 Thread Jan Lukavský
Looks to me to be failing consistently. At least appears so both on PRs 
and tested locally on master branch.


On 7/5/21 1:34 PM, Alexey Romanenko wrote:

Hello,

JavaPrecommit fails because of 
“org.apache.beam.fn.harness.control.HarnessMonitoringInfosInstructionHandlerTest.testReturnsProcessWideMonitoringInfos”
 test.
Is it a regression or just flaky?

—
Alexey




Re: Specifying environment for cross-language transform expansion

2021-07-02 Thread Jan Lukavský

I'm not sure what is the conclusion here. I'd suggest the following:

 a) merge fix for BEAM-12538 ([1]), so that it can be fixed in 2.32.0, 
because the current state is not good


 b) make a decision about whether to send expansion options via 
ExpansionRequest, or not


 c) in either case, we probably should somehow narrow the options that 
can be passed to the expansion service via command-line, because most of 
them will be (silently) ignored otherwise. In the discussion on the PR, 
there are two options - either refactor the PipelineOptions to extract 
PortableEnvironmentOptions (which would then probably have to extend 
ExperimentalOptions) and accept only those (if possible), or add a 
marker annotation on the fields that should be supported by the 
expansion service. I tend to prefer the latter option, because it seems 
a kind of more flexible (but I'm fine with the first option as well).


 Jan

[1] https://github.com/apache/beam/pull/15082

On 7/1/21 11:37 PM, Robert Bradshaw wrote:

IIRC, Flink has special support for "native" read operations. But +1
this should be handled by the runner swapping things out rather than
during pipeline construction.

On Thu, Jul 1, 2021 at 2:11 PM Jan Lukavský  wrote:

This really does not match my experience. Passing the correct 
"use_deprecated_read" flag to the expansion service had the expected impact on 
the Flink's execution DAG and - most of all - it started to work (at least seems so). The 
UI in Flink also started to reflect that and stopped using SDF (no Impulse in the DAG).

On 7/1/21 10:26 PM, Luke Cwik wrote:

There is no implementation for executing UnboundedSource directly within the 
Java SDK harness, it only supports executing SDFs and UnboundedSource via the 
wrapper over SDF. The runner would have to execute the source directly itself. 
It could attempt to deserialize the SDF ptransform and see if there is an 
UnboundedSource inside and then do whatever it wants with it.

On Thu, Jul 1, 2021 at 11:39 AM Jan Lukavský  wrote:

I don't have complete comprehension of the topic, but from what I have observed, the 
runner gets (possibly cross-language) proto description of the pipeline, and the 
post-processing there might be limited.  That is mainly due to the fact, that we have 
inverted the expansion flow - we expand Read to SDF and only when 
"use_deprecated_read" is on, we revert it back to UnboundedSource. The portable 
runner cannot interfere with that.

On 7/1/21 8:31 PM, Kyle Weaver wrote:

I thought the runner was expected to call 
convertReadBasedSplittableDoFnsToPrimitiveReadsIfNecessary. Why do we need to 
do that in the expansion service?

On Thu, Jul 1, 2021 at 11:16 AM Jan Lukavský  wrote:

Hi,

after today's experience I think I have some arguments about why we *should* 
pass (at least some) of the PipelineOptions from SDK to expansion service.

  1) there are lots, and lots, and lots of bugs around SDF and around the 
"use_deprecated_read", sorry, but the switch to SDF as the default *way* too 
premature

  2) therefore, the expansion *is* runner dependent (because whether to use "use_deprecated_read" 
or not is runner dependent), only the client of the expansion service (the SDK, the driver code) knows the 
target runner - i.e. if the target runner can use "new" Read or "deprecated" Read

  3) currently, my opinion is that we hold many portable Flink users on 2.24.0, because 
from 2.25.0, the combination of Kafka + Python SDK + Flink is simply not working - until 
now, there is no way to pass arguments to expansion service, and even after that, 
"use_deprecated_read" is simply ignored by the service (pretty much the same as 
was in DirectRunner, see [1])

We should consider making use_deprecated_read the default for Flink (at least), 
not sure what is the state of other runners regarding that. It would be good to 
rename it, if we do not have plans to correctly support SDF (Read), including 
portability of other runners.

Yes, this might be a temporary issue, but the fact, that expansion is runner 
dependent remains valid, because such situation might reappear.

  Jan

[1] 
https://github.com/apache/beam/pull/15082/commits/5a46664ceb9f03da3089925b30ecd0a802e8b3eb

On 7/1/21 9:33 AM, Jan Lukavský wrote:

On 7/1/21 3:26 AM, Kyle Weaver wrote:

I think it should accept complete list of PipelineOptions (or at least some 
defined subset - PortabilityPipelineOptions, ExperimentalOptions, ...?)


I'm not totally opposed to redefining some options, either. Using 
PipelineOptions could be confusing because only very few options would actually 
be respected -- even PortablePipelineOptions includes many options that 
wouldn't make sense in this context. Maybe better to have a small list of 
options that are guaranteed to work.

That makes sense. How would we define the subset? I think that would probably 
require some sort of annotation analogous to 

Re: Specifying environment for cross-language transform expansion

2021-07-01 Thread Jan Lukavský
This really does not match my experience. Passing the correct 
"use_deprecated_read" flag to the expansion service had the expected 
impact on the Flink's execution DAG and - most of all - it started to 
work (at least seems so). The UI in Flink also started to reflect that 
and stopped using SDF (no Impulse in the DAG).


On 7/1/21 10:26 PM, Luke Cwik wrote:
There is no implementation for executing UnboundedSource directly 
within the Java SDK harness, it only supports executing SDFs and 
UnboundedSource via the wrapper over SDF. The runner would have to 
execute the source directly itself. It could attempt to deserialize 
the SDF ptransform and see if there is an UnboundedSource inside and 
then do whatever it wants with it.


On Thu, Jul 1, 2021 at 11:39 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


I don't have complete comprehension of the topic, but from what I
have observed, the runner gets (possibly cross-language) proto
description of the pipeline, and the post-processing there might
be limited.  That is mainly due to the fact, that we have inverted
the expansion flow - we expand Read to SDF and only when
"use_deprecated_read" is on, we revert it back to UnboundedSource.
The portable runner cannot interfere with that.

On 7/1/21 8:31 PM, Kyle Weaver wrote:

I thought the runner was expected to
call convertReadBasedSplittableDoFnsToPrimitiveReadsIfNecessary.
Why do we need to do that in the expansion service?

On Thu, Jul 1, 2021 at 11:16 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

Hi,

after today's experience I think I have some arguments about
why we *should* pass (at least some) of the PipelineOptions
from SDK to expansion service.

 1) there are lots, and lots, and lots of bugs around SDF and
around the "use_deprecated_read", sorry, but the switch to
SDF as the default *way* too premature

 2) therefore, the expansion *is* runner dependent (because
whether to use "use_deprecated_read" or not is runner
dependent), only the client of the expansion service (the
SDK, the driver code) knows the target runner - i.e. if the
target runner can use "new" Read or "deprecated" Read

 3) currently, my opinion is that we hold many portable Flink
users on 2.24.0, because from 2.25.0, the combination of
Kafka + Python SDK + Flink is simply not working - until now,
there is no way to pass arguments to expansion service, and
even after that, "use_deprecated_read" is simply ignored by
the service (pretty much the same as was in DirectRunner, see
[1])

We should consider making use_deprecated_read the default for
Flink (at least), not sure what is the state of other runners
regarding that. It would be good to rename it, if we do not
have plans to correctly support SDF (Read), including
portability of other runners.

Yes, this might be a temporary issue, but the fact, that
expansion is runner dependent remains valid, because such
situation might reappear.

 Jan

[1]

https://github.com/apache/beam/pull/15082/commits/5a46664ceb9f03da3089925b30ecd0a802e8b3eb

<https://github.com/apache/beam/pull/15082/commits/5a46664ceb9f03da3089925b30ecd0a802e8b3eb>

On 7/1/21 9:33 AM, Jan Lukavský wrote:

On 7/1/21 3:26 AM, Kyle Weaver wrote:


I think it should accept complete list of
PipelineOptions (or at least some defined subset -
PortabilityPipelineOptions, ExperimentalOptions, ...?)


I'm not totally opposed to redefining some options, either.
Using PipelineOptions could be confusing because only very
few options would actually be respected -- even
PortablePipelineOptions includes many options that wouldn't
make sense in this context. Maybe better to have a small
list of options that are guaranteed to work.


That makes sense. How would we define the subset? I think
that would probably require some sort of annotation
analogous to @Validation.Required, maybe
@Validation.ExpansionSupported or similar. I'm fine with
implementing that, but I would need now to get the 'hotfix'
to upcoming 2.32.0 release. Could we make that for 2.33.0?
Will you help me review the current PR [1]?

[1] https://github.com/apache/beam/pull/15082
    <https://github.com/apache/beam/pull/15082>



On Wed, Jun 30, 2021 at 8:48 AM Jan Lukavský
mailto:je...@seznam.cz>> wrote:

 > Not sure why we need the hacks with NoOpRunner

As noted earlier (and that was why I started this
thread in the first
place :)), adding :runner

Re: Specifying environment for cross-language transform expansion

2021-07-01 Thread Jan Lukavský
I don't have complete comprehension of the topic, but from what I have 
observed, the runner gets (possibly cross-language) proto description of 
the pipeline, and the post-processing there might be limited.  That is 
mainly due to the fact, that we have inverted the expansion flow - we 
expand Read to SDF and only when "use_deprecated_read" is on, we revert 
it back to UnboundedSource. The portable runner cannot interfere with that.


On 7/1/21 8:31 PM, Kyle Weaver wrote:
I thought the runner was expected to 
call convertReadBasedSplittableDoFnsToPrimitiveReadsIfNecessary. Why 
do we need to do that in the expansion service?


On Thu, Jul 1, 2021 at 11:16 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


Hi,

after today's experience I think I have some arguments about why
we *should* pass (at least some) of the PipelineOptions from SDK
to expansion service.

 1) there are lots, and lots, and lots of bugs around SDF and
around the "use_deprecated_read", sorry, but the switch to SDF as
the default *way* too premature

 2) therefore, the expansion *is* runner dependent (because
whether to use "use_deprecated_read" or not is runner dependent),
only the client of the expansion service (the SDK, the driver
code) knows the target runner - i.e. if the target runner can use
"new" Read or "deprecated" Read

 3) currently, my opinion is that we hold many portable Flink
users on 2.24.0, because from 2.25.0, the combination of Kafka +
Python SDK + Flink is simply not working - until now, there is no
way to pass arguments to expansion service, and even after that,
"use_deprecated_read" is simply ignored by the service (pretty
much the same as was in DirectRunner, see [1])

We should consider making use_deprecated_read the default for
Flink (at least), not sure what is the state of other runners
regarding that. It would be good to rename it, if we do not have
plans to correctly support SDF (Read), including portability of
other runners.

Yes, this might be a temporary issue, but the fact, that expansion
is runner dependent remains valid, because such situation might
reappear.

 Jan

[1]

https://github.com/apache/beam/pull/15082/commits/5a46664ceb9f03da3089925b30ecd0a802e8b3eb

<https://github.com/apache/beam/pull/15082/commits/5a46664ceb9f03da3089925b30ecd0a802e8b3eb>

On 7/1/21 9:33 AM, Jan Lukavský wrote:

On 7/1/21 3:26 AM, Kyle Weaver wrote:


I think it should accept complete list of PipelineOptions
(or at least some defined subset -
PortabilityPipelineOptions, ExperimentalOptions, ...?)


I'm not totally opposed to redefining some options, either.
Using PipelineOptions could be confusing because only very few
options would actually be respected -- even
PortablePipelineOptions includes many options that wouldn't
make sense in this context. Maybe better to have a small list of
options that are guaranteed to work.


That makes sense. How would we define the subset? I think that
would probably require some sort of annotation analogous to
@Validation.Required, maybe @Validation.ExpansionSupported or
similar. I'm fine with implementing that, but I would need now to
get the 'hotfix' to upcoming 2.32.0 release. Could we make that
for 2.33.0? Will you help me review the current PR [1]?

[1] https://github.com/apache/beam/pull/15082
<https://github.com/apache/beam/pull/15082>



On Wed, Jun 30, 2021 at 8:48 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

 > Not sure why we need the hacks with NoOpRunner

As noted earlier (and that was why I started this thread in
the first
place :)), adding :runners:direct-java as runtime dependency
of the
expansion service causes something like 200 tests in
pre-commit to fail.
Looks like there is some kind of conflict among Flink and
Direct runner.
I didn't dig too deep into that, though.

 > You could use the Python utilities in your script to
start/stop it
manually.

Yes, that is possible. I'll probably follow that path.

 > This is where the runner's ability to customize
environments would
come in handy--e.g. a Java runner could decide to swap out
the Java
docker environment for EMBEDDED or LOOPBACK (and a
Python-based runner
could do the same for the Python docker env).

That would be just perfect, as that would make it possible
to finally
unify 'classical' and 'portable' runners. But that is a
whole different
story. :)

  Jan

    On 6/30/21 5:35 PM, Robert Bradshaw wrote:
> On Wed, Jun 30, 2021 at 7:41 AM Jan Lukavský
m

Re: Specifying environment for cross-language transform expansion

2021-07-01 Thread Jan Lukavský

Hi,

after today's experience I think I have some arguments about why we 
*should* pass (at least some) of the PipelineOptions from SDK to 
expansion service.


 1) there are lots, and lots, and lots of bugs around SDF and around 
the "use_deprecated_read", sorry, but the switch to SDF as the default 
*way* too premature


 2) therefore, the expansion *is* runner dependent (because whether to 
use "use_deprecated_read" or not is runner dependent), only the client 
of the expansion service (the SDK, the driver code) knows the target 
runner - i.e. if the target runner can use "new" Read or "deprecated" Read


 3) currently, my opinion is that we hold many portable Flink users on 
2.24.0, because from 2.25.0, the combination of Kafka + Python SDK + 
Flink is simply not working - until now, there is no way to pass 
arguments to expansion service, and even after that, 
"use_deprecated_read" is simply ignored by the service (pretty much the 
same as was in DirectRunner, see [1])


We should consider making use_deprecated_read the default for Flink (at 
least), not sure what is the state of other runners regarding that. It 
would be good to rename it, if we do not have plans to correctly support 
SDF (Read), including portability of other runners.


Yes, this might be a temporary issue, but the fact, that expansion is 
runner dependent remains valid, because such situation might reappear.


 Jan

[1] 
https://github.com/apache/beam/pull/15082/commits/5a46664ceb9f03da3089925b30ecd0a802e8b3eb


On 7/1/21 9:33 AM, Jan Lukavský wrote:

On 7/1/21 3:26 AM, Kyle Weaver wrote:


I think it should accept complete list of PipelineOptions (or at
least some defined subset - PortabilityPipelineOptions,
ExperimentalOptions, ...?)


I'm not totally opposed to redefining some options, either. Using 
PipelineOptions could be confusing because only very few options 
would actually be respected -- even PortablePipelineOptions includes 
many options that wouldn't make sense in this context. Maybe better 
to have a small list of options that are guaranteed to work.


That makes sense. How would we define the subset? I think that would 
probably require some sort of annotation analogous to 
@Validation.Required, maybe @Validation.ExpansionSupported or similar. 
I'm fine with implementing that, but I would need now to get the 
'hotfix' to upcoming 2.32.0 release. Could we make that for 2.33.0? 
Will you help me review the current PR [1]?


[1] https://github.com/apache/beam/pull/15082



On Wed, Jun 30, 2021 at 8:48 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


 > Not sure why we need the hacks with NoOpRunner

As noted earlier (and that was why I started this thread in the
first
place :)), adding :runners:direct-java as runtime dependency of the
expansion service causes something like 200 tests in pre-commit
to fail.
Looks like there is some kind of conflict among Flink and Direct
runner.
I didn't dig too deep into that, though.

 > You could use the Python utilities in your script to
start/stop it
manually.

Yes, that is possible. I'll probably follow that path.

 > This is where the runner's ability to customize environments
would
come in handy--e.g. a Java runner could decide to swap out the Java
docker environment for EMBEDDED or LOOPBACK (and a Python-based
runner
could do the same for the Python docker env).

That would be just perfect, as that would make it possible to
finally
unify 'classical' and 'portable' runners. But that is a whole
different
story. :)

  Jan

On 6/30/21 5:35 PM, Robert Bradshaw wrote:
> On Wed, Jun 30, 2021 at 7:41 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:
>>> java -jar beam-sdks-java-io-expansion-service-2.30.0.jar 
>> This does not accept any other parameters than the port. That
is the first part of this thread - the intent was to enable this
to accept additional arguments, but there are (still waiting to
be addressed unresolved) issues. There currently even seems to be
no other way to adapt ExpansionService than to copy the
code and modify it, because it simply is not extensible. What
would be enough is wrapping Pipeline.create() [1] call to a
protected method, or add (protected) constructor that would
accept PipelineOptions (probably better in this regard). That
would make it more easy for users to create customized
ExpansionService and it would (sort of) help solving described
issues.
> Yes, let's make it easy to extend/customize/start up a custom
> ExpansionService, including adding optional command line
arguments to
> the "default" one. Not sure why we need the hacks with NoOpRunner
> (IMHO, the direct runner should just be part of the SDK, but that's
> not where 

Re: Specifying environment for cross-language transform expansion

2021-07-01 Thread Jan Lukavský

On 7/1/21 3:26 AM, Kyle Weaver wrote:


I think it should accept complete list of PipelineOptions (or at
least some defined subset - PortabilityPipelineOptions,
ExperimentalOptions, ...?)


I'm not totally opposed to redefining some options, either. Using 
PipelineOptions could be confusing because only very few options would 
actually be respected -- even PortablePipelineOptions includes many 
options that wouldn't make sense in this context. Maybe better to have 
a small list of options that are guaranteed to work.


That makes sense. How would we define the subset? I think that would 
probably require some sort of annotation analogous to 
@Validation.Required, maybe @Validation.ExpansionSupported or similar. 
I'm fine with implementing that, but I would need now to get the 
'hotfix' to upcoming 2.32.0 release. Could we make that for 2.33.0? Will 
you help me review the current PR [1]?


[1] https://github.com/apache/beam/pull/15082



On Wed, Jun 30, 2021 at 8:48 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


 > Not sure why we need the hacks with NoOpRunner

As noted earlier (and that was why I started this thread in the first
place :)), adding :runners:direct-java as runtime dependency of the
expansion service causes something like 200 tests in pre-commit to
fail.
Looks like there is some kind of conflict among Flink and Direct
runner.
I didn't dig too deep into that, though.

 > You could use the Python utilities in your script to start/stop it
manually.

Yes, that is possible. I'll probably follow that path.

 > This is where the runner's ability to customize environments would
come in handy--e.g. a Java runner could decide to swap out the Java
docker environment for EMBEDDED or LOOPBACK (and a Python-based
runner
could do the same for the Python docker env).

That would be just perfect, as that would make it possible to finally
unify 'classical' and 'portable' runners. But that is a whole
different
story. :)

  Jan

On 6/30/21 5:35 PM, Robert Bradshaw wrote:
> On Wed, Jun 30, 2021 at 7:41 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:
>>> java -jar beam-sdks-java-io-expansion-service-2.30.0.jar 
>> This does not accept any other parameters than the port. That
is the first part of this thread - the intent was to enable this
to accept additional arguments, but there are (still waiting to be
addressed unresolved) issues. There currently even seems to be no
other way to adapt ExpansionService than to copy the code
and modify it, because it simply is not extensible. What would be
enough is wrapping Pipeline.create() [1] call to a protected
method, or add (protected) constructor that would accept
PipelineOptions (probably better in this regard). That would make
it more easy for users to create customized ExpansionService and
it would (sort of) help solving described issues.
> Yes, let's make it easy to extend/customize/start up a custom
> ExpansionService, including adding optional command line
arguments to
> the "default" one. Not sure why we need the hacks with NoOpRunner
> (IMHO, the direct runner should just be part of the SDK, but that's
> not where we live now).
>
>> But even if we do that, we still need to deal with the
expansion service on two places:
>>
>>   a) run it (and stop it)
>>
>>   b) specify it in the
>>
>> Using the default expansion service is much, much easier, it is
started and stopped automatically for the user. Morever, the
JavaJarExpansionService actually even presumes that there can be
additional arguments passed to the service ([2]), the
ExpansionService only does not accept them (and kafka IO does not
expose that - that could be worked-around by users by manually
creating the JavaJarExpansionService from own jar, yes). I would
find it natural to add the command-line parsing (somehow!) to the
ExpansionService itself, so that it doesn't need end-user
modifications and then to figure out how to most easily expose
there command-line arguments to end-users.
> You could use the Python utilities in your script to start/stop
it manually.
>
>> Yes, I verified that Flink can use Python Kafka IO over PROCESS
environment with some hacking of the ExpansionService as shown in
one of the linked PRs (though there is probably still some bugs
regarding SDF - [3]). Adding --experiments seems have the same
issues, need expose that to the CLI of ExpansionService. And I'm
not sure if this [4] is not in conflict with
--experiments=use_deprecated_read. That is something I still need
to investigate.
>>
>> LOOPBACK is currently not supported by Flink. That is
   

Re: Specifying environment for cross-language transform expansion

2021-06-30 Thread Jan Lukavský

> Not sure why we need the hacks with NoOpRunner

As noted earlier (and that was why I started this thread in the first 
place :)), adding :runners:direct-java as runtime dependency of the 
expansion service causes something like 200 tests in pre-commit to fail. 
Looks like there is some kind of conflict among Flink and Direct runner. 
I didn't dig too deep into that, though.


> You could use the Python utilities in your script to start/stop it 
manually.


Yes, that is possible. I'll probably follow that path.

> This is where the runner's ability to customize environments would 
come in handy--e.g. a Java runner could decide to swap out the Java 
docker environment for EMBEDDED or LOOPBACK (and a Python-based runner 
could do the same for the Python docker env).


That would be just perfect, as that would make it possible to finally 
unify 'classical' and 'portable' runners. But that is a whole different 
story. :)


 Jan

On 6/30/21 5:35 PM, Robert Bradshaw wrote:

On Wed, Jun 30, 2021 at 7:41 AM Jan Lukavský  wrote:

java -jar beam-sdks-java-io-expansion-service-2.30.0.jar 

This does not accept any other parameters than the port. That is the first part of 
this thread - the intent was to enable this to accept additional arguments, but 
there are (still waiting to be addressed unresolved) issues. There currently even 
seems to be no other way to adapt ExpansionService than to copy the code 
and modify it, because it simply is not extensible. What would be enough is 
wrapping Pipeline.create() [1] call to a protected method, or add (protected) 
constructor that would accept PipelineOptions (probably better in this regard). 
That would make it more easy for users to create customized ExpansionService and it 
would (sort of) help solving described issues.

Yes, let's make it easy to extend/customize/start up a custom
ExpansionService, including adding optional command line arguments to
the "default" one. Not sure why we need the hacks with NoOpRunner
(IMHO, the direct runner should just be part of the SDK, but that's
not where we live now).


But even if we do that, we still need to deal with the expansion service on two 
places:

  a) run it (and stop it)

  b) specify it in the

Using the default expansion service is much, much easier, it is started and 
stopped automatically for the user. Morever, the JavaJarExpansionService 
actually even presumes that there can be additional arguments passed to the 
service ([2]), the ExpansionService only does not accept them (and kafka IO 
does not expose that - that could be worked-around by users by manually 
creating the JavaJarExpansionService from own jar, yes). I would find it 
natural to add the command-line parsing (somehow!) to the ExpansionService 
itself, so that it doesn't need end-user modifications and then to figure out 
how to most easily expose there command-line arguments to end-users.

You could use the Python utilities in your script to start/stop it manually.


Yes, I verified that Flink can use Python Kafka IO over PROCESS environment 
with some hacking of the ExpansionService as shown in one of the linked PRs 
(though there is probably still some bugs regarding SDF - [3]). Adding 
--experiments seems have the same issues, need expose that to the CLI of 
ExpansionService. And I'm not sure if this [4] is not in conflict with 
--experiments=use_deprecated_read. That is something I still need to 
investigate.

LOOPBACK is currently not supported by Flink. That is nice-to-have feature.

Local Flink does support LOOPBACK mode. If you just want to run
locally, just specifying "FlinkRunner" is enough. It's distributed
Flink that does not. It seems a lot of complexities are due to trying
to using minikube, which acts like it's distributed, but trying to
make it as easy as if it were all local (and the docker deficiencies
as well, which would make it just work...) Which is a worthy goal.

This is where the runner's ability to customize environments would
come in handy--e.g. a Java runner could decide to swap out the Java
docker environment for EMBEDDED or LOOPBACK (and a Python-based runner
could do the same for the Python docker env).


[1] 
https://github.com/apache/beam/blob/f9a4bfcb027f2e3a8e32578adf49981aeef3586a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L394

[2] 
https://github.com/apache/beam/blob/f9a4bfcb027f2e3a8e32578adf49981aeef3586a/sdks/python/apache_beam/transforms/external.py#L481

[3] https://issues.apache.org/jira/browse/BEAM-11998

[4] 
https://github.com/apache/beam/blob/b86fcf94af26a240777f30f8193a314cb7ffc87e/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L398

On 6/30/21 3:57 PM, Chamikara Jayalath wrote:



On Wed, Jun 30, 2021 at 6:54 AM Chamikara Jayalath  wrote:



On Wed, Jun 30, 2021 at 1:20 AM Jan Lukavský  wrote:

On 6/30/21 1:16 AM, Robert Bradshaw wrote:

Why doesn't docke

Re: Specifying environment for cross-language transform expansion

2021-06-30 Thread Jan Lukavský
> I'm totally fine with changes to ExpansionService (java) to support 
additional features.


Looks like this is consensus, I'm with it as well, for the first round. 
The problem is how exactly to modify it. I think it should accept 
complete list of PipelineOptions (or at least some defined subset - 
PortabilityPipelineOptions, ExperimentalOptions, ...?), the problem is 
that in order to use Pipeline.create(options), the options *must* 
include runner. Adding :runners:direct-java to dependency of 
:sdks:java:expansion_service does not work, because it conflicts with 
other runners.


I just modified the PR [1] to include a NoOpRunner, which seems to 
resolve the issues and the tests pass. Feels a little hackish, but I 
might be OK with it.


[1] 
https://github.com/apache/beam/pull/15082/files#diff-fc9fd0c0d7d5f3ad6d5db7ec63ca1d75080a6527fd053a2ad36333d760da5b70R552



On 6/30/21 4:46 PM, Chamikara Jayalath wrote:



On Wed, Jun 30, 2021 at 7:41 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


> java -jar beam-sdks-java-io-expansion-service-2.30.0.jar 

This does not accept any other parameters than the port. That is
the first part of this thread - the intent was to enable this to
accept additional arguments, but there are (still waiting to be
addressed unresolved) issues. There currently even seems to be no
other way to adapt ExpansionService than to copy the code
and modify it, because it simply is not extensible. What would be
enough is wrapping Pipeline.create() [1] call to a protected
method, or add (protected) constructor that would accept
PipelineOptions (probably better in this regard). That would make
it more easy for users to create customized ExpansionService and
it would (sort of) help solving described issues.


I'm totally fine with changes to ExpansionService (java) to support 
additional features.


But even if we do that, we still need to deal with the expansion
service on two places:

 a) run it (and stop it)

 b) specify it in the

Using the default expansion service is much, much easier, it is
started and stopped automatically for the user. Morever, the
JavaJarExpansionService actually even presumes that there can be
additional arguments passed to the service ([2]), the
ExpansionService only does not accept them (and kafka IO does not
expose that - that could be worked-around by users by manually
creating the JavaJarExpansionService from own jar, yes). I would
find it natural to add the command-line parsing (somehow!) to the
ExpansionService itself, so that it doesn't need end-user
modifications and then to figure out how to most easily expose
there command-line arguments to end-users.

> Or PROCESS mode.

Yes, I verified that Flink can use Python Kafka IO over PROCESS
environment with some hacking of the ExpansionService as shown in
one of the linked PRs (though there is probably still some bugs
regarding SDF - [3]). Adding --experiments seems have the same
issues, need expose that to the CLI of ExpansionService. And I'm
not sure if this [4] is not in conflict with
--experiments=use_deprecated_read. That is something I still need
to investigate.


This is very good to know. Thanks.


LOOPBACK is currently not supported by Flink. That is nice-to-have
feature.

 Jan

[1]

https://github.com/apache/beam/blob/f9a4bfcb027f2e3a8e32578adf49981aeef3586a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L394

<https://github.com/apache/beam/blob/f9a4bfcb027f2e3a8e32578adf49981aeef3586a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L394>

[2]

https://github.com/apache/beam/blob/f9a4bfcb027f2e3a8e32578adf49981aeef3586a/sdks/python/apache_beam/transforms/external.py#L481

<https://github.com/apache/beam/blob/f9a4bfcb027f2e3a8e32578adf49981aeef3586a/sdks/python/apache_beam/transforms/external.py#L481>

[3] https://issues.apache.org/jira/browse/BEAM-11998
<https://issues.apache.org/jira/browse/BEAM-11998>

[4]

https://github.com/apache/beam/blob/b86fcf94af26a240777f30f8193a314cb7ffc87e/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L398

<https://github.com/apache/beam/blob/b86fcf94af26a240777f30f8193a314cb7ffc87e/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L398>

On 6/30/21 3:57 PM, Chamikara Jayalath wrote:



On Wed, Jun 30, 2021 at 6:54 AM Chamikara Jayalath
mailto:chamik...@google.com>> wrote:



On Wed, Jun 30, 2021 at 1:20 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

On 6/30/21 1:16 AM, Robert Bradshaw wrote:
> Why doesn't docker in docker just work, rather

Re: Specifying environment for cross-language transform expansion

2021-06-30 Thread Jan Lukavský

> java -jar beam-sdks-java-io-expansion-service-2.30.0.jar 

This does not accept any other parameters than the port. That is the 
first part of this thread - the intent was to enable this to accept 
additional arguments, but there are (still waiting to be addressed 
unresolved) issues. There currently even seems to be no other way to 
adapt ExpansionService than to copy the code and modify it, 
because it simply is not extensible. What would be enough is wrapping 
Pipeline.create() [1] call to a protected method, or add (protected) 
constructor that would accept PipelineOptions (probably better in this 
regard). That would make it more easy for users to create customized 
ExpansionService and it would (sort of) help solving described issues.


But even if we do that, we still need to deal with the expansion service 
on two places:


 a) run it (and stop it)

 b) specify it in the

Using the default expansion service is much, much easier, it is started 
and stopped automatically for the user. Morever, the 
JavaJarExpansionService actually even presumes that there can be 
additional arguments passed to the service ([2]), the ExpansionService 
only does not accept them (and kafka IO does not expose that - that 
could be worked-around by users by manually creating the 
JavaJarExpansionService from own jar, yes). I would find it natural to 
add the command-line parsing (somehow!) to the ExpansionService itself, 
so that it doesn't need end-user modifications and then to figure out 
how to most easily expose there command-line arguments to end-users.


> Or PROCESS mode.

Yes, I verified that Flink can use Python Kafka IO over PROCESS 
environment with some hacking of the ExpansionService as shown in one of 
the linked PRs (though there is probably still some bugs regarding SDF - 
[3]). Adding --experiments seems have the same issues, need expose that 
to the CLI of ExpansionService. And I'm not sure if this [4] is not in 
conflict with --experiments=use_deprecated_read. That is something I 
still need to investigate.


LOOPBACK is currently not supported by Flink. That is nice-to-have feature.

 Jan

[1] 
https://github.com/apache/beam/blob/f9a4bfcb027f2e3a8e32578adf49981aeef3586a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L394


[2] 
https://github.com/apache/beam/blob/f9a4bfcb027f2e3a8e32578adf49981aeef3586a/sdks/python/apache_beam/transforms/external.py#L481


[3] https://issues.apache.org/jira/browse/BEAM-11998

[4] 
https://github.com/apache/beam/blob/b86fcf94af26a240777f30f8193a314cb7ffc87e/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L398


On 6/30/21 3:57 PM, Chamikara Jayalath wrote:



On Wed, Jun 30, 2021 at 6:54 AM Chamikara Jayalath 
mailto:chamik...@google.com>> wrote:




On Wed, Jun 30, 2021 at 1:20 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

On 6/30/21 1:16 AM, Robert Bradshaw wrote:
> Why doesn't docker in docker just work, rather than
having to do
> ugly hacks when composing two technologies that both rely on
> docker...
>
> Presumably you're setting up a node for Kafka and Flink; why
not set
> one up for the expansion service as well? The UX of
>
>
ReadFromKafka(default_expansion_service_args={"defaultEnvironmentType":
> "PROCESS", "defaultEnvironmentConfig": ""{\"os\": \"linux\",
\"arch\":
> \"amd64\", \"command\": \"/path/to/launcher/boot
> cp=/some/other/long/path\" ...}")"})
>
> isn't that great either. Rather than pass arbitrary
arguments to a
> default expansion service, I still think once you get to
this level
> it's better to just start your own expansion service.

Sure, that is possible (seems to me, that it would still
require some
changes to ExpansionService to be extendable, but yes, kind of
tiny
changes). The problem is not with Flink or Kafka - those are
technologies you are actually expecting to set up, because you
want to
use them. The problem is what everything else you must set up
for making
something that seems as easy as "read a few messages from
kafka in beam
python" to work. You must have:

  a) Python SDK harness (OK, that is something that should be
probably
expected) - there are few problems with it, namely it is somewhat
hardcoded that it must run in the same pod as Flink's
taskmanager to be
able to use EXTERNAL environment, but ok, let's go on

  b) Java SDK harness, at least installed in docker image of
taskmanager
(to be

Re: Specifying environment for cross-language transform expansion

2021-06-30 Thread Jan Lukavský

On 6/30/21 1:16 AM, Robert Bradshaw wrote:

Why doesn't docker in docker just work, rather than having to do
ugly hacks when composing two technologies that both rely on
docker...

Presumably you're setting up a node for Kafka and Flink; why not set
one up for the expansion service as well? The UX of

 ReadFromKafka(default_expansion_service_args={"defaultEnvironmentType":
"PROCESS", "defaultEnvironmentConfig": ""{\"os\": \"linux\", \"arch\":
\"amd64\", \"command\": \"/path/to/launcher/boot
cp=/some/other/long/path\" ...}")"})

isn't that great either. Rather than pass arbitrary arguments to a
default expansion service, I still think once you get to this level
it's better to just start your own expansion service.


Sure, that is possible (seems to me, that it would still require some 
changes to ExpansionService to be extendable, but yes, kind of tiny 
changes). The problem is not with Flink or Kafka - those are 
technologies you are actually expecting to set up, because you want to 
use them. The problem is what everything else you must set up for making 
something that seems as easy as "read a few messages from kafka in beam 
python" to work. You must have:


 a) Python SDK harness (OK, that is something that should be probably 
expected) - there are few problems with it, namely it is somewhat 
hardcoded that it must run in the same pod as Flink's taskmanager to be 
able to use EXTERNAL environment, but ok, let's go on


 b) Java SDK harness, at least installed in docker image of taskmanager 
(to be usable via PROCESS environment) - OK, that starts to be weird, 
taskmanager is java, right? Something like LOOPBACK would be cool there, 
but never mind. You create custom docker image for your Flink JM and TM 
and continue.


 c) Implement (extend) and deploy own expansion service - ouch, that 
starts to hurt, that is even going to be a pod that is running even 
though there is nothing using it (yes, can be scaled down).


The complexity of a simple task starts to be somewhat extraordinary. And 
most of the users will not be willing to follow this path, I'm afraid. 
People generally don't like to set up complex environment for something 
that looks it should "just work".  There is non-trivial work necessary 
to make all of this working, mostly when you are starting to evaluate 
Beam and don't have much experience with it.


We can get rid of b) (implement LOOPBACK in Flink) and c) (enable Python 
SDK Kafka IO to spawn expansion service with the LOOPBACK environment 
when submitting to Flink). That is why I still think that this 
simplification matters a lot.




On Tue, Jun 29, 2021 at 3:33 PM Jan Lukavský  wrote:

I believe we could change that more or less the same as we can deprecate / stop 
supporting any other parameter of any method. If python starts to support 
natively Kafka IO, then we can simply log warning / raise exception (one after 
the other). That seems like natural development.

Maybe I should have described the case - I'm trying to setup a "simple" use-case for 
users that want to try Python SDK to read using Flink from Kafka using Minikube (both Kafka and 
Flink are running inside Minikube). There are tons of problems to use docker from within Minkube 
and I would not say that is the "simple" way we would like to present to users. Setting 
up own expansion service is possibility - but that also lacks the UX approach. I pretty much think 
that understanding portability on it's own is already a burden we put on users (yes, we do that for 
a reason, but everything else should be as simple as possible).

On 6/30/21 12:16 AM, Chamikara Jayalath wrote:

So I think one downside to this PR is that we assume that the default expansion 
service used by the transform (Kafka in this case) will not change. Currently 
it's fully opaque. In the default case we just promise that the transform will 
work (if conditions I mentioned above are met). Nothing else.
If we add a "param default_expansion_service_args", we leak the nature of the 
default expansion service to the API and it will be hard to change it in the future.

Thanks,
Cham

On Tue, Jun 29, 2021 at 3:07 PM Jan Lukavský  wrote:

I would absolutely understand this, if it would be mostly impossible or at 
least really hard to get the user friendly behavior. But we are mostly there in 
this case. When we can actually quite simply pass the supported environment via 
parameter, I think we should go for it.

I have created a sketch (I verified that when the ExpansionService is patched 
'enough' this works) in [1]. This is only a sketch, because we first must know 
how to support the default execution environment in ExpansionService.

[1] https://github.com/apache/beam/pull/15099/files

On 6/29/21 11:51 PM, Chamikara Jayalath wrote:



On Tue, Jun 29, 2021 at 2:39 PM Jan Lukavský  wrote

Re: Specifying environment for cross-language transform expansion

2021-06-29 Thread Jan Lukavský
I believe we could change that more or less the same as we can deprecate 
/ stop supporting any other parameter of any method. If python starts to 
support natively Kafka IO, then we can simply log warning / raise 
exception (one after the other). That seems like natural development.


Maybe I should have described the case - I'm trying to setup a "simple" 
use-case for users that want to try Python SDK to read using Flink from 
Kafka using Minikube (both Kafka and Flink are running inside Minikube). 
There are tons of problems to use docker from within Minkube and I would 
not say that is the "simple" way we would like to present to users. 
Setting up own expansion service is possibility - but that also lacks 
the UX approach. I pretty much think that understanding portability on 
it's own is already a burden we put on users (yes, we do that for a 
reason, but everything else should be as simple as possible).


On 6/30/21 12:16 AM, Chamikara Jayalath wrote:
So I think one downside to this PR is that we assume that the default 
expansion service used by the transform (Kafka in this case) will not 
change. Currently it's fully opaque. In the default case we just 
promise that the transform will work (if conditions I mentioned above 
are met). Nothing else.
If we add a "param default_expansion_service_args", we leak the nature 
of the default expansion service to the API and it will be hard to 
change it in the future.


Thanks,
Cham

On Tue, Jun 29, 2021 at 3:07 PM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


I would absolutely understand this, if it would be mostly
impossible or at least really hard to get the user friendly
behavior. But we are mostly there in this case. When we can
actually quite simply pass the supported environment via
parameter, I think we should go for it.

I have created a sketch (I verified that when the ExpansionService
is patched 'enough' this works) in [1]. This is only a sketch,
because we first must know how to support the default execution
environment in ExpansionService.

[1] https://github.com/apache/beam/pull/15099/files
<https://github.com/apache/beam/pull/15099/files>

On 6/29/21 11:51 PM, Chamikara Jayalath wrote:



    On Tue, Jun 29, 2021 at 2:39 PM Jan Lukavský mailto:je...@seznam.cz>> wrote:

On 6/29/21 11:04 PM, Robert Bradshaw wrote:
> You can configure the environment in the current state, you
just have
> to run your own expansion service that has a different
environment
> backed into it (or, makes this configurable).
Yes, that is true. On the other hand that lacks some
user-friendliness,
because ideally, you don't want to worry about expansion
services,
mostly when it comes to some mostly standard IO. The ideal
case is that
you either do not basically know that you use external
transform (which
is probably the case when you can use docker), or you are
able to
overcome the problem within the SDK (Python) by passing some
argument to
the input transform.


Arguments passed to the pipeline level apply to the whole
pipeline (not just one transform). So if you pass in a default
environment (and configs) at pipeline level, that would mean the
default environment and configs used by the pipeline (so Python
SDK in this case) not a specific transform.
I believe we have made usage of external transforms used-friendly
for the general case. But we had to make some assumptions. For
example we assumed,
* user will be using the default environment of the expansion
service (Docker in this case)
* User will be using the pre-specified dependency only
(sdks:java:io:expansion-service:shadowJar for Kafka)
* User will be in an environment where the jar can be downloaded.

I would consider any use-case where these basic assumptions
cannot be met as an advanced use-case. The solution in such a
case would be to start a custom expansion service and pass the
address of it as a parameter to the transform [1]. I'm fine with
extending the capabilities of Java expansion service by adding
more parameters (for example, for overriding the environment, for
specifying dependencies, for providing pipeline options).

Thanks,
Cham

[1]

https://github.com/apache/beam/blob/b86fcf94af26a240777f30f8193a314cb7ffc87e/sdks/python/apache_beam/io/kafka.py#L133

<https://github.com/apache/beam/blob/b86fcf94af26a240777f30f8193a314cb7ffc87e/sdks/python/apache_beam/io/kafka.py#L133>

>
> Is option (1) updating the default expansion service such
that one can
> override default environment properties on the command
line? (You
> would still have to start it up manually to use it.)
Ye

Re: Specifying environment for cross-language transform expansion

2021-06-29 Thread Jan Lukavský
I would absolutely understand this, if it would be mostly impossible or 
at least really hard to get the user friendly behavior. But we are 
mostly there in this case. When we can actually quite simply pass the 
supported environment via parameter, I think we should go for it.


I have created a sketch (I verified that when the ExpansionService is 
patched 'enough' this works) in [1]. This is only a sketch, because we 
first must know how to support the default execution environment in 
ExpansionService.


[1] https://github.com/apache/beam/pull/15099/files

On 6/29/21 11:51 PM, Chamikara Jayalath wrote:



On Tue, Jun 29, 2021 at 2:39 PM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


On 6/29/21 11:04 PM, Robert Bradshaw wrote:
> You can configure the environment in the current state, you just
have
> to run your own expansion service that has a different environment
> backed into it (or, makes this configurable).
Yes, that is true. On the other hand that lacks some
user-friendliness,
because ideally, you don't want to worry about expansion services,
mostly when it comes to some mostly standard IO. The ideal case is
that
you either do not basically know that you use external transform
(which
is probably the case when you can use docker), or you are able to
overcome the problem within the SDK (Python) by passing some
argument to
the input transform.


Arguments passed to the pipeline level apply to the whole pipeline 
(not just one transform). So if you pass in a default environment (and 
configs) at pipeline level, that would mean the default environment 
and configs used by the pipeline (so Python SDK in this case) not a 
specific transform.
I believe we have made usage of external transforms used-friendly for 
the general case. But we had to make some assumptions. For example we 
assumed,
* user will be using the default environment of the expansion service 
(Docker in this case)
* User will be using the pre-specified dependency only 
(sdks:java:io:expansion-service:shadowJar for Kafka)

* User will be in an environment where the jar can be downloaded.

I would consider any use-case where these basic assumptions cannot be 
met as an advanced use-case. The solution in such a case would be to 
start a custom expansion service and pass the address of it as a 
parameter to the transform [1]. I'm fine with extending the 
capabilities of Java expansion service by adding more parameters (for 
example, for overriding the environment, for specifying dependencies, 
for providing pipeline options).


Thanks,
Cham

[1] 
https://github.com/apache/beam/blob/b86fcf94af26a240777f30f8193a314cb7ffc87e/sdks/python/apache_beam/io/kafka.py#L133 
<https://github.com/apache/beam/blob/b86fcf94af26a240777f30f8193a314cb7ffc87e/sdks/python/apache_beam/io/kafka.py#L133>


>
> Is option (1) updating the default expansion service such that
one can
> override default environment properties on the command line? (You
> would still have to start it up manually to use it.)
Yes and no. :) Updating ExpansionService so that you can specify
default
environment on command like makes this accessible to
JavaJarExpansionService, and that makes it possible to add (optional)
argument to Python Kafka IO, that would delegate this to the
(automatically) started expansion service. It is important to note
that
both ReadFromKafka and WriteToKafka have expansion that involves only
single external (Java) SDK. That simplifies things.
>
> Maybe it would help to make things more concrete. Suppose I have
a Go
> pipeline that uses a library which invokes a Python external
transform
> to do ML (say, via TFX), and two Java IOs (which happen to have
> mutually exclusive dependencies). The ML transform itself uses
Java to
> invoke some SQL.
>
> The way things work currently is each external transform will
have an
> associated fully specified environment and a runner can use
docker to
> start up the required workers at the expected time.
>
> Now, suppose one doesn't have docker on the workers. One wants
to run this with
>
>      ./my_pipeline --someFlag=someValue
--someOtherFlag=someOtherValue ...
>
> such that docker is no longer needed. What someFlags would we need,
> and what would their values be? (And how to make this feasible to
> implement.)
>
> Are there meaningful intermediate points that extend to a general
> solution (or at least aren't hostile to it)?
I believe that in the option 2) the best way would to use each
SDK's URN
Then the arguments could be something like
"--expansionEnvironments={"apache:beam:go:2.33.0:latest"={"env"="DOCKER",

config=""}, "apache:

Re: Specifying environment for cross-language transform expansion

2021-06-29 Thread Jan Lukavský

On 6/29/21 11:04 PM, Robert Bradshaw wrote:

You can configure the environment in the current state, you just have
to run your own expansion service that has a different environment
backed into it (or, makes this configurable).
Yes, that is true. On the other hand that lacks some user-friendliness, 
because ideally, you don't want to worry about expansion services, 
mostly when it comes to some mostly standard IO. The ideal case is that 
you either do not basically know that you use external transform (which 
is probably the case when you can use docker), or you are able to 
overcome the problem within the SDK (Python) by passing some argument to 
the input transform.


Is option (1) updating the default expansion service such that one can
override default environment properties on the command line? (You
would still have to start it up manually to use it.)
Yes and no. :) Updating ExpansionService so that you can specify default 
environment on command like makes this accessible to 
JavaJarExpansionService, and that makes it possible to add (optional) 
argument to Python Kafka IO, that would delegate this to the 
(automatically) started expansion service. It is important to note that 
both ReadFromKafka and WriteToKafka have expansion that involves only 
single external (Java) SDK. That simplifies things.


Maybe it would help to make things more concrete. Suppose I have a Go
pipeline that uses a library which invokes a Python external transform
to do ML (say, via TFX), and two Java IOs (which happen to have
mutually exclusive dependencies). The ML transform itself uses Java to
invoke some SQL.

The way things work currently is each external transform will have an
associated fully specified environment and a runner can use docker to
start up the required workers at the expected time.

Now, suppose one doesn't have docker on the workers. One wants to run this with

 ./my_pipeline --someFlag=someValue --someOtherFlag=someOtherValue ...

such that docker is no longer needed. What someFlags would we need,
and what would their values be? (And how to make this feasible to
implement.)

Are there meaningful intermediate points that extend to a general
solution (or at least aren't hostile to it)?
I believe that in the option 2) the best way would to use each SDK's URN 
Then the arguments could be something like 
"--expansionEnvironments={"apache:beam:go:2.33.0:latest"={"env"="DOCKER", 
config=""}, "apache:beam:python:2.33.0:latest"={env="PROCESS", 
config={...}}". Yes, it would require a lot of "syntactic sugar" to 
configure that. :) (sorry if I don't have URNs for SDKs 100% correct)



I still think in the long run having runners understand environments,
and saying "oh, whenever I see 'apache:beam:java:2.33.0:latest' I'll
swap that out for 'path/to/my/java -cp ...' is the right way to go
long-term. (I would put this in runners, not SDKs, though a common
runners library could be used.)
Yes, I also agree, that expansion service should be runner-dependent (or 
at least runner-aware), as that brings optimizations. Runner could 
ignore settings from previous point when it can be *sure* it can do so.



On Tue, Jun 29, 2021 at 1:29 PM Jan Lukavský  wrote:

Thanks for pointing to that thread.

1) I'm - as well as Kyle - fine with the approach that we use a
"preferred environment" for the expansion service. We only need to pass
it via command line. Yes, the command line might be generally
SDK-dependent, and that makes it expansion dependent, because whether or
not particular transform is "external" or not is implementation detail.
That is the nasty part. The rest of my original question is about, how
exactly to do that, because it seems to be tricky, due to the fact, that
it is not possible to include runtime dependency on DirectRunner (fails
many, many tests) and it is not possible to extract PipelineOptions as a
Map either.

2) Regarding SDK injecting environment, I still think that is the
correct way. The SDK (the driver code) own the execution environment. It
should be able to define (or at least prioritize) runtime environments
of all transforms. If we cannot know in advance, which transform is
going to expand to how many nested (and possibly external) transforms, I
think that the SDK could be fine with providing a Map(SDK ->
environment). That is: "Run Java using PROCESS", "Run Python using
DOCKER", and so on. A default mapping might exist on the expansion
service as well (which might be passed through command line and that is
the point 1)). Yes, the Map approach is definitely not universal,
because one can imagine that the SDK itself is not enough for specifying
the environment, but seems that vast majority of cases would fit into that.

3) The best might be for the SDK to provide a list of supported
environments with additional metrics which the expansion service might
choose fro

Re: Specifying environment for cross-language transform expansion

2021-06-29 Thread Jan Lukavský

Thanks for pointing to that thread.

1) I'm - as well as Kyle - fine with the approach that we use a 
"preferred environment" for the expansion service. We only need to pass 
it via command line. Yes, the command line might be generally 
SDK-dependent, and that makes it expansion dependent, because whether or 
not particular transform is "external" or not is implementation detail. 
That is the nasty part. The rest of my original question is about, how 
exactly to do that, because it seems to be tricky, due to the fact, that 
it is not possible to include runtime dependency on DirectRunner (fails 
many, many tests) and it is not possible to extract PipelineOptions as a 
Map either.


2) Regarding SDK injecting environment, I still think that is the 
correct way. The SDK (the driver code) own the execution environment. It 
should be able to define (or at least prioritize) runtime environments 
of all transforms. If we cannot know in advance, which transform is 
going to expand to how many nested (and possibly external) transforms, I 
think that the SDK could be fine with providing a Map(SDK -> 
environment). That is: "Run Java using PROCESS", "Run Python using 
DOCKER", and so on. A default mapping might exist on the expansion 
service as well (which might be passed through command line and that is 
the point 1)). Yes, the Map approach is definitely not universal, 
because one can imagine that the SDK itself is not enough for specifying 
the environment, but seems that vast majority of cases would fit into that.


3) The best might be for the SDK to provide a list of supported 
environments with additional metrics which the expansion service might 
choose from.


These three approaches are all extensions to the current state. Current 
state has predefined environment without possibility to change it. 
Option 1) changes it to single configurable environment, option 2) to N 
environments based on SDK and option 3) to M environments based on 
SDK-dependent metrics (and/or capabilitites of particular environment). 
Seems like gradual extensions of the current state, so maybe we can 
focus on the first one, and maybe add other, when there is a need?


If this could be the first conclusion, then the next one would be, what 
should be the preferred way to implement it.


WDYT?

On 6/29/21 9:15 PM, Robert Bradshaw wrote:

+1, thanks for digging up that thread.

I am still of the same opinion that I wrote there. To touch on some
things brought up here, copying something like
defaultEnvironmentConfig doesn't make sense from language to language
(e.g. the docker image name or CLI arguments for subprocess mode just
isn't going to work for all of Python, Java, and Go, and clearly
embedded type is only going to work for one.)

In the short term, to change environment (or anything else) about the
"default" expansions service, the thing to do is build and start your
own expansion service that sets up the environment for its transforms
in a custom way.

FYI, in Python, one can use --beam_services to use a custom expansion
service. E.g.

--beam_services='{":sdks:java:extensions:sql:expansion-service:shadowJar":
"localhost:port"}'

would override the default one when using SqlTransform.

On Tue, Jun 29, 2021 at 11:47 AM Kyle Weaver  wrote:

For context, there was a previous thread which touched on many of the same 
points: 
https://lists.apache.org/thread.html/r6f6fc207ed62e1bf2a1d41deeeab554e35cd2af389ce38289a303cea%40%3Cdev.beam.apache.org%3E

On Tue, Jun 29, 2021 at 11:16 AM Jan Lukavský  wrote:

I would slightly disagree that this breaks the black box nature of the expansion, the "how the 
transform expands" remains unknown to the SDK requesting the expansion, the "how the 
transform executes" - on the other hand - is something that the SDK must cooperate on - it 
knows (or could or should know) what is the environment that the pipeline is going to be executed 
on looks like. That is why expansion service on its own cannot correctly define the execution 
environment. It could, if it would be bound to runner (and its environemnt) - for instance 
FlinkRunnerExpansionService could probably expand KafkaIO to something more 'native'. But that 
requires knowledge of the target runner. If the expansion service is not dedicated to a runner, the 
only place where it can be defined, is the SDK - and therefore the expansion request.


Power users can always modify the output produced by the expansion service as 
well.

I'm not sure if I follow this, do you mean that power users, who run the 
expansion service can modify the output? Or is the output (protobuf) of the 
expansion service easily transferable between different execution 
environments?- I had the impression, that execution environments do not 
necessarily have to have the same payloads associated with them, and therefore 
it is impossible to 'postprocess' the output of the e

Re: Specifying environment for cross-language transform expansion

2021-06-29 Thread Jan Lukavský
I would slightly disagree that this breaks the black box nature of the 
expansion, the "how the transform expands" remains unknown to the SDK 
requesting the expansion, the "how the transform executes" - on the 
other hand - is something that the SDK must cooperate on - it knows (or 
could or should know) what is the environment that the pipeline is going 
to be executed on looks like. That is why expansion service on its own 
cannot correctly define the execution environment. It could, if it would 
be bound to runner (and its environemnt) - for instance 
FlinkRunnerExpansionService could probably expand KafkaIO to something 
more 'native'. But that requires knowledge of the target runner. If the 
expansion service is not dedicated to a runner, the only place where it 
can be defined, is the SDK - and therefore the expansion request.


> Power users can always modify the output produced by the expansion 
service as well.


I'm not sure if I follow this, do you mean that power users, who run the 
expansion service can modify the output? Or is the output (protobuf) of 
the expansion service easily transferable between different execution 
environments?- I had the impression, that execution environments do not 
necessarily have to have the same payloads associated with them, and 
therefore it is impossible to 'postprocess' the output of the expansion. 
Is that wrong assumption?


On 6/29/21 7:55 PM, Luke Cwik wrote:
This would "break" the black box where the expansion service is 
supposed to hide the implementation internals from the caller and 
pushes compatibility of these kinds of environment overrides on to the 
expansion service and its implementer.


Power users can always modify the output produced by the expansion 
service as well.


On Tue, Jun 29, 2021 at 10:08 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


The argument for being able to accept (possibly ordered list of)
execution environments is in that this could make a single
instance of execution service reusable by various clients with
different requirements. Moreover, the two approaches are probably
orthogonal - users could specify 'defaultExecutionEnvironment' for
the service which could be used in case when there is no
preference given by the client.

On 6/29/21 7:03 PM, Luke Cwik wrote:

I would be much more inclined for the user being able to
configure the expansion service for their needs instead of
changing the expansion service API.

On Tue, Jun 29, 2021 at 9:42 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

If I understand it correctly, there is currently no place to
set the
defaultEnvironmentType - python's KafkaIO uses either
'expansion_service' given by the user (which might be a
host:port, or an
object that has appropriate method), or calls
'default_io_expansion_service' - which in turn runs
ExpansionService
using gradle. Either way, it ends up in ExpansionService#main
[1]. It
could be possible to adapt ExpansionService and call it
locally -
provided ExpansionService would provide a way to extend it
(using
protected method createPipeline()) seems to be enough - but
that is not
too much user-friendly. If we could specify the
defaultEnvironmentConfig
when starting the ExpansionService, it would be possible to
add these
parameters in the python SDK's KafkaIO, which would mean
users do not
have to worry about the expansion service at all (leaving
aside that
using too many ReafFromKafka or WriteToKafka transforms would
somewhat
hurt performance during pipeline build, but that applies to
the pipeline
build time only). I have created [2] to track that.

Does that make sense, or is my analysis incorrect?

  Jan

[1]

https://github.com/apache/beam/blob/22205ee1a84581e9206c5c61bad88a799779b4bc/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L511

<https://github.com/apache/beam/blob/22205ee1a84581e9206c5c61bad88a799779b4bc/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L511>

[2] https://issues.apache.org/jira/browse/BEAM-12539
<https://issues.apache.org/jira/browse/BEAM-12539>


On 6/29/21 6:24 PM, Alexey Romanenko wrote:
> I’m sorry if I missed something but do you mean that
PortablePipelineOptions.setDefaultEnvironmentType(String)
doesn’t work for you? Or it’s only a specific case while
using portable KafkaIO?
>
>> On 29 Jun 2021, at 09:51, Jan Lukavský
mailto:x666je...@gmail.com>> wrote:
>>
>> Hi,
>>
&

Re: Specifying environment for cross-language transform expansion

2021-06-29 Thread Jan Lukavský
The argument for being able to accept (possibly ordered list of) 
execution environments is in that this could make a single instance of 
execution service reusable by various clients with different 
requirements. Moreover, the two approaches are probably orthogonal - 
users could specify 'defaultExecutionEnvironment' for the service which 
could be used in case when there is no preference given by the client.


On 6/29/21 7:03 PM, Luke Cwik wrote:
I would be much more inclined for the user being able to configure the 
expansion service for their needs instead of changing the expansion 
service API.


On Tue, Jun 29, 2021 at 9:42 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


If I understand it correctly, there is currently no place to set the
defaultEnvironmentType - python's KafkaIO uses either
'expansion_service' given by the user (which might be a host:port,
or an
object that has appropriate method), or calls
'default_io_expansion_service' - which in turn runs ExpansionService
using gradle. Either way, it ends up in ExpansionService#main [1]. It
could be possible to adapt ExpansionService and call it locally -
provided ExpansionService would provide a way to extend it (using
protected method createPipeline()) seems to be enough - but that
is not
too much user-friendly. If we could specify the
defaultEnvironmentConfig
when starting the ExpansionService, it would be possible to add these
parameters in the python SDK's KafkaIO, which would mean users do not
have to worry about the expansion service at all (leaving aside that
using too many ReafFromKafka or WriteToKafka transforms would
somewhat
hurt performance during pipeline build, but that applies to the
pipeline
build time only). I have created [2] to track that.

Does that make sense, or is my analysis incorrect?

  Jan

[1]

https://github.com/apache/beam/blob/22205ee1a84581e9206c5c61bad88a799779b4bc/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L511

<https://github.com/apache/beam/blob/22205ee1a84581e9206c5c61bad88a799779b4bc/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L511>

[2] https://issues.apache.org/jira/browse/BEAM-12539
<https://issues.apache.org/jira/browse/BEAM-12539>


On 6/29/21 6:24 PM, Alexey Romanenko wrote:
> I’m sorry if I missed something but do you mean that
PortablePipelineOptions.setDefaultEnvironmentType(String) doesn’t
work for you? Or it’s only a specific case while using portable
KafkaIO?
>
>> On 29 Jun 2021, at 09:51, Jan Lukavský mailto:x666je...@gmail.com>> wrote:
>>
>> Hi,
>>
>> I have come across an issue with cross-language transforms. My
setup is I have working environment type PROCESS and I cannot use
DOCKER. When I use Python's KafkaIO, it unfortunately - by default
- expands to docker environment, which then fails due to missing
'docker' command. I didn't find a solution without tackling the
expansion service, yet.
>>
>> I see several possible solutions to that:
>>
>>   a) I would say, that the cleanest solution would be to add
preferred environment type to the expansion request to the
expansion service (probably along with additional flags, probably
--experiments?). This requires deeper changes to the expansion RPC
defintion, probably serializing the PipelineOptions from the
client environment into the ExpansionRequest.
>>
>>   b) Another option would be to allow specifying some of the
command-line arguments when starting the expansion service, which
currently accepts only port on command line, see [1]. The
straightforward 'fix' (see [2]) unfortunately does not work,
because it requires DirectRunner to be on the classpath, which
then breaks other runners (see [3]). It seems possible to copy
hand selected options from command line to the Pipeline, but that
feels hackish. It would require to either be able to construct the
Pipeline without a runner specified (which seems possible when
calling Pipeline.create(), but not when using PipelineOptions
create by parsing command-line arguments) or to be able to create
a Map from PIpelineOptions and then the ability to
copy all options into the Pipeline's options.
>>
>> My proposal would be to create a hackish shortcut and just copy
the --defaultEnvironmentType, --defaultEnvironmentConfig and
--experiments into Pipeline's options for now, and create an issue
for a proper solution (possible a)?).
>>
>> WDYT? Or did I miss a way to override the default expansion?
>>
>> Thanks for comments,
>>
>>   Jan
&

Re: Specifying environment for cross-language transform expansion

2021-06-29 Thread Jan Lukavský
If I understand it correctly, there is currently no place to set the 
defaultEnvironmentType - python's KafkaIO uses either 
'expansion_service' given by the user (which might be a host:port, or an 
object that has appropriate method), or calls 
'default_io_expansion_service' - which in turn runs ExpansionService 
using gradle. Either way, it ends up in ExpansionService#main [1]. It 
could be possible to adapt ExpansionService and call it locally - 
provided ExpansionService would provide a way to extend it (using 
protected method createPipeline()) seems to be enough - but that is not 
too much user-friendly. If we could specify the defaultEnvironmentConfig 
when starting the ExpansionService, it would be possible to add these 
parameters in the python SDK's KafkaIO, which would mean users do not 
have to worry about the expansion service at all (leaving aside that 
using too many ReafFromKafka or WriteToKafka transforms would somewhat 
hurt performance during pipeline build, but that applies to the pipeline 
build time only). I have created [2] to track that.


Does that make sense, or is my analysis incorrect?

 Jan

[1] 
https://github.com/apache/beam/blob/22205ee1a84581e9206c5c61bad88a799779b4bc/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L511


[2] https://issues.apache.org/jira/browse/BEAM-12539


On 6/29/21 6:24 PM, Alexey Romanenko wrote:

I’m sorry if I missed something but do you mean that 
PortablePipelineOptions.setDefaultEnvironmentType(String) doesn’t work for you? 
Or it’s only a specific case while using portable KafkaIO?


On 29 Jun 2021, at 09:51, Jan Lukavský  wrote:

Hi,

I have come across an issue with cross-language transforms. My setup is I have 
working environment type PROCESS and I cannot use DOCKER. When I use Python's 
KafkaIO, it unfortunately - by default - expands to docker environment, which 
then fails due to missing 'docker' command. I didn't find a solution without 
tackling the expansion service, yet.

I see several possible solutions to that:

  a) I would say, that the cleanest solution would be to add preferred 
environment type to the expansion request to the expansion service (probably 
along with additional flags, probably --experiments?). This requires deeper 
changes to the expansion RPC defintion, probably serializing the 
PipelineOptions from the client environment into the ExpansionRequest.

  b) Another option would be to allow specifying some of the command-line arguments 
when starting the expansion service, which currently accepts only port on command 
line, see [1]. The straightforward 'fix' (see [2]) unfortunately does not work, 
because it requires DirectRunner to be on the classpath, which then breaks other 
runners (see [3]). It seems possible to copy hand selected options from command line 
to the Pipeline, but that feels hackish. It would require to either be able to 
construct the Pipeline without a runner specified (which seems possible when calling 
Pipeline.create(), but not when using PipelineOptions create by parsing command-line 
arguments) or to be able to create a Map from PIpelineOptions 
and then the ability to copy all options into the Pipeline's options.

My proposal would be to create a hackish shortcut and just copy the 
--defaultEnvironmentType, --defaultEnvironmentConfig and --experiments into 
Pipeline's options for now, and create an issue for a proper solution (possible 
a)?).

WDYT? Or did I miss a way to override the default expansion?

Thanks for comments,

  Jan

[1] 
https://github.com/apache/beam/blob/22205ee1a84581e9206c5c61bad88a799779b4bc/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L511

[2] https://github.com/apache/beam/pull/15082

[3] https://ci-beam.apache.org/job/beam_PreCommit_Java_Commit/18169/



Specifying environment for cross-language transform expansion

2021-06-29 Thread Jan Lukavský

Hi,

I have come across an issue with cross-language transforms. My setup is 
I have working environment type PROCESS and I cannot use DOCKER. When I 
use Python's KafkaIO, it unfortunately - by default - expands to docker 
environment, which then fails due to missing 'docker' command. I didn't 
find a solution without tackling the expansion service, yet.


I see several possible solutions to that:

 a) I would say, that the cleanest solution would be to add preferred 
environment type to the expansion request to the expansion service 
(probably along with additional flags, probably --experiments?). This 
requires deeper changes to the expansion RPC defintion, probably 
serializing the PipelineOptions from the client environment into the 
ExpansionRequest.


 b) Another option would be to allow specifying some of the 
command-line arguments when starting the expansion service, which 
currently accepts only port on command line, see [1]. The 
straightforward 'fix' (see [2]) unfortunately does not work, because it 
requires DirectRunner to be on the classpath, which then breaks other 
runners (see [3]). It seems possible to copy hand selected options from 
command line to the Pipeline, but that feels hackish. It would require 
to either be able to construct the Pipeline without a runner specified 
(which seems possible when calling Pipeline.create(), but not when using 
PipelineOptions create by parsing command-line arguments) or to be able 
to create a Map from PIpelineOptions and then the 
ability to copy all options into the Pipeline's options.


My proposal would be to create a hackish shortcut and just copy the 
--defaultEnvironmentType, --defaultEnvironmentConfig and --experiments 
into Pipeline's options for now, and create an issue for a proper 
solution (possible a)?).


WDYT? Or did I miss a way to override the default expansion?

Thanks for comments,

 Jan

[1] 
https://github.com/apache/beam/blob/22205ee1a84581e9206c5c61bad88a799779b4bc/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L511


[2] https://github.com/apache/beam/pull/15082

[3] https://ci-beam.apache.org/job/beam_PreCommit_Java_Commit/18169/



Specifying environment for cross-language transform expansion

2021-06-29 Thread Jan Lukavský

Hi,

I have come across an issue with cross-language transforms. My setup is 
I have working environment type PROCESS and I cannot use DOCKER. When I 
use Python's KafkaIO, it unfortunately - by default - expands to docker 
environment, which then fails due to missing 'docker' command. I didn't 
find a solution without tackling the expansion service, yet.


I see several possible solutions to that:

 a) I would say, that the cleanest solution would be to add preferred 
environment type to the expansion request to the expansion service 
(probably along with additional flags, probably --experiments?). This 
requires deeper changes to the expansion gRPC defintion, probably 
serializing the PipelineOptions from the client environment into the 
ExpansionRequest.


 b) Another option would be to allow specifying some of the 
command-line arguments when starting the expansion service, which 
currently accepts only port on command line, see [1]. The 
straightforward 'fix' (see [2]) unfortunately does not work, because it 
requires DirectRunner to be on the classpath, which then breaks other 
runners (see [3]). It seems possible to copy hand selected options from 
command line to the Pipeline, but that feels hackish. It would require 
to either be able to construct the Pipeline without a runner specified 
(which seems possible when calling Pipeline.create(), but not when using 
PipelineOptions create by parsing command-line arguments) or to be able 
to create a Map from PIpelineOptions and then the 
ability to copy all options into the Pipeline's options.


My proposal would be to create a hackish shortcut and just copy the 
--defaultEnvironmentType, --defaultEnvironmentConfig and --experiments 
into Pipeline's options for now, and create an issue for a proper 
solution (possible a)?).


WDYT? Or did I miss a way to override the default expansion?

Thanks for comments,

 Jan

[1] 
https://github.com/apache/beam/blob/22205ee1a84581e9206c5c61bad88a799779b4bc/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java#L511


[2] https://github.com/apache/beam/pull/15082

[3] https://ci-beam.apache.org/job/beam_PreCommit_Java_Commit/18169/



Re: [DISCUSS] Do we have all the building block(s) to support iterations in Beam?

2021-06-24 Thread Jan Lukavský

Hi,

I went through the papers and first of all, sorry, I misinterpreted the 
name, this is not specific to Cloud Dataflow. I think that the biggest 
issue is how to ensure that the iterative transform can make progress 
_even trough the watermark cannot make progress_ (chicken-egg problem, 
watermark can move after the cycle finishes, but the cycle finishes when 
the watermark moves). That is where the vector watermark must come in 
place and where all the stuff gets complicated.


Thanks for this discussion, it helped me clarify some things.

 Jan

On 6/24/21 1:44 AM, Jan Lukavský wrote:


Hi Kenn,

thanks for the pointers, that is really interesting reading that 
probably could (should) be part of the Beam docs. On the other hand 
Beam is no longer Dataflow only - and that could mean that some of the 
concepts can be reiterated, possibly?


I don't quite understand where is the difference of "source watermark" 
- where the "source SDF" can use output any downstream watermark - and 
an "iterative SDF", cannot ... this feels like it should be the same.


On 6/24/21 12:47 AM, Kenneth Knowles wrote:
Most of the theory is particularly well-treated in "Timely Dataflow" 
and "Differential Dataflow". There is a brief summary of the latter 
at https://blog.acolyer.org/2015/06/17/differential-dataflow/ 
<https://blog.acolyer.org/2015/06/17/differential-dataflow/> but I 
recommend actually reading both papers. It uses clock ticks rather 
than Beam's continuous style of watermark, but I don't think this 
changes the general approach.


There are very few implementations of watermark-correct cycles AFAIK. 
For Beam runners where the watermark is simulated (for example using 
Spark's state) we could possibly implement at the Beam layer. For 
engines where the Beam watermark is implemented more directly (for 
example Dataflow & Flink) there would be a lot of added complexity, 
probably performance loss, if it could be done at all.


Kenn

On Wed, Jun 23, 2021 at 3:31 PM Reuven Lax <mailto:re...@google.com>> wrote:




On Wed, Jun 23, 2021 at 2:33 PM Jan Lukavský mailto:je...@seznam.cz>> wrote:

On 6/23/21 11:13 PM, Reuven Lax wrote:



On Wed, Jun 23, 2021 at 2:00 PM Jan Lukavský
mailto:je...@seznam.cz>> wrote:

The most qualitatively import use-case I see are ACID
transactions - transactions naturally involve cycles,
because the most natural implementation would be of
something like "optimistic locking" where the
transaction is allowed to progress until a downstream
"commit" sees a conflict, when it needs to return the
transaction back to the beginning (pretty much the same
as how git resolves conflict in a push).

True, however within a transform one could use timers to
implement this (there are currently some bugs around looping
timers I believe, but those are easier to fix than
implementing a brand new programming model). Iterative is
only really necessary if you need to iterate an entire
subgraph, including GroupByKeys, etc.


There necessarily needs to be GBK to support transactions. If
we take the most prominent example of a transaction - moving
some amount of cash between two bank accounts - we can have a
state for the current amount of cash per user. This state
would be keyed, request to transfer some amount would come as
a atomic event "move amount X from user A to user B". This
would lead to updates of state in keyed state of A and B, but
we don't know if A or B have the required amount on their
account. And this is where we need to do two GBKs - one will
assign a sequential ID to each transaction (that is a
serialized order) and the other downstream will verify that
the result was computed from the correct data.


Fair point. This could be done with state/timers in an
eventually-consistent way (though not fully ACID) by simply
sending messages. However in these sorts of workflow scenarios,
the need for back edges will probably come up regardless (e.g. if
a failure happens and you want to cancel, you might need a back
edge to tell the previous key to cancel).

However I'm still not convinced watermarks are needed.

This is maybe too complex to describe in short, but there
definitely has to be GBK (actually GroupAll operation)
downstream and a cycle when a post-condition fails.



Another application would be graph algorithms on
changing graphs, where adding or removing an edge might
trigger an iterative algorithm on the graph (and I'm
absolutely not sure that the discussed approach can do
that, t

Re: [DISCUSS] Do we have all the building block(s) to support iterations in Beam?

2021-06-23 Thread Jan Lukavský

Hi Kenn,

thanks for the pointers, that is really interesting reading that 
probably could (should) be part of the Beam docs. On the other hand Beam 
is no longer Dataflow only - and that could mean that some of the 
concepts can be reiterated, possibly?


I don't quite understand where is the difference of "source watermark" - 
where the "source SDF" can use output any downstream watermark - and an 
"iterative SDF", cannot ... this feels like it should be the same.


On 6/24/21 12:47 AM, Kenneth Knowles wrote:
Most of the theory is particularly well-treated in "Timely Dataflow" 
and "Differential Dataflow". There is a brief summary of the latter at 
https://blog.acolyer.org/2015/06/17/differential-dataflow/ 
<https://blog.acolyer.org/2015/06/17/differential-dataflow/> but I 
recommend actually reading both papers. It uses clock ticks rather 
than Beam's continuous style of watermark, but I don't think this 
changes the general approach.


There are very few implementations of watermark-correct cycles AFAIK. 
For Beam runners where the watermark is simulated (for example using 
Spark's state) we could possibly implement at the Beam layer. For 
engines where the Beam watermark is implemented more directly (for 
example Dataflow & Flink) there would be a lot of added complexity, 
probably performance loss, if it could be done at all.


Kenn

On Wed, Jun 23, 2021 at 3:31 PM Reuven Lax <mailto:re...@google.com>> wrote:




On Wed, Jun 23, 2021 at 2:33 PM Jan Lukavský mailto:je...@seznam.cz>> wrote:

On 6/23/21 11:13 PM, Reuven Lax wrote:



On Wed, Jun 23, 2021 at 2:00 PM Jan Lukavský mailto:je...@seznam.cz>> wrote:

The most qualitatively import use-case I see are ACID
transactions - transactions naturally involve cycles,
because the most natural implementation would be of
something like "optimistic locking" where the transaction
is allowed to progress until a downstream "commit" sees a
conflict, when it needs to return the transaction back to
the beginning (pretty much the same as how git resolves
conflict in a push).

True, however within a transform one could use timers to
implement this (there are currently some bugs around looping
timers I believe, but those are easier to fix than
implementing a brand new programming model). Iterative is
only really necessary if you need to iterate an entire
subgraph, including GroupByKeys, etc.


There necessarily needs to be GBK to support transactions. If
we take the most prominent example of a transaction - moving
some amount of cash between two bank accounts - we can have a
state for the current amount of cash per user. This state
would be keyed, request to transfer some amount would come as
a atomic event "move amount X from user A to user B". This
would lead to updates of state in keyed state of A and B, but
we don't know if A or B have the required amount on their
account. And this is where we need to do two GBKs - one will
assign a sequential ID to each transaction (that is a
serialized order) and the other downstream will verify that
the result was computed from the correct data.


Fair point. This could be done with state/timers in an
eventually-consistent way (though not fully ACID) by simply
sending messages. However in these sorts of workflow scenarios,
the need for back edges will probably come up regardless (e.g. if
a failure happens and you want to cancel, you might need a back
edge to tell the previous key to cancel).

However I'm still not convinced watermarks are needed.

This is maybe too complex to describe in short, but there
definitely has to be GBK (actually GroupAll operation)
downstream and a cycle when a post-condition fails.



Another application would be graph algorithms on changing
graphs, where adding or removing an edge might trigger an
iterative algorithm on the graph (and I'm absolutely not
sure that the discussed approach can do that, this is
just something, that would be cool to do :)).

Yes, that's what I had in mind. I'm just not sure that these
algorithms lend themselves to windowing. I.e. if we added
iterative support, but did not have support for windowing or
watermarks across iterations, have we solved most of the
problem?


I don't think there is any windowing involved. When a new road
is built between cities A and B it _immediately_ makes
traveling between these two cities faster. There is no
discrete boundary.

I don't understand why we would drop support for watermarks

Re: [DISCUSS] Do we have all the building block(s) to support iterations in Beam?

2021-06-23 Thread Jan Lukavský
> If that timer is triggered by the watermark of the previous step and 
that watermark is being held up by the entire iteration, then this timer 
will never fire and the whole transform could deadlock. This was one 
reason for multi-dimensional watermarks - the timer can fire based on 
the watermark from the previous iterations, and so never deadlocks 
(though figuring out how to efficiently implement watermarks of 
unbounded dimensionality might be difficult).


That makes sense. However, if you implement endless "for" cycle, that 
will cycle for ever - is that something that the tool you are using 
should avoid? Should we ban for-cycles just because it can lead to 
infinite loops? Another important detail - there is no change in the 
Beam model needed. The purpose of this thread is - are we already there? 
Is really SDF filling the gap in both the source, but as well the 
iteration "gap"? Iteration is for the time being the domain of batch, 
which is where the unified approach looses its points.


On 6/24/21 12:31 AM, Reuven Lax wrote:



On Wed, Jun 23, 2021 at 2:33 PM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


On 6/23/21 11:13 PM, Reuven Lax wrote:



On Wed, Jun 23, 2021 at 2:00 PM Jan Lukavský mailto:je...@seznam.cz>> wrote:

The most qualitatively import use-case I see are ACID
transactions - transactions naturally involve cycles, because
the most natural implementation would be of something like
"optimistic locking" where the transaction is allowed to
progress until a downstream "commit" sees a conflict, when it
needs to return the transaction back to the beginning (pretty
much the same as how git resolves conflict in a push).

True, however within a transform one could use timers to
implement this (there are currently some bugs around looping
timers I believe, but those are easier to fix than implementing a
brand new programming model). Iterative is only really necessary
if you need to iterate an entire subgraph, including GroupByKeys,
etc.


There necessarily needs to be GBK to support transactions. If we
take the most prominent example of a transaction - moving some
amount of cash between two bank accounts - we can have a state for
the current amount of cash per user. This state would be keyed,
request to transfer some amount would come as a atomic event "move
amount X from user A to user B". This would lead to updates of
state in keyed state of A and B, but we don't know if A or B have
the required amount on their account. And this is where we need to
do two GBKs - one will assign a sequential ID to each transaction
(that is a serialized order) and the other downstream will verify
that the result was computed from the correct data.


Fair point. This could be done with state/timers in an 
eventually-consistent way (though not fully ACID) by simply sending 
messages. However in these sorts of workflow scenarios, the need for 
back edges will probably come up regardless (e.g. if a failure happens 
and you want to cancel, you might need a back edge to tell the 
previous key to cancel).


However I'm still not convinced watermarks are needed.

This is maybe too complex to describe in short, but there
definitely has to be GBK (actually GroupAll operation) downstream
and a cycle when a post-condition fails.



Another application would be graph algorithms on changing
graphs, where adding or removing an edge might trigger an
iterative algorithm on the graph (and I'm absolutely not sure
that the discussed approach can do that, this is just
something, that would be cool to do :)).

Yes, that's what I had in mind. I'm just not sure that these
algorithms lend themselves to windowing. I.e. if we added
iterative support, but did not have support for windowing or
watermarks across iterations, have we solved most of the problem?


I don't think there is any windowing involved. When a new road is
built between cities A and B it _immediately_ makes traveling
between these two cities faster. There is no discrete boundary.

I don't understand why we would drop support for watermarks - they
would be perfectly supported, every iteration key will have a
watermark hold that would be released when the key finished
iterating - or was terminated due to timeout. I'm not sure if
windowing as such plays any role in this, but maybe can.


You'd have to make sure things don't deadlock. If a step inside the 
transform that was being iterated had an event-time timer, what 
triggers that timer? If that timer is triggered by the watermark of 
the previous step and that watermark is being held up by the entire 
iteration, then this timer will never fire and the whole transform 
could deadlock. This was one reason for multi-dime

Re: [DISCUSS] Do we have all the building block(s) to support iterations in Beam?

2021-06-23 Thread Jan Lukavský

On 6/23/21 11:13 PM, Reuven Lax wrote:



On Wed, Jun 23, 2021 at 2:00 PM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


The most qualitatively import use-case I see are ACID transactions
- transactions naturally involve cycles, because the most natural
implementation would be of something like "optimistic locking"
where the transaction is allowed to progress until a downstream
"commit" sees a conflict, when it needs to return the transaction
back to the beginning (pretty much the same as how git resolves
conflict in a push).

True, however within a transform one could use timers to implement 
this (there are currently some bugs around looping timers I believe, 
but those are easier to fix than implementing a brand new programming 
model). Iterative is only really necessary if you need to iterate an 
entire subgraph, including GroupByKeys, etc.


There necessarily needs to be GBK to support transactions. If we take 
the most prominent example of a transaction - moving some amount of cash 
between two bank accounts - we can have a state for the current amount 
of cash per user. This state would be keyed, request to transfer some 
amount would come as a atomic event "move amount X from user A to user 
B". This would lead to updates of state in keyed state of A and B, but 
we don't know if A or B have the required amount on their account. And 
this is where we need to do two GBKs - one will assign a sequential ID 
to each transaction (that is a serialized order) and the other 
downstream will verify that the result was computed from the correct data.


This is maybe too complex to describe in short, but there definitely has 
to be GBK (actually GroupAll operation) downstream and a cycle when a 
post-condition fails.




Another application would be graph algorithms on changing graphs,
where adding or removing an edge might trigger an iterative
algorithm on the graph (and I'm absolutely not sure that the
discussed approach can do that, this is just something, that would
be cool to do :)).

Yes, that's what I had in mind. I'm just not sure that these 
algorithms lend themselves to windowing. I.e. if we added iterative 
support, but did not have support for windowing or watermarks across 
iterations, have we solved most of the problem?


I don't think there is any windowing involved. When a new road is built 
between cities A and B it _immediately_ makes traveling between these 
two cities faster. There is no discrete boundary.


I don't understand why we would drop support for watermarks - they would 
be perfectly supported, every iteration key will have a watermark hold 
that would be released when the key finished iterating - or was 
terminated due to timeout. I'm not sure if windowing as such plays any 
role in this, but maybe can.



On 6/23/21 10:53 PM, Reuven Lax wrote:

One question I have is whether the use cases for cyclic graphs
overlap substantially with the use cases for event-time
watermarks. Many of the uses I'm aware of are ML-type algorithms
(e.g. clustering) or iterative algorithms on large graphs
(connected components, etc.), and it's unclear how many of these
problems have a natural time component.

    On Wed, Jun 23, 2021 at 1:49 PM Jan Lukavský mailto:je...@seznam.cz>> wrote:

Reuven, can you please elaborate a little on that? Why do you
need watermark per iteration? Letting the watermark progress
as soon as all the keys arriving before the upstream
watermark terminate the cycle seems like a valid definition
without the need to make the watermark multidimensional. Yes,
it introduces (possibly unbounded) latency in downstream
processing, but that is something that should be probably
expected. The unboundness of the latency can be limited by
either fixed timeout or number of iterations.

On 6/23/21 8:39 PM, Reuven Lax wrote:

This was explored in the past, though the design started
getting very complex (watermarks of unbounded dimension,
where each iteration has its own watermark dimension). At
the time, the exploration petered out.

On Wed, Jun 23, 2021 at 10:13 AM Jan Lukavský
mailto:je...@seznam.cz>> wrote:

Hi,

I'd like to discuss a very rough idea. I didn't walk
through all the
corner cases and the whole idea has a lot of rough
edges, so please bear
with me. I was thinking about non-IO applications of
splittable DoFn,
and the main idea - and why it is called splittable - is
that it can
handle unbounded outputs per element. Then I was
thinking about what can
generate unbounded outputs per element _without reading
from external
source_ (as that would b

Re: [DISCUSS] Do we have all the building block(s) to support iterations in Beam?

2021-06-23 Thread Jan Lukavský
The most qualitatively import use-case I see are ACID transactions - 
transactions naturally involve cycles, because the most natural 
implementation would be of something like "optimistic locking" where the 
transaction is allowed to progress until a downstream "commit" sees a 
conflict, when it needs to return the transaction back to the beginning 
(pretty much the same as how git resolves conflict in a push).


Another application would be graph algorithms on changing graphs, where 
adding or removing an edge might trigger an iterative algorithm on the 
graph (and I'm absolutely not sure that the discussed approach can do 
that, this is just something, that would be cool to do :)).


On 6/23/21 10:53 PM, Reuven Lax wrote:
One question I have is whether the use cases for cyclic graphs overlap 
substantially with the use cases for event-time watermarks. Many of 
the uses I'm aware of are ML-type algorithms (e.g. clustering) or 
iterative algorithms on large graphs (connected components, etc.), and 
it's unclear how many of these problems have a natural time component.


On Wed, Jun 23, 2021 at 1:49 PM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


Reuven, can you please elaborate a little on that? Why do you need
watermark per iteration? Letting the watermark progress as soon as
all the keys arriving before the upstream watermark terminate the
cycle seems like a valid definition without the need to make the
watermark multidimensional. Yes, it introduces (possibly
unbounded) latency in downstream processing, but that is something
that should be probably expected. The unboundness of the latency
can be limited by either fixed timeout or number of iterations.

On 6/23/21 8:39 PM, Reuven Lax wrote:

This was explored in the past, though the design started getting
very complex (watermarks of unbounded dimension, where each
iteration has its own watermark dimension). At the time, the
exploration petered out.

On Wed, Jun 23, 2021 at 10:13 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

Hi,

I'd like to discuss a very rough idea. I didn't walk through
all the
corner cases and the whole idea has a lot of rough edges, so
please bear
with me. I was thinking about non-IO applications of
splittable DoFn,
and the main idea - and why it is called splittable - is that
it can
handle unbounded outputs per element. Then I was thinking
about what can
generate unbounded outputs per element _without reading from
external
source_ (as that would be IO application) - and then I
realized that the
data can - at least theoretically - come from a downstream
transform. It
would have to be passed over an RPC (gRPC probably)
connection, it would
probably require some sort of service discovery - as the
feedback loop
would have to be correctly targeted based on key - and so on
(those are
the rough edges).

But supposing this can be solved - what iterations actually
mean is the
we have a side channel, that come from downstream processing
- and we
need a watermark estimator for this channel, that is able to
hold the
watermark back until the very last element (at a certain
watermark)
finishes the iteration. The idea is then we could - in theory
- create
an Iteration PTransform, that would take another PTransform
(probably
something like PTransform>,
PCollection>>, where the IterationResult
would contain
the original KV and a stopping condition (true, false)
and by
creating the feedback loop from the output of this
PCollection we could
actually implement this without any need of support on the
side of runners.

Does that seem like something that might be worth exploring?

  Jan



Re: [DISCUSS] Do we have all the building block(s) to support iterations in Beam?

2021-06-23 Thread Jan Lukavský
BTW, the iterations might break the (otherwise very useful) concept that 
elements arriving ON_TIME should stay ON_TIME throughout the complete 
computation. If an element has an excessive amount of iterations to 
complete, it _could_ be output late even though it would have arrived 
ON_TIME. But that would only mean, we might need two timeouts - one for 
releasing the watermark hold and another for cancelling the iteration 
completely.


On 6/23/21 10:43 PM, Jan Lukavský wrote:


Right, one can "outsource" this functionality through external source, 
but that is a sort-of hackish solution. The most serious problem is 
that it "disconnects" the watermark of the feedback loop which can 
make it tricky to correctly compute the downstream watermark. The SDF 
approach seems to compute the watermark correctly (using per-key 
watermark hold until the end of the cycle).


On 6/23/21 10:25 PM, Luke Cwik wrote:
SDF isn't required as users already try to do things like this using 
UnboundedSource and Pubsub.


On Wed, Jun 23, 2021 at 11:39 AM Reuven Lax <mailto:re...@google.com>> wrote:


This was explored in the past, though the design started getting
very complex (watermarks of unbounded dimension, where each
iteration has its own watermark dimension). At the time, the
exploration petered out.

On Wed, Jun 23, 2021 at 10:13 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

Hi,

I'd like to discuss a very rough idea. I didn't walk through
all the
corner cases and the whole idea has a lot of rough edges, so
please bear
with me. I was thinking about non-IO applications of
splittable DoFn,
and the main idea - and why it is called splittable - is that
it can
handle unbounded outputs per element. Then I was thinking
about what can
generate unbounded outputs per element _without reading from
external
source_ (as that would be IO application) - and then I
realized that the
data can - at least theoretically - come from a downstream
transform. It
would have to be passed over an RPC (gRPC probably)
connection, it would
probably require some sort of service discovery - as the
feedback loop
would have to be correctly targeted based on key - and so on
(those are
the rough edges).

But supposing this can be solved - what iterations actually
mean is the
we have a side channel, that come from downstream processing
- and we
need a watermark estimator for this channel, that is able to
hold the
watermark back until the very last element (at a certain
watermark)
finishes the iteration. The idea is then we could - in theory
- create
an Iteration PTransform, that would take another PTransform
(probably
something like PTransform>,
PCollection>>, where the IterationResult
would contain
the original KV and a stopping condition (true, false)
and by
creating the feedback loop from the output of this
PCollection we could
actually implement this without any need of support on the
side of runners.

Does that seem like something that might be worth exploring?

  Jan



Re: [DISCUSS] Do we have all the building block(s) to support iterations in Beam?

2021-06-23 Thread Jan Lukavský
Reuven, can you please elaborate a little on that? Why do you need 
watermark per iteration? Letting the watermark progress as soon as all 
the keys arriving before the upstream watermark terminate the cycle 
seems like a valid definition without the need to make the watermark 
multidimensional. Yes, it introduces (possibly unbounded) latency in 
downstream processing, but that is something that should be probably 
expected. The unboundness of the latency can be limited by either fixed 
timeout or number of iterations.


On 6/23/21 8:39 PM, Reuven Lax wrote:
This was explored in the past, though the design started getting very 
complex (watermarks of unbounded dimension, where each iteration has 
its own watermark dimension). At the time, the exploration petered out.


On Wed, Jun 23, 2021 at 10:13 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


Hi,

I'd like to discuss a very rough idea. I didn't walk through all the
corner cases and the whole idea has a lot of rough edges, so
please bear
with me. I was thinking about non-IO applications of splittable DoFn,
and the main idea - and why it is called splittable - is that it can
handle unbounded outputs per element. Then I was thinking about
what can
generate unbounded outputs per element _without reading from external
source_ (as that would be IO application) - and then I realized
that the
data can - at least theoretically - come from a downstream
transform. It
would have to be passed over an RPC (gRPC probably) connection, it
would
probably require some sort of service discovery - as the feedback
loop
would have to be correctly targeted based on key - and so on
(those are
the rough edges).

But supposing this can be solved - what iterations actually mean
is the
we have a side channel, that come from downstream processing - and we
need a watermark estimator for this channel, that is able to hold the
watermark back until the very last element (at a certain watermark)
finishes the iteration. The idea is then we could - in theory -
create
an Iteration PTransform, that would take another PTransform (probably
something like PTransform>, PCollection>>, where the IterationResult would
contain
the original KV and a stopping condition (true, false) and by
creating the feedback loop from the output of this PCollection we
could
actually implement this without any need of support on the side of
runners.

Does that seem like something that might be worth exploring?

  Jan



Re: [DISCUSS] Do we have all the building block(s) to support iterations in Beam?

2021-06-23 Thread Jan Lukavský
Right, one can "outsource" this functionality through external source, 
but that is a sort-of hackish solution. The most serious problem is that 
it "disconnects" the watermark of the feedback loop which can make it 
tricky to correctly compute the downstream watermark. The SDF approach 
seems to compute the watermark correctly (using per-key watermark hold 
until the end of the cycle).


On 6/23/21 10:25 PM, Luke Cwik wrote:
SDF isn't required as users already try to do things like this using 
UnboundedSource and Pubsub.


On Wed, Jun 23, 2021 at 11:39 AM Reuven Lax <mailto:re...@google.com>> wrote:


This was explored in the past, though the design started getting
very complex (watermarks of unbounded dimension, where each
iteration has its own watermark dimension). At the time, the
exploration petered out.

On Wed, Jun 23, 2021 at 10:13 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

Hi,

I'd like to discuss a very rough idea. I didn't walk through
all the
corner cases and the whole idea has a lot of rough edges, so
please bear
with me. I was thinking about non-IO applications of
splittable DoFn,
and the main idea - and why it is called splittable - is that
it can
handle unbounded outputs per element. Then I was thinking
about what can
generate unbounded outputs per element _without reading from
external
source_ (as that would be IO application) - and then I
realized that the
data can - at least theoretically - come from a downstream
transform. It
would have to be passed over an RPC (gRPC probably)
connection, it would
probably require some sort of service discovery - as the
feedback loop
would have to be correctly targeted based on key - and so on
(those are
the rough edges).

But supposing this can be solved - what iterations actually
mean is the
we have a side channel, that come from downstream processing -
and we
need a watermark estimator for this channel, that is able to
hold the
watermark back until the very last element (at a certain
watermark)
finishes the iteration. The idea is then we could - in theory
- create
an Iteration PTransform, that would take another PTransform
(probably
something like PTransform>,
PCollection>>, where the IterationResult would
contain
the original KV and a stopping condition (true, false)
and by
creating the feedback loop from the output of this PCollection
we could
actually implement this without any need of support on the
side of runners.

Does that seem like something that might be worth exploring?

  Jan



[DISCUSS] Do we have all the building block(s) to support iterations in Beam?

2021-06-23 Thread Jan Lukavský

Hi,

I'd like to discuss a very rough idea. I didn't walk through all the 
corner cases and the whole idea has a lot of rough edges, so please bear 
with me. I was thinking about non-IO applications of splittable DoFn, 
and the main idea - and why it is called splittable - is that it can 
handle unbounded outputs per element. Then I was thinking about what can 
generate unbounded outputs per element _without reading from external 
source_ (as that would be IO application) - and then I realized that the 
data can - at least theoretically - come from a downstream transform. It 
would have to be passed over an RPC (gRPC probably) connection, it would 
probably require some sort of service discovery - as the feedback loop 
would have to be correctly targeted based on key - and so on (those are 
the rough edges).


But supposing this can be solved - what iterations actually mean is the 
we have a side channel, that come from downstream processing - and we 
need a watermark estimator for this channel, that is able to hold the 
watermark back until the very last element (at a certain watermark) 
finishes the iteration. The idea is then we could - in theory - create 
an Iteration PTransform, that would take another PTransform (probably 
something like PTransform>, PCollectionIterationResult>>, where the IterationResult would contain 
the original KV and a stopping condition (true, false) and by 
creating the feedback loop from the output of this PCollection we could 
actually implement this without any need of support on the side of runners.


Does that seem like something that might be worth exploring?

 Jan



Re: FileIO with custom sharding function

2021-06-17 Thread Jan Lukavský
Correct, by the external shuffle service I pretty much meant "offloading 
the contents of a shuffle phase out of the system". Looks like that is 
what the Spark's checkpoint does as well. On the other hand (if I 
understand the concept correctly), that implies some performance penalty 
- the data has to be moved to external distributed filesystem. Which 
then feels weird if we optimize code to avoid computing random numbers, 
but are okay with moving complete datasets back and forth. I think in 
this particular case making the Pipeline deterministic - idempotent to 
be precise - (within the limits, yes, hashCode of enum is not stable 
between JVMs) would seem more practical to me.


 Jan

On 6/17/21 7:09 AM, Reuven Lax wrote:
I have some thoughts here, as Eugene Kirpichov and I spent a lot of 
time working through these semantics in the past.


First - about the problem of duplicates:

A "deterministic" sharding - e.g. hashCode based (though Java makes no 
guarantee that hashCode is stable across JVM instances, so this 
technique ends up not being stable) doesn't really help matters in 
Beam. Unlike other systems, Beam makes no assumptions that transforms 
are idempotent or deterministic. What's more, _any_ transform that has 
any sort of triggered grouping (whether the trigger used is watermark 
based or otherwise) is non deterministic.


Forcing a hash of every element imposed quite a CPU cost; even 
generating a random number per-element slowed things down too much, 
which is why the current code generates a random number only in 
startBundle.


Any runner that does not implement RequiresStableInput will not 
properly execute FileIO. Dataflow and Flink both support this. I 
believe that the Spark runner implicitly supports it by manually 
calling checkpoint as Ken mentioned (unless someone removed that from 
the Spark runner, but if so that would be a correctness regression). 
Implementing this has nothing to do with external shuffle services - 
neither Flink, Spark, nor Dataflow appliance (classic shuffle) have 
any problem correctly implementing RequiresStableInput.


On Wed, Jun 16, 2021 at 11:18 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


I think that the support for @RequiresStableInput is rather
limited. AFAIK it is supported by streaming Flink (where it is not
needed in this situation) and by Dataflow. Batch runners without
external shuffle service that works as in the case of Dataflow
have IMO no way to implement it correctly. In the case of FileIO
(which do not use @RequiresStableInput as it would not be
supported on Spark) the randomness is easily avoidable (hashCode
of key?) and I it seems to me it should be preferred.

 Jan

On 6/16/21 6:23 PM, Kenneth Knowles wrote:


On Wed, Jun 16, 2021 at 5:18 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

Hi,

maybe a little unrelated, but I think we definitely should
not use random assignment of shard keys
(RandomShardingFunction), at least for bounded workloads
(seems to be fine for streaming workloads). Many batch
runners simply recompute path in the computation DAG from the
failed node (transform) to the root (source). In the case
there is any non-determinism involved in the logic, then it
can result in duplicates (as the 'previous' attempt might
have ended in DAG path that was not affected by the fail).
That addresses the option 2) of what Jozef have mentioned.

This is the reason we introduced "@RequiresStableInput".

When things were only Dataflow, we knew that each shuffle was a
checkpoint, so we inserted a Reshuffle after the random numbers
were generated, freezing them so it was safe for replay. Since
other engines do not checkpoint at every shuffle, we needed a way
to communicate that this checkpointing was required for
correctness. I think we still have many IOs that are written
using Reshuffle instead of @RequiresStableInput, and I don't know
which runners process @RequiresStableInput properly.

By the way, I believe the SparkRunner explicitly calls
materialize() after a GBK specifically so that it gets correct
results for IOs that rely on Reshuffle. Has that changed?

I agree that we should minimize use of RequiresStableInput. It
has a significant cost, and the cost varies across runners. If we
can use a deterministic function, we should.

Kenn

 Jan

On 6/16/21 1:43 PM, Jozef Vilcek wrote:



On Wed, Jun 16, 2021 at 1:38 AM Kenneth Knowles
mailto:k...@apache.org>> wrote:

In general, Beam only deals with keys and grouping by
key. I think expanding this idea to some more abstract
notion of a sharding function could make sense.

For FileIO specifically, I wonder if you can use
writeDynamic() to get

Re: FileIO with custom sharding function

2021-06-16 Thread Jan Lukavský
I think that the support for @RequiresStableInput is rather limited. 
AFAIK it is supported by streaming Flink (where it is not needed in this 
situation) and by Dataflow. Batch runners without external shuffle 
service that works as in the case of Dataflow have IMO no way to 
implement it correctly. In the case of FileIO (which do not use 
@RequiresStableInput as it would not be supported on Spark) the 
randomness is easily avoidable (hashCode of key?) and I it seems to me 
it should be preferred.


 Jan

On 6/16/21 6:23 PM, Kenneth Knowles wrote:


On Wed, Jun 16, 2021 at 5:18 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


Hi,

maybe a little unrelated, but I think we definitely should not use
random assignment of shard keys (RandomShardingFunction), at least
for bounded workloads (seems to be fine for streaming workloads).
Many batch runners simply recompute path in the computation DAG
from the failed node (transform) to the root (source). In the case
there is any non-determinism involved in the logic, then it can
result in duplicates (as the 'previous' attempt might have ended
in DAG path that was not affected by the fail). That addresses the
option 2) of what Jozef have mentioned.

This is the reason we introduced "@RequiresStableInput".

When things were only Dataflow, we knew that each shuffle was a 
checkpoint, so we inserted a Reshuffle after the random numbers were 
generated, freezing them so it was safe for replay. Since other 
engines do not checkpoint at every shuffle, we needed a way to 
communicate that this checkpointing was required for correctness. I 
think we still have many IOs that are written using Reshuffle instead 
of @RequiresStableInput, and I don't know which runners process 
@RequiresStableInput properly.


By the way, I believe the SparkRunner explicitly calls materialize() 
after a GBK specifically so that it gets correct results for IOs that 
rely on Reshuffle. Has that changed?


I agree that we should minimize use of RequiresStableInput. It has a 
significant cost, and the cost varies across runners. If we can use a 
deterministic function, we should.


Kenn

 Jan

On 6/16/21 1:43 PM, Jozef Vilcek wrote:



On Wed, Jun 16, 2021 at 1:38 AM Kenneth Knowles mailto:k...@apache.org>> wrote:

In general, Beam only deals with keys and grouping by key. I
think expanding this idea to some more abstract notion of a
sharding function could make sense.

For FileIO specifically, I wonder if you can use
writeDynamic() to get the behavior you are seeking.


The change in mind looks like this:

https://github.com/JozoVilcek/beam/commit/9c5a7fe35388f06f72972ec4c1846f1dbe85eb18

<https://github.com/JozoVilcek/beam/commit/9c5a7fe35388f06f72972ec4c1846f1dbe85eb18>

Dynamic Destinations in my mind is more towards the need for
"partitioning" data (destination as directory level) or if one
needs to handle groups of events differently, e.g. write some
events in FormatA and others in FormatB.
Shards are now used for distributing writes or bucketing of
events within a particular destination group. More specifically,
currently, each element is assigned `ShardedKey` [1]
before GBK operation. Sharded key is a compound of destination
and assigned shard.

Having said that, I might be able to use dynamic destination for
this, possibly with the need of custom FileNaming, and set shards
to be always 1. But it feels less natural than allowing the user
to swap already present `RandomShardingFunction` [2] with
something of his own choosing.


[1]

https://github.com/apache/beam/blob/release-2.29.0/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ShardedKey.java

<https://github.com/apache/beam/blob/release-2.29.0/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ShardedKey.java>

[2]

https://github.com/apache/beam/blob/release-2.29.0/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java#L856

<https://github.com/apache/beam/blob/release-2.29.0/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java#L856>

Kenn

On Tue, Jun 15, 2021 at 3:49 PM Tyson Hamilton
mailto:tyso...@google.com>> wrote:

Adding sharding to the model may require a wider
discussion than FileIO alone. I'm not entirely sure how
wide, or if this has been proposed before, but IMO it
warrants a design doc or proposal.

A couple high level questions I can think of are,
  - What runners support sharding?
      * There will be some work in Dataflow required to
support this but I'm not sure how much.
  - What does sharding mean for streaming pipelines?

A more nitty-detail question:
  - How can this be a

Re: FileIO with custom sharding function

2021-06-16 Thread Jan Lukavský

Hi,

maybe a little unrelated, but I think we definitely should not use 
random assignment of shard keys (RandomShardingFunction), at least for 
bounded workloads (seems to be fine for streaming workloads). Many batch 
runners simply recompute path in the computation DAG from the failed 
node (transform) to the root (source). In the case there is any 
non-determinism involved in the logic, then it can result in duplicates 
(as the 'previous' attempt might have ended in DAG path that was not 
affected by the fail). That addresses the option 2) of what Jozef have 
mentioned.


 Jan

On 6/16/21 1:43 PM, Jozef Vilcek wrote:



On Wed, Jun 16, 2021 at 1:38 AM Kenneth Knowles > wrote:


In general, Beam only deals with keys and grouping by key. I think
expanding this idea to some more abstract notion of a sharding
function could make sense.

For FileIO specifically, I wonder if you can use writeDynamic() to
get the behavior you are seeking.


The change in mind looks like this:
https://github.com/JozoVilcek/beam/commit/9c5a7fe35388f06f72972ec4c1846f1dbe85eb18 



Dynamic Destinations in my mind is more towards the need for 
"partitioning" data (destination as directory level) or if one needs 
to handle groups of events differently, e.g. write some events in 
FormatA and others in FormatB.
Shards are now used for distributing writes or bucketing of events 
within a particular destination group. More specifically, currently, 
each element is assigned `ShardedKey` [1] before GBK 
operation. Sharded key is a compound of destination and assigned shard.


Having said that, I might be able to use dynamic destination for this, 
possibly with the need of custom FileNaming, and set shards to be 
always 1. But it feels less natural than allowing the user to swap 
already present `RandomShardingFunction` [2] with something of his own 
choosing.



[1] 
https://github.com/apache/beam/blob/release-2.29.0/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ShardedKey.java 
 

[2] 
https://github.com/apache/beam/blob/release-2.29.0/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java#L856 



Kenn

On Tue, Jun 15, 2021 at 3:49 PM Tyson Hamilton mailto:tyso...@google.com>> wrote:

Adding sharding to the model may require a wider discussion
than FileIO alone. I'm not entirely sure how wide, or if this
has been proposed before, but IMO it warrants a design doc or
proposal.

A couple high level questions I can think of are,
  - What runners support sharding?
      * There will be some work in Dataflow required to
support this but I'm not sure how much.
  - What does sharding mean for streaming pipelines?

A more nitty-detail question:
  - How can this be achieved performantly? For example, if a
shuffle is required to achieve a particular sharding
constraint, should we allow transforms to declare they don't
modify the sharding property (e.g. key preserving) which may
allow a runner to avoid an additional shuffle if a preceding
shuffle can guarantee the sharding requirements?

Where X is the shuffle that could be avoided: input -> shuffle
(key sharding fn A) -> transform1 (key preserving) ->
transform 2 (key preserving) -> X -> fileio (key sharding fn A)

On Tue, Jun 15, 2021 at 1:02 AM Jozef Vilcek
mailto:jozo.vil...@gmail.com>> wrote:

I would like to extend FileIO with possibility to specify
a custom sharding function:
https://issues.apache.org/jira/browse/BEAM-12493


I have 2 use-cases for this:

 1. I need to generate shards which are compatible with
Hive bucketing and therefore need to decide shard
assignment based on data fields of input element
 2. When running e.g. on Spark and job encounters kind of
failure which cause a loss of some data from previous
stages, Spark does issue recompute of necessary task
in necessary stages to recover data. Because the shard
assignment function is random as default, some data
will end up in different shards and cause duplicates
in the final output.

Please let me know your thoughts in case you see a reason
to not to add such improvement.

Thanks,
Jozef



Re: [DISCUSS] Drop support for Flink 1.10

2021-05-31 Thread Jan Lukavský

Hi,

+1 to remove the support for 1.10.

 Jan

On 5/28/21 10:00 PM, Ismaël Mejía wrote:

Hello,

With Beam support for Flink 1.13 just merged it is the time to discuss 
the end of
support for Flink 1.10 following the agreed policy on supporting only 
the latest

three Flink releases [1].

I would like to propose that for Beam 2.31.0 we stop supporting Flink 
1.10 [2].
I prepared a PR for this [3] but of course I wanted to bring the 
subject here
(and to user@) for your attention and in case someone has a different 
opinion or

reason to still support the older version.

WDYT?

Regards,
Ismael

[1] 
https://lists.apache.org/thread.html/rfb5ac9d889d0e3f4400471de3c25000a15352bde879622c899d97581%40%3Cdev.beam.apache.org%3E 

[2] https://issues.apache.org/jira/browse/BEAM-12281 

[3] https://github.com/apache/beam/pull/14906 





Re: Some questions around GroupIntoBatches

2021-05-22 Thread Jan Lukavský
Yes, using end-of-window timer would work as well. The problem with 
output timestamp can be solved using setAllowedTimestampSkew, which is 
deprecated, but AFAIK there is no working alternative.


Maybe the best option would be to enable modification of the output 
timestamp only, not the firing timestamp? I think there are cases, that 
do that and actually store the fire timestamp in a ValueState now. I'm 
not sure if creating timer.updateOutputTimestampTo could spare us from 
saving the fire timestamp in a ValueState, though. What is the exact 
reason we don't know the firing timestamp? I'm not 100% sure of the 
runner details, but just from the logic - if we can reset the timer by 
first cancelling it and then setting to a new firing timestamp, there 
has to be the knowledge, what the current time was, otherwise, how would 
do you cancel it?


 Jan

On 5/21/21 4:53 PM, Reuven Lax wrote:
Ah I see, you want to set the event-time timer in the future. Could 
also accomplish this with an end-of-window timer for which we keep 
updating the hold.


Unfortunately, this still doesn't quite work. The output will need to 
happen from the processing-time timer, so Beam will likely reject it 
due to being an output "in the past". In the past is determined by 
comparing the output timestamp to the timestamp of the current element 
being processed (for the case of timers, the timestamp of the timer is 
the output timestamp of that timer). The fact that there is 
another timer extant with that output timestamp doesn't help here.


On Fri, May 21, 2021 at 2:06 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


If I understand it correctly (and what I have observed from the
actual behavior on both FlinkRunner and DirectRunner) a relative
timer with zero duration will not fire immediately. It has to wait
for the watermark to advance. It requires to fix [1] for the
relative timer with output timestamp to work reliably.

 Jan

[1] https://issues.apache.org/jira/browse/BEAM-12276
<https://issues.apache.org/jira/browse/BEAM-12276>

On 5/20/21 7:08 PM, Reuven Lax wrote:

I don't think that would work well. It's entirely possible that
the input watermark will already have passed the timestamp of the
hold, in which case an event-time timer would fire immediately.
You could make it a looping timer, but the new timer would also
fire immediately after being set, and a tight timer loop isn't a
good idea.

A Timer.get() is one solution, though I think the only way we
have to implement it is to store the timer's timestamp in a
ValueState; doing this for every timer would add a lot of cost to
pipelines.

Another option is a Timer.getCurrentTime() method, that would
return the current base time that setRelative is based off of. It
seems like a strange function to add to Timer though.

Another option is to use TimerMap to bucket timers. Every minute
we round the current processing time to the nearest minute and
set a timer with an expiration of that minute (and with the
minute timestamp as its tag as well). This way we would have a
continuous sequence of expiring timers, and we wouldn't have to
set just the first one. The biggest problem with this approach is
that we would also have to use MapState to store the desired
watermark hold per  processing-time bucket. MapState is not
supported by many runners yet, so I don't want to use it in a
basic transform like GroupIntoBatches (furthermore - a transform
that works on most runners today).

Reuven


On Thu, May 20, 2021 at 2:11 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

Sounds like you could solve that using second event time
timer, that would be actually used only to hold the output
timestamp (watermark hold). Something like


eventTimer.withOutputTimestamp(currentBufferMinimalStamp).offset(Duration.ZERO).setRelative()

when the timer fires, you would only reset the minimum.

It is sort of ugly, though. It would be cool to have a way to
get the current timestamp a timer is set to (if any).

 Jan

On 5/20/21 3:12 AM, Reuven Lax wrote:

100% - the contract should not change because things are in
a bundle. IIRC there are some open bugs in Beam around this
that really should be fixed.

My issue with GroupIntoBatches is different. This transform
works as follows:

if (this is the first element in batch - checked by reading
a count stored in a ValueState)
 timer.offset(bufferingDuration).setRelative()

This makes it tricky to use setTimer.withOutputTimestamp.
Inputs are not guaranteed to be in order, so simply adding a
withOutputTimestamp would set the timestamp to be whatever
the first element happened to be; it really should be the
mi

Re: Some questions around GroupIntoBatches

2021-05-21 Thread Jan Lukavský
If I understand it correctly (and what I have observed from the actual 
behavior on both FlinkRunner and DirectRunner) a relative timer with 
zero duration will not fire immediately. It has to wait for the 
watermark to advance. It requires to fix [1] for the relative timer with 
output timestamp to work reliably.


 Jan

[1] https://issues.apache.org/jira/browse/BEAM-12276

On 5/20/21 7:08 PM, Reuven Lax wrote:
I don't think that would work well. It's entirely possible that the 
input watermark will already have passed the timestamp of the hold, in 
which case an event-time timer would fire immediately. You could make 
it a looping timer, but the new timer would also fire immediately 
after being set, and a tight timer loop isn't a good idea.


A Timer.get() is one solution, though I think the only way we have to 
implement it is to store the timer's timestamp in a ValueState; doing 
this for every timer would add a lot of cost to pipelines.


Another option is a Timer.getCurrentTime() method, that would return 
the current base time that setRelative is based off of. It seems like 
a strange function to add to Timer though.


Another option is to use TimerMap to bucket timers. Every minute we 
round the current processing time to the nearest minute and set a 
timer with an expiration of that minute (and with the minute timestamp 
as its tag as well). This way we would have a continuous sequence of 
expiring timers, and we wouldn't have to set just the first one. The 
biggest problem with this approach is that we would also have to use 
MapState to store the desired watermark hold per  processing-time 
bucket. MapState is not supported by many runners yet, so I don't want 
to use it in a basic transform like GroupIntoBatches (furthermore - a 
transform that works on most runners today).


Reuven


On Thu, May 20, 2021 at 2:11 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


Sounds like you could solve that using second event time timer,
that would be actually used only to hold the output timestamp
(watermark hold). Something like


eventTimer.withOutputTimestamp(currentBufferMinimalStamp).offset(Duration.ZERO).setRelative()

when the timer fires, you would only reset the minimum.

It is sort of ugly, though. It would be cool to have a way to get
the current timestamp a timer is set to (if any).

 Jan

On 5/20/21 3:12 AM, Reuven Lax wrote:

100% - the contract should not change because things are in a
bundle. IIRC there are some open bugs in Beam around this that
really should be fixed.

My issue with GroupIntoBatches is different. This transform works
as follows:

if (this is the first element in batch - checked by reading a
count stored in a ValueState)
   timer.offset(bufferingDuration).setRelative()

This makes it tricky to use setTimer.withOutputTimestamp. Inputs
are not guaranteed to be in order, so simply adding a
withOutputTimestamp would set the timestamp to be whatever the
first element happened to be; it really should be the minimum
timestamp of all elements in the buffer. If we started setting
the timer on every element, then timer.offset.setRelative would
keep bumping the (processing-time) timer into the future and it
would never expire.

One solution would be to store the timer timestamp in a
ValueState, and use Timer.set to set the timer to an absolute
timestamp. This would allow us to always reset the timer to the
same expiration target, just modifying the output timestamp each
time. However, this will break DirectRunner tests. The
DirectRunner allows the user to control the advancement of
processing time when using TestStream, but this facility doesn't
work well if the transform sets the processing-time timer using
absolute set() calls.

I'm not sure how to solve this using the existing Timer API.

On Wed, May 19, 2021 at 4:39 PM Robert Bradshaw
mailto:rober...@google.com>> wrote:

+1. It was my understanding as well that consensus was that
timers
must be delivered in timestamp order, and "within bundle"
resetting/clearing of timers should be respected (as if each
timer was
in its own bundle).

On Wed, May 19, 2021 at 3:01 PM Kenneth Knowles
mailto:k...@apache.org>> wrote:
>
> Reading over the other thread, there was consensus to
implement.
>
> Reading commentary on the PR, there were good questions
raised about the semantics. Questions which I feel able to
have an opinion about :-)
>
> The questions surrounded bundling and timers in the same
bundling clearing each other. Actually the same questions
    apply to timers re-setting later timers and +Jan Lukavský has
raised this already (among other people) so we kind of know
the an

Re: Some questions around GroupIntoBatches

2021-05-20 Thread Jan Lukavský
Sounds like you could solve that using second event time timer, that 
would be actually used only to hold the output timestamp (watermark 
hold). Something like


eventTimer.withOutputTimestamp(currentBufferMinimalStamp).offset(Duration.ZERO).setRelative()

when the timer fires, you would only reset the minimum.

It is sort of ugly, though. It would be cool to have a way to get the 
current timestamp a timer is set to (if any).


 Jan

On 5/20/21 3:12 AM, Reuven Lax wrote:
100% - the contract should not change because things are in a bundle. 
IIRC there are some open bugs in Beam around this that really should 
be fixed.


My issue with GroupIntoBatches is different. This transform works as 
follows:


if (this is the first element in batch - checked by reading a count 
stored in a ValueState)

   timer.offset(bufferingDuration).setRelative()

This makes it tricky to use setTimer.withOutputTimestamp. Inputs are 
not guaranteed to be in order, so simply adding a withOutputTimestamp 
would set the timestamp to be whatever the first element happened to 
be; it really should be the minimum timestamp of all elements in the 
buffer. If we started setting the timer on every element, then 
timer.offset.setRelative would keep bumping the (processing-time) 
timer into the future and it would never expire.


One solution would be to store the timer timestamp in a ValueState, 
and use Timer.set to set the timer to an absolute timestamp. This 
would allow us to always reset the timer to the same expiration 
target, just modifying the output timestamp each time. However, this 
will break DirectRunner tests. The DirectRunner allows the user to 
control the advancement of processing time when using TestStream, but 
this facility doesn't work well if the transform sets the 
processing-time timer using absolute set() calls.


I'm not sure how to solve this using the existing Timer API.

On Wed, May 19, 2021 at 4:39 PM Robert Bradshaw <mailto:rober...@google.com>> wrote:


+1. It was my understanding as well that consensus was that timers
must be delivered in timestamp order, and "within bundle"
resetting/clearing of timers should be respected (as if each timer was
in its own bundle).

On Wed, May 19, 2021 at 3:01 PM Kenneth Knowles mailto:k...@apache.org>> wrote:
>
> Reading over the other thread, there was consensus to implement.
>
> Reading commentary on the PR, there were good questions raised
about the semantics. Questions which I feel able to have an
opinion about :-)
>
> The questions surrounded bundling and timers in the same
bundling clearing each other. Actually the same questions apply to
    timers re-setting later timers and +Jan Lukavský has raised this
already (among other people) so we kind of know the answer now,
and I think +Boyuan Zhang code was good (from my quick read). What
has changed is that we have a better idea of the contract with the
runner. I'm not sure if portability makes this more complex. I
will share all my thoughts on this:
>
> I think one key to the Beam model is that bundles are for
performance and also intended as the unit of commitment (until
FinishBundle is called, there may be unfinished work). They can
affect *behavior* (what the program does - including what can be
observed) ... but not *semantics* (what the output means).
>
> So, for example, bundling affects how many files are written but
you are expected to read all the files and the number or ordering
you must not depend on. The behavior is different, but the
semantics are the same.
>
> When it comes to timers, behavior and semantics are very tightly
coupled; timers are like a self loop. The firing of a timer is a
behavior w.r.t. the output of the stateful ParDo but it is
semantics of the timer itself (my definitions don't actually apply
so simply so don't dig too hard :-). So to get bundling-invariant
semantics, we should try for bundling-invariant behavior. When
some clock moves from T to T+D between bundles then all the timers
in the range [T, T+D) may fire so they are delivered in the
bundle. I believe we have in the prior re-setting issue agreed
that timers should always be called back in timestamp order.
Bundling-invariance then implies that earlier timers can clear
later timers just like they can re-set them. So even though a
timer is delivered in a bundle, the local state of the timer wins.
This matches how state works as well; no matter how things are
bundled, the state you read is always whatever was written last.
>
> Kenn
>
> On Tue, May 11, 2021 at 1:24 PM Siyuan Chen mailto:syc...@google.com>> wrote:
>>
>>
>>
>> On Tue, May 11, 2021 at 11:08 AM Reuven Lax mailto:re...@google.com>> wrote:
&

Re: [VOTE] Vendored Dependencies Release Byte Buddy 1.11.0

2021-05-19 Thread Jan Lukavský

+1 (non-binding)

verified correct shading.

 Jan

On 5/19/21 8:53 PM, Ismaël Mejía wrote:
This release is only to publish the vendored dependency artifacts. We 
need those to integrate it and be able to verify if it causes problems 
or not. The PR for this is already opened but it needs the artifacts 
of this vote to be ran.
https://github.com/apache/beam/pull/14824 



For ref there is a document on how to release and validate releases of 
Beam's vendored dependencies that can be handy to anyone wishing to 
help validate:
https://s.apache.org/beam-release-vendored-artifacts 



On Wed, May 19, 2021 at 8:45 PM Tyson Hamilton > wrote:


I'd like to help, but I don't know how to determine whether this
upgrade is going to cause problems or not. Are there tests I
should look at, or some validation I should perform?

On Wed, May 19, 2021 at 11:29 AM Ismaël Mejía mailto:ieme...@gmail.com>> wrote:

Kind reminder, the vote is ongoing

On Mon, May 17, 2021 at 5:32 PM Ismaël Mejía
mailto:ieme...@gmail.com>> wrote:

Please review the release of the following artifacts that
we vendor:
 * beam-vendor-bytebuddy-1_11_0

Hi everyone,
Please review and vote on the release candidate #1 for the
version 0.1, as follows:
[ ] +1, Approve the release
[ ] -1, Do not approve the release (please provide
specific comments)

The complete staging area is available for your review,
which includes:
* the official Apache source release to be deployed to
dist.apache.org  [1], which is
signed with the key with fingerprint
3415631729E15B33051ADB670A9DAF6713B86349 [2],
* all artifacts to be deployed to the Maven Central
Repository [3],
* commit hash "d93c591deb21237ddb656583d7ef7a4debba" [4],

The vote will be open for at least 72 hours. It is adopted
by majority approval, with at least 3 PMC affirmative votes.

Thanks,
Release Manager

[1] https://dist.apache.org/repos/dist/dev/beam/vendor/

[2] https://dist.apache.org/repos/dist/release/beam/KEYS

[3]

https://repository.apache.org/content/repositories/orgapachebeam-1166/


[4]

https://github.com/apache/beam/commit/d93c591deb21237ddb656583d7ef7a4debba





Re: Ordered PCollections eventually?

2021-05-11 Thread Jan Lukavský
I'll just remind that Beam already supports (experimental) 
@RequiresTimeSortedInput (which has several limitations, mostly in that 
it orders only by timestamp and not some - time related - user field; 
and of course - missing retractions). An arbitrary sorting seems to be 
hard, even per-key, it seems it will always have to be somewhat 
time-bounded, as otherwise it might require unbounded states. The batch 
case on the other hand typically has a way to order inputs arbitrarily 
with virtually zero cost, as many implementations use sort-merge-group 
to perform reduction operations.


 Jan

On 5/11/21 5:56 PM, Kenneth Knowles wrote:
Per-key ordered delivery makes a ton of sense. I'd guess CDC has the 
same needs as retractions, so that the changelog can be applied in 
order as it arrives. And since it is per-key you still get parallelism.


Global ordering is quite different. I know that SQL and Dataframes 
have global sorting operations. The question has always been how does 
"embarassingly paralllel" processing interact with sorting/ordering. I 
imagine some other systems have the features so we can look at how it 
is used?


Kenn

Kenn

On Mon, May 10, 2021 at 4:39 PM Sam Rohde > wrote:


Awesome, thanks Pablo!

On Mon, May 10, 2021 at 4:05 PM Pablo Estrada mailto:pabl...@google.com>> wrote:

CDC would also benefit. I am working on a proposal for this
that is concerned with streaming pipelines, and per-key
ordered delivery. I will share with you as soon as I have a
draft.
Best
-P.

On Mon, May 10, 2021 at 2:56 PM Reuven Lax mailto:re...@google.com>> wrote:

There has been talk, but nothing concrete.

On Mon, May 10, 2021 at 1:42 PM Sam Rohde
mailto:sro...@google.com>> wrote:

Hi All,

I was wondering if there had been any plans for
creating ordered PCollections in the Beam model? Or if
there might be plans for them in the future?

I know that Beam SQL and Beam DataFrames would
directly benefit from an ordered PCollection.

Regards,
Sam



Re: LGPL-2.1 in beam-vendor-grpc

2021-05-10 Thread Jan Lukavský
+1 for blocking the release - I think we should not release something 
about which we _know_ that it might be legally problematic. And we 
should definitely create a check in the build process that will warn 
about such issues in the future.


 Jan

On 5/10/21 3:44 PM, Ismaël Mejía wrote:
Tomo just confirmed in the ticket that if we update the gRPC vendored 
version we won't need the JBoss dependency anymore so we should be 
good to go with the upgrade. The open question is if this should be 
blocking for the upcoming Beam 2.31.0 release or we can fix it afterwards.



On Mon, May 10, 2021 at 2:46 PM Ismaël Mejía <mailto:ieme...@gmail.com>> wrote:


We have been discussing about updating the vendored dependency in
BEAM-11227 <https://issues.apache.org/jira/browse/BEAM-11227>, if
I remember correctly the newer version of gRPC does not require
the jboss dependency, so probably is the best upgrade path, can
you confirm Tomo Suzuki
<https://issues.apache.org/jira/secure/ViewProfile.jspa?name=suztomo>
?

On Mon, May 10, 2021 at 2:33 PM Jarek Potiuk mailto:ja...@potiuk.com>> wrote:

Also we have very similar discussion about it in
https://issues.apache.org/jira/browse/LEGAL-572
<https://issues.apache.org/jira/browse/LEGAL-572>
Just to be clear about the context of it, it's not a legal
requirement of Apache Licence, it's Apache Software Foundation
policy, that we should not limit our users in using our
software. If the LGPL dependency is "optional", it's fine to
add such optional dependency. If it is "required" to run your
software, then it is not allowed as it limits the users of ASF
software in further redistributing the software in the way
they want (this is at least my understanding of it).

On Mon, May 10, 2021 at 12:58 PM JB Onofré mailto:j...@nanthrax.net>> wrote:

Hi

You can take a look on

https://www.apache.org/legal/resolved.html
<https://www.apache.org/legal/resolved.html>

Regards
JB


Le 10 mai 2021 à 12:56, Elliotte Rusty Harold
mailto:elh...@ibiblio.org>> a écrit :

Anyone have a link to the official Apache policy about
        this? Thanks.

On Mon, May 10, 2021 at 10:07 AM Jan Lukavský
mailto:je...@seznam.cz>> wrote:


Hi,

we are bundling dependencies with LGPL-2.1, according to
license header
in
META-INF/maven/org.jboss.modules/jboss-modules/pom.xml.
I think is
might be an issue, already reported here: [1]. I created
[2] to track it
on our side.

 Jan

[1] https://issues.apache.org/jira/browse/FLINK-22555
<https://issues.apache.org/jira/browse/FLINK-22555>

[2] https://issues.apache.org/jira/browse/BEAM-12316
<https://issues.apache.org/jira/browse/BEAM-12316>




-- 
Elliotte Rusty Harold

elh...@ibiblio.org <mailto:elh...@ibiblio.org>




-- 
+48 660 796 129




LGPL-2.1 in beam-vendor-grpc

2021-05-10 Thread Jan Lukavský

Hi,

we are bundling dependencies with LGPL-2.1, according to license header 
in META-INF/maven/org.jboss.modules/jboss-modules/pom.xml. I think is 
might be an issue, already reported here: [1]. I created [2] to track it 
on our side.


 Jan

[1] https://issues.apache.org/jira/browse/FLINK-22555

[2] https://issues.apache.org/jira/browse/BEAM-12316



Re: Timer.withOutputTimestamp().offset().setRelative seems unusable with event time

2021-05-05 Thread Jan Lukavský
Yes, what I meant was the distinguishing of cleanup timers from plain 
user other timers - that seems to be due to the fact that they fire base 
on different watermark (input/output). And firing timers based on output 
watermark might be actually a good user-facing feature, because that 
might help tracking output watermark in transforms that want to deal 
with potentially droppable data downstream (the input would have to be 
re-windowed to global window, of course). I don't know if there are 
other use-cases, if not maybe it might be sufficient to create a 
DroppableDataSplit transform, that would create a PCollectionTuple with 
droppable and other data. But that was just an idea when Kenn mentioned 
that the cleanup timers "fire differently" - I generally think that when 
there is a need for a different behavior, than it might signal there is 
something possibly fundamental.


 Jan

On 5/5/21 5:11 PM, Reuven Lax wrote:
This is one way to put it. I think in practice Beam guarantees that 
timers fire in order for a given key (though there is still a bit of a 
bug around looping timers - the fix for that got rolled back). This 
means that as long as the runner sets the cleanup timer to be 1ms 
passed the end of the window (plus allowed lateness), it's 
guaranteed to be the last timer that fires for that window.




On Wed, May 5, 2021 at 2:41 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


Hm, one thing in the last paragraph seems there might be some
logical gap.

> For a runner author to implement a "cleanup timer" requires a
different mechanism. A window expires when *both* the input
element watermark *and* the timer watermark are past the expiry
time. In other words, the cleanup timer fires according to the
minimum of these watermarks, combined. It *cannot* fire according
to the input element watermark. If you naively try to implement it
as a user timer, it will be incorrect. Incidentally this is
why @OnWindowExpiration is a meaningful feature.

The description describes a timer that fires not according to
input watermark, but according to the output watermark (once the
output watermark reaches certain point in time). That logically
implies, that such a timer cannot have non-droppable output (at
least if its output timestamp belongs to the respective window)
and cannot create a watermark hold (because that would block the
progress of the output watermark and might cause the timer to not
fire ever). This maybe might be a useful user-feature as well,
probably again mostly related to how user-code might want to deal
with droppable data.

 Jan

On 5/4/21 6:41 PM, Kenneth Knowles wrote:

Mean to also add +Reuven Lax <mailto:re...@google.com>

On Tue, May 4, 2021 at 9:41 AM Kenneth Knowles mailto:k...@apache.org>> wrote:

Explicitly pinging a couple folks who were involved in the
original change which yours essentially reverts. There's a
model question here that I want to clarify on-list:

When you have a ParDo setting timers, you have an additional
watermark that must be considered:

 - input element watermark
 - output watermark
 - *(user) timer watermark*

The timer watermark is an input to the ParDo. Sometimes you
might think of the "timer channel" as a self loop, where each
timer is an element. Each timer has a timestamp (the output
timestamp) and separately some instructions on when to
deliver that timer. This is the same as the usual difference
between event time and processing time.

The instruction on when to deliver a timer can have two forms:

 - wait a certain amount of processing time
 - deliver the timer when the *input element watermark*
reaches a time X

Here is an important point: "cleanup timers" are *not* user
timers. They are an implementation detail. They are not part
of the model. The runner's job is to reclaim resources as
windows expire. A user should never be reasoning about how
their timers relate to cleanup timers (except for resource
consumption). Because there is no relationship except that
the cleanup should happen "eventually" and invisibly.

For a runner author to implement a "cleanup timer" requires a
different mechanism. A window expires when *both* the input
element watermark *and* the timer watermark are past the
expiry time. In other words, the cleanup timer fires
according to the minimum of these watermarks, combined. It
*cannot* fire according to the input element watermark. If
you naively try to implement it as a user timer, it will be
incorrect. Incidentally this is why @OnWindowExpiration is a
    meaningful feature.


Re: Timer.withOutputTimestamp().offset().setRelative seems unusable with event time

2021-05-05 Thread Jan Lukavský

Hm, one thing in the last paragraph seems there might be some logical gap.

> For a runner author to implement a "cleanup timer" requires a 
different mechanism. A window expires when *both* the input element 
watermark *and* the timer watermark are past the expiry time. In other 
words, the cleanup timer fires according to the minimum of these 
watermarks, combined. It *cannot* fire according to the input element 
watermark. If you naively try to implement it as a user timer, it will 
be incorrect. Incidentally this is why @OnWindowExpiration is a 
meaningful feature.


The description describes a timer that fires not according to input 
watermark, but according to the output watermark (once the output 
watermark reaches certain point in time). That logically implies, that 
such a timer cannot have non-droppable output (at least if its output 
timestamp belongs to the respective window) and cannot create a 
watermark hold (because that would block the progress of the output 
watermark and might cause the timer to not fire ever). This maybe might 
be a useful user-feature as well, probably again mostly related to how 
user-code might want to deal with droppable data.


 Jan

On 5/4/21 6:41 PM, Kenneth Knowles wrote:

Mean to also add +Reuven Lax <mailto:re...@google.com>

On Tue, May 4, 2021 at 9:41 AM Kenneth Knowles <mailto:k...@apache.org>> wrote:


Explicitly pinging a couple folks who were involved in the
original change which yours essentially reverts. There's a model
question here that I want to clarify on-list:

When you have a ParDo setting timers, you have an additional
watermark that must be considered:

 - input element watermark
 - output watermark
 - *(user) timer watermark*

The timer watermark is an input to the ParDo. Sometimes you might
think of the "timer channel" as a self loop, where each timer is
an element. Each timer has a timestamp (the output timestamp) and
separately some instructions on when to deliver that timer. This
is the same as the usual difference between event time and
processing time.

The instruction on when to deliver a timer can have two forms:

 - wait a certain amount of processing time
 - deliver the timer when the *input element watermark* reaches a
time X

Here is an important point: "cleanup timers" are *not* user
timers. They are an implementation detail. They are not part of
the model. The runner's job is to reclaim resources as windows
expire. A user should never be reasoning about how their timers
relate to cleanup timers (except for resource consumption).
Because there is no relationship except that the cleanup should
happen "eventually" and invisibly.

For a runner author to implement a "cleanup timer" requires a
different mechanism. A window expires when *both* the input
element watermark *and* the timer watermark are past the expiry
time. In other words, the cleanup timer fires according to the
minimum of these watermarks, combined. It *cannot* fire according
to the input element watermark. If you naively try to implement it
as a user timer, it will be incorrect. Incidentally this is
why @OnWindowExpiration is a meaningful feature.

Kenn

On Tue, May 4, 2021 at 4:29 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

Hi Kenn,

I created BEAM-12276 [1] with PR [2].

 Jan

[1] https://issues.apache.org/jira/browse/BEAM-12276
<https://issues.apache.org/jira/browse/BEAM-12276>

[2] https://github.com/apache/beam/pull/14718
<https://github.com/apache/beam/pull/14718>

On 5/3/21 7:46 PM, Kenneth Knowles wrote:

This seems like just a bug. If you set a timer for X and have
output timestamp Y where X < Y this should be fine. Is the
problem the current input watermark? Are you trying to set a
timer with output timestamp that is already past? I think
that should be allowed, too, as long as the window is not
expired, but I may be missing something.

Some greater detail would be useful - maybe the full stack
trace and/or a failing unit test in a PR?

Kenn

On Thu, Apr 29, 2021 at 12:51 AM Jan Lukavský
mailto:je...@seznam.cz>> wrote:

Hi,

I have come across a bug with timer output timestamp -
when using event
time and relative timers, setting the timer can
arbitrarily throw
IllegalArgumentException if the firing timestamp (input
watermark) is
ahead of the output timestamp (like
.java.lang.IllegalArgumentException:
Attempted to set an event-time timer with an output
timestamp of
2021-04-29T07:16:19.369Z that is after the timer firing
  

Re: Timer.withOutputTimestamp().offset().setRelative seems unusable with event time

2021-05-04 Thread Jan Lukavský

Hi Kenn,

I created BEAM-12276 [1] with PR [2].

 Jan

[1] https://issues.apache.org/jira/browse/BEAM-12276

[2] https://github.com/apache/beam/pull/14718

On 5/3/21 7:46 PM, Kenneth Knowles wrote:
This seems like just a bug. If you set a timer for X and have output 
timestamp Y where X < Y this should be fine. Is the problem the 
current input watermark? Are you trying to set a timer with output 
timestamp that is already past? I think that should be allowed, too, 
as long as the window is not expired, but I may be missing something.


Some greater detail would be useful - maybe the full stack trace 
and/or a failing unit test in a PR?


Kenn

On Thu, Apr 29, 2021 at 12:51 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


Hi,

I have come across a bug with timer output timestamp - when using
event
time and relative timers, setting the timer can arbitrarily throw
IllegalArgumentException if the firing timestamp (input watermark) is
ahead of the output timestamp (like
.java.lang.IllegalArgumentException:
Attempted to set an event-time timer with an output timestamp of
2021-04-29T07:16:19.369Z that is after the timer firing timestamp
-290308-12-21T19:59:05.225Z). But there is no way to access the
firing
timestamp from user code. This means that the use has to either catch
the IllegalArgumentException, or not use this construct at all.

Catching the exception should probably not be part of a contract,
so we
should do one of the following:

  a) either throw the exception right away and disable using relative
timers with output timestamp completely, or

  b) support it correctly

What is the actual reason not to support output timestamps, that are
ahead of firing timestamp? From my understanding, that should not
be an
issue, because there is TimestampCombiner.EARLIEST on the
watermarkholdstate that corresponds to the output timestamp. If
that is
correct can we simply remove the check?

  Jan



Timer.withOutputTimestamp().offset().setRelative seems unusable with event time

2021-04-29 Thread Jan Lukavský

Hi,

I have come across a bug with timer output timestamp - when using event 
time and relative timers, setting the timer can arbitrarily throw 
IllegalArgumentException if the firing timestamp (input watermark) is 
ahead of the output timestamp (like .java.lang.IllegalArgumentException: 
Attempted to set an event-time timer with an output timestamp of 
2021-04-29T07:16:19.369Z that is after the timer firing timestamp 
-290308-12-21T19:59:05.225Z). But there is no way to access the firing 
timestamp from user code. This means that the use has to either catch 
the IllegalArgumentException, or not use this construct at all.


Catching the exception should probably not be part of a contract, so we 
should do one of the following:


 a) either throw the exception right away and disable using relative 
timers with output timestamp completely, or


 b) support it correctly

What is the actual reason not to support output timestamps, that are 
ahead of firing timestamp? From my understanding, that should not be an 
issue, because there is TimestampCombiner.EARLIEST on the 
watermarkholdstate that corresponds to the output timestamp. If that is 
correct can we simply remove the check?


 Jan



Re: Event time matching of side input

2021-04-27 Thread Jan Lukavský
Actually yes, tagging PCollections and using stateful DoFn can solve 
that in all cases - the memory requirements for batch should be solvable 
using @RequiresTimeSortedInput, as that is what causes the issues on 
batch on runners that support it.


Another yes, this is somewhat related to a full stream-to-stream join 
with exact time characteristics, where each side should be matched to 
the closest preceding update on the other side (again, some 
oversimplification).


"Event time side inputs" could be a handy tool for solving these 
problems in cases where one side is "reasonably small", so that it can 
simply fir into memory, yet still be arbitrarily fast updatable (as 
opposed to the "slowly changing caches" patterns that we sometimes see 
using side inputs - the reasons for this "slowness" seem to be in the 
inherent processing time matching in global windows). It should also 
give a more efficient runtime characteristics.


On 4/27/21 10:27 PM, Robert Bradshaw wrote:

So, I was just thinking when Jan mentioned "timestamp indexed" that
side input and state happen to be implemented on top of roughly the
same thing (in streaming Dataflow, and over the FnAPI) but the
connection might be deeper than that. Side inputs could be viewed as a
special type of (typically bag or map) state that is fully populated
by another PCollection and has a set eviction property. Maybe there's
some deeper unification here that could generalize side inputs.

On the other hand, it seems you can get any desired side input
behavior by taking as your main input a tagged union of the various
PCollections and using state+timers. (This may, however, require
infeasibly large memory requirements for batch.) The value is in a
more general, modular pattern that both pipeline authors and systems
can reason about.

On Tue, Apr 27, 2021 at 1:01 PM Reuven Lax  wrote:

The watermark discussion may be a red herring here.

I think it's interesting to think about the case where the side input is 
produced by a state/timers DoFn. in this scenario, there may be more 
interesting ways to match the side input than simply using windowing. It might 
be worth thinking about this topic more deeply.

On Tue, Apr 27, 2021 at 12:51 PM Jan Lukavský  wrote:

On 4/27/21 9:26 PM, Robert Bradshaw wrote:


On Tue, Apr 27, 2021 at 12:05 PM Jan Lukavský  wrote:

On 4/27/21 8:51 PM, Robert Bradshaw wrote:

On Tue, Apr 27, 2021 at 11:25 AM Jan Lukavský  wrote:

Are you asking for a way to ignore early triggers on side input mapping, and 
only map to on-time triggered values for the window?

No, that could for sure be done before applying the View transform. I'd like a 
know if it would be possible to create mode of the matching which would be 
deterministic. One possibility to make it deterministic seems to be, that main 
input elements would be pushed back until side input watermark 'catches up' 
with main input. Whenever the side input watermark would be delayed after the 
main input watermark, elements would start to be pushed back again. Not sure if 
I'm explaining it using the right words. The side input watermark can be 
controlled using timer in an upstream transform, so this defines which elements 
in main input would be matched onto which pane of the side input.

Perhaps I'm not following the request correctly, but this is exactly
how side inputs work by default. It is only when one explicitly
requests a non-deterministic trigger upstream of the side input (e.g.
one that may fire multiple times or ahead of the watermark) that one
sees a side input with multiple variations or data in the side input
before the watermark of the side input is caught up to the main input.

Yes, exactly. But take the example of side input in global windows (on
both the main input and side input). Then there has to be multiple
firings per window, because otherwise the side input would be available
at the end of time, which is not practical. The trigger doesn't have to
be non-deterministic, the data might come from a stateful ParDo, using a
timer with output timestamp, which would make the downstream watermark
progress quite well defined. The matching would still be
nondeterministic in this case.

If everything is in the global window, things get non-deterministic
across PCollections. For example, say the main input has element m20
and the side input has elements s10 and s30 (with the obvious
timestamps). Suppose we have a total ordering of events as follows.

  s10 arrives
  side input watermark advances to 25
  s30 arrives
  side input watermark advances to 100
  m20 arrives

In this case, while processing m20, one would see both s10 and s30.
Alternatively we could have had

  s10 arrives
  side input watermark advances to 25
  m20 arrives
  s30 arrives
  side input watermark advances to 100

in which case m20 would only see s10.

Yes, this is absolutely true, I didn't want to m

Re: Event time matching of side input

2021-04-27 Thread Jan Lukavský

On 4/27/21 9:26 PM, Robert Bradshaw wrote:


On Tue, Apr 27, 2021 at 12:05 PM Jan Lukavský  wrote:

On 4/27/21 8:51 PM, Robert Bradshaw wrote:

On Tue, Apr 27, 2021 at 11:25 AM Jan Lukavský  wrote:

Are you asking for a way to ignore early triggers on side input mapping, and 
only map to on-time triggered values for the window?

No, that could for sure be done before applying the View transform. I'd like a 
know if it would be possible to create mode of the matching which would be 
deterministic. One possibility to make it deterministic seems to be, that main 
input elements would be pushed back until side input watermark 'catches up' 
with main input. Whenever the side input watermark would be delayed after the 
main input watermark, elements would start to be pushed back again. Not sure if 
I'm explaining it using the right words. The side input watermark can be 
controlled using timer in an upstream transform, so this defines which elements 
in main input would be matched onto which pane of the side input.

Perhaps I'm not following the request correctly, but this is exactly
how side inputs work by default. It is only when one explicitly
requests a non-deterministic trigger upstream of the side input (e.g.
one that may fire multiple times or ahead of the watermark) that one
sees a side input with multiple variations or data in the side input
before the watermark of the side input is caught up to the main input.

Yes, exactly. But take the example of side input in global windows (on
both the main input and side input). Then there has to be multiple
firings per window, because otherwise the side input would be available
at the end of time, which is not practical. The trigger doesn't have to
be non-deterministic, the data might come from a stateful ParDo, using a
timer with output timestamp, which would make the downstream watermark
progress quite well defined. The matching would still be
nondeterministic in this case.

If everything is in the global window, things get non-deterministic
across PCollections. For example, say the main input has element m20
and the side input has elements s10 and s30 (with the obvious
timestamps). Suppose we have a total ordering of events as follows.

 s10 arrives
 side input watermark advances to 25
 s30 arrives
 side input watermark advances to 100
 m20 arrives

In this case, while processing m20, one would see both s10 and s30.
Alternatively we could have had

 s10 arrives
 side input watermark advances to 25
 m20 arrives
 s30 arrives
 side input watermark advances to 100

in which case m20 would only see s10.
Yes, this is absolutely true, I didn't want to make things too 
complicated, so I ignored this fact, that absolutely correct solution 
would require the PCollectioView to be timestamp indexed so that if it 
would receive both s10 and s30 it would correctly return s10 (as s30 
didn't exist at m20, right?). I simplified this at this moment for this 
discussion, thanks for clarifying.


Windowing is exactly the mechanism that gives us a cross-PCollection
barrier with which to line things up.

Should not watermark do this?





On 4/27/21 8:03 PM, Reuven Lax wrote:



On Tue, Apr 27, 2021 at 10:53 AM Jan Lukavský  wrote:

If using early triggers, then the side input loads the latest trigger for that 
window.

Does not the the word 'latest' imply processing time matching? The watermark of main 
input might be arbitrarily delayed from the watermark of side input. If we consider 
GlobalWindows on both sides, than "latest" trigger in side input looks exactly 
as processing time matching. Yes, between different windows, the matching is in event 
time. But within the same window (ignoring the window mapping for now), the matching 
looks like processing time, right?

Not really. Think of each trigger of a window as a refinement - so the latest trigger for 
that window is our best approximation of the "correct" value for that window.  
For this reason, window panes are indexed by an integer (pane_index), not by a timestamp. 
The idea here is that the main input element maps to the side input for the best, 
most-recent knowledge of the window.

Are you asking for a way to ignore early triggers on side input mapping, and 
only map to on-time triggered values for the window?


If we look at the SimplePushbackSideInputDoFnRunner used by runners exactly for 
the side input matching, there is no testing of side input watermark to 
determine if an element should be 'pushed back' or processed. Element is 
processed if, and only if, all side inputs for particular window are ready.

On 4/27/21 7:24 PM, Reuven Lax wrote:

The windows shouldn't need to match - at least if the FlinkRunner 
implementation is correct. By default, the side-input's WindowFn should be used 
to map the main input's timestamp into a window, and that window is used to 
determine which version of the side input to load. A custom WindowFn can be 
used t

Re: Event time matching of side input

2021-04-27 Thread Jan Lukavský

On 4/27/21 8:51 PM, Robert Bradshaw wrote:

On Tue, Apr 27, 2021 at 11:25 AM Jan Lukavský  wrote:

Are you asking for a way to ignore early triggers on side input mapping, and 
only map to on-time triggered values for the window?

No, that could for sure be done before applying the View transform. I'd like a 
know if it would be possible to create mode of the matching which would be 
deterministic. One possibility to make it deterministic seems to be, that main 
input elements would be pushed back until side input watermark 'catches up' 
with main input. Whenever the side input watermark would be delayed after the 
main input watermark, elements would start to be pushed back again. Not sure if 
I'm explaining it using the right words. The side input watermark can be 
controlled using timer in an upstream transform, so this defines which elements 
in main input would be matched onto which pane of the side input.

Perhaps I'm not following the request correctly, but this is exactly
how side inputs work by default. It is only when one explicitly
requests a non-deterministic trigger upstream of the side input (e.g.
one that may fire multiple times or ahead of the watermark) that one
sees a side input with multiple variations or data in the side input
before the watermark of the side input is caught up to the main input.
Yes, exactly. But take the example of side input in global windows (on 
both the main input and side input). Then there has to be multiple 
firings per window, because otherwise the side input would be available 
at the end of time, which is not practical. The trigger doesn't have to 
be non-deterministic, the data might come from a stateful ParDo, using a 
timer with output timestamp, which would make the downstream watermark 
progress quite well defined. The matching would still be 
nondeterministic in this case.




On 4/27/21 8:03 PM, Reuven Lax wrote:



On Tue, Apr 27, 2021 at 10:53 AM Jan Lukavský  wrote:

If using early triggers, then the side input loads the latest trigger for that 
window.

Does not the the word 'latest' imply processing time matching? The watermark of main 
input might be arbitrarily delayed from the watermark of side input. If we consider 
GlobalWindows on both sides, than "latest" trigger in side input looks exactly 
as processing time matching. Yes, between different windows, the matching is in event 
time. But within the same window (ignoring the window mapping for now), the matching 
looks like processing time, right?


Not really. Think of each trigger of a window as a refinement - so the latest trigger for 
that window is our best approximation of the "correct" value for that window.  
For this reason, window panes are indexed by an integer (pane_index), not by a timestamp. 
The idea here is that the main input element maps to the side input for the best, 
most-recent knowledge of the window.

Are you asking for a way to ignore early triggers on side input mapping, and 
only map to on-time triggered values for the window?


If we look at the SimplePushbackSideInputDoFnRunner used by runners exactly for 
the side input matching, there is no testing of side input watermark to 
determine if an element should be 'pushed back' or processed. Element is 
processed if, and only if, all side inputs for particular window are ready.

On 4/27/21 7:24 PM, Reuven Lax wrote:

The windows shouldn't need to match - at least if the FlinkRunner 
implementation is correct. By default, the side-input's WindowFn should be used 
to map the main input's timestamp into a window, and that window is used to 
determine which version of the side input to load. A custom WindowFn can be 
used to to even more - e.g. if you want the main input element to map to the 
_previous_ window in the side input (you would do this by overloading 
getDefaultWindowMappingFn).

If using early triggers, then the side input loads the latest trigger for that 
window. This is still an event-time mapping - for example two main-input events 
in two different windows will still map the the side input for the matching 
window. However if the side input PCollection is triggered, than the latest 
trigger for each window's side input will be loaded.

It's possible that the FlinkRunner implementation is incomplete, in which case 
it should be fixed.

On Tue, Apr 27, 2021 at 9:36 AM Jan Lukavský  wrote:

It seems to me, that this is true only with matching windows on both sides and 
default trigger of the side input. Then it will (due to condition a)) work as 
if the matching happenned in event time. But when using any early triggers then 
it will work in processing time. At least, that is my understanding from 
studying the code in FlinkRunner.

On 4/27/21 4:05 PM, Robert Burke wrote:

I thought the matching happened with elements in the matching window, in Event 
time, not in Processing time.

Granted, I'm not that familiar with this area myself, but one key part of Beam 
is nearly ever

Re: Event time matching of side input

2021-04-27 Thread Jan Lukavský
> Are you asking for a way to ignore early triggers on side input 
mapping, and only map to on-time triggered values for the window?


No, that could for sure be done before applying the View transform. I'd 
like a know if it would be possible to create mode of the matching which 
would be deterministic. One possibility to make it deterministic seems 
to be, that main input elements would be pushed back until side input 
watermark 'catches up' with main input. Whenever the side input 
watermark would be delayed after the main input watermark, elements 
would start to be pushed back again. Not sure if I'm explaining it using 
the right words. The side input watermark can be controlled using timer 
in an upstream transform, so this defines which elements in main input 
would be matched onto which pane of the side input.


On 4/27/21 8:03 PM, Reuven Lax wrote:



On Tue, Apr 27, 2021 at 10:53 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


> If using early triggers, then the side input loads the latest
trigger for that window.

Does not the the word 'latest' imply processing time matching? The
watermark of main input might be arbitrarily delayed from the
watermark of side input. If we consider GlobalWindows on both
sides, than "latest" trigger in side input looks exactly as
processing time matching. Yes, between different windows, the
matching is in event time. But within the same window (ignoring
the window mapping for now), the matching looks like processing
time, right?


Not really. Think of each trigger of a window as a refinement - so the 
latest trigger for that window is our best approximation of the 
"correct" value for that window. For this reason, window panes are 
indexed by an integer (pane_index), not by a timestamp. The idea here 
is that the main input element maps to the side input for the best, 
most-recent knowledge of the window.


Are you asking for a way to ignore early triggers on side input 
mapping, and only map to on-time triggered values for the window?


If we look at the SimplePushbackSideInputDoFnRunner used by
runners exactly for the side input matching, there is no testing
of side input watermark to determine if an element should be
'pushed back' or processed. Element is processed if, and only if,
all side inputs for particular window are ready.

On 4/27/21 7:24 PM, Reuven Lax wrote:

The windows shouldn't need to match - at least if the FlinkRunner
implementation is correct. By default, the side-input's WindowFn
should be used to map the main input's timestamp into a window,
and that window is used to determine which version of the side
input to load. A custom WindowFn can be used to to even more -
e.g. if you want the main input element to map to the _previous_
window in the side input (you would do this by overloading
getDefaultWindowMappingFn).

If using early triggers, then the side input loads the latest
trigger for that window. This is still an event-time mapping -
for example two main-input events in two different windows will
still map the the side input for the matching window. However if
the side input PCollection is triggered, than the latest trigger
for each window's side input will be loaded.

It's possible that the FlinkRunner implementation is incomplete,
in which case it should be fixed.

    On Tue, Apr 27, 2021 at 9:36 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

It seems to me, that this is true only with matching windows
on both sides and default trigger of the side input. Then it
will (due to condition a)) work as if the matching happenned
in event time. But when using any early triggers then it will
work in processing time. At least, that is my understanding
from studying the code in FlinkRunner.

On 4/27/21 4:05 PM, Robert Burke wrote:

I thought the matching happened with elements in the
matching window, in Event time, not in Processing time.

Granted, I'm not that familiar with this area myself, but
one key part of Beam is nearly everything is Event time by
default, not Processing time.

    On Tue, Apr 27, 2021, 12:43 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

Hi,

I have a question about matching side inputs to main
input. First a
recap, to make sure I understand the current state
correctly:

  a) elements from main input are pushed back (stored in
state) until a
first side input pane arrives (that might be on time, or
early)

  b) after that, elements from the main input are
matched to the current
side input view - the view is updated as new data
arrives, but is
matche

Re: Event time matching of side input

2021-04-27 Thread Jan Lukavský
> If using early triggers, then the side input loads the latest trigger 
for that window.


Does not the the word 'latest' imply processing time matching? The 
watermark of main input might be arbitrarily delayed from the watermark 
of side input. If we consider GlobalWindows on both sides, than "latest" 
trigger in side input looks exactly as processing time matching. Yes, 
between different windows, the matching is in event time. But within the 
same window (ignoring the window mapping for now), the matching looks 
like processing time, right?


If we look at the SimplePushbackSideInputDoFnRunner used by runners 
exactly for the side input matching, there is no testing of side input 
watermark to determine if an element should be 'pushed back' or 
processed. Element is processed if, and only if, all side inputs for 
particular window are ready.


On 4/27/21 7:24 PM, Reuven Lax wrote:
The windows shouldn't need to match - at least if the FlinkRunner 
implementation is correct. By default, the side-input's WindowFn 
should be used to map the main input's timestamp into a window, and 
that window is used to determine which version of the side input to 
load. A custom WindowFn can be used to to even more - e.g. if you want 
the main input element to map to the _previous_ window in the side 
input (you would do this by overloading getDefaultWindowMappingFn).


If using early triggers, then the side input loads the latest trigger 
for that window. This is still an event-time mapping - for example two 
main-input events in two different windows will still map the the side 
input for the matching window. However if the side input PCollection 
is triggered, than the latest trigger for each window's side input 
will be loaded.


It's possible that the FlinkRunner implementation is incomplete, in 
which case it should be fixed.


On Tue, Apr 27, 2021 at 9:36 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


It seems to me, that this is true only with matching windows on
both sides and default trigger of the side input. Then it will
(due to condition a)) work as if the matching happenned in event
time. But when using any early triggers then it will work in
processing time. At least, that is my understanding from studying
the code in FlinkRunner.

On 4/27/21 4:05 PM, Robert Burke wrote:

I thought the matching happened with elements in the matching
window, in Event time, not in Processing time.

Granted, I'm not that familiar with this area myself, but one key
part of Beam is nearly everything is Event time by default, not
Processing time.

On Tue, Apr 27, 2021, 12:43 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

Hi,

I have a question about matching side inputs to main input.
First a
recap, to make sure I understand the current state correctly:

  a) elements from main input are pushed back (stored in
state) until a
first side input pane arrives (that might be on time, or early)

  b) after that, elements from the main input are matched to
the current
side input view - the view is updated as new data arrives,
but is
matched to the main input elements in processing time

If this is the current state, my question is, would it be
possible to
add a new mode of matching of side inputs? Something like

  ParDo.of(new MyDoFn()).withSideInput("name", myView,
TimeDomain.EVENT_TIME)

the semantics would be that the elements from the main
PCollection would
be stored into state as pairs with the value of the current
main input
watermark and on update of side-input watermark only main
input elements
with associated input watermark less than that of the side
input would
be matched with the side input and sent downstream. Although
this
approach is necessarily more expensive and introducing
additional
latency than processing time matching, there are situations when
processing time matching is inapropriate for correctness reasons.

WDYT?

  Jan



Re: Event time matching of side input

2021-04-27 Thread Jan Lukavský

Hi Reuven,

the code in FlinkRunner really does not seem to work with watermarks. 
Moreover, what I was able to find, is this quote in [1]:


"The global window side input triggers on processing time, so the main 
pipeline nondeterministically matches the side input to elements in 
event time."


 Jan

[1] https://beam.apache.org/documentation/patterns/side-inputs/

On 4/27/21 6:31 PM, Reuven Lax wrote:

Side inputs are matched based on event time, not processing time.

On Tue, Apr 27, 2021 at 12:43 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


Hi,

I have a question about matching side inputs to main input. First a
recap, to make sure I understand the current state correctly:

  a) elements from main input are pushed back (stored in state)
until a
first side input pane arrives (that might be on time, or early)

  b) after that, elements from the main input are matched to the
current
side input view - the view is updated as new data arrives, but is
matched to the main input elements in processing time

If this is the current state, my question is, would it be possible to
add a new mode of matching of side inputs? Something like

  ParDo.of(new MyDoFn()).withSideInput("name", myView,
TimeDomain.EVENT_TIME)

the semantics would be that the elements from the main PCollection
would
be stored into state as pairs with the value of the current main
input
watermark and on update of side-input watermark only main input
elements
with associated input watermark less than that of the side input
would
be matched with the side input and sent downstream. Although this
approach is necessarily more expensive and introducing additional
latency than processing time matching, there are situations when
processing time matching is inapropriate for correctness reasons.

WDYT?

  Jan



Re: Event time matching of side input

2021-04-27 Thread Jan Lukavský
It seems to me, that this is true only with matching windows on both 
sides and default trigger of the side input. Then it will (due to 
condition a)) work as if the matching happenned in event time. But when 
using any early triggers then it will work in processing time. At least, 
that is my understanding from studying the code in FlinkRunner.


On 4/27/21 4:05 PM, Robert Burke wrote:
I thought the matching happened with elements in the matching window, 
in Event time, not in Processing time.


Granted, I'm not that familiar with this area myself, but one key part 
of Beam is nearly everything is Event time by default, not Processing 
time.


On Tue, Apr 27, 2021, 12:43 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


Hi,

I have a question about matching side inputs to main input. First a
recap, to make sure I understand the current state correctly:

  a) elements from main input are pushed back (stored in state)
until a
first side input pane arrives (that might be on time, or early)

  b) after that, elements from the main input are matched to the
current
side input view - the view is updated as new data arrives, but is
matched to the main input elements in processing time

If this is the current state, my question is, would it be possible to
add a new mode of matching of side inputs? Something like

  ParDo.of(new MyDoFn()).withSideInput("name", myView,
TimeDomain.EVENT_TIME)

the semantics would be that the elements from the main PCollection
would
be stored into state as pairs with the value of the current main
input
watermark and on update of side-input watermark only main input
elements
with associated input watermark less than that of the side input
would
be matched with the side input and sent downstream. Although this
approach is necessarily more expensive and introducing additional
latency than processing time matching, there are situations when
processing time matching is inapropriate for correctness reasons.

WDYT?

  Jan



Event time matching of side input

2021-04-27 Thread Jan Lukavský

Hi,

I have a question about matching side inputs to main input. First a 
recap, to make sure I understand the current state correctly:


 a) elements from main input are pushed back (stored in state) until a 
first side input pane arrives (that might be on time, or early)


 b) after that, elements from the main input are matched to the current 
side input view - the view is updated as new data arrives, but is 
matched to the main input elements in processing time


If this is the current state, my question is, would it be possible to 
add a new mode of matching of side inputs? Something like


 ParDo.of(new MyDoFn()).withSideInput("name", myView, 
TimeDomain.EVENT_TIME)


the semantics would be that the elements from the main PCollection would 
be stored into state as pairs with the value of the current main input 
watermark and on update of side-input watermark only main input elements 
with associated input watermark less than that of the side input would 
be matched with the side input and sent downstream. Although this 
approach is necessarily more expensive and introducing additional 
latency than processing time matching, there are situations when 
processing time matching is inapropriate for correctness reasons.


WDYT?

 Jan



Re: Should WindowFn have a mininal Duration?

2021-04-26 Thread Jan Lukavský

Hi Robert and Reuven,

I was not aware that implementing custom windowing logic is that much 
"common" practice. If so, I think that probably makes little sense to 
"force" users specify the minimal duration - though it could be made 
somewhat "user-friendly", but still, it would require some work on user 
side. Maybe I'll rephrase the motivation is actually the ability to 
generate a set of BoundedWindow labels, that cover a specific time 
interval - and does not leave any window behind. This is obviously 
possible only for time-only windows (which is not the case Reuven 
mentioned with "terminating sessions", which are data-sensitive 
windows). Maybe that would boil down to only the set of built-in 
WindowFns? Can we reasonable presume that users would create custom 
windows not sensitive to data? If so, that would seem like a 
generic-type of windows that could be suitable to include to the 
built-in ones?


 Jan

On 4/26/21 8:28 PM, Reuven Lax wrote:
I've often seen custom windowfns with no static minimum duration. e.g. 
a common customization of sessions is to identify a specific "logout" 
event to end the session.


On Mon, Apr 26, 2021 at 11:08 AM Robert Bradshaw <mailto:rober...@google.com>> wrote:


I do think minimal window duration is a meaningful concept for
WindowFns, but from the pragmatic perspective I would ask is it useful
enough to require all implementers of WindowFn to specify it (given
that a default value of 0 would not be very useful).

On Mon, Apr 26, 2021 at 10:05 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:
>
> Hi Kenn,
>
> On 4/26/21 5:59 PM, Kenneth Knowles wrote:
>
> In +Reza Rokni's example of looping timers, it is necessary to
"seed" each key, for just the reason you say. The looping timer
itself for a key should be in the global window. The outputs of
the looping timer are windowed.
>
> Yes, exactly.
>
>
> All that said, your example seems possibly easier if you are OK
with no output for windows with no data.
>
> The problem is actually not with windows with no data. But with
windows containing only droppable data. This "toy example" is
interestingly much more complex than I expected. Pretty much due
to the reason, that there is no access to watermark while
processing elements. But yes, there are probably more efficient
ways to solve that, the best option would be to have access to the
input watermark (e.g. at the start of the bundle, that seems to be
well defined, though I understand there is some negative
experience with that approach). But I don't want to discuss the
solutions, actually.
>
> My "motivating example" was merely a motivation for me to ask
this question (and possible one more about side inputs is to
follow :)), but - giving all examples and possible solutions
aside, the question is - is a minimal duration an intrinsic
property of a WindowFn, or not? If yes, I think there are reasons
to include this property into the model. If no, then we can
discuss the reason why is it the case. I see the only problem with
data-driven windows, all other windows are time-based and as such,
probably carry this property. The data-driven WindowFns could have
this property defined as zero. This is not a super critical
request, more of a philosophical discussion.
>
>  Jan
>
> It sounds like you don't actually want to drop the data, yes?
You want to partition elements at some time X that is in the
middle of some event time interval. If I understand your chosen
approach, you could buffer the element w/ metadata and set the
timer in @ProcessElement. It is no problem if the timestamp of the
timer has already passed. It will fire immediately then. In the
@OnTimer you output from the buffer. I think there may be more
efficient ways to achieve this output.
>
> Kenn
>
> On Thu, Apr 22, 2021 at 2:48 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:
>>
>> Hi,
>>
>> I have come across a "problem" while implementing some toy
Pipeline. I
>> would like to split input PCollection into two parts -
droppable data
>> (delayed for more than allowed lateness from the end of the
window) from
>> the rest. I will not go into details, as that is not relevant, the
>> problem is that I need to setup something like "looping timer"
to be
>> able to create state for a window, even when there is no data,
yet (to
>> be able to setup timer for the end of a window, to be able to
recognize
>> droppable data). I would like the solution to be generic, so I
would
 

Re: Should WindowFn have a mininal Duration?

2021-04-26 Thread Jan Lukavský

Hi Kenn,

On 4/26/21 5:59 PM, Kenneth Knowles wrote:
In +Reza Rokni <mailto:rezaro...@google.com>'s example of looping 
timers, it is necessary to "seed" each key, for just the reason you 
say. The looping timer itself for a key should be in the global 
window. The outputs of the looping timer are windowed.

Yes, exactly.


All that said, your example seems possibly easier if you are OK with 
no output for windows with no data.


The problem is actually not with windows with no data. But with windows 
containing only droppable data. This "toy example" is interestingly much 
more complex than I expected. Pretty much due to the reason, that there 
is no access to watermark while processing elements. But yes, there are 
probably more efficient ways to solve that, the best option would be to 
have access to the input watermark (e.g. at the start of the bundle, 
that seems to be well defined, though I understand there is some 
negative experience with that approach). But I don't want to discuss the 
solutions, actually.


My "motivating example" was merely a motivation for me to ask this 
question (and possible one more about side inputs is to follow :)), but 
- giving all examples and possible solutions aside, the question is - is 
a minimal duration an intrinsic property of a WindowFn, or not? If yes, 
I think there are reasons to include this property into the model. If 
no, then we can discuss the reason why is it the case. I see the only 
problem with data-driven windows, all other windows are time-based and 
as such, probably carry this property. The data-driven WindowFns could 
have this property defined as zero. This is not a super critical 
request, more of a philosophical discussion.


 Jan

It sounds like you don't actually want to drop the data, yes? You want 
to partition elements at some time X that is in the middle of some 
event time interval. If I understand your chosen approach, you could 
buffer the element w/ metadata and set the timer in @ProcessElement. 
It is no problem if the timestamp of the timer has already passed. It 
will fire immediately then. In the @OnTimer you output from the 
buffer. I think there may be more efficient ways to achieve this output.


Kenn

On Thu, Apr 22, 2021 at 2:48 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


Hi,

I have come across a "problem" while implementing some toy
Pipeline. I
would like to split input PCollection into two parts - droppable data
(delayed for more than allowed lateness from the end of the
window) from
the rest. I will not go into details, as that is not relevant, the
problem is that I need to setup something like "looping timer" to be
able to create state for a window, even when there is no data, yet
(to
be able to setup timer for the end of a window, to be able to
recognize
droppable data). I would like the solution to be generic, so I would
like to "infer" the duration of the looping timer from the input
PCollection. What I would need is to know a _minimal guaranteed
duration
of a window that a WindowFn can generate_. I would then setup the
looping timer to tick with interval of this minimal duration and that
would guarantee the timer will hit all the windows.

I could try to infer this duration from the input windowing with some
hackish ways - e.g. using some "instanceof" approach, or by using the
WindowFn to generate set of windows for some fixed timestamp (without
data element) and then infer the time from maxTimestamp of the
returned
windows. That would probably break for sliding windows, because the
result would be the duration of the slide, not the duration of the
window (at least when doing naive computation).

It seems to me, that all WindowFns have such a minimal Duration -
obvious for Fixed Windows, but every other window type seems to have
such property (including Sessions - that is the gap duration). The
only
problem would be with data-driven windows, but we don't have
currently
strong support for these.

The question is then - would it make sense to introduce
WindowFn.getMinimalWindowDuration() to the model? Default value
could be
zero, which would mean such WindowFn would be unsupported in my
motivating example.

  Jan



Should WindowFn have a mininal Duration?

2021-04-22 Thread Jan Lukavský

Hi,

I have come across a "problem" while implementing some toy Pipeline. I 
would like to split input PCollection into two parts - droppable data 
(delayed for more than allowed lateness from the end of the window) from 
the rest. I will not go into details, as that is not relevant, the 
problem is that I need to setup something like "looping timer" to be 
able to create state for a window, even when there is no data, yet (to 
be able to setup timer for the end of a window, to be able to recognize 
droppable data). I would like the solution to be generic, so I would 
like to "infer" the duration of the looping timer from the input 
PCollection. What I would need is to know a _minimal guaranteed duration 
of a window that a WindowFn can generate_. I would then setup the 
looping timer to tick with interval of this minimal duration and that 
would guarantee the timer will hit all the windows.


I could try to infer this duration from the input windowing with some 
hackish ways - e.g. using some "instanceof" approach, or by using the 
WindowFn to generate set of windows for some fixed timestamp (without 
data element) and then infer the time from maxTimestamp of the returned 
windows. That would probably break for sliding windows, because the 
result would be the duration of the slide, not the duration of the 
window (at least when doing naive computation).


It seems to me, that all WindowFns have such a minimal Duration - 
obvious for Fixed Windows, but every other window type seems to have 
such property (including Sessions - that is the gap duration). The only 
problem would be with data-driven windows, but we don't have currently 
strong support for these.


The question is then - would it make sense to introduce 
WindowFn.getMinimalWindowDuration() to the model? Default value could be 
zero, which would mean such WindowFn would be unsupported in my 
motivating example.


 Jan



Re: [ANNOUNCE] New committer: Yichi Zhang

2021-04-22 Thread Jan Lukavský

Congrats Yichi!

On 4/22/21 4:58 AM, Ahmet Altay wrote:

Congratulations Yichi! 

On Wed, Apr 21, 2021 at 6:48 PM Chamikara Jayalath 
mailto:chamik...@google.com>> wrote:


Congrats Yichi!

On Wed, Apr 21, 2021 at 6:14 PM Heejong Lee mailto:heej...@google.com>> wrote:

Congratulations :)

On Wed, Apr 21, 2021 at 5:20 PM Tomo Suzuki
mailto:suzt...@google.com>> wrote:

Congratulations!

On Wed, Apr 21, 2021 at 7:48 PM Tyson Hamilton
mailto:tyso...@google.com>> wrote:

Congrats!

On Wed, Apr 21, 2021 at 4:37 PM Valentyn Tymofieiev
mailto:valen...@google.com>> wrote:

Well deserved and congrats, Yichi!

On Wed, Apr 21, 2021 at 4:23 PM Pablo Estrada
mailto:pabl...@google.com>>
wrote:

Hi all,

Please join me and the rest of the Beam PMC in
welcoming a new committer: Yichi Zhang

Yichi has been working in Beam for a while. He
has contributed to various areas, including
Nexmark tests, test health, Python's streaming
capabilities, he has answered questions on
StackOverflow, and helped with release
validations, among many other things that
Yichi has contributed to the Beam community.

Considering these contributions, the Beam PMC
trusts Yichi with the responsibilities of a
Beam committer.[1]

Thanks Yichi!
-P.

[1]

https://beam.apache.org/contribute/become-a-committer/#an-apache-beam-committer





-- 
Regards,

Tomo



Re: Long term support versions of Beam Java

2021-04-08 Thread Jan Lukavský
Understand that, my point was just that we cannot care about API changes 
only. What an LTS has to maintain is binary compatibility of state or 
have any kind of upgrade process without the running Pipeline being 
terminated and completely replaced with another one. Bootstrapping the 
state is what is - from my point of view - the most expensive part of 
upgrade with state-incompatibility. If an update to LTS breaks the state 
compatibility, then upgrading to the new (bugfixed) LTS version seems to 
be (about) the same pain as upgrading to the newest version of Beam. 
That is what somewhat breaks the "LTS" expectations users might have. 
But yes, fixes which break state might not be that frequent, but are 
possible, that's for sure. In that case, we essentially could not 
release a fix, even if it was critical security fix. My question was 
then if these concerns will be somehow part of the proposed LTS support 
process.


 Jan

On 4/7/21 4:23 PM, Elliotte Rusty Harold wrote:

On Tue, Apr 6, 2021 at 9:43 PM Jan Lukavský  wrote:

Hi,
do we know what is the reason users stay on an older version of Beam? My guess would be that it is 
not related to API changes, but more likely to state incompatibility. Maybe if we could figure out 
a way which would enable a smooth migration of state (and timers) between Beam versions, that might 
help? The migration would probably have to be runner-dependent, but Beam might offer some tooling 
to make this easier. One example would be coder evolution, where we currently do not have the 
option of "reading old way, writing new way" with some 
"coder-version-registry". I suppose there might have been a discussion about this in the 
past, does anyone know of any conclusion?


Different customers have different requirements. The projects and
teams I'm talking to simply don't want to invest engineering time in
migrating as often as they have to now. They want to be able to ship a
working product and not worry about it for a year or more. They
recognize the possibility of unforeseen bugs and security issues that
might require an update. However they'd like that update to be minimal
and not require them to update their own code. Nor do they want to run
the risk of performance regressions and newly introduced bugs and
security issues.



Re: Long term support versions of Beam Java

2021-04-06 Thread Jan Lukavský

Hi,
do we know what is the reason users stay on an older version of Beam? My 
guess would be that it is not related to API changes, but more likely to 
state incompatibility. Maybe if we could figure out a way which would 
enable a smooth migration of state (and timers) between Beam versions, 
that might help? The migration would probably have to be 
runner-dependent, but Beam might offer some tooling to make this easier. 
One example would be coder evolution, where we currently do not have the 
option of "reading old way, writing new way" with some 
"coder-version-registry". I suppose there might have been a discussion 
about this in the past, does anyone know of any conclusion?


 Jan

On 4/6/21 10:54 PM, Robert Bradshaw wrote:
I do think there's value in having an LTS release, if there's 
sufficient interest to fund it (specifically, figuring out who would 
be backporting fixes and cutting the new releases).


On Mon, Apr 5, 2021 at 1:14 PM Elliotte Rusty Harold 
mailto:elh...@ibiblio.org>> wrote:


Hi,

I'd like to return to the discussion around a long term support
release that first came up here in 2018:


https://lists.apache.org/thread.html/6ec572d8edfe93225edebec18792cbcf44ef447ffe54ea35549cdafe%40%3Cdev.beam.apache.org%3E



This is important to some Google Cloud Dataflow Java customers, and
likely others as well.

Specifically, I'd like to propose cutting an LTS release off a branch
and maintaining it with critical bug fixes and security updates for 18
months. Right now we're finding that the current one year support
period and six week release cycle is a tad fast for some customers.

There's some wiggle room in terms of what's "critical", but in
that category I include security fixes and data integrity issues.
Essentially, this is any bug so bad that, if found in a new release,
we'd recommend customers wait for the fix before upgrading to the
latest and greatest. The difference is we'd backport the patch to the
not-latest-and-greatest release.

To run something up the flagpole, I propose:

1. 2.28.0 becomes the first LTS release.
2. New patch versions are released as 2.28.1, 2.28.2, etc.
3. Patch releases do not change API, at all, except in the unlikely
event this is absolutely required for a security fix.
4. Dependencies are not upgraded in patch releases unless required to
fix a critical bug or security issue.
5. In a year, cut a new LTS release from whatever is then current so
there's some overlap to give customers time to switch over.

I picked 2.28.0 since it's the most recent release, and I prefer to
stay off the bleeding edge for longterm support. This would also
enable customers to develop on top of it sooner. However I understand
others may well prefer to pick a different release such as 2.29.0 or
2.30.0. I'm OK with whatever recent version the community picks.

Thoughts?

-- 
Elliotte Rusty Harold

elh...@ibiblio.org 



Re: Null checking in Beam

2021-04-06 Thread Jan Lukavský
I agree that there are _some_ added annotations at _some_, that are 
useful - most notably @NonNull on method arguments, possibly return 
values. Adding @NonNull into exception type being thrown seems awkward. 
The @UnknownKeyFor probably should not be there, as it brings no value. 
Did we raise the issue with the checkerframework? It seems to me, that 
the biggest problem lies there. It might have two modes of operation - 
after the check it could have a way of specifying which (and where) 
annotations should be in the compiled byte-code and which should be 
removed. Or can we post-process that with some different tool?


 Jan

On 4/5/21 6:03 PM, Kenneth Knowles wrote:


On Thu, Apr 1, 2021 at 9:57 AM Brian Hulette <mailto:bhule...@google.com>> wrote:


What value does it add? Is it that it enables them to use
checkerframework with our interfaces?


Actually if they are also using checkerframework the defaults are the 
same so it is not usually needed (though some defaults can be 
changed). Making defaults explicit is most useful for interfacing with 
other tools with different defaults, such as Spotbugs [1], IDEs [2] 
[3], or JVM languages with null safety bult-in, etc [4] [5].


Kenn

[1] 
https://spotbugs.readthedocs.io/en/stable/annotations.html#edu-umd-cs-findbugs-annotations-nullable 
<https://spotbugs.readthedocs.io/en/stable/annotations.html#edu-umd-cs-findbugs-annotations-nullable>
[2] 
https://www.jetbrains.com/help/idea/2021.1/nullable-and-notnull-annotations.html 
<https://www.jetbrains.com/help/idea/2021.1/nullable-and-notnull-annotations.html>
[3] https://wiki.eclipse.org/JDT_Core/Null_Analysis 
<https://wiki.eclipse.org/JDT_Core/Null_Analysis>
[4] https://kotlinlang.org/docs/null-safety.html 
<https://kotlinlang.org/docs/null-safety.html>
[5] 
https://kotlinlang.org/docs/java-interop.html#null-safety-and-platform-types 
<https://kotlinlang.org/docs/java-interop.html#null-safety-and-platform-types>


On Thu, Apr 1, 2021 at 8:54 AM Kenneth Knowles mailto:k...@apache.org>> wrote:

Thanks for filing that. Once it is fixed in IntelliJ, the
annotations actually add value for downstream users.

Kenn

    On Thu, Apr 1, 2021 at 1:10 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

Hi,

I created the issue in JetBrains tracker [1]. I'm still
not 100% convinced that it is correct for the checker to
actually modify the bytecode. An open questions is - in
guava this does not happen. Does guava apply the check on
code being released? From what is in this thread is seems
to me, that the answer is no.

 Jan

[1] https://youtrack.jetbrains.com/issue/IDEA-265658
<https://youtrack.jetbrains.com/issue/IDEA-265658>

On 4/1/21 6:15 AM, Kenneth Knowles wrote:

Hi all,

About the IntelliJ automatic method stub issue: I raised
it to the checkerframework list and got a helpful
response:

https://groups.google.com/g/checker-framework-discuss/c/KHQdjF4lesk/m/dJ4u1BBNBgAJ

<https://groups.google.com/g/checker-framework-discuss/c/KHQdjF4lesk/m/dJ4u1BBNBgAJ>

It eventually reached back to Jetbrains and they would
appreciate a detailed report with steps to reproduce,
preferably a sample project. Would you - Jan or Ismaël or
Reuven - provide them with this issue report? It sounds
like Jan you have an example ready to go.

    Kenn

On Mon, Mar 15, 2021 at 1:29 PM Jan Lukavský
mailto:je...@seznam.cz>> wrote:

Yes, annotations that we add to the code base on
purpose (like @Nullable or @SuppressWarnings) are
aboslutely fine. What is worse is that the checked is
not only checked, but a code generator. :)

For example when one wants to implement Coder by
extending CustomCoder and use auto-generating the
overridden methods, they look like

@Override public void encode(Long value, @UnknownKeyFor 
@NonNull @Initialized OutputStream outStream)throws 
@UnknownKeyFor@NonNull@Initialized CoderException, 
@UnknownKeyFor@NonNull@Initialized IOException {

}

Which is really ugly. :-(

 Jan

On 3/15/21 6:37 PM, Ismaël Mejía wrote:

+1

Even if I like the strictness for Null checking, I also think 
that
this is adding too much extra time for builds (that I noticed 
locally
when enabled) and also I agree with Jan that the annotations are
really an undesired side effect. For reference when you try to 
auto
complete some method signatures on IntelliJ on do

Re: Null checking in Beam

2021-04-01 Thread Jan Lukavský

Hi,

I created the issue in JetBrains tracker [1]. I'm still not 100% 
convinced that it is correct for the checker to actually modify the 
bytecode. An open questions is - in guava this does not happen. Does 
guava apply the check on code being released? From what is in this 
thread is seems to me, that the answer is no.


 Jan

[1] https://youtrack.jetbrains.com/issue/IDEA-265658

On 4/1/21 6:15 AM, Kenneth Knowles wrote:

Hi all,

About the IntelliJ automatic method stub issue: I raised it to the 
checkerframework list and got a helpful response: 
https://groups.google.com/g/checker-framework-discuss/c/KHQdjF4lesk/m/dJ4u1BBNBgAJ 
<https://groups.google.com/g/checker-framework-discuss/c/KHQdjF4lesk/m/dJ4u1BBNBgAJ>


It eventually reached back to Jetbrains and they would appreciate a 
detailed report with steps to reproduce, preferably a sample project. 
Would you - Jan or Ismaël or Reuven - provide them with this issue 
report? It sounds like Jan you have an example ready to go.


Kenn

On Mon, Mar 15, 2021 at 1:29 PM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


Yes, annotations that we add to the code base on purpose (like
@Nullable or @SuppressWarnings) are aboslutely fine. What is worse
is that the checked is not only checked, but a code generator. :)

For example when one wants to implement Coder by extending
CustomCoder and use auto-generating the overridden methods, they
look like

@Override public void encode(Long value, @UnknownKeyFor @NonNull 
@Initialized OutputStream outStream)throws @UnknownKeyFor@NonNull@Initialized 
CoderException, @UnknownKeyFor@NonNull@Initialized IOException {

}

Which is really ugly. :-(

 Jan

On 3/15/21 6:37 PM, Ismaël Mejía wrote:

+1

Even if I like the strictness for Null checking, I also think that
this is adding too much extra time for builds (that I noticed locally
when enabled) and also I agree with Jan that the annotations are
really an undesired side effect. For reference when you try to auto
complete some method signatures on IntelliJ on downstream projects
with C-A-v it generates some extra Checkers annotations like @NonNull
and others even if the user isn't using them which is not desirable.



On Mon, Mar 15, 2021 at 6:04 PM Kyle Weaver  
<mailto:kcwea...@google.com>  wrote:

Big +1 for moving this to separate CI job. I really don't like what 
annotations are currently added to the code we ship. Tools like Idea add these 
annotations to code they generate when overriding classes and that's very 
annoying. Users should not be exposed to internal tools like nullability 
checking.

I was only planning on moving this to a separate CI job. The job would 
still be release blocking, so the same annotations would still be required.

I'm not sure which annotations you are concerned about. There are two 
annotations involved with nullness checking, @SuppressWarnings and @Nullable. 
@SuppressWarnings has retention policy SOURCE, so it shouldn't be exposed to 
users at all. @Nullable is not just for internal tooling, it also provides 
useful information about our APIs to users. The user should not have to guess 
whether a method argument etc. can be null or not, and for better or worse, 
these annotations are the standard way of expressing that in Java.




Re: Null checking in Beam

2021-03-17 Thread Jan Lukavský
If there is no way to configure which annotations should be generated, 
then I'd be +1 for removing the checker to separated CI and adding an 
opt-in flag for the check when run locally.


We need to solve the issue for dev@ as well, as the undesirable 
annotations are already digging their way to the code base:


git/apache/beam$ git grep UnknownKeyFor

Another strange thing is that it seems, that we are pulling the 
checkerframework as a runtime dependency (at least for some submodules). 
When I run `mvn dependency:tree` on one of my projects that uses maven I see


[INFO] +- com.google.guava:guava:jar:30.1-jre:provided
[INFO] |  +- com.google.guava:failureaccess:jar:1.0.1:provided
[INFO] |  +- 
com.google.guava:listenablefuture:jar:.0-empty-to-avoid-conflict-with-guava:provided

[INFO] |  +- org.checkerframework:checker-qual:jar:3.5.0:provided

which is fine, but when I add beam-sdks-java-extensions-euphoria it 
changes to


[INFO] +- 
org.apache.beam:beam-sdks-java-extensions-euphoria:jar:2.28.0:compile

[INFO] |  \- org.checkerframework:checker-qual:jar:3.7.0:compile

I'm not a gradle guru, so I cannot tell how this happens, there seems to 
be nothing special about euphoria in the gradle.


 Jan

On 3/16/21 7:12 PM, Kenneth Knowles wrote:
I've asked on checkerframework users mailing list if they or any users 
have recommendations for the IntelliJ integration issue.


It is worth noting that the default annotations inserted into the 
bytecode do add value: the @NonNull type annotations are default for 
checkerframework but not default for spotbugs. So having the default 
inserted enables downstream users to have betters spotbugs heuristic 
analysis. Further, the defaults can be adjusted by users so freezing 
them at the values we use them at is valuable in general across all tools.


I think it makes sense to sacrifice these minor value-adds to keep 
things simple-looking for IntelliJ users.


Kenn

On Tue, Mar 16, 2021 at 10:05 AM Kenneth Knowles <mailto:k...@apache.org>> wrote:


Seems it is an FAQ with no solution:
https://checkerframework.org/manual/#faq-classfile-annotations
<https://checkerframework.org/manual/#faq-classfile-annotations>

On Tue, Mar 16, 2021 at 10:01 AM Kenneth Knowles mailto:k...@apache.org>> wrote:

Adding -PskipCheckerframework when releasing will solve it for
users, but not for dev@.

Making it off by default and a separate CI check that turns it
on would solve it overall but has the downsides mentioned before.

It would be very nice to simply have a flag to not insert
default annotations.

Kenn

On Tue, Mar 16, 2021 at 9:37 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

I believe it is not a problem of Idea. At least I didn't
notice behavior like that with Guava, although Guava uses
the framework as well. Maybe there is a way to tune which
annotations should be generated? Or Guava handles that
somehow differently?

On 3/16/21 5:22 PM, Reuven Lax wrote:

I've also been annoyed at IntelliJ autogenenerating all
these annotations. I believe Kenn said that this was not
the intention - maybe there's an IntelliJ setting that
would stop this from happening?

On Tue, Mar 16, 2021 at 2:14 AM Jan Lukavský
mailto:je...@seznam.cz>> wrote:

I don't know the details of the checkerframework, but
there seems a contradiction between what code is
(currently) generated and what we therefore release
and what actually the checkerframework states [1]:

@UnknownKeyFor:

Used internally by the type system; should never be
written by a programmer.

If this annotation is generated for overwritten
methods, then I'd say, that it means we place a great
burden to our users - either not using autogenerated
methods, or erase all the generated annotations
afterwards. Either way, that is not "polite" from Beam.

What we should judge is not only a formal purity of
code, but what stands on the other side is how usable
APIs we provide. We should not head for 100% pure
code and sacrificing use comfort. Every check that
leaks to user code is at a price and we should not
ignore that. No free lunch.

From my point of view:

 a) if a check does not modify the bytecode, it is
fine and we can use it - we are absolutely free to
use any tooling we agree on, if our users cannot be
affected anyhow

 b) if a tool needs to be leaked to user, it should
be as sm

Re: Null checking in Beam

2021-03-16 Thread Jan Lukavský
I believe it is not a problem of Idea. At least I didn't notice behavior 
like that with Guava, although Guava uses the framework as well. Maybe 
there is a way to tune which annotations should be generated? Or Guava 
handles that somehow differently?


On 3/16/21 5:22 PM, Reuven Lax wrote:
I've also been annoyed at IntelliJ autogenenerating all these 
annotations. I believe Kenn said that this was not the intention - 
maybe there's an IntelliJ setting that would stop this from happening?


On Tue, Mar 16, 2021 at 2:14 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


I don't know the details of the checkerframework, but there seems
a contradiction between what code is (currently) generated and
what we therefore release and what actually the checkerframework
states [1]:

@UnknownKeyFor:

Used internally by the type system; should never be written by a
programmer.

If this annotation is generated for overwritten methods, then I'd
say, that it means we place a great burden to our users - either
not using autogenerated methods, or erase all the generated
annotations afterwards. Either way, that is not "polite" from Beam.

What we should judge is not only a formal purity of code, but what
stands on the other side is how usable APIs we provide. We should
not head for 100% pure code and sacrificing use comfort. Every
check that leaks to user code is at a price and we should not
ignore that. No free lunch.

From my point of view:

 a) if a check does not modify the bytecode, it is fine and we can
use it - we are absolutely free to use any tooling we agree on, if
our users cannot be affected anyhow

 b) if a tool needs to be leaked to user, it should be as small
leakage as possible

 c) if a check significantly affects compile performance, it
should be possible to opt-out

I think that our current setup violates all these three points.

Moving the check to different CI is a possibility (a)), it would
then require opt-in flag to be able to run the check locally. It
would also stop the leakage (if we would release code without this
check).

If we want to keep some annotations for user's benefit (which
might be fine), it should be really limited to the bare minimum
(e.g. only @Nullable for method arguments and return values,
possibly more, I don't know if and how we can configure that).
Definitely not @UnknownKeyFor, that is simply internal to the
checker. We should then have opt-out flag for local development
before committing changes.

 Jan

[1]

https://checkerframework.org/api/org/checkerframework/checker/nullness/qual/UnknownKeyFor.html

<https://checkerframework.org/api/org/checkerframework/checker/nullness/qual/UnknownKeyFor.html>

On 3/16/21 8:33 AM, Reuven Lax wrote:



On Mon, Mar 15, 2021 at 11:42 PM Reuven Lax mailto:re...@google.com>> wrote:



On Mon, Mar 15, 2021 at 9:12 PM Kenneth Knowles
mailto:k...@apache.org>> wrote:

I will be blunt about my opinions about the general issue:

- NullPointerExceptions (and similar) are a solved problem.
   * They have been since 2003 at the latest [1] (this is
when the types were hacked into Java - the foundation
dates back to the 70s or earlier)


Huh - Fahndrich tried to hire me once to work on a project
called Singularity. Small world. Also note that Sanjay
Ghemawat is listed in the citations!


Umm, I was confusing names. Fahndrich is actually a former
coworker at Google :)

   * Checkerframework is a _pluggable_ system where that
nullness type system is a "hello, world" level demo,
since 2008 at the latest [2].
   * Our users should know this and judge us accordingly.

- Checkerframework should be thought of and described as
type checking, because it is. It is not heuristic nor
approximate.
- If your code was unclear about whether something could
be null, it was probably unclear to a person reading it
also, and very likely to have actual bugs.
- APIs that accept a lot of nullable parameters are,
generally speaking, bad APIs. They are hard to use
correctly, less readable, and very likely to cause actual
bugs. You are forcing your users to deal with accidental
complexity you left behind.
  * Corollary to the above two points: Almost all the
time, the changes to clearify nullness make your code
better, more readable, easier for users or editors.
- It is true that there is a learning curve to
programming in this way.


I agree with the above in a closed system. However as
   

Re: Null checking in Beam

2021-03-16 Thread Jan Lukavský
I don't know the details of the checkerframework, but there seems a 
contradiction between what code is (currently) generated and what we 
therefore release and what actually the checkerframework states [1]:


@UnknownKeyFor:

Used internally by the type system; should never be written by a 
programmer.


If this annotation is generated for overwritten methods, then I'd say, 
that it means we place a great burden to our users - either not using 
autogenerated methods, or erase all the generated annotations 
afterwards. Either way, that is not "polite" from Beam.


What we should judge is not only a formal purity of code, but what 
stands on the other side is how usable APIs we provide. We should not 
head for 100% pure code and sacrificing use comfort. Every check that 
leaks to user code is at a price and we should not ignore that. No free 
lunch.


From my point of view:

 a) if a check does not modify the bytecode, it is fine and we can use 
it - we are absolutely free to use any tooling we agree on, if our users 
cannot be affected anyhow


 b) if a tool needs to be leaked to user, it should be as small leakage 
as possible


 c) if a check significantly affects compile performance, it should be 
possible to opt-out


I think that our current setup violates all these three points.

Moving the check to different CI is a possibility (a)), it would then 
require opt-in flag to be able to run the check locally. It would also 
stop the leakage (if we would release code without this check).


If we want to keep some annotations for user's benefit (which might be 
fine), it should be really limited to the bare minimum (e.g. only 
@Nullable for method arguments and return values, possibly more, I don't 
know if and how we can configure that). Definitely not @UnknownKeyFor, 
that is simply internal to the checker. We should then have opt-out flag 
for local development before committing changes.


 Jan

[1] 
https://checkerframework.org/api/org/checkerframework/checker/nullness/qual/UnknownKeyFor.html


On 3/16/21 8:33 AM, Reuven Lax wrote:



On Mon, Mar 15, 2021 at 11:42 PM Reuven Lax > wrote:




On Mon, Mar 15, 2021 at 9:12 PM Kenneth Knowles mailto:k...@apache.org>> wrote:

I will be blunt about my opinions about the general issue:

- NullPointerExceptions (and similar) are a solved problem.
   * They have been since 2003 at the latest [1] (this is when
the types were hacked into Java - the foundation dates back to
the 70s or earlier)


Huh - Fahndrich tried to hire me once to work on a project called
Singularity. Small world. Also note that Sanjay Ghemawat is listed
in the citations!


Umm, I was confusing names. Fahndrich is actually a former coworker at 
Google :)


   * Checkerframework is a _pluggable_ system where that
nullness type system is a "hello, world" level demo, since
2008 at the latest [2].
   * Our users should know this and judge us accordingly.

- Checkerframework should be thought of and described as type
checking, because it is. It is not heuristic nor approximate.
- If your code was unclear about whether something could be
null, it was probably unclear to a person reading it also, and
very likely to have actual bugs.
- APIs that accept a lot of nullable parameters are, generally
speaking, bad APIs. They are hard to use correctly, less
readable, and very likely to cause actual bugs. You are
forcing your users to deal with accidental complexity you left
behind.
  * Corollary to the above two points: Almost all the time,
the changes to clearify nullness make your code better, more
readable, easier for users or editors.
- It is true that there is a learning curve to programming in
this way.


I agree with the above in a closed system. However as mentioned,
some of the APIs we use suffer from this.

In a previous life, I saw up close an effort to add such analysis
to a large codebase. Two separate tools called Prefix and Prefast
were used (the difference between the two is actually quite
interesting, but not relevant here). However in order to make this
analysis useful, there was a massive effort to properly annotate
the entire codebase, including all libraries used. This isn't a
perfect example - this was a C++ codebase which is much harder to
analyze, and these tools identified far more than simply nullness
errors (resource leaks, array indices, proper string null
termination, exception behavior, etc.). However the closer we can
get to properly annotating the transitive closure of our
dependencies, the better this framework will work.

- There are certainly common patterns in Java that do not work
very well, and suppression is sometimes the best option.
   * Example: JUnit's 

Re: Null checking in Beam

2021-03-15 Thread Jan Lukavský
Yes, annotations that we add to the code base on purpose (like @Nullable 
or @SuppressWarnings) are aboslutely fine. What is worse is that the 
checked is not only checked, but a code generator. :)


For example when one wants to implement Coder by extending CustomCoder 
and use auto-generating the overridden methods, they look like


@Override public void encode(Long value, @UnknownKeyFor @NonNull @Initialized 
OutputStream outStream)throws @UnknownKeyFor@NonNull@Initialized 
CoderException, @UnknownKeyFor@NonNull@Initialized IOException {

}

Which is really ugly. :-(

 Jan

On 3/15/21 6:37 PM, Ismaël Mejía wrote:

+1

Even if I like the strictness for Null checking, I also think that
this is adding too much extra time for builds (that I noticed locally
when enabled) and also I agree with Jan that the annotations are
really an undesired side effect. For reference when you try to auto
complete some method signatures on IntelliJ on downstream projects
with C-A-v it generates some extra Checkers annotations like @NonNull
and others even if the user isn't using them which is not desirable.



On Mon, Mar 15, 2021 at 6:04 PM Kyle Weaver  wrote:

Big +1 for moving this to separate CI job. I really don't like what annotations 
are currently added to the code we ship. Tools like Idea add these annotations 
to code they generate when overriding classes and that's very annoying. Users 
should not be exposed to internal tools like nullability checking.


I was only planning on moving this to a separate CI job. The job would still be 
release blocking, so the same annotations would still be required.

I'm not sure which annotations you are concerned about. There are two 
annotations involved with nullness checking, @SuppressWarnings and @Nullable. 
@SuppressWarnings has retention policy SOURCE, so it shouldn't be exposed to 
users at all. @Nullable is not just for internal tooling, it also provides 
useful information about our APIs to users. The user should not have to guess 
whether a method argument etc. can be null or not, and for better or worse, 
these annotations are the standard way of expressing that in Java.


Re: Null checking in Beam

2021-03-15 Thread Jan Lukavský
 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 mailto:bhule...@google.com>> wrote:



On Fri, Jan 22, 2021 at 1:18 AM Jan
Lukavský mailto:je...@seznam.cz>> 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
 

Re: Do we need synchronized processing time? / What to do about "continuation triggers"?

2021-02-25 Thread Jan Lukavský

Hi Kenn,

> The idea with sink triggering is that the level of abstraction is 
raised. You have a DoFn (more generally IO transform) that writes to 
some external system, and you request updates every ten seconds. This 
specification is propagated to cause all the GBKs in the pipeline to 
emit data at a rate to enable updates to that IO every ten seconds.


I get this, this makes totally sense. But - what else could the 
propagation meaningfully do, then to propagate the 10 seconds triggering 
to the very first GBK(s) and then try to push the outcome of these 
PTransforms as fast as possible through the pipeline? Yes, seems it 
would require retractions, at least in cases when the DAG contains 
multiple paths from root(s) to leaf. It seems to me, that the 
intermediate GBK(s) play no role, because if they do not trigger as fast 
as possible (and retract wrongly triggered outputs due to 
out-of-orderness), what they do, is they add latency and actually make 
the "sink triggering" not trigger at the configured frequency. 
Everything else seems clear to me, I just don't get this part. Is is 
possible to describe a specific a example where an inner GBK would 
trigger with some different trigger than with each pane?


 Jan

On 2/25/21 12:44 AM, Kenneth Knowles wrote:



On Wed, Feb 24, 2021 at 12:44 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


Hi Robert,

> Here "sink" is really any observable outside effect, so I
think "how often output should be written" and "how quickly output
should react to the change of input" are the same.

The difference is in the input trigger - let's imagine, that I
have two chained GBKs (A and B). If I trigger A every minute, but
B every second, I will output 60 records per minute, but 59 of
them will be the same. That's why it seems to me, that meaningful
"sink" triggering has to start at the input and then propagate
with each pane.

The idea with sink triggering is that the level of abstraction is 
raised. You have a DoFn (more generally IO transform) that writes to 
some external system, and you request updates every ten seconds. This 
specification is propagated to cause all the GBKs in the pipeline to 
emit data at a rate to enable updates to that IO every ten seconds.


Sinks will need separate configurations to handle multiple panes 
(updates/retractions) vs final values, and we can validate that a sink 
can support a particular triggering strategy. Sinks already need this, 
but we haven't solved the problem very formally or systematically. In 
many cases, these are just two different sinks - for example a CSV 
file with an extra column to indicate overwrite/retraction is really a 
different sink than just appending. They write to the same storage 
system, but the relationship of the input records to the output 
storage differs.


There's a lot of unsolved problems in terms of exactly how the 
triggering requirements of a sink can feed back to upstream 
aggregations to cause them to trigger at appropriate times. It could 
be static (inferring upstream triggering) but seems like it might have 
to be dynamic (running a state machine at the sink that broadcasts 
messages). I don't think this is straightforward, nor is it guaranteed 
to be doable without knobs or some fresh ideas.


Kenn

> As an example, if I want, say, hourly output, triggering hourly
at the source and then as quickly as possible from then on may be
wasteful. It may also be desirable to arrange such that certain
transforms only have a single pane per window, which is easier to
propagate up than down. As another example, consider accumulating
vs. discarding. If I have CombineValues(sum) followed by a
re-keying and another CombineValues(sum), and I want the final
output to be accumulating, the first must be discarding (or,
better, retractions). Propagating upwards is possible in a way
propagating downward is not.

I'm not sure I understand this. If I want hourly output, I cannot
trigger source with lower frequency. If I trigger source with
hourly, but do not propagate this as fast as possible, I'm
inevitably introducing additional latency (that's the definition
of "not as fast as possible") in downstream processing. Therefore
the final triggering cannot be "hourly output" at least not with
regard to the rate of change in inputs.

On 2/23/21 5:47 PM, Robert Bradshaw wrote:

On Tue, Feb 23, 2021 at 1:07 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

First, +1 to the conclusion of this thread.

One note regarding the composite transforms and triggers
*inside* those transforms - I think that propagating the
triggering from input PCollection might be even dangerous and
composite PTransforms that would be sensitive to the change
of triggering will (sh

Re: Do we need synchronized processing time? / What to do about "continuation triggers"?

2021-02-24 Thread Jan Lukavský

Hi Robert,

> Here "sink" is really any observable outside effect, so I think "how 
often output should be written" and "how quickly output should react to 
the change of input" are the same.


The difference is in the input trigger - let's imagine, that I have two 
chained GBKs (A and B). If I trigger A every minute, but B every second, 
I will output 60 records per minute, but 59 of them will be the same. 
That's why it seems to me, that meaningful "sink" triggering has to 
start at the input and then propagate with each pane.


> As an example, if I want, say, hourly output, triggering hourly at 
the source and then as quickly as possible from then on may be wasteful. 
It may also be desirable to arrange such that certain transforms only 
have a single pane per window, which is easier to propagate up than 
down. As another example, consider accumulating vs. discarding. If I 
have CombineValues(sum) followed by a re-keying and another 
CombineValues(sum), and I want the final output to be accumulating, the 
first must be discarding (or, better, retractions). Propagating upwards 
is possible in a way propagating downward is not.


I'm not sure I understand this. If I want hourly output, I cannot 
trigger source with lower frequency. If I trigger source with hourly, 
but do not propagate this as fast as possible, I'm inevitably 
introducing additional latency (that's the definition of "not as fast as 
possible") in downstream processing. Therefore the final triggering 
cannot be "hourly output" at least not with regard to the rate of change 
in inputs.


On 2/23/21 5:47 PM, Robert Bradshaw wrote:
On Tue, Feb 23, 2021 at 1:07 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


First, +1 to the conclusion of this thread.

One note regarding the composite transforms and triggers *inside*
those transforms - I think that propagating the triggering from
input PCollection might be even dangerous and composite
PTransforms that would be sensitive to the change of triggering
will (should!) override the input triggering, and therefore
adjusting it upfront will not work. There is clear option for
composite PTransform (which includes one or more GBKs) to create
API to specify the _input_ triggering of the composite as a whole,
i.e.

 input.apply(MyComposite.create().triggering())

which (consistently with how triggering works for pure GBK) would
change the input triggering (if we define trigger as "buffer input
in state, flush buffer when trigger fires") of the PTransform. The
PTransform knows how it expands and so it is quite easy to do the
output triggering correctly.

When we originally explored this (for windowing, before triggering 
existed) we looked at the number of composite operations (combining, 
joining, cogbk, ...) that contained GBKs and realized it would add a 
lot of boilerplate to manually pass through the windowing information 
to each. Worse, this is a burden placed on every author of a composite 
operation (and omitting this argument, or hard coding a default, would 
be strictly worse). Triggering doesn't flow as nicely, but requiring 
it on every subtransform invocation during pipeline construction would 
have the same downsides of verbosity.


Regarding the sink triggering - out of curiosity, how does that
differ from applying the triggering on the very first GBK(s) and
the subsequently trigger all downstream GBKs using
AfterPane.elementCountAtLeast(1)? It seems to me, that from user
perspective what I will want to define is not "how often output
should be written", but "how quickly output should react to the
change of input" - therefore I *must* trigger with at least this
frequency from the source and then propagate each pane as quickly
as possible to the output. Am I missing something?

Here "sink" is really any observable outside effect, so I think "how 
often output should be written" and "how quickly output should react 
to the change of input" are the same.


As an example, if I want, say, hourly output, triggering hourly at the 
source and then as quickly as possible from then on may be wasteful. 
It may also be desirable to arrange such that certain transforms only 
have a single pane per window, which is easier to propagate up than 
down. As another example, consider accumulating vs. discarding. If I 
have CombineValues(sum) followed by a re-keying and another 
CombineValues(sum), and I want the final output to be accumulating, 
the first must be discarding (or, better, retractions). Propagating 
upwards is possible in a way propagating downward is not.



 Jan


On 2/22/21 9:53 PM, Reuven Lax wrote:

I really wish that we had found the time to build sink triggers.
Jan is right - specifying triggers up front and having them
propa

Re: Do we need synchronized processing time? / What to do about "continuation triggers"?

2021-02-23 Thread Jan Lukavský

First, +1 to the conclusion of this thread.

One note regarding the composite transforms and triggers *inside* those 
transforms - I think that propagating the triggering from input 
PCollection might be even dangerous and composite PTransforms that would 
be sensitive to the change of triggering will (should!) override the 
input triggering, and therefore adjusting it upfront will not work. 
There is clear option for composite PTransform (which includes one or 
more GBKs) to create API to specify the _input_ triggering of the 
composite as a whole, i.e.


 input.apply(MyComposite.create().triggering())

which (consistently with how triggering works for pure GBK) would change 
the input triggering (if we define trigger as "buffer input in state, 
flush buffer when trigger fires") of the PTransform. The PTransform 
knows how it expands and so it is quite easy to do the output triggering 
correctly.


Regarding the sink triggering - out of curiosity, how does that differ 
from applying the triggering on the very first GBK(s) and the 
subsequently trigger all downstream GBKs using 
AfterPane.elementCountAtLeast(1)? It seems to me, that from user 
perspective what I will want to define is not "how often output should 
be written", but "how quickly output should react to the change of 
input" - therefore I *must* trigger with at least this frequency from 
the source and then propagate each pane as quickly as possible to the 
output. Am I missing something?


 Jan


On 2/22/21 9:53 PM, Reuven Lax wrote:
I really wish that we had found the time to build sink triggers. Jan 
is right - specifying triggers up front and having them propagate down 
is confusing (it's also a bit confusing for Windows, but with Windows 
the propagation at least makes sense). The fact that users rarely have 
access to the actual GBK operation means that allowing them to specify 
triggers on their sinks is the best approach.


On Mon, Feb 22, 2021 at 12:48 PM Robert Bradshaw <mailto:rober...@google.com>> wrote:


On Mon, Feb 22, 2021 at 11:51 AM Kenneth Knowles mailto:k...@apache.org>> wrote:

I agree completely: Triggers control the output of the GBK.

The issue is composite transforms, where there will be a GBK
deep inside some code and the user cannot adjust the triggering.

What a user really wants is "sink triggers
<https://s.apache.org/beam-sink-triggers>" [1], a purely
hypothetical feature where they specify the latency
requirements on each _output_ and everything else is figured
out automatically. Unfortunately, sink triggers require
retractions, so each PCollection can be a complete changelog.
Otherwise transformations cannot be transparently correct
throughout a pipeline and triggers cannot be decoupled from
pipeline logic. Retractions themselves are not necessarily
complex in some cases (Flink SQL has them - they are extra
easy for "pure" code) but require a massive working of the
library of transforms, particularly IOs. And backwards
compatibility concerns for existing DoFns are somewhat tricky.
We've had two prototypes [2] [3] and some important design
investigations [4], but no time to really finish adding them,
even as just an optional experiment. And once we have
retractions, there is still a lot to figure out to finish sink
triggers. They may not even really be possible!

So for now, we do our best with the user setting up triggering
at the beginning of the pipeline instead of the end of the
pipeline. The very first GBK (which may be deep in library
code) is controlled by the triggering they set up and all the
rest get the "continuation trigger" which tries to just let
the data flow. Unless they set up another bit of triggering.
Some of our transforms do this for various reasons.

I think the conclusion of this particular thread is:

 - make all the SDKs use AfterSynchronizedProcessingTime triggers
 - allow runners to do whatever they want when they see
AfterSynchronizedProcessingTime trigger
 - remove TimeDomain.afterSynchronizedProcessingTime from the
proto since it is only for timers and they should not use this
 - later, figure out if we want to add support for making
downstream triggering optional (could be useful prep for sink
triggers)


+1

[1] https://s.apache.org/beam-sink-triggers
[2] https://github.com/apache/beam/pull/4742
[3] https://github.com/apache/beam/pull/9199
[4] https://s.apache.org/beam-retractions

On Mon, Feb 22, 2021 at 1:28 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

The same holds true for pane accumulation mode.

 Jan

On 2/2

Re: Do we need synchronized processing time? / What to do about "continuation triggers"?

2021-02-22 Thread Jan Lukavský

The same holds true for pane accumulation mode.

 Jan

On 2/22/21 10:21 AM, Jan Lukavský wrote:


Hi,

I'm not sure if I got everything from this thread right, but from my 
point of view, triggers are property of GBK. They are property of 
neither windowing, nor PCollection, but relate solely to GBK. This can 
be seen from the fact, that unlike windowFn, triggers are completely 
ignored in stateful ParDo (there is no semantics for them, which is 
fine). It would be cool if the model could be adjusted for that - this 
would actually mean, that the correct place, where to specify 
triggering is not Window PTransform, but the GBK, i.e.


 input.apply(GroupByKey.create().triggering(...))

That would imply we simply have default trigger for all GBKs, unless 
explicitly changed, but for that particular instance only. I'm not 
sure what the impacts on pipeline compatibility would be, though.


 Jan

On 2/19/21 12:09 AM, Robert Bradshaw wrote:
On Wed, Feb 17, 2021 at 1:56 PM Kenneth Knowles <mailto:k...@apache.org>> wrote:



On Wed, Feb 17, 2021 at 1:06 PM Robert Bradshaw
mailto:rober...@google.com>> wrote:

I would prefer to leave downstream triggering up to the
runner (or, better, leave upstream triggering up to the
runner, a la sink triggers), but one problem is that without
an explicit AfterSynchronizedProcessingTime one can't tell if
the downstream ProcessingTime between two groupings is due to
an explicit re-triggering between them or inherited from one
to the other.


I mean to propose that there should be no triggering specified
unless due to explicit re-triggering.


You're saying that we leave the trigger (and perhaps other) fields of 
the WindowingStrategy attached to PCollections downstream the first 
GBK unset in the proto? And let runners walk over the graph to infer 
it? I could be OK with making this legal, though updating all SDKs 
and Runners to handle this doesn't seem high priority at the moment.



(and BTW yes I agree about sink triggers, but we need retractions
and probably some theoretical work before we can aim for that)

Kenn

On Wed, Feb 17, 2021 at 12:37 PM Kenneth Knowles
mailto:k...@apache.org>> wrote:

Just for the thread I want to comment on another, more
drastic approach: eliminate continuation triggers from
the model, leaving downstream triggering up to a runner.
This approach is not viable because transforms may need
to change their behavior based on whether or not a
trigger will fire more than once. Transforms can and do
inspect the windowing strategy to do things differently.

Kenn

On Wed, Feb 17, 2021 at 11:47 AM Reuven Lax
mailto:re...@google.com>> wrote:

I'll say that synchronized processing time has
confused users before. Users sometimes use
processing-time triggers to optimize latency, banking
that that will decouple stage latency from the
long-tail latency of previous stages. However
continuation triggers silently switching to
synchronized processing time has defeated that, and
it wasn't clear to users why.

On Wed, Feb 17, 2021 at 11:12 AM Robert Bradshaw
mailto:rober...@google.com>> wrote:

On Fri, Feb 12, 2021 at 9:09 AM Kenneth Knowles
mailto:k...@apache.org>> wrote:


On Thu, Feb 11, 2021 at 9:38 PM Robert
Bradshaw mailto:rober...@google.com>> wrote:

Of course the right answer is to just
implement sink triggers and sidestep the
question altogether :).

In the meantime, I think leaving
AfterSynchronizedProcessingTime in the
model makes the most sense, and runners
can choose an implementation between
firing eagerly and waiting some amount of
time until they think all (most?)
downstream results are in before firing,
depending on how smart the runner wants
to be. As you point out, they're all
correct, and we'll have multiple firings
due to the upstream trigger anyway, and
this is safer than it used to be (though
still possibly requires work).


Just to clarify, as I got a little confused,
is your suggestion: Leave
AfterSynchronizedProcessingTime* 

Re: Do we need synchronized processing time? / What to do about "continuation triggers"?

2021-02-22 Thread Jan Lukavský

Hi,

I'm not sure if I got everything from this thread right, but from my 
point of view, triggers are property of GBK. They are property of 
neither windowing, nor PCollection, but relate solely to GBK. This can 
be seen from the fact, that unlike windowFn, triggers are completely 
ignored in stateful ParDo (there is no semantics for them, which is 
fine). It would be cool if the model could be adjusted for that - this 
would actually mean, that the correct place, where to specify triggering 
is not Window PTransform, but the GBK, i.e.


 input.apply(GroupByKey.create().triggering(...))

That would imply we simply have default trigger for all GBKs, unless 
explicitly changed, but for that particular instance only. I'm not sure 
what the impacts on pipeline compatibility would be, though.


 Jan

On 2/19/21 12:09 AM, Robert Bradshaw wrote:
On Wed, Feb 17, 2021 at 1:56 PM Kenneth Knowles > wrote:



On Wed, Feb 17, 2021 at 1:06 PM Robert Bradshaw
mailto:rober...@google.com>> wrote:

I would prefer to leave downstream triggering up to the runner
(or, better, leave upstream triggering up to the runner, a la
sink triggers), but one problem is that without an
explicit AfterSynchronizedProcessingTime one can't tell if the
downstream ProcessingTime between two groupings is due to an
explicit re-triggering between them or inherited from one to
the other.


I mean to propose that there should be no triggering specified
unless due to explicit re-triggering.


You're saying that we leave the trigger (and perhaps other) fields of 
the WindowingStrategy attached to PCollections downstream the first 
GBK unset in the proto? And let runners walk over the graph to infer 
it? I could be OK with making this legal, though updating all SDKs and 
Runners to handle this doesn't seem high priority at the moment.



(and BTW yes I agree about sink triggers, but we need retractions
and probably some theoretical work before we can aim for that)

Kenn

On Wed, Feb 17, 2021 at 12:37 PM Kenneth Knowles
mailto:k...@apache.org>> wrote:

Just for the thread I want to comment on another, more
drastic approach: eliminate continuation triggers from the
model, leaving downstream triggering up to a runner. This
approach is not viable because transforms may need to
change their behavior based on whether or not a trigger
will fire more than once. Transforms can and do inspect
the windowing strategy to do things differently.

Kenn

On Wed, Feb 17, 2021 at 11:47 AM Reuven Lax
mailto:re...@google.com>> wrote:

I'll say that synchronized processing time has
confused users before. Users sometimes use
processing-time triggers to optimize latency, banking
that that will decouple stage latency from the
long-tail latency of previous stages. However
continuation triggers silently switching to
synchronized processing time has defeated that, and it
wasn't clear to users why.

On Wed, Feb 17, 2021 at 11:12 AM Robert Bradshaw
mailto:rober...@google.com>> wrote:

On Fri, Feb 12, 2021 at 9:09 AM Kenneth Knowles
mailto:k...@apache.org>> wrote:


On Thu, Feb 11, 2021 at 9:38 PM Robert
Bradshaw mailto:rober...@google.com>> wrote:

Of course the right answer is to just
implement sink triggers and sidestep the
question altogether :).

In the meantime, I think leaving
AfterSynchronizedProcessingTime in the
model makes the most sense, and runners
can choose an implementation between
firing eagerly and waiting some amount of
time until they think all (most?)
downstream results are in before firing,
depending on how smart the runner wants to
be. As you point out, they're all correct,
and we'll have multiple firings due to the
upstream trigger anyway, and this is safer
than it used to be (though still possibly
requires work).


Just to clarify, as I got a little confused,
is your suggestion: Leave
AfterSynchronizedProcessingTime* triggers in
the model/proto, let the SDK put them in where
they want, and let runners decide how to
 

Re: Null checking in Beam

2021-01-22 Thread Jan Lukavský
> 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(!!).


Agree, if we can throw a better exception that is what should be done. 
That's what I meant by "tell user what specifically is missing or going 
wrong". I'm not sure, if this solution would be applicable at all 
places, though.


> 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.


That's understandable as well. Essentially, what I think is the most 
questionable part, is that _the checker modified source code_, by adding 
@Nonnull annotations (and others). That is something, that can annoy 
users (warnings in IDEs, or source code full of annotations) even more 
than (mostly) quite easy-to-debug NPEs. I'm not saying it is doing more 
harm than good, I'm just saying we don't know that.


Is it an option for the checked to be only really checker and not change 
the source code? If yes, I think it might be good idea to switch it to 
that mode only.


 Jan

On 1/22/21 7:37 PM, Brian Hulette wrote:



On Fri, Jan 22, 2021 at 1:18 AM Jan Lukavský <mailto:je...@seznam.cz>> 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 `getDescrip

Re: Null checking in Beam

2021-01-22 Thread Jan Lukavský

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.


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.


 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 your code than to suppress the issue.


It would be very cool for the proto compiler to annotate enough that 
new-and-improved type checkers could make things more convenient.


Kenn

On Mon, Jan 11, 2021 at 8:53 PM Reuven Lax > wrote:


I can use that trick. However I'm surprised that the check appears
to be so simplistic.

For example, the following code triggers the check on
getDescription().contains(), since getDescription returns a
Nullable string. However even a simplistic analysis should realize
that getDescription() was checked for null first! I have a dozen
or so cases like this, and I question how useful such a simplistic
check it will be.

if (statusCode.equals(Code.INVALID_ARGUMENT)
&().getDescription() !=null 
&().getDescription().contains("finalized")) {return false;
}


On Mon, Jan 11, 2021 at 8:32 PM Boyuan Zhang mailto:boyu...@google.com>> wrote:

Yeah it seems like the checker is enabled:
https://issues.apache.org/jira/browse/BEAM-10402. I used
@SuppressWarnings({"nullness" )}) to suppress the error when I
think it's not really a concern.

On Mon, Jan 11, 2021 at 8:28 PM Reuven Lax mailto:re...@google.com>> wrote:

Has extra Nullable checking been enabled in the Beam
project? I have a PR that was on hold for several 

Re: [VOTE] Release 2.27.0, release candidate #4

2021-01-07 Thread Jan Lukavský

+1 (non-binding).

I've validated the RC against my dependent projects (mainly Java SDK, 
Flink and DirectRunner).


Thanks,

 Jan

On 1/7/21 2:15 AM, Ahmet Altay wrote:

+1 (binding) - validated python quickstarts.

Thank you Pablo.

On Wed, Jan 6, 2021 at 1:57 PM Pablo Estrada > wrote:


+1 (binding)
I've built and unit tested existing Dataflow Templates with the
new version.
Best
-P.

On Tue, Jan 5, 2021 at 11:17 PM Pablo Estrada mailto:pabl...@google.com>> wrote:

Hi everyone,
Please review and vote on the release candidate #4 for the
version 2.27.0, as follows:
[ ] +1, Approve the release
[ ] -1, Do not approve the release (please provide specific
comments)

*NOTE*. What happened to RC #2? I started building RC2 before
completing all the cherry-picks, so the tag for RC2 was
created on an incorrect commit.

*NOTE*. What happened to RC #3? I started building RC3, but a
new bug was discovered (BEAM-11569) that required amending the
branch. Thus this is now RC4.

Reviewers are encouraged to test their own use cases with the
release candidate, and vote +1
 if no issues are found.

The complete staging area is available for your review, which
includes:
* JIRA release notes [1],
* the official Apache source release to be deployed to
dist.apache.org  [2], which is signed
with the key with fingerprint
C79DDD47DAF3808F0B9DDFAC02B2D9F742008494 [3],
* all artifacts to be deployed to the Maven Central Repository
[4],
* source code tag "v2.27.0-RC4" [5],
* website pull request listing the release [6], publishing the
API reference manual [7], and the blog post [8].
* Python artifacts are deployed along with the source release
to the dist.apache.org  [2].
* Validation sheet with a tab for 2.27.0 release to help with
validation [9].
* Docker images published to Docker Hub [10].

The vote will be open for at least 72 hours, but given the
holidays, we will likely extend for a few more days. The
release will be adopted by majority approval, with at least 3
PMC affirmative votes.

Thanks,
-P.

[1]

https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12319527=12349380

[2] https://dist.apache.org/repos/dist/dev/beam/2.27.0/
[3] https://dist.apache.org/repos/dist/release/beam/KEYS
[4]
https://repository.apache.org/content/repositories/orgapachebeam-1149/

[5] https://github.com/apache/beam/tree/v2.27.0-RC4
[6] https://github.com/apache/beam/pull/13602
[7] https://github.com/apache/beam-site/pull/610
[8] https://github.com/apache/beam/pull/13603
[9]

https://docs.google.com/spreadsheets/d/1qk-N5vjXvbcEk68GjbkSZTR8AGqyNUM-oLFo_ZXBpJw/edit#gid=194829106

[10] https://hub.docker.com/search?q=apache%2Fbeam=image



Re: Compatibility between Beam v2.23 and Beam v2.26

2021-01-07 Thread Jan Lukavský

Hi Antonio,

can you please create one?

Thanks,

 Jan

On 1/6/21 10:31 PM, Antonio Si wrote:

Thanks for the information. Do we have a jira to track this issue or do you 
want me to create a jira for this?

Thanks.

Antonio.

On 2021/01/06 17:59:47, Kenneth Knowles  wrote:

Agree with Boyuan & Kyle. That PR is the problem, and we probably do not
have adequate testing. We have a cultural understanding of not breaking
encoded data forms but this is the encoded form of the TypeSerializer, and
actually there are two problems.

1. When you have a serialized object that does not have the
serialVersionUid explicitly set, the UID is generated based on many details
that are irrelevant for binary compatibility. Any Java-serialized object
that is intended for anything other than transient transmission *must* have
a serialVersionUid set and an explicit serialized form. Else it is
completely normal for it to break due to irrelevant changes. The
serialVersionUid has no mechanism for upgrade/downgrade so you *must* keep
it the same forever, and any versioning or compat scheme exists within the
single serialVersionUid.
2. In this case there was an actual change to the fields of the object
stored, so you need to explicitly add the serialized form and also the
ability to read from prior serialized forms.

I believe explicitly setting the serialVersionUid to the original (and
keeping it that way forever) and adding the ability to decode prior forms
will regain the ability to read the snapshot. But also this seems like
something that would be part of Flink best practice documentation since
naive use of Java serialization often hits this problem.

Kenn

On Tue, Jan 5, 2021 at 4:30 PM Kyle Weaver  wrote:


This raises a few related questions from me:

1. Do we claim to support resuming Flink checkpoints made with previous
Beam versions?
2. Does 1. require full binary compatibility between different versions of
runner internals like CoderTypeSerializer?


3. Do we have tests for 1.?
Kenn



On Tue, Jan 5, 2021 at 4:05 PM Boyuan Zhang  wrote:


https://github.com/apache/beam/pull/13240 seems suspicious to me.

  +Maximilian Michels  Any insights here?

On Tue, Jan 5, 2021 at 8:48 AM Antonio Si  wrote:


Hi,

I would like to followup with this question to see if there is a
solution/workaround for this issue.

Thanks.

Antonio.

On 2020/12/19 18:33:48, Antonio Si  wrote:

Hi,

We were using Beam v2.23 and recently, we are testing upgrade to Beam

v2.26. For Beam v2.26, we are passing --experiments=use_deprecated_read and
--fasterCopy=true.

We run into this exception when we resume our pipeline:

Caused by: java.io.InvalidClassException:

org.apache.beam.runners.flink.translation.types.CoderTypeSerializer; local
class incompatible: stream classdesc serialVersionUID =
5241803328188007316, local class serialVersionUID = 7247319138941746449

   at

java.io.ObjectStreamClass.initNonProxy(ObjectStreamClass.java:699)

   at

java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1942)

   at

java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1808)

   at

java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2099)

   at

java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1625)

   at

java.io.ObjectInputStream.readObject(ObjectInputStream.java:465)

   at

java.io.ObjectInputStream.readObject(ObjectInputStream.java:423)

   at

org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil$TypeSerializerSerializationProxy.read(TypeSerializerSerializationUtil.java:301)

   at

org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil.tryReadSerializer(TypeSerializerSerializationUtil.java:116)

   at

org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot.readSnapshot(TypeSerializerConfigSnapshot.java:113)

   at

org.apache.flink.api.common.typeutils.TypeSerializerSnapshot.readVersionedSnapshot(TypeSerializerSnapshot.java:174)

   at

org.apache.flink.api.common.typeutils.TypeSerializerSnapshotSerializationUtil$TypeSerializerSnapshotSerializationProxy.deserializeV2(TypeSerializerSnapshotSerializationUtil.java:179)

   at

org.apache.flink.api.common.typeutils.TypeSerializerSnapshotSerializationUtil$TypeSerializerSnapshotSerializationProxy.read(TypeSerializerSnapshotSerializationUtil.java:150)

   at

org.apache.flink.api.common.typeutils.TypeSerializerSnapshotSerializationUtil.readSerializerSnapshot(TypeSerializerSnapshotSerializationUtil.java:76)

   at

org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshotReadersWriters$CurrentReaderImpl.readStateMetaInfoSnapshot(StateMetaInfoSnapshotReadersWriters.java:219)

   at

org.apache.flink.runtime.state.OperatorBackendSerializationProxy.read(OperatorBackendSerializationProxy.java:119)

   at

org.apache.flink.runtime.state.OperatorStateRestoreOperation.restore(OperatorStateRestoreOperation.java:83)

It looks like it is not able to 

Re: Usability regression using SDF Unbounded Source wrapper + DirectRunner

2021-01-06 Thread Jan Lukavský

Sorry for the typo in your name. :-)

On 1/6/21 10:11 AM, Jan Lukavský wrote:

Hi Antonie,

yes, for instance. I'd just like to rule out possibility that a single 
DoFn processing multiple partitions (restrictions) brings some 
overhead in your case.


Jan

On 12/31/20 10:36 PM, Antonio Si wrote:

Hi Jan,

Sorry for the late reply. My topic has 180 partitions. Do you mean 
run with a

parallelism set to 900?

Thanks.

Antonio.

On 2020/12/23 20:30:34, Jan Lukavský  wrote:

OK,

could you make an experiment and increase the parallelism to something
significantly higher than the total number of partitions? Say 5 times
higher? Would that have impact on throughput in your case?

Jan

On 12/23/20 7:03 PM, Antonio Si wrote:

Hi Jan,

The performance data that I reported was run with parallelism = 8. 
We also ran with parallelism = 15 and we observed similar behaviors 
although I don't have the exact numbers. I can get you the numbers 
if needed.


Regarding number of partitions, since we have multiple topics, the 
number of partitions varies from 180 to 12. The highest TPS topic 
has 180 partitions, while the lowest TPS topic has 12 partitions.


Thanks.

Antonio.

On 2020/12/23 12:28:42, Jan Lukavský  wrote:

Hi Antonio,

can you please clarify a few things:

    a) what parallelism you use for your sources

    b) how many partitions there is in your topic(s)

Thanks,

    Jan

On 12/22/20 10:07 PM, Antonio Si wrote:

Hi Boyuan,

Let me clarify, I have tried with and without using 
--experiments=beam_fn_api,use_sdf_kafka_read option:


-  with --experiments=use_deprecated_read --fasterrCopy=true, I 
am able to achieve 13K TPS
-  with --experiments="beam_fn_api,use_sdf_kafka_read" 
--fasterCopy=true, I am able to achieve 10K

-  with --fasterCopy=true alone, I am only able to achieve 5K TPS

In our testcase, we have multiple topics, checkpoint intervals is 
60s. Some topics have a lot higher traffics than others. We look 
at the case with --experiments="beam_fn_api,use_sdf_kafka_read" 
--fasterCopy=true options a little. Based on our observation, 
each consumer poll() in ReadFromKafkaDoFn.processElement() takes 
about 0.8ms. So for topic with high traffics, it will continue in 
the loop because every poll() will return some records. Every 
poll returns about 200 records. So, it takes about 0.8ms for 
every 200 records. I am not sure if that is part of the reason 
for the performance.


Thanks.

Antonio.

On 2020/12/21 19:03:19, Boyuan Zhang  wrote:

Hi Antonio,

Thanks for the data point. That's very valuable information!

I didn't use DirectRunner. I am using FlinkRunner.
We measured the number of Kafka messages that we can processed 
per second.

With Beam v2.26 with --experiments=use_deprecated_read and
--fasterCopy=true,
we are able to consume 13K messages per second, but with Beam 
v2.26
without the use_deprecated_read option, we are only able to 
process 10K

messages
per second for the same pipeline.
We do have SDF implementation of Kafka Read instead of using the 
wrapper.
Would you like to have a try to see whether it helps you improve 
your
situation?  You can use 
--experiments=beam_fn_api,use_sdf_kafka_read to

switch to the Kafka SDF Read.

On Mon, Dec 21, 2020 at 10:54 AM Boyuan Zhang 
 wrote:



Hi Jan,
it seems that what we would want is to couple the lifecycle of 
the Reader

not with the restriction but with the particular instance of
(Un)boundedSource (after being split). That could be done in 
the processing
DoFn, if it contained a cache mapping instance of the source 
to the
(possibly null - i.e. not yet open) reader. In @NewTracker we 
could assign
(or create) the reader to the tracker, as the tracker is 
created for each

restriction.

WDYT?

I was thinking about this but it seems like it is not 
applicable to the

way how UnboundedSource and UnboundedReader work together.
Please correct me if I'm wrong. The UnboundedReader is created 
from
UnboundedSource per CheckpointMark[1], which means for certain 
sources, the
CheckpointMark could affect some attributes like start position 
of the
reader when resuming. So a single UnboundedSource could be 
mapped to
multiple readers because of different instances of 
CheckpointMarl. That's

also the reason why we use CheckpointMark as the restriction.

Please let me know if I misunderstand your suggestion.

[1]
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java#L73-L78 



On Mon, Dec 21, 2020 at 9:18 AM Antonio Si 
 wrote:



Hi Boyuan,

Sorry for my late reply. I was off for a few days.

I didn't use DirectRunner. I am using FlinkRunner.

We measured the number of Kafka messages that we can processed 
per second.

With Beam v2.26 with --experiments=use_deprecated_read and
--fasterCopy=true,
we are able to consume 13K messages per second, but with Beam 
v2.26
without the use_deprecated_read option, we are only able to 
process 10K

messages
per second for the sa

Re: Usability regression using SDF Unbounded Source wrapper + DirectRunner

2021-01-06 Thread Jan Lukavský

Hi Antonie,

yes, for instance. I'd just like to rule out possibility that a single 
DoFn processing multiple partitions (restrictions) brings some overhead 
in your case.


Jan

On 12/31/20 10:36 PM, Antonio Si wrote:

Hi Jan,

Sorry for the late reply. My topic has 180 partitions. Do you mean run with a
parallelism set to 900?

Thanks.

Antonio.

On 2020/12/23 20:30:34, Jan Lukavský  wrote:

OK,

could you make an experiment and increase the parallelism to something
significantly higher than the total number of partitions? Say 5 times
higher? Would that have impact on throughput in your case?

Jan

On 12/23/20 7:03 PM, Antonio Si wrote:

Hi Jan,

The performance data that I reported was run with parallelism = 8. We also ran 
with parallelism = 15 and we observed similar behaviors although I don't have 
the exact numbers. I can get you the numbers if needed.

Regarding number of partitions, since we have multiple topics, the number of 
partitions varies from 180 to 12. The highest TPS topic has 180 partitions, 
while the lowest TPS topic has 12 partitions.

Thanks.

Antonio.

On 2020/12/23 12:28:42, Jan Lukavský  wrote:

Hi Antonio,

can you please clarify a few things:

    a) what parallelism you use for your sources

    b) how many partitions there is in your topic(s)

Thanks,

    Jan

On 12/22/20 10:07 PM, Antonio Si wrote:

Hi Boyuan,

Let me clarify, I have tried with and without using 
--experiments=beam_fn_api,use_sdf_kafka_read option:

-  with --experiments=use_deprecated_read --fasterrCopy=true, I am able to 
achieve 13K TPS
-  with --experiments="beam_fn_api,use_sdf_kafka_read" --fasterCopy=true, I am 
able to achieve 10K
-  with --fasterCopy=true alone, I am only able to achieve 5K TPS

In our testcase, we have multiple topics, checkpoint intervals is 60s. Some topics have a 
lot higher traffics than others. We look at the case with 
--experiments="beam_fn_api,use_sdf_kafka_read" --fasterCopy=true options a 
little. Based on our observation, each consumer poll() in 
ReadFromKafkaDoFn.processElement() takes about 0.8ms. So for topic with high traffics, it 
will continue in the loop because every poll() will return some records. Every poll 
returns about 200 records. So, it takes about 0.8ms for every 200 records. I am not sure 
if that is part of the reason for the performance.

Thanks.

Antonio.

On 2020/12/21 19:03:19, Boyuan Zhang  wrote:

Hi Antonio,

Thanks for the data point. That's very valuable information!

I didn't use DirectRunner. I am using FlinkRunner.

We measured the number of Kafka messages that we can processed per second.
With Beam v2.26 with --experiments=use_deprecated_read and
--fasterCopy=true,
we are able to consume 13K messages per second, but with Beam v2.26
without the use_deprecated_read option, we are only able to process 10K
messages
per second for the same pipeline.

We do have SDF implementation of Kafka Read instead of using the wrapper.
Would you like to have a try to see whether it helps you improve your
situation?  You can use --experiments=beam_fn_api,use_sdf_kafka_read to
switch to the Kafka SDF Read.

On Mon, Dec 21, 2020 at 10:54 AM Boyuan Zhang  wrote:


Hi Jan,

it seems that what we would want is to couple the lifecycle of the Reader
not with the restriction but with the particular instance of
(Un)boundedSource (after being split). That could be done in the processing
DoFn, if it contained a cache mapping instance of the source to the
(possibly null - i.e. not yet open) reader. In @NewTracker we could assign
(or create) the reader to the tracker, as the tracker is created for each
restriction.

WDYT?


I was thinking about this but it seems like it is not applicable to the
way how UnboundedSource and UnboundedReader work together.
Please correct me if I'm wrong. The UnboundedReader is created from
UnboundedSource per CheckpointMark[1], which means for certain sources, the
CheckpointMark could affect some attributes like start position of the
reader when resuming. So a single UnboundedSource could be mapped to
multiple readers because of different instances of CheckpointMarl. That's
also the reason why we use CheckpointMark as the restriction.

Please let me know if I misunderstand your suggestion.

[1]
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java#L73-L78

On Mon, Dec 21, 2020 at 9:18 AM Antonio Si  wrote:


Hi Boyuan,

Sorry for my late reply. I was off for a few days.

I didn't use DirectRunner. I am using FlinkRunner.

We measured the number of Kafka messages that we can processed per second.
With Beam v2.26 with --experiments=use_deprecated_read and
--fasterCopy=true,
we are able to consume 13K messages per second, but with Beam v2.26
without the use_deprecated_read option, we are only able to process 10K
messages
per second for the same pipeline.

Thanks and regards,

Antonio.

On 2020/12/11 22:19:40, Boyuan Zhang  wrote:

Hi Antonio,

Thanks for the de

Re: Usability regression using SDF Unbounded Source wrapper + DirectRunner

2020-12-23 Thread Jan Lukavský

OK,

could you make an experiment and increase the parallelism to something 
significantly higher than the total number of partitions? Say 5 times 
higher? Would that have impact on throughput in your case?


Jan

On 12/23/20 7:03 PM, Antonio Si wrote:

Hi Jan,

The performance data that I reported was run with parallelism = 8. We also ran 
with parallelism = 15 and we observed similar behaviors although I don't have 
the exact numbers. I can get you the numbers if needed.

Regarding number of partitions, since we have multiple topics, the number of 
partitions varies from 180 to 12. The highest TPS topic has 180 partitions, 
while the lowest TPS topic has 12 partitions.

Thanks.

Antonio.

On 2020/12/23 12:28:42, Jan Lukavský  wrote:

Hi Antonio,

can you please clarify a few things:

   a) what parallelism you use for your sources

   b) how many partitions there is in your topic(s)

Thanks,

   Jan

On 12/22/20 10:07 PM, Antonio Si wrote:

Hi Boyuan,

Let me clarify, I have tried with and without using 
--experiments=beam_fn_api,use_sdf_kafka_read option:

-  with --experiments=use_deprecated_read --fasterrCopy=true, I am able to 
achieve 13K TPS
-  with --experiments="beam_fn_api,use_sdf_kafka_read" --fasterCopy=true, I am 
able to achieve 10K
-  with --fasterCopy=true alone, I am only able to achieve 5K TPS

In our testcase, we have multiple topics, checkpoint intervals is 60s. Some topics have a 
lot higher traffics than others. We look at the case with 
--experiments="beam_fn_api,use_sdf_kafka_read" --fasterCopy=true options a 
little. Based on our observation, each consumer poll() in 
ReadFromKafkaDoFn.processElement() takes about 0.8ms. So for topic with high traffics, it 
will continue in the loop because every poll() will return some records. Every poll 
returns about 200 records. So, it takes about 0.8ms for every 200 records. I am not sure 
if that is part of the reason for the performance.

Thanks.

Antonio.

On 2020/12/21 19:03:19, Boyuan Zhang  wrote:

Hi Antonio,

Thanks for the data point. That's very valuable information!

I didn't use DirectRunner. I am using FlinkRunner.

We measured the number of Kafka messages that we can processed per second.
With Beam v2.26 with --experiments=use_deprecated_read and
--fasterCopy=true,
we are able to consume 13K messages per second, but with Beam v2.26
without the use_deprecated_read option, we are only able to process 10K
messages
per second for the same pipeline.

We do have SDF implementation of Kafka Read instead of using the wrapper.
Would you like to have a try to see whether it helps you improve your
situation?  You can use --experiments=beam_fn_api,use_sdf_kafka_read to
switch to the Kafka SDF Read.

On Mon, Dec 21, 2020 at 10:54 AM Boyuan Zhang  wrote:


Hi Jan,

it seems that what we would want is to couple the lifecycle of the Reader
not with the restriction but with the particular instance of
(Un)boundedSource (after being split). That could be done in the processing
DoFn, if it contained a cache mapping instance of the source to the
(possibly null - i.e. not yet open) reader. In @NewTracker we could assign
(or create) the reader to the tracker, as the tracker is created for each
restriction.

WDYT?


I was thinking about this but it seems like it is not applicable to the
way how UnboundedSource and UnboundedReader work together.
Please correct me if I'm wrong. The UnboundedReader is created from
UnboundedSource per CheckpointMark[1], which means for certain sources, the
CheckpointMark could affect some attributes like start position of the
reader when resuming. So a single UnboundedSource could be mapped to
multiple readers because of different instances of CheckpointMarl. That's
also the reason why we use CheckpointMark as the restriction.

Please let me know if I misunderstand your suggestion.

[1]
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java#L73-L78

On Mon, Dec 21, 2020 at 9:18 AM Antonio Si  wrote:


Hi Boyuan,

Sorry for my late reply. I was off for a few days.

I didn't use DirectRunner. I am using FlinkRunner.

We measured the number of Kafka messages that we can processed per second.
With Beam v2.26 with --experiments=use_deprecated_read and
--fasterCopy=true,
we are able to consume 13K messages per second, but with Beam v2.26
without the use_deprecated_read option, we are only able to process 10K
messages
per second for the same pipeline.

Thanks and regards,

Antonio.

On 2020/12/11 22:19:40, Boyuan Zhang  wrote:

Hi Antonio,

Thanks for the details! Which version of Beam SDK are you using? And are
you using --experiments=beam_fn_api with DirectRunner to launch your
pipeline?

For ReadFromKafkaDoFn.processElement(), it will take a Kafka
topic+partition as input element and a KafkaConsumer will be assigned to
this topic+partition then poll records continuously. The Kafka consumer
will resume reading and return from

Re: Usability regression using SDF Unbounded Source wrapper + DirectRunner

2020-12-23 Thread Jan Lukavský

Hi Antonio,

can you please clarify a few things:

 a) what parallelism you use for your sources

 b) how many partitions there is in your topic(s)

Thanks,

 Jan

On 12/22/20 10:07 PM, Antonio Si wrote:

Hi Boyuan,

Let me clarify, I have tried with and without using 
--experiments=beam_fn_api,use_sdf_kafka_read option:

-  with --experiments=use_deprecated_read --fasterrCopy=true, I am able to 
achieve 13K TPS
-  with --experiments="beam_fn_api,use_sdf_kafka_read" --fasterCopy=true, I am 
able to achieve 10K
-  with --fasterCopy=true alone, I am only able to achieve 5K TPS

In our testcase, we have multiple topics, checkpoint intervals is 60s. Some topics have a 
lot higher traffics than others. We look at the case with 
--experiments="beam_fn_api,use_sdf_kafka_read" --fasterCopy=true options a 
little. Based on our observation, each consumer poll() in 
ReadFromKafkaDoFn.processElement() takes about 0.8ms. So for topic with high traffics, it 
will continue in the loop because every poll() will return some records. Every poll 
returns about 200 records. So, it takes about 0.8ms for every 200 records. I am not sure 
if that is part of the reason for the performance.

Thanks.

Antonio.

On 2020/12/21 19:03:19, Boyuan Zhang  wrote:

Hi Antonio,

Thanks for the data point. That's very valuable information!

I didn't use DirectRunner. I am using FlinkRunner.

We measured the number of Kafka messages that we can processed per second.
With Beam v2.26 with --experiments=use_deprecated_read and
--fasterCopy=true,
we are able to consume 13K messages per second, but with Beam v2.26
without the use_deprecated_read option, we are only able to process 10K
messages
per second for the same pipeline.

We do have SDF implementation of Kafka Read instead of using the wrapper.
Would you like to have a try to see whether it helps you improve your
situation?  You can use --experiments=beam_fn_api,use_sdf_kafka_read to
switch to the Kafka SDF Read.

On Mon, Dec 21, 2020 at 10:54 AM Boyuan Zhang  wrote:


Hi Jan,

it seems that what we would want is to couple the lifecycle of the Reader
not with the restriction but with the particular instance of
(Un)boundedSource (after being split). That could be done in the processing
DoFn, if it contained a cache mapping instance of the source to the
(possibly null - i.e. not yet open) reader. In @NewTracker we could assign
(or create) the reader to the tracker, as the tracker is created for each
restriction.

WDYT?


I was thinking about this but it seems like it is not applicable to the
way how UnboundedSource and UnboundedReader work together.
Please correct me if I'm wrong. The UnboundedReader is created from
UnboundedSource per CheckpointMark[1], which means for certain sources, the
CheckpointMark could affect some attributes like start position of the
reader when resuming. So a single UnboundedSource could be mapped to
multiple readers because of different instances of CheckpointMarl. That's
also the reason why we use CheckpointMark as the restriction.

Please let me know if I misunderstand your suggestion.

[1]
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java#L73-L78

On Mon, Dec 21, 2020 at 9:18 AM Antonio Si  wrote:


Hi Boyuan,

Sorry for my late reply. I was off for a few days.

I didn't use DirectRunner. I am using FlinkRunner.

We measured the number of Kafka messages that we can processed per second.
With Beam v2.26 with --experiments=use_deprecated_read and
--fasterCopy=true,
we are able to consume 13K messages per second, but with Beam v2.26
without the use_deprecated_read option, we are only able to process 10K
messages
per second for the same pipeline.

Thanks and regards,

Antonio.

On 2020/12/11 22:19:40, Boyuan Zhang  wrote:

Hi Antonio,

Thanks for the details! Which version of Beam SDK are you using? And are
you using --experiments=beam_fn_api with DirectRunner to launch your
pipeline?

For ReadFromKafkaDoFn.processElement(), it will take a Kafka
topic+partition as input element and a KafkaConsumer will be assigned to
this topic+partition then poll records continuously. The Kafka consumer
will resume reading and return from the process fn when

- There are no available records currently(this is a feature of SDF
which calls SDF self-initiated checkpoint)
- The OutputAndTimeBoundedSplittableProcessElementInvoker issues
checkpoint request to ReadFromKafkaDoFn for getting partial results.

The

checkpoint frequency for DirectRunner is every 100 output records or

every

1 seconds.

It seems like either the self-initiated checkpoint or DirectRunner

issued

checkpoint gives you the performance regression since there is overhead
when rescheduling residuals. In your case, it's more like that the
checkpoint behavior of

OutputAndTimeBoundedSplittableProcessElementInvoker

gives you 200 elements a batch. I want to understand what kind of
performance regression you are noticing? Is it 

Re: Usability regression using SDF Unbounded Source wrapper + DirectRunner

2020-12-21 Thread Jan Lukavský

Sure. My ID is je-ik.

Thanks,

 Jan

On 12/21/20 8:43 PM, Boyuan Zhang wrote:
Thanks for your explanation, Jan. Now I can see what you mean here. I 
can try to have a PR to do such optimization. Would you like to share 
your github ID with me to review the PR later?


On Mon, Dec 21, 2020 at 11:15 AM Robert Bradshaw <mailto:rober...@google.com>> wrote:


If readers are expensive to create, this seems like an important
(and not too difficult) optimization.

On Mon, Dec 21, 2020 at 11:04 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

Hi Boyuan,

I think your analysis is correct - with one exception. It
should  be possible to reuse the reader if and only if the
last taken CheckpointMark equals to the new CheckpointMark the
reader would be created from. But - this equality is on the
happy path and should be satisfied for vast majority of
invocations, so it will spare many call to createReader.
Actually, it should be non-equal only after recovery from
checkpoint, but then there should be no reader. So to be
technically correct, we should keep the last CheckpointMark
along with the open reader, but that might turn out to be
non-necessary (I'm not sure about that and I would definitely
keep the last CheckpointMark, because it is better safe than
sorry :))

Jan

On 12/21/20 7:54 PM, Boyuan Zhang wrote:

Hi Jan,

it seems that what we would want is to couple the
lifecycle of the Reader not with the restriction but with
the particular instance of (Un)boundedSource (after being
split). That could be done in the processing DoFn, if it
contained a cache mapping instance of the source to the
(possibly null - i.e. not yet open) reader. In
@NewTracker we could assign (or create) the reader to the
tracker, as the tracker is created for each restriction.

WDYT?

I was thinking about this but it seems like it is not
applicable to the way how UnboundedSource and UnboundedReader
work together.
Please correct me if I'm wrong. The UnboundedReader is
created from UnboundedSource per CheckpointMark[1], which
means for certain sources, the CheckpointMark could affect
some attributes like start position of the reader when
resuming. So a single UnboundedSource could be mapped to
multiple readers because of different instances of
CheckpointMarl. That's also the reason why we use
CheckpointMark as the restriction.

Please let me know if I misunderstand your suggestion.

[1]

https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java#L73-L78

On Mon, Dec 21, 2020 at 9:18 AM Antonio Si
mailto:antonio...@gmail.com>> wrote:

Hi Boyuan,

Sorry for my late reply. I was off for a few days.

I didn't use DirectRunner. I am using FlinkRunner.

We measured the number of Kafka messages that we can
processed per second.
With Beam v2.26 with --experiments=use_deprecated_read
and --fasterCopy=true,
we are able to consume 13K messages per second, but with
Beam v2.26
without the use_deprecated_read option, we are only able
to process 10K messages
per second for the same pipeline.

Thanks and regards,

Antonio.

On 2020/12/11 22:19:40, Boyuan Zhang mailto:boyu...@google.com>> wrote:
> Hi Antonio,
>
> Thanks for the details! Which version of Beam SDK are
you using? And are
> you using --experiments=beam_fn_api with DirectRunner
to launch your
> pipeline?
>
> For ReadFromKafkaDoFn.processElement(), it will take a
Kafka
> topic+partition as input element and a KafkaConsumer
will be assigned to
> this topic+partition then poll records continuously.
The Kafka consumer
> will resume reading and return from the process fn when
>
>    - There are no available records currently(this is a
feature of SDF
>    which calls SDF self-initiated checkpoint)
>    - The
OutputAndTimeBoundedSplittableProcessElementInvoker issues
>    checkpoint request to ReadFromKafkaDoFn for getting
partial results. The
>    checkpoint frequency for DirectRunner is every 100
output records or every
>    1 seconds.
>
> It seems like either the self-initia

Re: Usability regression using SDF Unbounded Source wrapper + DirectRunner

2020-12-21 Thread Jan Lukavský

Hi Boyuan,

I think your analysis is correct - with one exception. It should be 
possible to reuse the reader if and only if the last taken 
CheckpointMark equals to the new CheckpointMark the reader would be 
created from. But - this equality is on the happy path and should be 
satisfied for vast majority of invocations, so it will spare many call 
to createReader. Actually, it should be non-equal only after recovery 
from checkpoint, but then there should be no reader. So to be 
technically correct, we should keep the last CheckpointMark along with 
the open reader, but that might turn out to be non-necessary (I'm not 
sure about that and I would definitely keep the last CheckpointMark, 
because it is better safe than sorry :))


Jan

On 12/21/20 7:54 PM, Boyuan Zhang wrote:

Hi Jan,

it seems that what we would want is to couple the lifecycle of the
Reader not with the restriction but with the particular instance
of (Un)boundedSource (after being split). That could be done in
the processing DoFn, if it contained a cache mapping instance of
the source to the (possibly null - i.e. not yet open) reader. In
@NewTracker we could assign (or create) the reader to the tracker,
as the tracker is created for each restriction.

WDYT?

I was thinking about this but it seems like it is not applicable to 
the way how UnboundedSource and UnboundedReader work together.
Please correct me if I'm wrong. The UnboundedReader is created from 
UnboundedSource per CheckpointMark[1], which means for certain 
sources, the CheckpointMark could affect some attributes like start 
position of the reader when resuming. So a single UnboundedSource 
could be mapped to multiple readers because of different instances of 
CheckpointMarl. That's also the reason why we use CheckpointMark as 
the restriction.


Please let me know if I misunderstand your suggestion.

[1] 
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java#L73-L78


On Mon, Dec 21, 2020 at 9:18 AM Antonio Si > wrote:


Hi Boyuan,

Sorry for my late reply. I was off for a few days.

I didn't use DirectRunner. I am using FlinkRunner.

We measured the number of Kafka messages that we can processed per
second.
With Beam v2.26 with --experiments=use_deprecated_read and
--fasterCopy=true,
we are able to consume 13K messages per second, but with Beam v2.26
without the use_deprecated_read option, we are only able to
process 10K messages
per second for the same pipeline.

Thanks and regards,

Antonio.

On 2020/12/11 22:19:40, Boyuan Zhang mailto:boyu...@google.com>> wrote:
> Hi Antonio,
>
> Thanks for the details! Which version of Beam SDK are you using?
And are
> you using --experiments=beam_fn_api with DirectRunner to launch your
> pipeline?
>
> For ReadFromKafkaDoFn.processElement(), it will take a Kafka
> topic+partition as input element and a KafkaConsumer will be
assigned to
> this topic+partition then poll records continuously. The Kafka
consumer
> will resume reading and return from the process fn when
>
>    - There are no available records currently(this is a feature
of SDF
>    which calls SDF self-initiated checkpoint)
>    - The OutputAndTimeBoundedSplittableProcessElementInvoker issues
>    checkpoint request to ReadFromKafkaDoFn for getting partial
results. The
>    checkpoint frequency for DirectRunner is every 100 output
records or every
>    1 seconds.
>
> It seems like either the self-initiated checkpoint or
DirectRunner issued
> checkpoint gives you the performance regression since there is
overhead
> when rescheduling residuals. In your case, it's more like that the
> checkpoint behavior of
OutputAndTimeBoundedSplittableProcessElementInvoker
> gives you 200 elements a batch. I want to understand what kind of
> performance regression you are noticing? Is it slower to output
the same
> amount of records?
>
> On Fri, Dec 11, 2020 at 1:31 PM Antonio Si mailto:antonio...@gmail.com>> wrote:
>
> > Hi Boyuan,
> >
> > This is Antonio. I reported the KafkaIO.read() performance
issue on the
> > slack channel a few days ago.
> >
> > I am not sure if this is helpful, but I have been doing some
debugging on
> > the SDK KafkaIO performance issue for our pipeline and I would
like to
> > provide some observations.
> >
> > It looks like in my case the
ReadFromKafkaDoFn.processElement()  was
> > invoked within the same thread and every time
kafaconsumer.poll() is
> > called, it returns some records, from 1 up to 200 records. So,
it will
> > proceed to run the pipeline steps. Each kafkaconsumer.poll()
takes about
> > 0.8ms. So, in this case, the polling and running of the
 

Re: Usability regression using SDF Unbounded Source wrapper + DirectRunner

2020-12-18 Thread Jan Lukavský

Hi Boyuan,

I understand, that this could be difficult in the current 
implementation, my intent was just to point out that this should be 
possible, even in the general case. From the top of my head (and I 
didn't walk this though entirely, so please don't take me too literaly), 
it seems that what we would want is to couple the lifecycle of the 
Reader not with the restriction but with the particular instance of 
(Un)boundedSource (after being split). That could be done in the 
processing DoFn, if it contained a cache mapping instance of the source 
to the (possibly null - i.e. not yet open) reader. In @NewTracker we 
could assign (or create) the reader to the tracker, as the tracker is 
created for each restriction.


WDYT?

 Jan

On 12/18/20 1:03 AM, Boyuan Zhang wrote:

Hi Jan,

I'm not saying that it's completely impossible to do so but I want to 
explain why it's hard to apply these changes to existing SDF wrapper.


In the current SDF UnboundedSource wrapper[1], the restriction is the 
. The reader is binded to the 
UnboundedSourceAsSDFRestrictionTracker[2]. The reader is created from 
CheckpointMark and is started when it's the first time to call 
tracker.tryClaim(). The reader is closed when trySplit() happens 
successfully. The DoFn only has access to the RestrictionTracker in 
the @ProcessElement function, which means the SDF UnboundedSource 
wrapper DoFn is not able to manage the reader directly though it's 
lifecycle. In terms of the lifecycle of one RestrictionTracker 
instance, it is managed by the invoker(or in portable execution, it's 
managed by the FnApiDoFnRunner). The RestrictionTracker is created 
before @ProcessElement function is invoked by calling @NewTracker, and 
it will be deconstructed when the process function finishes.


It also makes sense to have CheckpointMark as the restriction because 
we want to perform checkpoint on UnboundedSource.


[1] 
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java#L436
[2] 
https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java#L750


On Thu, Dec 17, 2020 at 2:42 PM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


Hi Boyuan,

> Several changes could be made into PubSub SDF implementation
specially. For example, the PuSub SDF can choose not respond to
the checkpoint request when it thinks it's not a good time to do
so. Besides, if the expensive connection can be binded to the
lifecycle of the SDF instance instead of per restriction, the
PubSub SDF implementation can choose to start the connection when
the DoFn is started and close the connection when tearDown is called.

Why the same cannot be applied to the general case? If we think
about the "connection" and the "reader" as two abstract objects,
it has the same methods - namely open and close, which is what
defines the scope of the object. I think still think it should be
possible to implement that generally.

Jan

On 12/17/20 11:19 PM, Boyuan Zhang wrote:

Hi Jan, thanks for the quick followup!

I'm not sure if I see the difference between writing a
"native" SDF for PubSub source and the UnboundedSource
wrapper. With regards to the relation between reader and
checkpoint, wouldn't the native implementation be at the same
position?

Several changes could be made into PubSub SDF implementation
specially. For example, the PuSub SDF can choose not respond to
the checkpoint request when it thinks it's not a good time to do
so. Besides, if the expensive connection can be binded to the
lifecycle of the SDF instance instead of per restriction, the
PubSub SDF implementation can choose to start the connection when
the DoFn is started and close the connection when tearDown is called.

We might not be able to do so on SDF wrapper since it's a kind of
general solution for all sources and not all sources don't have
the connection binded to the restriction.

Another workaround for using PubSub on DirectRunner might be
using --experiments=enable_custom_pubsub_source, This flag will
make the pipeline to use a DoFn to read from PubSub instead of
using UnboundedSource. Hope it will be helpful as well.


On Thu, Dec 17, 2020 at 1:09 PM Jan Lukavský mailto:je...@seznam.cz>> wrote:

Hi Boyuan,

I'm not sure if I see the difference between writing a
"native" SDF for PubSub source and the UnboundedSource
wrapper. With regards to the relation between reader and
checkpoint, wouldn't the native implementation be at the same
position?

In my point of view, the decision to close the reader is
simply a matter of lifecycle of the reader. Currently, it is
tightly bound to the restriction being processed, but that
  

Re: Usability regression using SDF Unbounded Source wrapper + DirectRunner

2020-12-17 Thread Jan Lukavský

Hi Boyuan,

> Several changes could be made into PubSub SDF implementation 
specially. For example, the PuSub SDF can choose not respond to the 
checkpoint request when it thinks it's not a good time to do so. 
Besides, if the expensive connection can be binded to the lifecycle of 
the SDF instance instead of per restriction, the PubSub SDF 
implementation can choose to start the connection when the DoFn is 
started and close the connection when tearDown is called.


Why the same cannot be applied to the general case? If we think about 
the "connection" and the "reader" as two abstract objects, it has the 
same methods - namely open and close, which is what defines the scope of 
the object. I think still think it should be possible to implement that 
generally.


Jan

On 12/17/20 11:19 PM, Boyuan Zhang wrote:

Hi Jan, thanks for the quick followup!

I'm not sure if I see the difference between writing a "native"
SDF for PubSub source and the UnboundedSource wrapper. With
regards to the relation between reader and checkpoint, wouldn't
the native implementation be at the same position?

Several changes could be made into PubSub SDF implementation 
specially. For example, the PuSub SDF can choose not respond to the 
checkpoint request when it thinks it's not a good time to do so. 
Besides, if the expensive connection can be binded to the lifecycle of 
the SDF instance instead of per restriction, the PubSub SDF 
implementation can choose to start the connection when the DoFn is 
started and close the connection when tearDown is called.


We might not be able to do so on SDF wrapper since it's a kind of 
general solution for all sources and not all sources don't have the 
connection binded to the restriction.


Another workaround for using PubSub on DirectRunner might be using 
--experiments=enable_custom_pubsub_source, This flag will make the 
pipeline to use a DoFn to read from PubSub instead of using 
UnboundedSource. Hope it will be helpful as well.



On Thu, Dec 17, 2020 at 1:09 PM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


Hi Boyuan,

I'm not sure if I see the difference between writing a "native"
SDF for PubSub source and the UnboundedSource wrapper. With
regards to the relation between reader and checkpoint, wouldn't
the native implementation be at the same position?

In my point of view, the decision to close the reader is simply a
matter of lifecycle of the reader. Currently, it is tightly bound
to the restriction being processed, but that could be relaxed, so
that instead of immediately closing the reader, it could be only
_scheduled for closing in future_ (using processing time timer for
instance) provided it is not reused in the remaining restriction
after split (by the same instance of DoFn). That is an
optimization that could really make sense outside DirectRunner,
because for instance Flink has use cases, where user really
*wants* to configure quite often checkpoints (has relation to how
Flink implements @RequiresStableInput).

Jan

On 12/17/20 9:04 PM, Boyuan Zhang wrote:

Sorry for the confusion.

 Are you saying it *is* necessary to close the reader on
checkpoint, so the only solution is to reduce checkpoint
frequency? 


In the PubSub on DirectRunner with SDF wrapper case, my answer is
yes based on my understanding.
Closing the reader during checkpoint is the implementation
details of how the SDF wrapper wraps the Unbounded/Bounded
source. It's not controlled by the DirectRunner and the only
thing DirectRunner can control is the frequency of checkpoint,
which is hardcoded now. And closing the reader is the right
behavior since the work could be distributed to another instance
in the real world.

The ideal solution would be to offer a way to make the frequency
configurable, most possibly via PipelineOptions. Or we turn the
current PubSub UnboundedSource(and other source) implementation
into SDF. IIUC, the SDF wrapper is a migration phase of
Unbounded/Bounded source to SDF. Eventually we should have every
source in SDF.

On Thu, Dec 17, 2020 at 11:49 AM Brian Hulette
mailto:bhule...@google.com>> wrote:

Boyuan your suggestion seems at odds with Jan's. Are you
saying it *is* necessary to close the reader on checkpoint,
so the only solution is to reduce checkpoint frequency?

On Thu, Dec 17, 2020 at 10:39 AM Boyuan Zhang
mailto:boyu...@google.com>> wrote:

Thanks for your investigation, Steve! It seems like
preventing the checkpoint from happening so frequently
would be one workaround for you. Making the checkpoint
frequency configurable from pipeline option seems like
    the way to go.

On Thu, Dec 17, 2020 at 7:35 AM Jan Lukavský
 

Re: Usability regression using SDF Unbounded Source wrapper + DirectRunner

2020-12-17 Thread Jan Lukavský

Hi Boyuan,

I'm not sure if I see the difference between writing a "native" SDF for 
PubSub source and the UnboundedSource wrapper. With regards to the 
relation between reader and checkpoint, wouldn't the native 
implementation be at the same position?


In my point of view, the decision to close the reader is simply a matter 
of lifecycle of the reader. Currently, it is tightly bound to the 
restriction being processed, but that could be relaxed, so that instead 
of immediately closing the reader, it could be only _scheduled for 
closing in future_ (using processing time timer for instance) provided 
it is not reused in the remaining restriction after split (by the same 
instance of DoFn). That is an optimization that could really make sense 
outside DirectRunner, because for instance Flink has use cases, where 
user really *wants* to configure quite often checkpoints (has relation 
to how Flink implements @RequiresStableInput).


Jan

On 12/17/20 9:04 PM, Boyuan Zhang wrote:

Sorry for the confusion.

 Are you saying it *is* necessary to close the reader on
checkpoint, so the only solution is to reduce checkpoint frequency? 

In the PubSub on DirectRunner with SDF wrapper case, my answer is yes 
based on my understanding.
Closing the reader during checkpoint is the implementation details of 
how the SDF wrapper wraps the Unbounded/Bounded source. It's not 
controlled by the DirectRunner and the only thing DirectRunner can 
control is the frequency of checkpoint, which is hardcoded now. And 
closing the reader is the right behavior since the work could be 
distributed to another instance in the real world.


The ideal solution would be to offer a way to make the frequency 
configurable, most possibly via PipelineOptions. Or we turn the 
current PubSub UnboundedSource(and other source) implementation into 
SDF. IIUC, the SDF wrapper is a migration phase of Unbounded/Bounded 
source to SDF. Eventually we should have every source in SDF.


On Thu, Dec 17, 2020 at 11:49 AM Brian Hulette <mailto:bhule...@google.com>> wrote:


Boyuan your suggestion seems at odds with Jan's. Are you saying it
*is* necessary to close the reader on checkpoint, so the only
solution is to reduce checkpoint frequency?

On Thu, Dec 17, 2020 at 10:39 AM Boyuan Zhang mailto:boyu...@google.com>> wrote:

Thanks for your investigation, Steve! It seems like preventing
the checkpoint from happening so frequently would be one
workaround for you. Making the checkpoint frequency
configurable from pipeline option seems like the way to go.

On Thu, Dec 17, 2020 at 7:35 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:

Hi Steve,

I didn't mean we should deliberately make DirectRunner
slow, or we should not fix performance issues, if can be
fixed. What I meant was that if we are to choose between
short checkpoint time (and few elements processed before
checkpoint is taken) or performance, we should prefer
better tested code, in this particular case.

> After a bunch of debugging, I think I finally figured
out what the problem is though. During a checkpoint (in
trySplit), the UnboundedSourceViaSDF wrapper will close
the current source reader and create a new one.

That is actually a great example. The problem should be
fixed there (the reader probably need not to be closed on
checkpoint). And it is DirectRunner that manifested this,
due to short checkpointing.

Jan

On 12/17/20 4:14 PM, Steve Niemitz wrote:

> Primary purpose of DirectRunner is testing, not
performance

That's one argument, but it's very difficult to
effectively test a pipeline when I need to wait 15+
minutes for the first element to go through it.  I also,
disagree in general that we shouldn't care about the
performance of the DirectRunner. It's likely the first
runner new users of beam try (I know it was for us), and
if it doesn't provide enough performance to actually run
a representative pipeline, users may extrapolate that
performance onto other runners (I know we did). 
Anecdotally, the fact that the DirectRunner didn't work
for some of our initial test pipelines (because of
performance problems) probably delayed our adoption of
beam by at least 6 months.

> Steve, based on your findings, it seems like it takes
more time for the SDF pipeline to actually start to read
from PubSub and more time to output records.

Pubsub reads start ~instantly. but I'm not able to see
any elements actually output from it for a LONG time,

Re: Usability regression using SDF Unbounded Source wrapper + DirectRunner

2020-12-17 Thread Jan Lukavský

Hi Steve,

I didn't mean we should deliberately make DirectRunner slow, or we 
should not fix performance issues, if can be fixed. What I meant was 
that if we are to choose between short checkpoint time (and few elements 
processed before checkpoint is taken) or performance, we should prefer 
better tested code, in this particular case.


> After a bunch of debugging, I think I finally figured out what the 
problem is though.  During a checkpoint (in trySplit), the 
UnboundedSourceViaSDF wrapper will close the current source reader and 
create a new one.


That is actually a great example. The problem should be fixed there (the 
reader probably need not to be closed on checkpoint). And it is 
DirectRunner that manifested this, due to short checkpointing.


Jan

On 12/17/20 4:14 PM, Steve Niemitz wrote:

> Primary purpose of DirectRunner is testing, not performance

That's one argument, but it's very difficult to effectively test a 
pipeline when I need to wait 15+ minutes for the first element to go 
through it.  I also, disagree in general that we shouldn't care about 
the performance of the DirectRunner. It's likely the first runner new 
users of beam try (I know it was for us), and if it doesn't provide 
enough performance to actually run a representative pipeline, users 
may extrapolate that performance onto other runners (I know we did). 
Anecdotally, the fact that the DirectRunner didn't work for some of 
our initial test pipelines (because of performance problems) probably 
delayed our adoption of beam by at least 6 months.


> Steve, based on your findings, it seems like it takes more time for 
the SDF pipeline to actually start to read from PubSub and more time 
to output records.


Pubsub reads start ~instantly. but I'm not able to see any elements 
actually output from it for a LONG time, sometimes 30+ minutes.  I see 
the reader acking back to pubsub, so it IS committing, but no elements 
output.


After a bunch of debugging, I think I finally figured out what the 
problem is though.  During a checkpoint (in trySplit), the 
UnboundedSourceViaSDF wrapper will close the current source reader and 
create a new one.  The problem is, the pubsub reader needs some time 
to correctly estimate it's watermark [1], and because it gets closed 
and recreated so frequently due to checkpointing (either number of 
elements, or duration), it can never actually provide accurate 
estimates, and always returns the min watermark.  This seems like it 
prevents some internal timers from ever firing, effectively holding 
all elements in the pipeline state.  I can confirm this also by 
looking at WatermarkManager, where I see all the bundles pending.


[1] 
https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java#L959


On Thu, Dec 17, 2020 at 9:43 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


Hi Ismaël,

what I meant by the performance vs. testing argument is that when
choosing default values for certain (possibly configurable)
options, we
should prefer choices that result in better tested code, not better
performance. DirectRunner actually does quite many things that are
suboptimal performance-wise, but are good to be done for test
purposes
(immutability checks, as an example).

Regarding SDF in general, I can confirm we see some issues with
Flink,
most recently [1] (which I'm trying to fix right now). That is
actually
correctness, not performance issue. I personally didn't notice any
performance issues, so far.

Jan

[1] https://issues.apache.org/jira/browse/BEAM-11481

On 12/17/20 3:24 PM, Ismaël Mejía wrote:
> The influence of checkpointing on the output of the results
should be
> minimal in particular for Direct Runner. It seems what Steve reports
> here seems to be something different. Jan have you or others already
> checked the influence of this on Flink who is now using this new
> translation path?
>
> I think the argument that the Direct runner is mostly about testing
> and not about performance is an argument that is playing bad on
Beam,
> one should not necessarily exclude the other. Direct runner is our
> most used runner, basically every Beam user relies on the direct
> runners so every regression or improvement on it affects
everyone, but
> well that's a subject worth its own thread.
    >
> On Thu, Dec 17, 2020 at 10:55 AM Jan Lukavský mailto:je...@seznam.cz>> wrote:
>> Hi,
>>
>> from my point of view the number in DirectRunner are set
correctly. Primary purpose of DirectRunner is testing, not
performance, so DirectRunner makes intentionally frequent
checkpoints to easily exercise potential bugs in user code. It
might be possible to make the frequency conf

Re: Usability regression using SDF Unbounded Source wrapper + DirectRunner

2020-12-17 Thread Jan Lukavský

Hi Ismaël,

what I meant by the performance vs. testing argument is that when 
choosing default values for certain (possibly configurable) options, we 
should prefer choices that result in better tested code, not better 
performance. DirectRunner actually does quite many things that are 
suboptimal performance-wise, but are good to be done for test purposes 
(immutability checks, as an example).


Regarding SDF in general, I can confirm we see some issues with Flink, 
most recently [1] (which I'm trying to fix right now). That is actually 
correctness, not performance issue. I personally didn't notice any 
performance issues, so far.


Jan

[1] https://issues.apache.org/jira/browse/BEAM-11481

On 12/17/20 3:24 PM, Ismaël Mejía wrote:

The influence of checkpointing on the output of the results should be
minimal in particular for Direct Runner. It seems what Steve reports
here seems to be something different. Jan have you or others already
checked the influence of this on Flink who is now using this new
translation path?

I think the argument that the Direct runner is mostly about testing
and not about performance is an argument that is playing bad on Beam,
one should not necessarily exclude the other. Direct runner is our
most used runner, basically every Beam user relies on the direct
runners so every regression or improvement on it affects everyone, but
well that's a subject worth its own thread.

On Thu, Dec 17, 2020 at 10:55 AM Jan Lukavský  wrote:

Hi,

from my point of view the number in DirectRunner are set correctly. Primary 
purpose of DirectRunner is testing, not performance, so DirectRunner makes 
intentionally frequent checkpoints to easily exercise potential bugs in user 
code. It might be possible to make the frequency configurable, though.

Jan

On 12/17/20 12:20 AM, Boyuan Zhang wrote:

It's not a portable execution on DirectRunner so I would expect that outputs 
from OutputAndTimeBoundedSplittableProcessElementInvoker should be emitted 
immediately. For SDF execution on DirectRunner, the overhead could come from 
the SDF expansion, SDF wrapper and the invoker.

Steve, based on your findings, it seems like it takes more time for the SDF 
pipeline to actually start to read from PubSub and more time to output records. 
Are you able to tell how much time each part is taking?

On Wed, Dec 16, 2020 at 1:53 PM Robert Bradshaw  wrote:

If all it takes is bumping these numbers up a bit, that seems like a reasonable 
thing to do ASAP. (I would argue that perhaps they shouldn't be static, e.g. it 
might be preferable to start emitting results right away, but use larger 
batches for the steady state if there are performance benefits.)

That being said, it sounds like there's something deeper going on here. We 
should also verify that this performance impact is limited to the direct runner.

On Wed, Dec 16, 2020 at 1:36 PM Steve Niemitz  wrote:

I tried changing my build locally to 10 seconds and 10,000 elements but it 
didn't seem to make much of a difference, it still takes a few minutes for 
elements to begin actually showing up to downstream stages from the Pubsub 
read.  I can see elements being emitted from 
OutputAndTimeBoundedSplittableProcessElementInvoker, and bundles being 
committed by ParDoEvaluator.finishBundle, but after that, they seem to just 
kind of disappear somewhere.

On Wed, Dec 16, 2020 at 4:18 PM Boyuan Zhang  wrote:

Making it as the PipelineOptions was my another proposal but it might take some 
time to do so. On the other hand, tuning the number into something acceptable 
is low-hanging fruit.

On Wed, Dec 16, 2020 at 12:48 PM Ismaël Mejía  wrote:

It sounds reasonable. I am wondering also on the consequence of these
parameters for other runners (where it is every 10 seconds or 1
elements) + their own configuration e.g. checkpointInterval,
checkpointTimeoutMillis and minPauseBetweenCheckpoints for Flink. It
is not clear for me what would be chosen now in this case.

I know we are a bit anti knobs but maybe it makes sense to make this
configurable via PipelineOptions at least for Direct runner.

On Wed, Dec 16, 2020 at 7:29 PM Boyuan Zhang  wrote:

I agree, Ismael.

 From my current investigation, the performance overhead should majorly come 
from the frequency of checkpoint in 
OutputAndTimeBoundedSplittableProcessElementinvoker[1], which is hardcoded in 
the DirectRunner(every 1 seconds or 100 elements)[2]. I believe configuring 
these numbers on DirectRunner should improve reported cases so far. My last 
proposal was to change the number to every 5 seconds or 1 elements. What do 
you think?

[1] 
https://github.com/apache/beam/blob/master/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java
[2] 
https://github.com/apache/beam/blob/3bb232fb098700de408f574585dfe74bbaff7230/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java#L178-L181

On Wed

Re: Usability regression using SDF Unbounded Source wrapper + DirectRunner

2020-12-17 Thread Jan Lukavský

Hi,

from my point of view the number in DirectRunner are set correctly. 
Primary purpose of DirectRunner is testing, not performance, so 
DirectRunner makes intentionally frequent checkpoints to easily exercise 
potential bugs in user code. It might be possible to make the frequency 
configurable, though.


Jan

On 12/17/20 12:20 AM, Boyuan Zhang wrote:
It's not a portable execution on DirectRunner so I would expect that 
outputs from OutputAndTimeBoundedSplittableProcessElementInvoker 
should be emitted immediately. For SDF execution on DirectRunner, the 
overhead could come from the SDF expansion, SDF wrapper and the invoker.


Steve, based on your findings, it seems like it takes more time for 
the SDF pipeline to actually start to read from PubSub and more time 
to output records. Are you able to tell how much time each part is taking?


On Wed, Dec 16, 2020 at 1:53 PM Robert Bradshaw > wrote:


If all it takes is bumping these numbers up a bit, that seems like
a reasonable thing to do ASAP. (I would argue that perhaps they
shouldn't be static, e.g. it might be preferable to start emitting
results right away, but use larger batches for the steady state if
there are performance benefits.)

That being said, it sounds like there's something deeper going on
here. We should also verify that this performance impact is
limited to the direct runner.

On Wed, Dec 16, 2020 at 1:36 PM Steve Niemitz mailto:sniem...@apache.org>> wrote:

I tried changing my build locally to 10 seconds and 10,000
elements but it didn't seem to make much of a difference, it
still takes a few minutes for elements to begin actually
showing up to downstream stages from the Pubsub read.  I can
see elements being emitted
from OutputAndTimeBoundedSplittableProcessElementInvoker, and
bundles being committed by ParDoEvaluator.finishBundle, but
after that, they seem to just kind of disappear somewhere.

On Wed, Dec 16, 2020 at 4:18 PM Boyuan Zhang
mailto:boyu...@google.com>> wrote:

Making it as the PipelineOptions was my another proposal
but it might take some time to do so. On the other hand,
tuning the number into something acceptable is low-hanging
fruit.

On Wed, Dec 16, 2020 at 12:48 PM Ismaël Mejía
mailto:ieme...@gmail.com>> wrote:

It sounds reasonable. I am wondering also on the
consequence of these
parameters for other runners (where it is every 10
seconds or 1
elements) + their own configuration e.g.
checkpointInterval,
checkpointTimeoutMillis and minPauseBetweenCheckpoints
for Flink. It
is not clear for me what would be chosen now in this case.

I know we are a bit anti knobs but maybe it makes
sense to make this
configurable via PipelineOptions at least for Direct
runner.

On Wed, Dec 16, 2020 at 7:29 PM Boyuan Zhang
mailto:boyu...@google.com>> wrote:
>
> I agree, Ismael.
>
> From my current investigation, the performance
overhead should majorly come from the frequency of
checkpoint in
OutputAndTimeBoundedSplittableProcessElementinvoker[1],
which is hardcoded in the DirectRunner(every 1 seconds
or 100 elements)[2]. I believe configuring these
numbers on DirectRunner should improve reported cases
so far. My last proposal was to change the number to
every 5 seconds or 1 elements. What do you think?
>
> [1]

https://github.com/apache/beam/blob/master/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java
> [2]

https://github.com/apache/beam/blob/3bb232fb098700de408f574585dfe74bbaff7230/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java#L178-L181
>
> On Wed, Dec 16, 2020 at 9:02 AM Ismaël Mejía
mailto:ieme...@gmail.com>> wrote:
>>
>> I can guess that the same issues mentioned here
probably will affect
>> the usability for people trying Beam's interactive
SQL on Unbounded IO
>> too.
>>
>> We should really take into account that the
performance of the SDF
>> based path should be as good or better than the
previous version
>> before considering its 

Re: Possible 80% reduction in overhead for flink runner, input needed

2020-11-24 Thread Jan Lukavský
Agree this should be made the default (in the long term). There is no 
current plan to achieve this I'm aware of. I think we could get there by


 a) first log a warning for pipelines without --fasterCopy about 
performance, that user should test his pipeline against DirectRunner 
(with mutability detector) and then add the `--fasterCopy` option. We 
should keep this for several releases, then


 b) log a warning that --fasterCopy will be the default in some future 
version (we might want to gather user feedback here, to make the 
decision about creating opt-out flag), and then


 c) switch to default and maybe deprecate --fasterCopy (maybe along 
with --objectReuse, as that might be switched to default as well)


But, there has not been any discussion about this, yet.

 Jan

On 11/24/20 6:39 PM, Kenneth Knowles wrote:
Is there a plan/timeline to resolve the flag? It would be good for 
this to be the default, or only, behavior.


The behavior with the flag activated is correct. It may still cause 
problems for users who accidentally depended on things outside the 
Beam model, so having a flag for them to opt-out until they can 
migrate might be good.


Meanwhile there is a larger thread we could start about requesting a 
safe-to-mutate copy of things so it only affects performance local to 
that DoFn. Users can do this themselves, but if it is metadata on a 
DoFn it may be optimized sometimes.


Kenn

On Tue, Nov 24, 2020 at 12:39 AM Jan Lukavský <mailto:je...@seznam.cz>> wrote:


Hi Antonio,

the PR is already merged [1]. It will be released as part of the
upcoming 2.26.0 release [2]. You must activate the feature by
using the
--fasterCopy flag.

Best,

  Jan

[1] https://github.com/apache/beam/pull/13240

[2] https://issues.apache.org/jira/browse/BEAM-11146

On 11/23/20 10:57 PM, Antonio Si wrote:
> Hi all,
>
> Our team recently did a similar experiment and came to a similar
observation as what Teodor did.
> The Beam slack channel points me to email thread discussion.
>
> It seems like there is a jira issue created and Teodor had a PR.
May I ask what is the decision on that and if the PR is approved?
May I also have the link to the jira issue?
>
> Much appreciated.
>
> Antonio.
>
>
> On 2020/10/30 20:08:32, Kenneth Knowles mailto:k...@apache.org>> wrote:
>> I have not had an easy time following all the discussion points
here. It
>> seems that the main issue is really something that has been
true since
>> before Beam started: " In Beam, the user is not supposed to
modify the input
>> collection and if they do, it's undefined behavior." and after
Beam started
>> we added features to help: "This is the reason the DirectRunner
checks for
>> this, to make sure the users are not relying on it."
>>
>> This is all true. A runner can do "chaining" or "fusion" with
zero copy and
>> this is allowed. An SDK harness can do the same. If a user
mutates an input
>> to a DoFn, or mutates a value after it is output, that is user
error. A
>> runner that eagerly cloning elements is wasting time and we
should remove
>> that.
>>
>> Kenn
>>
>> On Thu, Oct 29, 2020 at 8:03 AM Teodor Spæren
mailto:teodor_spae...@riseup.net>>
>> wrote:
>>
>>> Thanks Jan, this cleared some things up!
>>>
>>> Best regards,
>>> Teodor Spæren
>>>
>>> On Thu, Oct 29, 2020 at 02:13:50PM +0100, Jan Lukavský wrote:
>>>> Hi Teodor,
>>>>
>>>> the confusion here maybe comes from the fact, that there are two
>>>> (logical) representations of an element in PCollection. One
>>>> representation is the never mutable (most probably serialized
in a
>>>> binary form) form of a PCollection element, where no
modifications are
>>>> possible. Once a PCollection is created (e.g. read from
source, or
>>>> created by a PTransform) it cannot be modified further. The
second
>>>> form is an SDK-dependent representation of each PCollection
element in
>>>> user code. This representation is what UDFs work with. The
same source
>>>> (binary) form of element can have (and will have) different
>>>> representation in Java SDK and in Python SDK. The Beam model says
>>>> nothing about mutability of this SDK-dependent form.
Nevertheless,
>>>> even if you modify this element, it has no impact on the source
>>>> representation. Bu

Re: Possible 80% reduction in overhead for flink runner, input needed

2020-11-24 Thread Jan Lukavský

Hi Antonio,

the PR is already merged [1]. It will be released as part of the 
upcoming 2.26.0 release [2]. You must activate the feature by using the 
--fasterCopy flag.


Best,

 Jan

[1] https://github.com/apache/beam/pull/13240

[2] https://issues.apache.org/jira/browse/BEAM-11146

On 11/23/20 10:57 PM, Antonio Si wrote:

Hi all,

Our team recently did a similar experiment and came to a similar observation as 
what Teodor did.
The Beam slack channel points me to email thread discussion.

It seems like there is a jira issue created and Teodor had a PR. May I ask what 
is the decision on that and if the PR is approved? May I also have the link to 
the jira issue?

Much appreciated.

Antonio.


On 2020/10/30 20:08:32, Kenneth Knowles  wrote:

I have not had an easy time following all the discussion points here. It
seems that the main issue is really something that has been true since
before Beam started: " In Beam, the user is not supposed to modify the input
collection and if they do, it's undefined behavior." and after Beam started
we added features to help: "This is the reason the DirectRunner checks for
this, to make sure the users are not relying on it."

This is all true. A runner can do "chaining" or "fusion" with zero copy and
this is allowed. An SDK harness can do the same. If a user mutates an input
to a DoFn, or mutates a value after it is output, that is user error. A
runner that eagerly cloning elements is wasting time and we should remove
that.

Kenn

On Thu, Oct 29, 2020 at 8:03 AM Teodor Spæren 
wrote:


Thanks Jan, this cleared some things up!

Best regards,
Teodor Spæren

On Thu, Oct 29, 2020 at 02:13:50PM +0100, Jan Lukavský wrote:

Hi Teodor,

the confusion here maybe comes from the fact, that there are two
(logical) representations of an element in PCollection. One
representation is the never mutable (most probably serialized in a
binary form) form of a PCollection element, where no modifications are
possible. Once a PCollection is created (e.g. read from source, or
created by a PTransform) it cannot be modified further. The second
form is an SDK-dependent representation of each PCollection element in
user code. This representation is what UDFs work with. The same source
(binary) form of element can have (and will have) different
representation in Java SDK and in Python SDK. The Beam model says
nothing about mutability of this SDK-dependent form. Nevertheless,
even if you modify this element, it has no impact on the source
representation. But, it can lead to SDK-dependent errors, when the
element is mutated in a way that a runner might not expect.

Hope this helps.

Jan

On 10/29/20 1:58 PM, Teodor Spæren wrote:

Hey!

Just so I understand this correctly then, what does the following
quote from [1], section 3.2.3 mean:

A PCollection is immutable. Once created, you cannot add, remove, or
change individual elements. A Beam Transform might process each
element of a PCollection and generate new pipeline data (as a new
PCollection), *but it does not consume or modify the original input
collection.*

(Don't know what the normal way of highlighting is on mailing lists,
so I just put it between *)

I read this as meaning that it is the users responsibilty to make
sure that their transformations do not modify the input, but should
I rather read it as meaning the beam runner itself should make sure
the user cannot make such a mistake? I find this reading at odds
with the documentation about the direct runner and it's express
purpose being to make sure users doesn't rely on semantics the beam
model doesn't ensure. And modifying of input arguments being one of
the constraints listed. [2].

It doesn't change the outcome here, adding an opt out switch, but if
I've missunderstood the quote above, I think this might benefit by
being reworded, to make sure it is communicated that shooting
yourself in the foot is impossible and the direct runner testing of
modifying input should be removed, as there is no point in users
making sure to not modifying the input if all runners guarantee it.


Also, I ran the whole Flink test suite with a simple return instead
of the deep copy and all tests passed, so there is no such test in
there. Depending on the reading above, we should add such tests to
all runners.

Best regards,
Teodor Spæren

On Thu, Oct 29, 2020 at 10:16:30AM +0100, Maximilian Michels wrote:

Ok then we are on the same page, but I disagree with your
conclusion. The reason Flink has to do the deep copy is that it
doesn't state that the inputs are immutable and should not be
changed, and so have to do the deep copy. In Beam, the user is
not supposed to modify the input collection and if they do, it's
undefined behavior. This is the reason the DirectRunner checks
for this, to make sure the users are not relying on it.

It's not written anywhere that the input cannot be mutated. A
DirectRunner test is not a proof. Any runner could add a test
which prov

Re: PTransform Annotations Proposal

2020-11-16 Thread Jan Lukavský
Minor correction, the CoGBK broadcast vs. full shuffle is probably not 
ideal example, because it still requires grouping the larger PCollection 
(if not already grouped). If we take Join PTransform that acts on 
cartesian product of these groups, then it works well.


Jan

On 11/16/20 8:39 PM, Jan Lukavský wrote:


Hi,

could this proposal be generalized to annotations of PCollections as 
well? Maybe that reduces to several types of annotations of a 
PTransform - e.g.


 a) runtime annotations of a PTransform (that might be scheduling 
hints - i.e. schedule this task to nodes with GPUs, etc.)


 b) output annotations - i.e. annotations that actually apply to 
PCollections, as every PCollection has at most one producer (this is 
what have been actually discussed in the referenced mailing list threads)


It would be cool, if this added option to do PTransform expansions 
based on annotations of input PCollections. We tried to play with this 
in Euphoria DSL, but it turned out it would be best fitted in Beam SDK.


Example of input annotation sensitive expansion might be CoGBK, when 
one side is annotated i.e. FitsInMemoryPerWindow (or SmallPerWindow, 
or whatever), then CoGBK might be expanded using broadcast instead of 
full shuffle.


Absolutely agree that all this must not have anything to do with 
semantics and correctness, thus might be safely ignored, and that 
might answer the last question of @Reuven, when there are conflicting 
annotations, it would be possible to simple drop them as a last resort.


Jan

On 11/16/20 8:13 PM, Robert Burke wrote:
I imagine it has everything to do with the specific annotation to 
define that.


The runner notionally doesn't need to do anything with them, as they 
are optional, and not required for correctness.


On Mon, Nov 16, 2020, 10:56 AM Reuven Lax <mailto:re...@google.com>> wrote:


PTransforms are hierarchical - namely a PTransform contains other
PTransforms, and so on. Is the runner expected to resolve all
annotations down to leaf nodes? What happens if that results in
conflicting annotations?

On Mon, Nov 16, 2020 at 10:54 AM Robert Burke mailto:rob...@frantil.com>> wrote:

That's a good question.

I think the main difference is a matter of scope. Annotations
would apply to a PTransform while an environment applies to
sets of transforms. A difference is the optional nature of
the annotations they don't affect correctness. Runners don't
need to do anything with them and still execute the pipeline
correctly.

Consider a privacy analysis on a pipeline graph. An
annotation indicating that a transform provides a certain
level of anonymization can be used in an analysis to
determine if the downstream transforms are encountering raw
data or not.

From my understanding (which can be wrong) environments are
rigid. Transforms in different environments can't be fused.
"This is the python env", "this is the java env" can't be
merged together. It's not clear to me that we have defined
when environments are safely fuseable outside of equality.
There's value in that simplicity.

AFIACT environment has less to do with the machines a
pipeline is executing on than it does about the kinds of SDK
pipelines it understands and can execute.



On Mon, Nov 16, 2020, 10:36 AM Chad Dombrova
mailto:chad...@gmail.com>> wrote:


Another example of an optional annotation is marking
a transform to run on secure hardware, or to give
hints to profiling/dynamic analysis tools.


There seems to be a lot of overlap between this idea and
Environments.  Can you talk about how you feel they may
be different or related?  For example, I could see
annotations as a way of tagging transforms with an
Environment, or I could see Environments becoming a
specialized form of annotation.

-chad



Re: PTransform Annotations Proposal

2020-11-16 Thread Jan Lukavský

Hi,

could this proposal be generalized to annotations of PCollections as 
well? Maybe that reduces to several types of annotations of a PTransform 
- e.g.


 a) runtime annotations of a PTransform (that might be scheduling hints 
- i.e. schedule this task to nodes with GPUs, etc.)


 b) output annotations - i.e. annotations that actually apply to 
PCollections, as every PCollection has at most one producer (this is 
what have been actually discussed in the referenced mailing list threads)


It would be cool, if this added option to do PTransform expansions based 
on annotations of input PCollections. We tried to play with this in 
Euphoria DSL, but it turned out it would be best fitted in Beam SDK.


Example of input annotation sensitive expansion might be CoGBK, when one 
side is annotated i.e. FitsInMemoryPerWindow (or SmallPerWindow, or 
whatever), then CoGBK might be expanded using broadcast instead of full 
shuffle.


Absolutely agree that all this must not have anything to do with 
semantics and correctness, thus might be safely ignored, and that might 
answer the last question of @Reuven, when there are conflicting 
annotations, it would be possible to simple drop them as a last resort.


Jan

On 11/16/20 8:13 PM, Robert Burke wrote:
I imagine it has everything to do with the specific annotation to 
define that.


The runner notionally doesn't need to do anything with them, as they 
are optional, and not required for correctness.


On Mon, Nov 16, 2020, 10:56 AM Reuven Lax > wrote:


PTransforms are hierarchical - namely a PTransform contains other
PTransforms, and so on. Is the runner expected to resolve all
annotations down to leaf nodes? What happens if that results in
conflicting annotations?

On Mon, Nov 16, 2020 at 10:54 AM Robert Burke mailto:rob...@frantil.com>> wrote:

That's a good question.

I think the main difference is a matter of scope. Annotations
would apply to a PTransform while an environment applies to
sets of transforms. A difference is the optional nature of the
annotations they don't affect correctness. Runners don't need
to do anything with them and still execute the pipeline
correctly.

Consider a privacy analysis on a pipeline graph. An annotation
indicating that a transform provides a certain level of
anonymization can be used in an analysis to determine if the
downstream transforms are encountering raw data or not.

From my understanding (which can be wrong) environments are
rigid. Transforms in different environments can't be fused.
"This is the python env", "this is the java env" can't be
merged together. It's not clear to me that we have defined
when environments are safely fuseable outside of equality.
There's value in that simplicity.

AFIACT environment has less to do with the machines a pipeline
is executing on than it does about the kinds of SDK pipelines
it understands and can execute.



On Mon, Nov 16, 2020, 10:36 AM Chad Dombrova
mailto:chad...@gmail.com>> wrote:


Another example of an optional annotation is marking a
transform to run on secure hardware, or to give hints
to profiling/dynamic analysis tools.


There seems to be a lot of overlap between this idea and
Environments.  Can you talk about how you feel they may be
different or related? For example, I could see annotations
as a way of tagging transforms with an Environment, or I
could see Environments becoming a specialized form of
annotation.

-chad



Re: Beam 2.25.0 / Flink 1.11.2 - Job failing after upgrading from 2.24.0 / Flink 1.10.2

2020-11-04 Thread Jan Lukavský

Hi Tobias,

this looks like a bug, the clearGlobalState method has been introduced 
in 2.25.0, and it (seems to) might have issues related to rocksdb, can 
you file a Jira for that, please?


Thanks,

 Jan

On 11/4/20 9:50 AM, Kaymak, Tobias wrote:
When running our Kafka-To-BigQuery pipeline with the Flink 1.11.2 
Docker image,
the following exception is visible for the failing job on the *job 
manager*:


2020-11-04 09:27:14
java.lang.RuntimeException: Failed to cleanup global state.
at 
org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals.clearGlobalState(FlinkStateInternals.java:150)
at 
org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator.maybeEmitWatermark(DoFnOperator.java:791)
at 
org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator.processWatermark1(DoFnOperator.java:741)
at 
org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator.processWatermark(DoFnOperator.java:713)
at 
org.apache.flink.streaming.runtime.tasks.OneInputStreamTask$StreamTaskNetworkOutput.emitWatermark(OneInputStreamTask.java:167)
at 
org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve.findAndOutputNewMinWatermarkAcrossAlignedChannels(StatusWatermarkValve.java:179)
at 
org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve.inputWatermark(StatusWatermarkValve.java:101)
at org.apache.flink.streaming.runtime.io 
.StreamTaskNetworkInput.processElement(StreamTaskNetworkInput.java:180)
at org.apache.flink.streaming.runtime.io 
.StreamTaskNetworkInput.emitNext(StreamTaskNetworkInput.java:153)
at org.apache.flink.streaming.runtime.io 
.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:67)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:351)
at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxStep(MailboxProcessor.java:191)
at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:181)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:566)
at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:536)

at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:721)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:546)
at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.ClassCastException: java.lang.String cannot be 
cast to org.apache.flink.runtime.state.VoidNamespace
at 
org.apache.flink.runtime.state.VoidNamespaceSerializer.serialize(VoidNamespaceSerializer.java:32)
at 
org.apache.flink.contrib.streaming.state.RocksDBKeySerializationUtils.writeNameSpace(RocksDBKeySerializationUtils.java:77)
at 
org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend.getKeys(RocksDBKeyedStateBackend.java:291)
at 
org.apache.flink.runtime.state.AbstractKeyedStateBackend.applyToAllKeys(AbstractKeyedStateBackend.java:242)
at 
org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals.clearGlobalState(FlinkStateInternals.java:141)

... 17 more
This is from the *task manager's* logs:
2020-11-04 08:46:31,250 WARN 
 org.apache.flink.runtime.taskmanager.Task                    [] - 
BigQueryIO.Write/BatchLoads/JobIdCreationRoot_LOAD/Read(CreateSource)/ParDo(BoundedSourceAsSDFWrapper)/ParMultiDo(BoundedSourceAsSDFWrapper)/ProcessKeyedElements/Splittable 
ueryIO.Write/BatchLoads/CreateJobId_LOAD/ParMultiDo(Anonymous) -> 
BigQueryIO.Write/BatchLoads/JobIdSideInput_LOAD/Combine.GloballyAsSingletonView/Combine.globally(Singleton)/WithKeys/AddKeys/Map/ParMultiDo(Anonymous) 
-> ToKeyedWorkItem (1/1) (bebac6c581d1b8ece88007ec0 
java.lang.RuntimeException: Failed to cleanup global state.   at 
org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals.clearGlobalState(FlinkStateInternals.java:150) 
~[blob_p-656af447c7120652ba6a8f48516776effc33dc07-8df5e6b00c52050981a9af655c97d0c9:?] 
  at 
org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator.maybeEmitWatermark(DoFnOperator.java:791) 
~[blob_p-656af447c7120652ba6a8f48516776effc33dc07-8df5e6b00c52050981a9af655c97d0c9:?] 
  at 
org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator.processWatermark1(DoFnOperator.java:741) 
~[blob_p-656af447c7120652ba6a8f48516776effc33dc07-8df5e6b00c52050981a9af655c97d0c9:?] 
  at 
org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator.processWatermark(DoFnOperator.java:713) 
~[blob_p-656af447c7120652ba6a8f48516776effc33dc07-8df5e6b00c52050981a9af655c97d0c9:?] 
  at 
org.apache.flink.streaming.runtime.tasks.OneInputStreamTask$StreamTaskNetworkOutput.emitWatermark(OneInputStreamTask.java:167) 

<    1   2   3   4   5   6   >