avery-nguyen-verily opened a new issue, #33815:
URL: https://github.com/apache/beam/issues/33815
### What happened?
## What happened?
If you try running a pipeline that has multiple unbounded sources where 1 or
more do not emit any data within the window and try to flatten the pcollections
together Prism will not execute any subsequent ParDos until all sources emit
something even after the window ends.
## Expected Behaviour
Prism should finish processing the data once the window ends, even if not
all the sources that are being flattened have emitted data.
## Minimal Example
```go
package main
import (
"context"
"log"
"math"
"time"
"github.com/apache/beam/sdks/v2/go/pkg/beam"
"github.com/apache/beam/sdks/v2/go/pkg/beam/core/graph/window"
beamLog "github.com/apache/beam/sdks/v2/go/pkg/beam/log"
"github.com/apache/beam/sdks/v2/go/pkg/beam/register"
"github.com/apache/beam/sdks/v2/go/pkg/beam/transforms/periodic"
"github.com/apache/beam/sdks/v2/go/pkg/beam/x/beamx"
)
func init() {
register.Function3x0(count)
register.Function3x0(initCount)
register.Emitter1[int]()
}
func initCount(ctx context.Context, _ []byte, emit func(int)) {
beamLog.Info(ctx, "Starting count")
emit(1)
}
func count(ctx context.Context, currCount int, emit func(int)) {
beamLog.Infof(ctx, "Current Count = %d", currCount)
emit(currCount + 1)
}
func main() {
beam.Init()
ctx := context.Background()
pipeline, scope := beam.NewPipelineWithRoot()
duration := 5 * time.Second
fixedWindow := window.NewFixedWindows(duration)
unboundedSource1 := periodic.Impulse(scope, time.Now(),
time.UnixMilli(math.MaxInt64), duration, false)
unboundedSource2 := periodic.Impulse(scope,
time.UnixMilli(math.MaxInt32), time.UnixMilli(math.MaxInt64), duration, false)
trigger1 := beam.WindowInto(scope, fixedWindow, unboundedSource1)
trigger2 := beam.WindowInto(scope, fixedWindow, unboundedSource2)
s1 := scope.Scope("source")
c1 := beam.ParDo(s1, initCount, trigger1)
c2 := beam.ParDo(s1, initCount, trigger2)
s := beam.Flatten(s1, c1, c2) // prism will never run any ParDo that
depends on this
s2 := scope.Scope("count")
c3 := beam.ParDo(s2, count, s)
c4 := beam.ParDo(s2, count, c3)
beam.ParDo(s2, count, c4)
if err := beamx.Run(ctx, pipeline); err != nil {
log.Fatalf("Failed to execute job: %v", err)
}
}
```
**Output**
Prism only ever runs `initCount`, none of the `count` ParDo functions are
ever run because it is waiting for data from c2
```
2025/01/30 15:55:59 INFO log from SDK worker
worker.ID=job-001[go-job-1-1738252559500852940]_go
worker.endpoint=localhost:38569 sdk.transformID=e12
sdk.location=/home/user/workspace/beam-starter-go/main.go:25
sdk.time=2025-01-30T15:55:59.830Z sdk.msg="Starting count"
2025/01/30 15:55:59 INFO log from SDK worker
worker.ID=job-001[go-job-1-1738252559500852940]_go
worker.endpoint=localhost:38569 sdk.transformID=e12
sdk.location=/home/user/workspace/beam-starter-go/main.go:25
sdk.time=2025-01-30T15:55:59.830Z sdk.msg="Starting count"
2025/01/30 15:55:59 INFO log from SDK worker
worker.ID=job-001[go-job-1-1738252559500852940]_go
worker.endpoint=localhost:38569 sdk.transformID=e12
sdk.location=/home/user/workspace/beam-starter-go/main.go:25
sdk.time=2025-01-30T15:55:59.830Z sdk.msg="Starting count"
2025/01/30 15:55:59 INFO log from SDK worker
worker.ID=job-001[go-job-1-1738252559500852940]_go
worker.endpoint=localhost:38569 sdk.transformID=e12
sdk.location=/home/user/workspace/beam-starter-go/main.go:25
sdk.time=2025-01-30T15:55:59.830Z sdk.msg="Starting count"
2025/01/30 15:55:59 INFO log from SDK worker
worker.ID=job-001[go-job-1-1738252559500852940]_go
worker.endpoint=localhost:38569 sdk.transformID=e12
sdk.location=/home/user/workspace/beam-starter-go/main.go:25
sdk.time=2025-01-30T15:55:59.830Z sdk.msg="Starting count"
```
### Issue Priority
Priority: 2 (default / most bugs should be filed as P2)
### Issue Components
- [ ] Component: Python SDK
- [ ] Component: Java SDK
- [x] Component: Go SDK
- [ ] Component: Typescript SDK
- [ ] Component: IO connector
- [ ] Component: Beam YAML
- [ ] Component: Beam examples
- [ ] Component: Beam playground
- [ ] Component: Beam katas
- [ ] Component: Website
- [ ] Component: Infrastructure
- [ ] Component: Spark Runner
- [ ] Component: Flink Runner
- [ ] Component: Samza Runner
- [ ] Component: Twister2 Runner
- [ ] Component: Hazelcast Jet Runner
- [ ] Component: Google Cloud Dataflow Runner
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]