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

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

dawidwys commented on a change in pull request #6205: [FLINK-9642]Reduce the 
count to deal with state during a CEP process
URL: https://github.com/apache/flink/pull/6205#discussion_r209233907
 
 

 ##########
 File path: 
flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/SharedBufferAccessor.java
 ##########
 @@ -0,0 +1,362 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOVICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  Vhe 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.sharedbuffer;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.cep.nfa.DeweyNumber;
+import org.apache.flink.util.WrappingRuntimeException;
+
+import org.apache.commons.lang3.StringUtils;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Stack;
+
+import static 
org.apache.flink.cep.nfa.compiler.NFAStateNameHandler.getOriginalNameFromInternal;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A shared buffer implementation which stores values under according state. 
Additionally, the values can be
+ * versioned such that it is possible to retrieve their predecessor element in 
the buffer.
+ *
+ * <p>The idea of the implementation is to have a buffer for incoming events 
with unique ids assigned to them. This way
+ * we do not need to deserialize events during processing and we store only 
one copy of the event.
+ *
+ * <p>The entries in {@link SharedBufferAccessor} are {@link 
SharedBufferNode}. The shared buffer node allows to store
+ * relations between different entries. A dewey versioning scheme allows to 
discriminate between
+ * different relations (e.g. preceding element).
+ *
+ * <p>The implementation is strongly based on the paper "Efficient Pattern 
Matching over Event Streams".
+ *
+ * @param <V> Type of the values
+ * @see <a 
href="https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf";>
+ * https://people.cs.umass.edu/~yanlei/publications/sase-sigmod08.pdf</a>
+ */
+public class SharedBufferAccessor<V> implements AutoCloseable{
+
+       /** The cache of sharedBuffer.*/
+       private SharedBuffer<V> sharedBuffer;
+
+       public SharedBufferAccessor(SharedBuffer<V> sharedBuffer) {
+               this.sharedBuffer = sharedBuffer;
+       }
+
+       public void setSharedBuffer(SharedBuffer<V> sharedBuffer) {
+               this.sharedBuffer = sharedBuffer;
+       }
+
+       /**
+        * Notifies shared buffer that there will be no events with timestamp 
&lt;&eq; the given value. It allows to clear
+        * internal counters for number of events seen so far per timestamp.
+        *
+        * @param timestamp watermark, no earlier events will arrive
+        * @throws Exception Thrown if the system cannot access the state.
+        */
+       public void advanceTime(long timestamp) throws Exception {
+               sharedBuffer.advanceTime(timestamp);
+       }
+
+       /**
+        * Adds another unique event to the shared buffer and assigns a unique 
id for it. It automatically creates a
+        * lock on this event, so it won't be removed during processing of that 
event. Therefore the lock should be removed
+        * after processing all {@link 
org.apache.flink.cep.nfa.ComputationState}s
+        *
+        * <p><b>NOTE:</b>Should be called only once for each unique event!
+        *
+        * @param value event to be registered
+        * @return unique id of that event that should be used when putting 
entries to the buffer.
+        * @throws Exception Thrown if the system cannot access the state.
+        */
+       public EventId registerEvent(V value, long timestamp) throws Exception {
+               return sharedBuffer.registerEvent(value, timestamp);
+       }
+
+       /**
+        * Stores given value (value + timestamp) under the given state. It 
assigns a preceding element
+        * relation to the previous entry.
+        *
+        * @param stateName      name of the state that the event should be 
assigned to
+        * @param eventId        unique id of event assigned by this 
SharedBuffer
+        * @param previousNodeId id of previous entry (might be null if start 
of new run)
+        * @param version        Version of the previous relation
+        * @return assigned id of this element
+        * @throws Exception Thrown if the system cannot access the state.
+        */
+       public NodeId put(
+               final String stateName,
+               final EventId eventId,
+               @Nullable final NodeId previousNodeId,
+               final DeweyNumber version) throws Exception {
+
+               if (previousNodeId != null) {
+                       lockNode(previousNodeId);
+               }
+
+               NodeId currentNodeId = new NodeId(eventId, 
getOriginalNameFromInternal(stateName));
+               Lockable<SharedBufferNode> currentNode = 
sharedBuffer.getEntry(currentNodeId);
+               if (currentNode == null) {
+                       currentNode = new Lockable<>(new SharedBufferNode(), 0);
+                       lockEvent(eventId);
+               }
+
+               currentNode.getElement().addEdge(new SharedBufferEdge(
+                       previousNodeId,
+                       version));
+               sharedBuffer.cacheEntry(currentNodeId, currentNode);
+
+               return currentNodeId;
+       }
+
+       /**
+        * Returns all elements from the previous relation starting at the 
given entry.
+        *
+        * @param nodeId  id of the starting entry
+        * @param version Version of the previous relation which shall be 
extracted
+        * @return Collection of previous relations starting with the given 
value
+        * @throws Exception Thrown if the system cannot access the state.
+        */
+       public List<Map<String, List<EventId>>> extractPatterns(
+               final NodeId nodeId,
+               final DeweyNumber version) throws Exception {
+
+               List<Map<String, List<EventId>>> result = new ArrayList<>();
+
+               // stack to remember the current extraction states
+               Stack<SharedBufferAccessor.ExtractionState> extractionStates = 
new Stack<>();
+
+               // get the starting shared buffer entry for the previous 
relation
+               Lockable<SharedBufferNode> entryLock = 
sharedBuffer.getEntry(nodeId);
+
+               if (entryLock != null) {
+                       SharedBufferNode entry = entryLock.getElement();
+                       extractionStates.add(new 
SharedBufferAccessor.ExtractionState(Tuple2.of(nodeId, entry), version, new 
Stack<>()));
+
+                       // use a depth first search to reconstruct the previous 
relations
+                       while (!extractionStates.isEmpty()) {
+                               final SharedBufferAccessor.ExtractionState 
extractionState = extractionStates.pop();
+                               // current path of the depth first search
+                               final Stack<Tuple2<NodeId, SharedBufferNode>> 
currentPath = extractionState.getPath();
+                               final Tuple2<NodeId, SharedBufferNode> 
currentEntry = extractionState.getEntry();
+
+                               // termination criterion
+                               if (currentEntry == null) {
+                                       final Map<String, List<EventId>> 
completePath = new LinkedHashMap<>();
+
+                                       while (!currentPath.isEmpty()) {
+                                               final NodeId currentPathEntry = 
currentPath.pop().f0;
+
+                                               String page = 
currentPathEntry.getPageName();
+                                               List<EventId> values = 
completePath
+                                                       .computeIfAbsent(page, 
k -> new ArrayList<>());
+                                               
values.add(currentPathEntry.getEventId());
+                                       }
+                                       result.add(completePath);
+                               } else {
+
+                                       // append state to the path
+                                       currentPath.push(currentEntry);
+
+                                       boolean firstMatch = true;
+                                       for (SharedBufferEdge edge : 
currentEntry.f1.getEdges()) {
+                                               // we can only proceed if the 
current version is compatible to the version
+                                               // of this previous relation
+                                               final DeweyNumber 
currentVersion = extractionState.getVersion();
+                                               if 
(currentVersion.isCompatibleWith(edge.getDeweyNumber())) {
+                                                       final NodeId target = 
edge.getTarget();
+                                                       Stack<Tuple2<NodeId, 
SharedBufferNode>> newPath;
+
+                                                       if (firstMatch) {
+                                                               // for the 
first match we don't have to copy the current path
+                                                               newPath = 
currentPath;
+                                                               firstMatch = 
false;
+                                                       } else {
+                                                               newPath = new 
Stack<>();
+                                                               
newPath.addAll(currentPath);
+                                                       }
+
+                                                       
extractionStates.push(new SharedBufferAccessor.ExtractionState(
+                                                               target != null 
? Tuple2.of(target, sharedBuffer.getEntry(target).getElement()) : null,
+                                                               
edge.getDeweyNumber(),
+                                                               newPath));
+                                               }
+                                       }
+                               }
+
+                       }
+               }
+               return result;
+       }
+
+       public Map<String, List<V>> materializeMatch(Map<String, List<EventId>> 
match) {
+               return materializeMatch(match, new HashMap<>());
+       }
+
+       public Map<String, List<V>> materializeMatch(Map<String, List<EventId>> 
match, Map<EventId, V> cache) {
+
+               Map<String, List<V>> materializedMatch = new 
LinkedHashMap<>(match.size());
+
+               for (Map.Entry<String, List<EventId>> pattern : 
match.entrySet()) {
+                       List<V> events = new 
ArrayList<>(pattern.getValue().size());
+                       for (EventId eventId : pattern.getValue()) {
+                               V event = cache.computeIfAbsent(eventId, id -> {
+                                       try {
+                                               return 
sharedBuffer.getEvent(id).getElement();
+                                       } catch (Exception ex) {
+                                               throw new 
WrappingRuntimeException(ex);
+                                       }
+                               });
+                               events.add(event);
+                       }
+                       materializedMatch.put(pattern.getKey(), events);
+               }
+
+               return materializedMatch;
+       }
+
+       /**
+        * Increases the reference counter for the given entry so that it is not
+        * accidentally removed.
+        *
+        * @param node id of the entry
+        * @throws Exception Thrown if the system cannot access the state.
+        */
+       public void lockNode(final NodeId node) throws Exception {
+               Lockable<SharedBufferNode> sharedBufferNode = 
sharedBuffer.getEntry(node);
+               if (sharedBufferNode != null) {
+                       sharedBufferNode.lock();
+                       sharedBuffer.cacheEntry(node, sharedBufferNode);
+               }
+       }
+
+       /**
+        * Decreases the reference counter for the given entry so that it can be
+        * removed once the reference counter reaches 0.
+        *
+        * @param node id of the entry
+        * @throws Exception Thrown if the system cannot access the state.
+        */
+       public void releaseNode(final NodeId node) throws Exception {
+               Lockable<SharedBufferNode> sharedBufferNode = 
sharedBuffer.getEntry(node);
+               if (sharedBufferNode != null) {
+                       if (sharedBufferNode.release()) {
+                               removeNode(node, sharedBufferNode.getElement());
+                       } else {
+                               sharedBuffer.cacheEntry(node, sharedBufferNode);
+                       }
+               }
+       }
+
+       /**
+        * Remove the {@code SharedBufferNode}, when the ref is decreased to 
zero, and also
 
 Review comment:
   `Remove -> Removes`

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


> Reduce the count to deal with state during a CEP process 
> ---------------------------------------------------------
>
>                 Key: FLINK-9642
>                 URL: https://issues.apache.org/jira/browse/FLINK-9642
>             Project: Flink
>          Issue Type: Improvement
>          Components: CEP
>    Affects Versions: 1.6.0
>            Reporter: aitozi
>            Assignee: aitozi
>            Priority: Major
>              Labels: pull-request-available
>
> With the rework of sharedBuffer Flink-9418, the lock & release operation is 
> deal with rocksdb state which is different from the previous version which 
> will read the state of sharedBuffer all to memory, i think we can add a cache 
> or variable in sharedbuffer to cache the LockAble Object to mark the ref 
> change in once process in NFA, this will reduce the count when the events 
> point to the same NodeId.. And flush the result to MapState at the end of 
> process. 



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

Reply via email to