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/09 13:02:47 UTC

[GitHub] flink pull request #4331: [FLINK-7169][CEP] Support AFTER MATCH SKIP functio...

Github user dawidwys commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4331#discussion_r132132564
  
    --- Diff: flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/AfterMatchSkipStrategy.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.nfa;
    +
    +import org.apache.flink.api.common.typeutils.CompatibilityResult;
    +import org.apache.flink.api.common.typeutils.CompatibilityUtil;
    +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
    +import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
    +import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataInputViewStreamWrapper;
    +import org.apache.flink.core.memory.DataOutputView;
    +import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.ObjectInputStream;
    +import java.io.ObjectOutputStream;
    +import java.io.Serializable;
    +import java.util.List;
    +
    +
    +/**
    + * Indicate the skip strategy after a match process.
    + * <p>There're four kinds of strategies:
    + * SKIP_PAST_LAST_EVENT,
    + * SKIP_TO_NEXT_EVENT,
    + * SKIP_TO_FIRST_<code>PATTERN</code> and
    + * SKIP_TO_LAST_<code>PATTERN</code>
    + * </p>
    + */
    +public class AfterMatchSkipStrategy implements Serializable {
    +
    +	// default strategy
    +	SkipStrategy strategy = SkipStrategy.SKIP_TO_NEXT_EVENT;
    +
    +	// fields
    +	String patternName = null;
    +
    +	public AfterMatchSkipStrategy(){
    +		this(SkipStrategy.SKIP_TO_NEXT_EVENT, null);
    +	}
    +
    +	public AfterMatchSkipStrategy(SkipStrategy strategy) {
    +		this(strategy, null);
    +	}
    +
    +	public AfterMatchSkipStrategy(SkipStrategy strategy, String patternName) {
    --- End diff --
    
    I would change ctor's into factory methods. I think that way the requirements for `patternName` will be more visible and the the code would be less bloated.
    
    I have sth like this in mind:
    
    	public static AfterMatchSkipStrategy skipToFirst(String patternName) {
    		return new AfterMatchSkipStrategy(SkipStrategy.SKIP_TO_FIRST, patternName);
    	}
    
    	public static AfterMatchSkipStrategy skipToLast(String patternName) {
    		return new AfterMatchSkipStrategy(SkipStrategy.SKIP_TO_LAST, patternName);
    	}
    
    	public static AfterMatchSkipStrategy skipPastLastEvent() {
    		return new AfterMatchSkipStrategy(SkipStrategy.SKIP_PAST_LAST_EVENT);
    	}
    
    	public static AfterMatchSkipStrategy skipToNextEvent() {
    		return new AfterMatchSkipStrategy(SkipStrategy.SKIP_TO_NEXT_EVENT);
    	}
    
    Then the usage will be:
    
        AfterMatchSkipStrategy.skipToLast("end")
    
    instead of:
    
        new AfterMatchSkipStrategy(AfterMatchSkipStrategy.SkipStrategy.SKIP_TO_LAST, "end")



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