[ 
https://issues.apache.org/jira/browse/FLINK-8159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16691744#comment-16691744
 ] 

ASF GitHub Bot commented on FLINK-8159:
---------------------------------------

dawidwys commented on a change in pull request #7110: [FLINK-8159] [cep] 
Pattern(Flat)SelectFunctions should support RichFunction interface
URL: https://github.com/apache/flink/pull/7110#discussion_r234631211
 
 

 ##########
 File path: 
flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/RichAndCondition.java
 ##########
 @@ -0,0 +1,83 @@
+/*
+ * 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.flink.cep.pattern.conditions;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.functions.util.FunctionUtils;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * A {@link RichIterativeCondition condition} which combines two conditions 
with a logical
+ * {@code AND} and returns {@code true} if both are {@code true}.
+ *
+ * @param <T> Type of the element to filter
+ */
+public class RichAndCondition<T> extends RichIterativeCondition<T> {
 
 Review comment:
   How about we introduce `RichCompositeIterativeCondition` that will get list 
of conditions in the ctor and will handle the `open/close/setRuntimeContext` 
methods in one place?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> Pattern(Flat)SelectFunctions should support RichFunction interface
> ------------------------------------------------------------------
>
>                 Key: FLINK-8159
>                 URL: https://issues.apache.org/jira/browse/FLINK-8159
>             Project: Flink
>          Issue Type: Sub-task
>          Components: CEP
>            Reporter: Dian Fu
>            Assignee: Dian Fu
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.8.0
>
>
> {{SelectWrapper}} and {{FlatSelectWrapper}} should extends 
> {{AbstractRichFucntion}} and process properly if the underlying functions 
> extend RichFunction.
> Things to be very careful about:
> * backwards compatibility (we previously serialized conditions) - changes to 
> those interfaces have to be done carefully
> * we want to be able to add dynamic patterns in the future, so at some point 
> we have to open also on control message arrival



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to