You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2019/08/06 20:02:39 UTC

[GitHub] [incubator-druid] clintropolis commented on a change in pull request #8222: Fix bugs in overshadowableManager and add unit tests

clintropolis commented on a change in pull request #8222: Fix bugs in overshadowableManager and add unit tests
URL: https://github.com/apache/incubator-druid/pull/8222#discussion_r311245679
 
 

 ##########
 File path: core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java
 ##########
 @@ -189,89 +247,362 @@ private void transitAtomicUpdateGroupState(AtomicUpdateGroup<T> atomicUpdateGrou
   }
 
   /**
-   * Find all atomicUpdateGroups of the given state overshadowed by the given rootPartitionRange and minorVersion.
+   * Find all atomicUpdateGroups of the given state overshadowed by the minorVersion in the given rootPartitionRange.
    * The atomicUpdateGroup of a higher minorVersion can have a wider RootPartitionRange.
    * To find all atomicUpdateGroups overshadowed by the given rootPartitionRange and minorVersion,
    * we first need to find the first key contained by the given rootPartitionRange.
    * Once we find such key, then we go through the entire map until we see an atomicUpdateGroup of which
    * rootRangePartition is not contained by the given rootPartitionRange.
+   *
+   * @param rangeOfAug   the partition range to search for overshadowed groups.
+   * @param minorVersion the minor version to check overshadow relation. The found groups will have lower minor versions
+   *                     than this.
+   * @param fromState    the state to search for overshadowed groups.
+   *
+   * @return a list of found atomicUpdateGroups. It could be empty if no groups are found.
+   */
+  @VisibleForTesting
+  List<AtomicUpdateGroup<T>> findOvershadowedBy(RootPartitionRange rangeOfAug, short minorVersion, State fromState)
+  {
+    final TreeMap<RootPartitionRange, Short2ObjectSortedMap<AtomicUpdateGroup<T>>> stateMap = getStateMap(fromState);
+    Entry<RootPartitionRange, Short2ObjectSortedMap<AtomicUpdateGroup<T>>> current = findLowestOverlappingEntry(
+        rangeOfAug,
+        stateMap,
+        true
+    );
+
+    if (current == null) {
+      return Collections.emptyList();
+    }
+
+    // Going through the map to find all entries of the RootPartitionRange contained by the given rangeOfAug.
+    // Note that RootPartitionRange of entries are always consecutive.
+    final List<AtomicUpdateGroup<T>> found = new ArrayList<>();
+    while (current != null && rangeOfAug.overlaps(current.getKey())) {
+      if (rangeOfAug.contains(current.getKey())) {
+        // versionToGroup is sorted by minorVersion.
+        // versionToGroup.headMap(minorVersion) below returns a map containing all entries of lower minorVersions
+        // than the given minorVersion.
+        final Short2ObjectSortedMap<AtomicUpdateGroup<T>> versionToGroup = current.getValue();
+        // Short2ObjectRBTreeMap.SubMap.short2ObjectEntrySet() implementation, especially size(), is not optimized.
+        // Note that size() is indirectly called in ArrayList.addAll() when ObjectSortedSet.toArray() is called.
+        // See AbstractObjectCollection.toArray().
+        // If you see performance degradation here, probably we need to improve the below line.
+        if (versionToGroup.firstShortKey() < minorVersion) {
+          found.addAll(versionToGroup.headMap(minorVersion).values());
+        }
+      }
+      current = stateMap.higherEntry(current.getKey());
+    }
+    return found;
+  }
+
+  private List<AtomicUpdateGroup<T>> findOvershadows(AtomicUpdateGroup<T> aug, State fromState)
+  {
+    return findOvershadows(RootPartitionRange.of(aug), aug.getMinorVersion(), fromState);
+  }
+
+  /**
+   * Find all atomicUpdateGroups which overshadow others of the given minorVersion in the given rootPartitionRange.
+   * Similar to {@link #findOvershadowedBy}.
+   *
+   * Note that one atommicUpdateGroup can overshadow multiple other groups. If you're finding overshadowing
+   * atomicUpdateGroups by calling this method in a loop, the results of this method can contain duplicate groups.
+   *
+   * @param rangeOfAug   the partition range to search for overshadowing groups.
+   * @param minorVersion the minor version to check overshadow relation. The found groups will have higher minor
+   *                     versions than this.
+   * @param fromState    the state to search for overshadowed groups.
+   *
+   * @return a list of found atomicUpdateGroups. It could be empty if no groups are found.
    */
-  private List<Short2ObjectMap.Entry<AtomicUpdateGroup<T>>> findOvershadowedBy(
+  @VisibleForTesting
+  List<AtomicUpdateGroup<T>> findOvershadows(RootPartitionRange rangeOfAug, short minorVersion, State fromState)
+  {
+    final TreeMap<RootPartitionRange, Short2ObjectSortedMap<AtomicUpdateGroup<T>>> stateMap = getStateMap(fromState);
+    Entry<RootPartitionRange, Short2ObjectSortedMap<AtomicUpdateGroup<T>>> current = findLowestOverlappingEntry(
+        rangeOfAug,
+        stateMap,
+        false
+    );
+
+    if (current == null) {
+      return Collections.emptyList();
+    }
+
+    // Going through the map to find all entries of the RootPartitionRange contains the given rangeOfAug.
+    // Note that RootPartitionRange of entries are always consecutive.
+    final List<AtomicUpdateGroup<T>> found = new ArrayList<>();
+    while (current != null && current.getKey().overlaps(rangeOfAug)) {
+      if (current.getKey().contains(rangeOfAug)) {
+        // versionToGroup is sorted by minorVersion.
+        // versionToGroup.tailMap(minorVersion) below returns a map containing all entries of equal to or higher
+        // minorVersions than the given minorVersion.
+        final Short2ObjectSortedMap<AtomicUpdateGroup<T>> versionToGroup = current.getValue();
+        // Short2ObjectRBTreeMap.SubMap.short2ObjectEntrySet() implementation, especially size(), is not optimized.
+        // Note that size() is indirectly called in ArrayList.addAll() when ObjectSortedSet.toArray() is called.
+        // See AbstractObjectCollection.toArray().
+        // If you see performance degradation here, probably we need to improve the below line.
+        if (versionToGroup.lastShortKey() > minorVersion) {
+          found.addAll(versionToGroup.tailMap(minorVersion).values());
+        }
+      }
+      current = stateMap.higherEntry(current.getKey());
+    }
+    return found;
+  }
+
+  private Entry<RootPartitionRange, Short2ObjectSortedMap<AtomicUpdateGroup<T>>> findLowestOverlappingEntry(
       RootPartitionRange rangeOfAug,
-      short minorVersion,
-      State fromState
+      TreeMap<RootPartitionRange, Short2ObjectSortedMap<AtomicUpdateGroup<T>>> stateMap,
+      boolean strictSameStartId
   )
   {
-    final TreeMap<RootPartitionRange, Short2ObjectSortedMap<AtomicUpdateGroup<T>>> stateMap = getStateMap(fromState);
     Entry<RootPartitionRange, Short2ObjectSortedMap<AtomicUpdateGroup<T>>> current = stateMap.floorEntry(rangeOfAug);
 
     if (current == null) {
-      return Collections.emptyList();
+      current = stateMap.ceilingEntry(rangeOfAug);
 
 Review comment:
   Could you add a comment of why `ceilingEntry`? I've been staring at this for a while and still don't understand it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org