private static DimFilter doSimplify()

in sql/src/main/java/org/apache/druid/sql/calcite/filtration/CombineAndSimplifyBounds.java [123:344]


  private static DimFilter doSimplify(final List<DimFilter> children, boolean disjunction)
  {
    // Copy the list of child filters. We'll modify the copy and eventually return it.
    // Filters we want to add and remove from "children".
    final List<DimFilter> childrenToAdd = new ArrayList<>();
    final IntOpenHashSet childrenToRemove = new IntOpenHashSet();

    // Group Bound filters by dimension, extractionFn, and comparator and compute a RangeSet for each one.
    // Each filter is paired with its position in the "children" array.
    final Map<BoundRefKey, List<ObjectIntPair<BoundDimFilter>>> bounds = new HashMap<>();
    // Group range filters by dimension, extractionFn, and matchValueType and compute a RangeSet for each one.
    // Each filter is paired with its position in the "children" array.
    final Map<RangeRefKey, List<ObjectIntPair<RangeFilter>>> ranges = new HashMap<>();
    final Map<String, ColumnType> leastRestrictiveNumericTypes = new HashMap<>();

    // all and/or filters have at least 1 child
    boolean allFalse = true;
    for (int childIndex = 0; childIndex < children.size(); childIndex++) {
      final DimFilter child = children.get(childIndex);
      if (child instanceof BoundDimFilter) {
        final BoundDimFilter bound = (BoundDimFilter) child;
        final BoundRefKey boundRefKey = BoundRefKey.from(bound);
        final List<ObjectIntPair<BoundDimFilter>> filterList =
            bounds.computeIfAbsent(boundRefKey, k -> new ArrayList<>());
        filterList.add(ObjectIntPair.of(bound, childIndex));
        allFalse = false;
      } else if (child instanceof RangeFilter) {
        final RangeFilter range = (RangeFilter) child;
        final RangeRefKey rangeRefKey = RangeRefKey.from(range);
        if (rangeRefKey.getMatchValueType().isNumeric()) {
          leastRestrictiveNumericTypes.compute(
              range.getColumn(),
              (c, existingType) -> ColumnType.leastRestrictiveType(existingType, range.getMatchValueType())
          );
        }

        final List<ObjectIntPair<RangeFilter>> filterList =
            ranges.computeIfAbsent(rangeRefKey, k -> new ArrayList<>());
        filterList.add(ObjectIntPair.of(range, childIndex));
        allFalse = false;
      } else {
        allFalse = allFalse && (child instanceof FalseDimFilter);
      }
    }

    // short circuit if can never be true
    if (allFalse) {
      return Filtration.matchNothing();
    }

    // Try to simplify "bound" filters within each group of "bounds".
    for (Map.Entry<BoundRefKey, List<ObjectIntPair<BoundDimFilter>>> entry : bounds.entrySet()) {
      final BoundRefKey boundRefKey = entry.getKey();
      final List<ObjectIntPair<BoundDimFilter>> filterList = entry.getValue();

      // Create a RangeSet for this group.
      final RangeSet<BoundValue> rangeSet =
          disjunction
          ? RangeSets.unionRanges(Bounds.toRanges(Lists.transform(filterList, Pair::left)))
          : RangeSets.intersectRanges(Bounds.toRanges(Lists.transform(filterList, Pair::left)));

      if (rangeSet.asRanges().size() < filterList.size()) {
        // We found a simplification. Remove the old filters and add new ones.
        for (final ObjectIntPair<BoundDimFilter> boundAndChildIndex : filterList) {
          childrenToRemove.add(boundAndChildIndex.rightInt());
        }

        if (rangeSet.asRanges().isEmpty()) {
          // range set matches nothing, equivalent to FALSE
          childrenToAdd.add(Filtration.matchNothing());
        }

        for (final Range<BoundValue> range : rangeSet.asRanges()) {
          if (!range.hasLowerBound() && !range.hasUpperBound()) {
            // range matches all, equivalent to TRUE
            childrenToAdd.add(Filtration.matchEverything());
          } else {
            childrenToAdd.add(Bounds.toFilter(boundRefKey, range));
          }
        }
      } else if (disjunction && Range.all().equals(rangeSet.span())) {
        // ranges in disjunction - spanning ALL
        // complementer must be a negated set of ranges
        for (final ObjectIntPair<BoundDimFilter> boundAndChildIndex : filterList) {
          childrenToRemove.add(boundAndChildIndex.rightInt());
        }
        Set<Range<BoundValue>> newRanges = rangeSet.complement().asRanges();
        List<DimFilter> newFilters = new ArrayList<>();
        for (Range<BoundValue> range : newRanges) {
          BoundDimFilter filter = Bounds.toFilter(boundRefKey, range);
          newFilters.add(filter);
        }
        childrenToAdd.add(new NotDimFilter(disjunction(newFilters)));
      }
    }

    // Consolidate groups of numeric ranges in "ranges", using the leastRestrictiveNumericTypes computed earlier.
    final Map<RangeRefKey, List<ObjectIntPair<RangeFilter>>> consolidatedRanges =
        Maps.newHashMapWithExpectedSize(ranges.size());
    for (Map.Entry<RangeRefKey, List<ObjectIntPair<RangeFilter>>> entry : ranges.entrySet()) {
      boolean refKeyChanged = false;
      RangeRefKey refKey = entry.getKey();
      if (entry.getKey().getMatchValueType().isNumeric()) {
        ColumnType numericTypeToUse = leastRestrictiveNumericTypes.get(refKey.getColumn());
        if (!numericTypeToUse.equals(refKey.getMatchValueType())) {
          refKeyChanged = true;
          refKey = new RangeRefKey(refKey.getColumn(), numericTypeToUse);
        }
      }
      final List<ObjectIntPair<RangeFilter>> consolidatedFilterList =
          consolidatedRanges.computeIfAbsent(refKey, k -> new ArrayList<>());

      if (refKeyChanged) {
        for (ObjectIntPair<RangeFilter> filterAndChildIndex : entry.getValue()) {
          final RangeFilter rewrite =
              Ranges.toFilter(refKey, Ranges.toRange(filterAndChildIndex.left(), refKey.getMatchValueType()));
          consolidatedFilterList.add(ObjectIntPair.of(rewrite, filterAndChildIndex.rightInt()));
        }
      } else {
        consolidatedFilterList.addAll(entry.getValue());
      }
    }

    // Try to simplify "range" filters within each group of "consolidatedRanges" (derived from "ranges").
    for (Map.Entry<RangeRefKey, List<ObjectIntPair<RangeFilter>>> entry : consolidatedRanges.entrySet()) {
      final RangeRefKey rangeRefKey = entry.getKey();
      final List<ObjectIntPair<RangeFilter>> filterList = entry.getValue();

      // Create a RangeSet for this group.
      final RangeSet<RangeValue> rangeSet =
          disjunction
          ? RangeSets.unionRanges(Ranges.toRanges(Lists.transform(filterList, Pair::left)))
          : RangeSets.intersectRanges(Ranges.toRanges(Lists.transform(filterList, Pair::left)));

      if (rangeSet.asRanges().size() < filterList.size()) {
        // We found a simplification. Remove the old filters and add new ones.
        for (final ObjectIntPair<RangeFilter> rangeAndChildIndex : filterList) {
          childrenToRemove.add(rangeAndChildIndex.rightInt());
        }

        if (rangeSet.asRanges().isEmpty()) {
          // range set matches nothing, equivalent to FALSE
          childrenToAdd.add(Filtration.matchNothing());
        }

        for (final Range<RangeValue> range : rangeSet.asRanges()) {
          if (!range.hasLowerBound() && !range.hasUpperBound()) {
            // range matches all, equivalent to TRUE
            childrenToAdd.add(Filtration.matchEverything());
          } else {
            childrenToAdd.add(Ranges.toFilter(rangeRefKey, range));
          }
        }
      } else if (disjunction && Range.all().equals(rangeSet.span())) {
        // ranges in disjunction - spanning ALL
        // complementer must be a negated set of ranges
        for (final ObjectIntPair<RangeFilter> boundAndChildIndex : filterList) {
          childrenToRemove.add(boundAndChildIndex.rightInt());
        }
        Set<Range<RangeValue>> newRanges = rangeSet.complement().asRanges();
        List<DimFilter> newFilters = new ArrayList<>();
        for (Range<RangeValue> range : newRanges) {
          RangeFilter filter = Ranges.toFilter(rangeRefKey, range);
          newFilters.add(filter);
        }
        childrenToAdd.add(new NotDimFilter(disjunction(newFilters)));
      }
    }

    // Create newChildren.
    final List<DimFilter> newChildren =
        new ArrayList<>(children.size() + childrenToAdd.size() - childrenToRemove.size());
    for (int i = 0; i < children.size(); i++) {
      if (!childrenToRemove.contains(i)) {
        newChildren.add(children.get(i));
      }
    }
    newChildren.addAll(childrenToAdd);

    // Finally: Go through newChildren, removing or potentially exiting early based on TRUE / FALSE marker filters.
    Preconditions.checkState(newChildren.size() > 0, "newChildren.size > 0");

    final Iterator<DimFilter> iterator = newChildren.iterator();
    while (iterator.hasNext()) {
      final DimFilter newChild = iterator.next();

      if (Filtration.matchNothing().equals(newChild)) {
        // Child matches nothing, equivalent to FALSE
        // OR with FALSE => ignore
        // AND with FALSE => always false, short circuit
        if (disjunction) {
          iterator.remove();
        } else {
          return Filtration.matchNothing();
        }
      } else if (Filtration.matchEverything().equals(newChild)) {
        // Child matches everything, equivalent to TRUE
        // OR with TRUE => always true, short circuit
        // AND with TRUE => ignore
        if (disjunction) {
          return Filtration.matchEverything();
        } else {
          iterator.remove();
        }
      }
    }

    if (newChildren.isEmpty()) {
      // If "newChildren" is empty at this point, it must have consisted entirely of TRUE / FALSE marker filters.
      if (disjunction) {
        // Must have been all FALSE filters (the only kind we would have removed above).
        return Filtration.matchNothing();
      } else {
        // Must have been all TRUE filters (the only kind we would have removed above).
        return Filtration.matchEverything();
      }
    } else if (newChildren.size() == 1) {
      return newChildren.get(0);
    } else {
      return disjunction ? new OrDimFilter(newChildren) : new AndDimFilter(newChildren);
    }
  }