forked from apache/beam
-
Notifications
You must be signed in to change notification settings - Fork 0
/
filter.go
106 lines (86 loc) · 3.51 KB
/
filter.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package main
// See: https://github.com/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/cookbook/FilterExamples.java
import (
"context"
"flag"
"reflect"
"github.com/apache/beam/sdks/go/pkg/beam"
"github.com/apache/beam/sdks/go/pkg/beam/io/bigqueryio"
"github.com/apache/beam/sdks/go/pkg/beam/log"
"github.com/apache/beam/sdks/go/pkg/beam/options/gcpopts"
"github.com/apache/beam/sdks/go/pkg/beam/transforms/stats"
"github.com/apache/beam/sdks/go/pkg/beam/x/beamx"
)
var (
input = flag.String("input", "clouddataflow-readonly:samples.weather_stations", "Weather data BQ table.")
output = flag.String("output", "", "Output BQ table.")
month = flag.Int("month_filter", 7, "Numerical month to analyze")
)
type WeatherDataRow struct {
Year int `bigquery:"year"`
Month int `bigquery:"month"`
Day int `bigquery:"day"`
MeanTemp float64 `bigquery:"mean_temp"`
}
// BelowGlobalMean computes the rows for the given month below the global mean. It takes a
// PCollection<WeatherDataRow> and returns a PCollection<WeatherDataRow>.
func BelowGlobalMean(s beam.Scope, m int, rows beam.PCollection) beam.PCollection {
s = s.Scope("BelowGlobalMean")
// Find the global mean of all the mean_temp readings in the weather data.
globalMeanTemp := stats.Mean(s, beam.ParDo(s, extractMeanTempFn, rows))
// Rows filtered to remove all but a single month
filtered := beam.ParDo(s, &filterMonthFn{Month: m}, rows)
// Then, use the global mean as a side input, to further filter the weather data.
// By using a side input to pass in the filtering criteria, we can use a value
// that is computed earlier in pipeline execution. We'll only output readings with
// temperatures below this mean.
return beam.ParDo(s, filterBelowMeanFn, filtered, beam.SideInput{Input: globalMeanTemp})
}
type filterMonthFn struct {
Month int `json:"month"`
}
func (f *filterMonthFn) ProcessElement(row WeatherDataRow, emit func(WeatherDataRow)) {
if row.Month == f.Month {
emit(row)
}
}
func extractMeanTempFn(row WeatherDataRow) float64 {
return row.MeanTemp
}
func filterBelowMeanFn(row WeatherDataRow, globalMeanTemp float64, emit func(WeatherDataRow)) {
if row.MeanTemp < globalMeanTemp {
emit(row)
}
}
func main() {
flag.Parse()
beam.Init()
ctx := context.Background()
if *output == "" {
log.Exit(ctx, "No output table specified. Use --output=<table>")
}
project := gcpopts.GetProject(ctx)
log.Info(ctx, "Running filter")
p := beam.NewPipeline()
s := p.Root()
rows := bigqueryio.Read(s, project, *input, reflect.TypeOf(WeatherDataRow{}))
out := BelowGlobalMean(s, *month, rows)
bigqueryio.Write(s, project, *output, out)
if err := beamx.Run(ctx, p); err != nil {
log.Exitf(ctx, "Failed to execute job: %v", err)
}
}