lostluck commented on a change in pull request #15239:
URL: https://github.com/apache/beam/pull/15239#discussion_r681872693



##########
File path: sdks/go/test/integration/primitives/windowinto.go
##########
@@ -91,3 +90,32 @@ func WindowSums_GBK(s beam.Scope) {
 func WindowSums_Lifted(s beam.Scope) {
        WindowSums(s.Scope("Lifted"), stats.SumPerKey)
 }
+
+// TriggerWindowSums, much like WindowSums described above has an addition of 
configuring
+// a trigger here. SetDefault works fine. Other triggers such as SetAlways 
throws
+// pane decoding error.
+func TriggerWindowSums(s beam.Scope, sumPerKey func(beam.Scope, 
beam.PCollection) beam.PCollection) {

Review comment:
       Consider re-writing this test to make it easier to validate *triggers* 
rather than windows. Certainly re-use what you can, but you are allowed to 
write new code.

##########
File path: sdks/go/pkg/beam/core/graph/window/trigger.go
##########
@@ -0,0 +1,41 @@
+// 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 window
+
+type TriggerType string

Review comment:
       Consider how we'd implement the more complicated triggers that have sub 
triggers. For example, AfterAll and AfterAny are missing their sub trigger 
configurations.

##########
File path: sdks/go/pkg/beam/pcollection.go
##########
@@ -49,7 +50,11 @@ func (p PCollection) IsValid() bool {
 }
 
 // TODO(herohde) 5/30/2017: add name for PCollections? Java supports it.
-// TODO(herohde) 5/30/2017: add windowing strategy and documentation.
+
+// WindowingStrategy returns the WindowingStrategy of PCollection.
+func (p PCollection) WindowingStrategy() *window.WindowingStrategy {
+       return p.n.WindowingStrategy()
+}

Review comment:
       We can probably leave out this method for now. It's easier to add 
something like this later than to remove it later.
   I'd still remove the TODO. We don't know what herohde's goal was with the 
TODO, and we've gone in our own design direction.

##########
File path: sdks/go/pkg/beam/core/graph/window/trigger.go
##########
@@ -0,0 +1,41 @@
+// 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 window
+
+type TriggerType string
+
+const (
+       Default  TriggerType = "Trigger_Default_"
+       Always   TriggerType = "Trigger_Always_"
+       AfterAny TriggerType = "Trigger_AfterAny_"
+       AfterAll TriggerType = "Trigger_AfterAny_"

Review comment:
       Typo: duplicated as AfterAny instead of AfterAll.




-- 
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]


Reply via email to