[ 
https://issues.apache.org/jira/browse/BEAM-9273?focusedWorklogId=385059&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-385059
 ]

ASF GitHub Bot logged work on BEAM-9273:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 11/Feb/20 10:17
            Start Date: 11/Feb/20 10:17
    Worklog Time Spent: 10m 
      Work Description: je-ik commented on pull request #10816: [BEAM-9273] 
Explicitly disable @RequiresTimeSortedInput on unsupported runners
URL: https://github.com/apache/beam/pull/10816#discussion_r377544106
 
 

 ##########
 File path: 
runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DoFnFeatures.java
 ##########
 @@ -0,0 +1,82 @@
+/*
+ * 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 org.apache.beam.runners.core.construction;
+
+import org.apache.beam.sdk.state.BagState;
+import org.apache.beam.sdk.state.MapState;
+import org.apache.beam.sdk.state.SetState;
+import org.apache.beam.sdk.state.State;
+import org.apache.beam.sdk.state.ValueState;
+import org.apache.beam.sdk.state.WatermarkHoldState;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignatures;
+import org.apache.beam.sdk.values.TypeDescriptor;
+
+/**
+ * Features a {@link DoFn} can posses. Each runner might implement a different 
(sub)set of this
 
 Review comment:
   Agree. The reason I created this class is that I wanted to demonstrate the 
approach of declaring pipeline requirements and validating that runner supports 
all requirements. 
   
   That would go as follows:
   ```java
    Set<Features> required = PipelineFeaturs.extract(pipeline);
    Runner.validateAllFeaturesSupported(required);
   ```
   That way, adding a new feature to pipeline would not require any change to 
runner core, but the runner would reject the pipeline.
   
   I then realized that this change would be too big and really not related to 
the annotation, so I returned back to the original approach, where runners 
explicitly reject features they do not support (and adding unsupported feature 
needs modification in runners code). I will rename the class as you suggest.
 
----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 385059)
    Time Spent: 3h 10m  (was: 3h)

> Explicitly fail pipeline with @RequiresTimeSortedInput with unsupported runner
> ------------------------------------------------------------------------------
>
>                 Key: BEAM-9273
>                 URL: https://issues.apache.org/jira/browse/BEAM-9273
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-java-core
>            Reporter: Jan Lukavský
>            Assignee: Jan Lukavský
>            Priority: Major
>          Time Spent: 3h 10m
>  Remaining Estimate: 0h
>
> Fail pipeline with @RequiresTimeSortedInput annotation in pipeline 
> translation time when being run with unsupported runner. Currently, 
> unsupported runners are:
>  - apex
>  - portable flink
>  - gearpump
>  - dataflow
>  - jet
>  - samza
>  - spark structured streaming
> These runners should reject the pipeline.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to