You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by dawidwys <gi...@git.apache.org> on 2017/08/25 09:54:25 UTC

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

GitHub user dawidwys opened a pull request:

    https://github.com/apache/flink/pull/4587

    [FLINK-7511] [cep] Remove dead code after dropping backward compatibi…

    …lity with <=1.2
    
    
    *Thank you very much for contributing to Apache Flink - we are happy that you want to help us improve Flink. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
    
    *Please understand that we do not do this to make contributions to Flink a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
    
    ## Contribution Checklist
    
      - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
      
      - Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where *FLINK-XXXX* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component.
      Typo fixes that have no associated JIRA issue should be named following this pattern: `[hotfix] [docs] Fix typo in event time introduction` or `[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
    
      - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
      
      - Make sure that the change passes the automated tests, i.e., `mvn clean verify` passes. You can set up Travis CI to do that following [this guide](http://flink.apache.org/contribute-code.html#best-practices).
    
      - Each pull request should address only one issue, not mix up code from multiple issues.
      
      - Each commit in the pull request has a meaningful commit message (including the JIRA id)
    
      - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
    
    
    **(The sections below can be removed for hotfixes of typos)**
    
    ## What is the purpose of the change
    
    *(For example: This pull request makes task deployment go through the blob server, rather than through RPC. That way we avoid re-transferring them on each deployment (during recovery).)*
    
    
    ## Brief change log
    
    *(for example:)*
      - *The TaskInfo is stored in the blob store on job creation time as a persistent artifact*
      - *Deployments RPC transmits only the blob storage reference*
      - *TaskManagers retrieve the TaskInfo from the blob cache*
    
    
    ## Verifying this change
    
    *(Please pick either of the following options)*
    
    This change is a trivial rework / code cleanup without any test coverage.
    
    *(or)*
    
    This change is already covered by existing tests, such as *(please describe tests)*.
    
    *(or)*
    
    This change added tests and can be verified as follows:
    
    *(example:)*
      - *Added integration tests for end-to-end deployment with large payloads (100MB)*
      - *Extended integration test for recovery after master (JobManager) failure*
      - *Added test that validates that TaskInfo is transferred only once across recoveries*
      - *Manually verified the change by running a 4 node cluser with 2 JobManagers and 4 TaskManagers, a stateful streaming program, and killing one JobManager and two TaskManagers during the execution, verifying that recovery happens correctly.*
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (yes / no)
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / no)
      - The serializers: (yes / no / don't know)
      - The runtime per-record code paths (performance sensitive): (yes / no / don't know)
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes / no / don't know)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (yes / no)
      - If yes, how is the feature documented? (not applicable / docs / JavaDocs / not documented)
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/dawidwys/flink cep-backward-drop

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/4587.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4587
    
----
commit 417b4a9ba474076b3252d5d16f88847967e377fe
Author: Dawid Wysakowicz <dw...@apache.org>
Date:   2017-08-25T09:53:27Z

    [FLINK-7511] [cep] Remove dead code after dropping backward compatibility with <=1.2

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r160700659
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java ---
    @@ -150,16 +126,13 @@
     	 */
     	private boolean nfaChanged;
     
    -	public NFA(
    -			final TypeSerializer<T> eventSerializer,
    +	public NFA(final TypeSerializer<T> eventSerializer,
     			final long windowTime,
     			final boolean handleTimeout) {
    -
     		this.eventSerializer = eventSerializer;
    --- End diff --
    
    If you check all of the `NFA`'s `eventSerializer` usages, there are all in the `extractCurrentMatches` method, which before your change was guarding against `(eventSerializer == null)` condition. Thus (maybe unintentionally) this class's `evenSerializer` field could be `@Nullable` or was supporting `new NFA(null, 1, false)` calls. From what I have checked (by following up all of the constructor invocations), there seems to be no code path for such construction to happen at the moment, but adding `checkNotNull(eventSerializer)` will guard against such mistakes in the future. 


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by dawidwys <gi...@git.apache.org>.
Github user dawidwys commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r162051801
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java ---
    @@ -1,55 +0,0 @@
    -/*
    - * 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;
    -
    -import org.apache.flink.api.common.functions.FilterFunction;
    -
    -/**
    - * A filter function which combines two filter functions with a logical and. Thus, the filter
    - * function only returns true, iff both filters return true.
    - *
    - * @param <T> Type of the element to filter
    - * @deprecated This is only used when migrating from an older Flink version.
    - * Use the {@link org.apache.flink.cep.pattern.conditions.AndCondition} instead.
    - */
    -@Deprecated
    -public class AndFilterFunction<T> implements FilterFunction<T> {
    --- End diff --
    
    Sorry for late response. I've added test that says explicitly it tests agains Subtype/And/Or conditions.
    
    I have not constructed this objects explicitly, as I think they are not intended to use them like this, but rather use chain of `where` for AND, `subtype` for SubtypeCondition and `or` for OrCondition.


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by dawidwys <gi...@git.apache.org>.
Github user dawidwys commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r162044383
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java ---
    @@ -1,55 +0,0 @@
    -/*
    - * 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;
    -
    -import org.apache.flink.api.common.functions.FilterFunction;
    -
    -/**
    - * A filter function which combines two filter functions with a logical and. Thus, the filter
    - * function only returns true, iff both filters return true.
    - *
    - * @param <T> Type of the element to filter
    - * @deprecated This is only used when migrating from an older Flink version.
    - * Use the {@link org.apache.flink.cep.pattern.conditions.AndCondition} instead.
    - */
    -@Deprecated
    -public class AndFilterFunction<T> implements FilterFunction<T> {
    --- End diff --
    
    Sorry for the late response. 
    
    Actually not sure about the value of testing those conditions explicitly. In fact it would just test if they are serializable. Anyway they are not intended for explicit usage (they could be annotated with `@Internal`, but it should be done as a separate task IMHO). Plus subtype is covered in migration tests (the `subtype` function on pattern).


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r160397308
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java ---
    @@ -1,55 +0,0 @@
    -/*
    - * 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;
    -
    -import org.apache.flink.api.common.functions.FilterFunction;
    -
    -/**
    - * A filter function which combines two filter functions with a logical and. Thus, the filter
    - * function only returns true, iff both filters return true.
    - *
    - * @param <T> Type of the element to filter
    - * @deprecated This is only used when migrating from an older Flink version.
    - * Use the {@link org.apache.flink.cep.pattern.conditions.AndCondition} instead.
    - */
    -@Deprecated
    -public class AndFilterFunction<T> implements FilterFunction<T> {
    --- End diff --
    
    Sorry
    > I'm kind of worried 
    
    I meant: "it seems to me like this change should be ok, the only thing that I'm not 100% sure". In other words I don't expect to have find any issues with this code basing on my understanding of it.
    
    The case and the test that I had in mind should make sure, that checkpoints created/taken in 1.3.x do not use those `*FilterFunction` classes. Correct me if I'm wrong, but I have an impression if 1.3.x somehow created a checkpoint using `*FilterFunction`(unlikely? impossible?), before your change such checkpoint would still correctly load. That's why I'm missing those `and`/`or` conditions in `CEPMigrationTest` from `1.3.x` -> `1.4+`. Also such test might be useful for the future.
    
    Maybe with some extra knowledge about the previous changes this code has undergone it's obvious that this code just works, but I'm missing this context (I'm new to the CEP stuff :( )
      


---

[GitHub] flink issue #4587: [FLINK-7511] [cep] Remove dead code after dropping backwa...

Posted by dawidwys <gi...@git.apache.org>.
Github user dawidwys commented on the issue:

    https://github.com/apache/flink/pull/4587
  
    Thanks for review @pnowojski. Merging.


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by dawidwys <gi...@git.apache.org>.
Github user dawidwys commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r161163221
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java ---
    @@ -1,55 +0,0 @@
    -/*
    - * 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;
    -
    -import org.apache.flink.api.common.functions.FilterFunction;
    -
    -/**
    - * A filter function which combines two filter functions with a logical and. Thus, the filter
    - * function only returns true, iff both filters return true.
    - *
    - * @param <T> Type of the element to filter
    - * @deprecated This is only used when migrating from an older Flink version.
    - * Use the {@link org.apache.flink.cep.pattern.conditions.AndCondition} instead.
    - */
    -@Deprecated
    -public class AndFilterFunction<T> implements FilterFunction<T> {
    --- End diff --
    
    I am for dropping support for 1.2. The only problem is regarding the 3rd point in @kl0u list. It is not possible to migrate 1.2 -> 1.4. Neither directly nor through 1.3.2. (that's what I was trying to say in my previous comment).
    
    I would be for dropping support for <= 1.2 completely, if it is ok with you


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r160349688
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java ---
    @@ -150,16 +126,13 @@
     	 */
     	private boolean nfaChanged;
     
    -	public NFA(
    -			final TypeSerializer<T> eventSerializer,
    +	public NFA(final TypeSerializer<T> eventSerializer,
     			final long windowTime,
     			final boolean handleTimeout) {
    -
     		this.eventSerializer = eventSerializer;
    --- End diff --
    
    shouldn't the `eventSerializer` be checked for not null here after this change?
      


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by dawidwys <gi...@git.apache.org>.
Github user dawidwys commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r160524740
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java ---
    @@ -150,16 +126,13 @@
     	 */
     	private boolean nfaChanged;
     
    -	public NFA(
    -			final TypeSerializer<T> eventSerializer,
    +	public NFA(final TypeSerializer<T> eventSerializer,
     			final long windowTime,
     			final boolean handleTimeout) {
    -
     		this.eventSerializer = eventSerializer;
    --- End diff --
    
    The difference in handling `null` seems strange to me. Could you tell me how did you check it?
    
    Anyway as I already said will add the check.


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by dawidwys <gi...@git.apache.org>.
Github user dawidwys commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r160885887
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java ---
    @@ -1,55 +0,0 @@
    -/*
    - * 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;
    -
    -import org.apache.flink.api.common.functions.FilterFunction;
    -
    -/**
    - * A filter function which combines two filter functions with a logical and. Thus, the filter
    - * function only returns true, iff both filters return true.
    - *
    - * @param <T> Type of the element to filter
    - * @deprecated This is only used when migrating from an older Flink version.
    - * Use the {@link org.apache.flink.cep.pattern.conditions.AndCondition} instead.
    - */
    -@Deprecated
    -public class AndFilterFunction<T> implements FilterFunction<T> {
    --- End diff --
    
    Unfortunately bad news. As I implemented a path of migration 1.2 -> 1.3 -> 1.4. I found out we broke it with 1.4 version (I mean it is impossible to migrate job that was first running with 1.2).
    
    It is due to the removal of NFA#Serializer class, which was used to serialize state in 1.2. We do not longer read or write anything there in 1.3+, but unfortunately the serializer is needed during the initialization. Will open a separate JIRA for that.
    
    As for this PR, I think if we want to restore the migration path we will need to drop those changes.


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by dawidwys <gi...@git.apache.org>.
Github user dawidwys commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r160523676
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java ---
    @@ -1,55 +0,0 @@
    -/*
    - * 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;
    -
    -import org.apache.flink.api.common.functions.FilterFunction;
    -
    -/**
    - * A filter function which combines two filter functions with a logical and. Thus, the filter
    - * function only returns true, iff both filters return true.
    - *
    - * @param <T> Type of the element to filter
    - * @deprecated This is only used when migrating from an older Flink version.
    - * Use the {@link org.apache.flink.cep.pattern.conditions.AndCondition} instead.
    - */
    -@Deprecated
    -public class AndFilterFunction<T> implements FilterFunction<T> {
    --- End diff --
    
    I am afraid your worries are justified :( 
    
    I analyzed the code once again and indeed it is possible to have a checkpoint taken in 1.3.x that has serialized  `*FilterFunction` classes through the `FilterWrapper` class. It is possible when the job was previously restored from 1.2.x checkpoint. Unfortunately I will need to restore those classes. I will do that tomorrow and I will also add test for that case.
    
    Thanks for catching that!


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r160895447
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java ---
    @@ -1,55 +0,0 @@
    -/*
    - * 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;
    -
    -import org.apache.flink.api.common.functions.FilterFunction;
    -
    -/**
    - * A filter function which combines two filter functions with a logical and. Thus, the filter
    - * function only returns true, iff both filters return true.
    - *
    - * @param <T> Type of the element to filter
    - * @deprecated This is only used when migrating from an older Flink version.
    - * Use the {@link org.apache.flink.cep.pattern.conditions.AndCondition} instead.
    - */
    -@Deprecated
    -public class AndFilterFunction<T> implements FilterFunction<T> {
    --- End diff --
    
    I am not sure if we should bring back these classes. 
    I would recommend to simply throw an exception saying that 1.2 version is no longer supported. The reason for this is:
    1) not many people seem to be using CEP in 1.2
    2) the code is already a bit complicated with all the migration magic.
    
    What do you think about that as a solution @dawidwys and @pnowojski ?


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r160363378
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java ---
    @@ -1,55 +0,0 @@
    -/*
    - * 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;
    -
    -import org.apache.flink.api.common.functions.FilterFunction;
    -
    -/**
    - * A filter function which combines two filter functions with a logical and. Thus, the filter
    - * function only returns true, iff both filters return true.
    - *
    - * @param <T> Type of the element to filter
    - * @deprecated This is only used when migrating from an older Flink version.
    - * Use the {@link org.apache.flink.cep.pattern.conditions.AndCondition} instead.
    - */
    -@Deprecated
    -public class AndFilterFunction<T> implements FilterFunction<T> {
    --- End diff --
    
    I'm kind of worried that migration test doesn't cover for and/or/subtype filters/conditions. Could you add a test which proves that migration from 1.3 with such expressions is still working?


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r161497658
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java ---
    @@ -1,55 +0,0 @@
    -/*
    - * 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;
    -
    -import org.apache.flink.api.common.functions.FilterFunction;
    -
    -/**
    - * A filter function which combines two filter functions with a logical and. Thus, the filter
    - * function only returns true, iff both filters return true.
    - *
    - * @param <T> Type of the element to filter
    - * @deprecated This is only used when migrating from an older Flink version.
    - * Use the {@link org.apache.flink.cep.pattern.conditions.AndCondition} instead.
    - */
    -@Deprecated
    -public class AndFilterFunction<T> implements FilterFunction<T> {
    --- End diff --
    
    Wouldn't it be better to have those conditions tested explicitly? And what about or/subtype conditions? Are they covered as well?


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r161179351
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java ---
    @@ -1,55 +0,0 @@
    -/*
    - * 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;
    -
    -import org.apache.flink.api.common.functions.FilterFunction;
    -
    -/**
    - * A filter function which combines two filter functions with a logical and. Thus, the filter
    - * function only returns true, iff both filters return true.
    - *
    - * @param <T> Type of the element to filter
    - * @deprecated This is only used when migrating from an older Flink version.
    - * Use the {@link org.apache.flink.cep.pattern.conditions.AndCondition} instead.
    - */
    -@Deprecated
    -public class AndFilterFunction<T> implements FilterFunction<T> {
    --- End diff --
    
    @dawidwys Let's drop support for <= 1.2. This will simplify the codebase and if someone complains, we can always re-introduce the missing classes.


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/4587


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by dawidwys <gi...@git.apache.org>.
Github user dawidwys commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r161483614
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java ---
    @@ -1,55 +0,0 @@
    -/*
    - * 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;
    -
    -import org.apache.flink.api.common.functions.FilterFunction;
    -
    -/**
    - * A filter function which combines two filter functions with a logical and. Thus, the filter
    - * function only returns true, iff both filters return true.
    - *
    - * @param <T> Type of the element to filter
    - * @deprecated This is only used when migrating from an older Flink version.
    - * Use the {@link org.apache.flink.cep.pattern.conditions.AndCondition} instead.
    - */
    -@Deprecated
    -public class AndFilterFunction<T> implements FilterFunction<T> {
    --- End diff --
    
    @kl0u I've reverted to the previous state (with removed *FilterFunctions).
    
    @pnowojski Tests that checked migration from 1.3 with those conditions where there (whenever we use both `subtype` + `where` it is converted into `And`), but I added savepoints for versions 1.4 and 1.5.


---

[GitHub] flink issue #4587: [FLINK-7511] [cep] Remove dead code after dropping backwa...

Posted by dawidwys <gi...@git.apache.org>.
Github user dawidwys commented on the issue:

    https://github.com/apache/flink/pull/4587
  
    @pnowojski I've updated the PR, will be grateful for your review, thx!


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r160908934
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java ---
    @@ -1,55 +0,0 @@
    -/*
    - * 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;
    -
    -import org.apache.flink.api.common.functions.FilterFunction;
    -
    -/**
    - * A filter function which combines two filter functions with a logical and. Thus, the filter
    - * function only returns true, iff both filters return true.
    - *
    - * @param <T> Type of the element to filter
    - * @deprecated This is only used when migrating from an older Flink version.
    - * Use the {@link org.apache.flink.cep.pattern.conditions.AndCondition} instead.
    - */
    -@Deprecated
    -public class AndFilterFunction<T> implements FilterFunction<T> {
    --- End diff --
    
    If you @kl0u think that's the best solution I will agree :)
    
    The only thing on which I would insist, is to extend `CEPMigrationTest` from 1.3 to 1.4/1.5 so it would cover the `and/or/subtype conditions` - that way it will be obvious from the automation tests that there is a backward compatibility with 1.3.


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by dawidwys <gi...@git.apache.org>.
Github user dawidwys commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r160372500
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java ---
    @@ -150,16 +126,13 @@
     	 */
     	private boolean nfaChanged;
     
    -	public NFA(
    -			final TypeSerializer<T> eventSerializer,
    +	public NFA(final TypeSerializer<T> eventSerializer,
     			final long windowTime,
     			final boolean handleTimeout) {
    -
     		this.eventSerializer = eventSerializer;
    --- End diff --
    
    this PR does not change anything in this manner (previously it also should not be null), but I agree we can add this check


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r162560468
  
    --- Diff: flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPMigrationTest.java ---
    @@ -535,6 +629,34 @@ private SinglePatternNFAFactory(boolean handleTimeout) {
     		}
     	}
     
    +	private static class NFAComplexConfitionsFactory implements NFACompiler.NFAFactory<Event> {
    --- End diff --
    
    `Confitions` typo?


---

[GitHub] flink issue #4587: [FLINK-7511] [cep] Remove dead code after dropping backwa...

Posted by dawidwys <gi...@git.apache.org>.
Github user dawidwys commented on the issue:

    https://github.com/apache/flink/pull/4587
  
    Sure, will rebase in the evening, thx!


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by pnowojski <gi...@git.apache.org>.
Github user pnowojski commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r160395375
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java ---
    @@ -150,16 +126,13 @@
     	 */
     	private boolean nfaChanged;
     
    -	public NFA(
    -			final TypeSerializer<T> eventSerializer,
    +	public NFA(final TypeSerializer<T> eventSerializer,
     			final long windowTime,
     			final boolean handleTimeout) {
    -
     		this.eventSerializer = eventSerializer;
    --- End diff --
    
    I checked the `NFA` code and currently in master it seems like it could handle `null` on this parameter while after this change that's no longer the case? But regardless as you said `checkNotNull` is a good thing anyway.


---

[GitHub] flink pull request #4587: [FLINK-7511] [cep] Remove dead code after dropping...

Posted by dawidwys <gi...@git.apache.org>.
Github user dawidwys commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4587#discussion_r160375193
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/AndFilterFunction.java ---
    @@ -1,55 +0,0 @@
    -/*
    - * 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;
    -
    -import org.apache.flink.api.common.functions.FilterFunction;
    -
    -/**
    - * A filter function which combines two filter functions with a logical and. Thus, the filter
    - * function only returns true, iff both filters return true.
    - *
    - * @param <T> Type of the element to filter
    - * @deprecated This is only used when migrating from an older Flink version.
    - * Use the {@link org.apache.flink.cep.pattern.conditions.AndCondition} instead.
    - */
    -@Deprecated
    -public class AndFilterFunction<T> implements FilterFunction<T> {
    --- End diff --
    
    But it was not possible to have those `*FilterFunction` in code in 1.3. There is no API that uses them. The only reason for those classes was migrating from 1.2 and as we drop that feature in this PR those classes are useless. After deserialization those functions were migrated to `*Condition`.
    
    If I wanted to test the migration that uses the `*FilterFunction` I would need to test migration from 1.2, which was dropped.
    
    Or did I misunderstand you?


---