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

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

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

    https://github.com/apache/flink/pull/6062#discussion_r190481987
  
    --- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java
 ---
    @@ -0,0 +1,235 @@
    +/*
    + * 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.streaming.api.operators;
    +
    +import org.apache.flink.annotation.Internal;
    +import org.apache.flink.api.common.typeutils.CompatibilityResult;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +import java.io.IOException;
    +
    +/**
    + * Implementation of {@link InternalTimer} for the {@link 
InternalTimerHeap}.
    + *
    + * @param <K> Type of the keys to which timers are scoped.
    + * @param <N> Type of the namespace to which timers are scoped.
    + */
    +@Internal
    +public final class TimerHeapInternalTimer<K, N> implements 
InternalTimer<K, N> {
    +
    +   /** The index that indicates that a tracked internal timer is not 
tracked. */
    +   private static final int NOT_MANAGED_BY_TIMER_QUEUE_INDEX = 
Integer.MIN_VALUE;
    +
    +   private final long timestamp;
    +
    +   private final K key;
    +
    +   private final N namespace;
    +
    +   /**
    +    * This field holds the current physical index if this timer when it is 
managed by a timer heap so that we can
    +    * support fast deletes.
    +    */
    +   private transient int timerHeapIndex;
    +
    +   TimerHeapInternalTimer(long timestamp, K key, N namespace) {
    +           this.timestamp = timestamp;
    +           this.key = key;
    +           this.namespace = namespace;
    +           this.timerHeapIndex = NOT_MANAGED_BY_TIMER_QUEUE_INDEX;
    +   }
    +
    +   @Override
    +   public long getTimestamp() {
    +           return timestamp;
    +   }
    +
    +   @Override
    +   public K getKey() {
    +           return key;
    +   }
    +
    +   @Override
    +   public N getNamespace() {
    +           return namespace;
    +   }
    +
    +   @Override
    +   public boolean equals(Object o) {
    +           if (this == o) {
    +                   return true;
    +           }
    +
    +           if (o instanceof InternalTimer) {
    +                   InternalTimer<?, ?> timer = (InternalTimer<?, ?>) o;
    +                   return timestamp == timer.getTimestamp()
    +                           && key.equals(timer.getKey())
    +                           && namespace.equals(timer.getNamespace());
    +           }
    +
    +           return false;
    +   }
    +
    +   /**
    +    * Returns the current index of this timer in the owning timer heap.
    +    */
    +   int getTimerHeapIndex() {
    +           return timerHeapIndex;
    +   }
    +
    +   /**
    +    * Sets the current index of this timer in the owning timer heap and 
should only be called by the managing heap.
    +    * @param timerHeapIndex the new index in the timer heap.
    +    */
    +   void setTimerHeapIndex(int timerHeapIndex) {
    +           this.timerHeapIndex = timerHeapIndex;
    +   }
    +
    +   /**
    +    * This method can be called to indicate that the timer is no longer 
managed be a timer heap, e.g. because it as
    +    * removed.
    +    */
    +   void removedFromTimerQueue() {
    +           setTimerHeapIndex(NOT_MANAGED_BY_TIMER_QUEUE_INDEX);
    +   }
    +
    +   @Override
    +   public int hashCode() {
    +           int result = (int) (timestamp ^ (timestamp >>> 32));
    +           result = 31 * result + key.hashCode();
    +           result = 31 * result + namespace.hashCode();
    +           return result;
    +   }
    +
    +   @Override
    +   public String toString() {
    +           return "Timer{" +
    +                           "timestamp=" + timestamp +
    +                           ", key=" + key +
    +                           ", namespace=" + namespace +
    +                           '}';
    +   }
    +
    +   /**
    +    * A {@link TypeSerializer} used to serialize/deserialize a {@link 
TimerHeapInternalTimer}.
    +    */
    +   public static class TimerSerializer<K, N> extends 
TypeSerializer<InternalTimer<K, N>> {
    +
    +           private static final long serialVersionUID = 
1119562170939152304L;
    +
    +           private final TypeSerializer<K> keySerializer;
    +
    +           private final TypeSerializer<N> namespaceSerializer;
    +
    +           TimerSerializer(TypeSerializer<K> keySerializer, 
TypeSerializer<N> namespaceSerializer) {
    +                   this.keySerializer = keySerializer;
    +                   this.namespaceSerializer = namespaceSerializer;
    +           }
    +
    +           @Override
    +           public boolean isImmutableType() {
    +                   return false;
    +           }
    +
    +           @Override
    +           public TypeSerializer<InternalTimer<K, N>> duplicate() {
    +                   return this;
    --- End diff --
    
    That is true, but this was already the case before this PR and it was never 
an issue because the timer service snapshots are not yet asynchronous. 
Nevertheless, it should be improved 👍 


> Implement efficient deletes for heap based timer service
> --------------------------------------------------------
>
>                 Key: FLINK-9423
>                 URL: https://issues.apache.org/jira/browse/FLINK-9423
>             Project: Flink
>          Issue Type: Improvement
>          Components: Streaming
>    Affects Versions: 1.6.0
>            Reporter: Stefan Richter
>            Assignee: Stefan Richter
>            Priority: Major
>
> The current data structures in the `HeapInternalTimerService` are not able to 
> support efficient timer deletes, the complexity is currently O\(n\), where n 
> is the number of registered timers.
>  
> We can keep track of timer's positions in the priority queue and (in 
> combination with the already existing set/map) have a more efficient 
> algorithm for deletes.



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

Reply via email to