diff --git a/CHANGELOG.md b/CHANGELOG.md index 86295cb34f8de..f386b092cf074 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -40,6 +40,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Introduce Template query ([#16818](https://github.com/opensearch-project/OpenSearch/pull/16818)) - Propagate the sourceIncludes and excludes fields from fetchSourceContext to FieldsVisitor. ([#17080](https://github.com/opensearch-project/OpenSearch/pull/17080)) - [Star Tree] [Search] Resolving Date histogram with metric aggregation using star-tree ([#16674](https://github.com/opensearch-project/OpenSearch/pull/16674)) +- [Star Tree] [Search] Extensible design to support different query and field types ([#17137](https://github.com/opensearch-project/OpenSearch/pull/17137)) ### Dependencies - Bump `com.google.cloud:google-cloud-core-http` from 2.23.0 to 2.47.0 ([#16504](https://github.com/opensearch-project/OpenSearch/pull/16504)) diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNode.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNode.java index df2ce9096bfc1..c6c4993290c16 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNode.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNode.java @@ -10,6 +10,7 @@ import org.apache.lucene.store.RandomAccessInput; import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNode; import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNodeType; +import org.opensearch.search.startree.StarTreeNodeCollector; import java.io.IOException; import java.io.UncheckedIOException; @@ -192,7 +193,7 @@ public StarTreeNode getChildStarNode() throws IOException { } @Override - public StarTreeNode getChildForDimensionValue(Long dimensionValue) throws IOException { + public StarTreeNode getChildForDimensionValue(Long dimensionValue, StarTreeNode lastMatchedChild) throws IOException { // there will be no children for leaf nodes if (isLeaf()) { return null; @@ -200,7 +201,7 @@ public StarTreeNode getChildForDimensionValue(Long dimensionValue) throws IOExce StarTreeNode resultStarTreeNode = null; if (null != dimensionValue) { - resultStarTreeNode = binarySearchChild(dimensionValue); + resultStarTreeNode = binarySearchChild(dimensionValue, lastMatchedChild); } return resultStarTreeNode; } @@ -240,21 +241,29 @@ private static FixedLengthStarTreeNode matchStarTreeNodeTypeOrNull(FixedLengthSt * @return The child node if found, null otherwise * @throws IOException If there's an error reading from the input */ - private FixedLengthStarTreeNode binarySearchChild(long dimensionValue) throws IOException { + private FixedLengthStarTreeNode binarySearchChild(long dimensionValue, StarTreeNode lastMatchedNode) throws IOException { int low = firstChildId; - // if the current node is star node, increment the low to reduce the search space - if (matchStarTreeNodeTypeOrNull(new FixedLengthStarTreeNode(in, firstChildId), StarTreeNodeType.STAR) != null) { - low++; - } - int high = getInt(LAST_CHILD_ID_OFFSET); // if the current node is null node, decrement the high to reduce the search space if (matchStarTreeNodeTypeOrNull(new FixedLengthStarTreeNode(in, high), StarTreeNodeType.NULL) != null) { high--; } + if (lastMatchedNode instanceof FixedLengthStarTreeNode) { + int lastMatchedNodeId = ((FixedLengthStarTreeNode) lastMatchedNode).nodeId(); + // Start the binary search from node after the last matched as low. + if ((lastMatchedNodeId + 1) <= high) { + low = lastMatchedNodeId + 1; + } else { + return null; + } + } else if (matchStarTreeNodeTypeOrNull(new FixedLengthStarTreeNode(in, low), StarTreeNodeType.STAR) != null) { + // if the current node is star node, increment the low to reduce the search space + low++; + } + while (low <= high) { int mid = low + (high - low) / 2; FixedLengthStarTreeNode midNode = new FixedLengthStarTreeNode(in, mid); @@ -271,6 +280,100 @@ private FixedLengthStarTreeNode binarySearchChild(long dimensionValue) throws IO return null; } + @Override + public void collectChildrenInRange(long low, long high, StarTreeNodeCollector collector) throws IOException { + if (low <= high) { + FixedLengthStarTreeNode lowStarTreeNode = binarySearchChild(low, true, null); + if (lowStarTreeNode != null) { + FixedLengthStarTreeNode highStarTreeNode = binarySearchChild(high, false, lowStarTreeNode); + if (highStarTreeNode != null) { + for (int lowNodeId = lowStarTreeNode.nodeId(); lowNodeId <= highStarTreeNode.nodeId(); ++lowNodeId) { + collector.collectStarTreeNode(new FixedLengthStarTreeNode(in, lowNodeId)); + } + } else if (lowStarTreeNode.getDimensionValue() <= high) { // Low StarTreeNode is the last default node for that dimension. + collector.collectStarTreeNode(lowStarTreeNode); + } + } + } + } + + /** + * + * @param dimensionValue : The dimension to match. + * @param matchNextHighest : If true then we try to return @dimensionValue or the next Highest. Else, we return @dimensionValue or the next Lowest. + * @param lastMatchedNode : If not null, we begin the binary search from the node after this. + * @return : Matched node or null. + * @throws IOException : + */ + private FixedLengthStarTreeNode binarySearchChild(long dimensionValue, boolean matchNextHighest, StarTreeNode lastMatchedNode) + throws IOException { + + int low = firstChildId; + int tempLow = low; + int starNodeId, nullNodeId; + starNodeId = nullNodeId = Integer.MIN_VALUE; + + // if the current node is star node, increment the tempLow to reduce the search space + if (matchStarTreeNodeTypeOrNull(new FixedLengthStarTreeNode(in, tempLow), StarTreeNodeType.STAR) != null) { + starNodeId = tempLow; + tempLow++; + } + + int high = getInt(LAST_CHILD_ID_OFFSET); + int tempHigh = high; + // if the current node is null node, decrement the tempHigh to reduce the search space + if (matchStarTreeNodeTypeOrNull(new FixedLengthStarTreeNode(in, tempHigh), StarTreeNodeType.NULL) != null) { + nullNodeId = tempHigh; + tempHigh--; + } + + if (lastMatchedNode instanceof FixedLengthStarTreeNode) { + int lastMatchedNodeId = ((FixedLengthStarTreeNode) lastMatchedNode).nodeId(); + // Start the binary search from node after the last matched as low. + if ((lastMatchedNodeId + 1) <= tempHigh) { + tempLow = lastMatchedNodeId + 1; + } else { + return null; + } + } + + while (tempLow <= tempHigh) { + int mid = tempLow + (tempHigh - tempLow) / 2; + FixedLengthStarTreeNode midNode = new FixedLengthStarTreeNode(in, mid); + long midDimensionValue = midNode.getDimensionValue(); + + if (midDimensionValue == dimensionValue) { + return midNode; + } else { + if (midDimensionValue < dimensionValue) { // Going to the right from mid to search next + tempLow = mid + 1; + // We are going out of bounds for this dimension on the right side. + if (tempLow > high || tempLow == nullNodeId) { + return matchNextHighest ? null : midNode; + } else { + FixedLengthStarTreeNode nodeGreaterThanMid = new FixedLengthStarTreeNode(in, tempLow); + if (nodeGreaterThanMid.getDimensionValue() > dimensionValue) { + return matchNextHighest ? nodeGreaterThanMid : midNode; + } + } + } else { // Going to the left from mid to search next + tempHigh = mid - 1; + // We are going out of bounds for this dimension on the left side. + if (tempHigh < low || tempHigh == starNodeId) { + return matchNextHighest ? midNode : null; + } else { + FixedLengthStarTreeNode nodeLessThanMid = new FixedLengthStarTreeNode(in, tempHigh); + if (nodeLessThanMid.getDimensionValue() < dimensionValue) { + return matchNextHighest ? midNode : nodeLessThanMid; + } + } + } + } + } + return null; + + } + @Override public Iterator getChildrenIterator() throws IOException { return new Iterator<>() { @@ -297,4 +400,8 @@ public void remove() { } }; } + + public int nodeId() { + return nodeId; + } } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNode.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNode.java index 3767f6850002a..40161a942ae4b 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNode.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNode.java @@ -9,6 +9,7 @@ package org.opensearch.index.compositeindex.datacube.startree.node; import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.search.startree.StarTreeNodeCollector; import java.io.IOException; import java.util.Iterator; @@ -107,7 +108,27 @@ public interface StarTreeNode { * @return the child node for the given dimension value or null if child is not present * @throws IOException if an I/O error occurs while retrieving the child node */ - StarTreeNode getChildForDimensionValue(Long dimensionValue) throws IOException; + default StarTreeNode getChildForDimensionValue(Long dimensionValue) throws IOException { + return getChildForDimensionValue(dimensionValue, null); + } + + /** + * Matches the given @dimensionValue amongst the child default nodes for this node. + * @param dimensionValue : Value to match + * @param lastMatchedChild : If not null, binary search will use this as the start/low + * @return : Matched StarTreeNode or null if not found + * @throws IOException : Any exception in reading the node data from index. + */ + StarTreeNode getChildForDimensionValue(Long dimensionValue, StarTreeNode lastMatchedChild) throws IOException; + + /** + * Collects all matching child nodes whose dimension values lie within the range of low and high, both inclusive. + * @param low : Starting of the range ( inclusive ) + * @param high : End of the range ( inclusive ) + * @param collector : Collector to collect the matched child StarTreeNode's + * @throws IOException : Any exception in reading the node data from index. + */ + void collectChildrenInRange(long low, long high, StarTreeNodeCollector collector) throws IOException; /** * Returns the child star node for a node in the star-tree. diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeQueryHelper.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeQueryHelper.java deleted file mode 100644 index e2414d9f6a8a1..0000000000000 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeQueryHelper.java +++ /dev/null @@ -1,306 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.index.compositeindex.datacube.startree.utils; - -import org.apache.lucene.index.DocValuesType; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.SegmentReader; -import org.apache.lucene.search.CollectionTerminatedException; -import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.util.FixedBitSet; -import org.opensearch.common.lucene.Lucene; -import org.opensearch.index.codec.composite.CompositeIndexFieldInfo; -import org.opensearch.index.codec.composite.CompositeIndexReader; -import org.opensearch.index.compositeindex.datacube.DateDimension; -import org.opensearch.index.compositeindex.datacube.Dimension; -import org.opensearch.index.compositeindex.datacube.Metric; -import org.opensearch.index.compositeindex.datacube.MetricStat; -import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; -import org.opensearch.index.compositeindex.datacube.startree.utils.date.DateTimeUnitAdapter; -import org.opensearch.index.compositeindex.datacube.startree.utils.date.DateTimeUnitRounding; -import org.opensearch.index.compositeindex.datacube.startree.utils.iterator.SortedNumericStarTreeValuesIterator; -import org.opensearch.index.mapper.CompositeDataCubeFieldType; -import org.opensearch.index.query.MatchAllQueryBuilder; -import org.opensearch.index.query.QueryBuilder; -import org.opensearch.index.query.TermQueryBuilder; -import org.opensearch.search.aggregations.AggregatorFactory; -import org.opensearch.search.aggregations.LeafBucketCollector; -import org.opensearch.search.aggregations.StarTreeBucketCollector; -import org.opensearch.search.aggregations.bucket.histogram.DateHistogramAggregatorFactory; -import org.opensearch.search.aggregations.metrics.MetricAggregatorFactory; -import org.opensearch.search.aggregations.support.ValuesSource; -import org.opensearch.search.builder.SearchSourceBuilder; -import org.opensearch.search.internal.SearchContext; -import org.opensearch.search.startree.StarTreeFilter; -import org.opensearch.search.startree.StarTreeQueryContext; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.function.BiConsumer; -import java.util.function.Consumer; -import java.util.stream.Collectors; - -/** - * Helper class for building star-tree query - * - * @opensearch.internal - * @opensearch.experimental - */ -public class StarTreeQueryHelper { - - /** - * Checks if the search context can be supported by star-tree - */ - public static boolean isStarTreeSupported(SearchContext context) { - return context.aggregations() != null && context.mapperService().isCompositeIndexPresent() && context.parsedPostFilter() == null; - } - - /** - * Gets StarTreeQueryContext from the search context and source builder. - * Returns null if the query and aggregation cannot be supported. - */ - public static StarTreeQueryContext getStarTreeQueryContext(SearchContext context, SearchSourceBuilder source) throws IOException { - // Current implementation assumes only single star-tree is supported - CompositeDataCubeFieldType compositeMappedFieldType = (CompositeDataCubeFieldType) context.mapperService() - .getCompositeFieldTypes() - .iterator() - .next(); - CompositeIndexFieldInfo starTree = new CompositeIndexFieldInfo( - compositeMappedFieldType.name(), - compositeMappedFieldType.getCompositeIndexType() - ); - - for (AggregatorFactory aggregatorFactory : context.aggregations().factories().getFactories()) { - // first check for aggregation is a metric aggregation - if (validateStarTreeMetricSupport(compositeMappedFieldType, aggregatorFactory)) { - continue; - } - - // if not a metric aggregation, check for applicable date histogram shape - if (validateDateHistogramSupport(compositeMappedFieldType, aggregatorFactory)) { - continue; - } - return null; - } - - // need to cache star tree values only for multiple aggregations - boolean cacheStarTreeValues = context.aggregations().factories().getFactories().length > 1; - int cacheSize = cacheStarTreeValues ? context.indexShard().segments(false).size() : -1; - - return StarTreeQueryHelper.tryCreateStarTreeQueryContext(starTree, compositeMappedFieldType, source.query(), cacheSize); - } - - /** - * Uses query builder and composite index info to form star-tree query context - */ - private static StarTreeQueryContext tryCreateStarTreeQueryContext( - CompositeIndexFieldInfo compositeIndexFieldInfo, - CompositeDataCubeFieldType compositeFieldType, - QueryBuilder queryBuilder, - int cacheStarTreeValuesSize - ) { - Map queryMap; - if (queryBuilder == null || queryBuilder instanceof MatchAllQueryBuilder) { - queryMap = null; - } else if (queryBuilder instanceof TermQueryBuilder termQueryBuilder) { - // TODO: Add support for keyword fields - Dimension matchedDimension = compositeFieldType.getDimensions() - .stream() - .filter(d -> (d.getField().equals(termQueryBuilder.fieldName()) && d.getDocValuesType() == DocValuesType.SORTED_NUMERIC)) - .findFirst() - .orElse(null); - if (matchedDimension == null) { - return null; - } - queryMap = Map.of(termQueryBuilder.fieldName(), Long.parseLong(termQueryBuilder.value().toString())); - } else { - return null; - } - return new StarTreeQueryContext(compositeIndexFieldInfo, queryMap, cacheStarTreeValuesSize); - } - - private static boolean validateStarTreeMetricSupport( - CompositeDataCubeFieldType compositeIndexFieldInfo, - AggregatorFactory aggregatorFactory - ) { - if (aggregatorFactory instanceof MetricAggregatorFactory metricAggregatorFactory - && metricAggregatorFactory.getSubFactories().getFactories().length == 0) { - String field; - Map> supportedMetrics = compositeIndexFieldInfo.getMetrics() - .stream() - .collect(Collectors.toMap(Metric::getField, Metric::getMetrics)); - - MetricStat metricStat = metricAggregatorFactory.getMetricStat(); - field = metricAggregatorFactory.getField(); - - return supportedMetrics.containsKey(field) && supportedMetrics.get(field).contains(metricStat); - } - return false; - } - - private static boolean validateDateHistogramSupport( - CompositeDataCubeFieldType compositeIndexFieldInfo, - AggregatorFactory aggregatorFactory - ) { - if (!(aggregatorFactory instanceof DateHistogramAggregatorFactory dateHistogramAggregatorFactory) - || aggregatorFactory.getSubFactories().getFactories().length < 1) { - return false; - } - - // Find the DateDimension in the dimensions list - DateDimension starTreeDateDimension = null; - for (Dimension dimension : compositeIndexFieldInfo.getDimensions()) { - if (dimension instanceof DateDimension) { - starTreeDateDimension = (DateDimension) dimension; - break; - } - } - - // If no DateDimension is found, validation fails - if (starTreeDateDimension == null) { - return false; - } - - // Ensure the rounding is not null - if (dateHistogramAggregatorFactory.getRounding() == null) { - return false; - } - - // Find the closest valid interval in the DateTimeUnitRounding class associated with star tree - DateTimeUnitRounding rounding = starTreeDateDimension.findClosestValidInterval( - new DateTimeUnitAdapter(dateHistogramAggregatorFactory.getRounding()) - ); - if (rounding == null) { - return false; - } - - // Validate all sub-factories - for (AggregatorFactory subFactory : aggregatorFactory.getSubFactories().getFactories()) { - if (!validateStarTreeMetricSupport(compositeIndexFieldInfo, subFactory)) { - return false; - } - } - return true; - } - - public static CompositeIndexFieldInfo getSupportedStarTree(SearchContext context) { - StarTreeQueryContext starTreeQueryContext = context.getStarTreeQueryContext(); - return (starTreeQueryContext != null) ? starTreeQueryContext.getStarTree() : null; - } - - public static StarTreeValues getStarTreeValues(LeafReaderContext context, CompositeIndexFieldInfo starTree) throws IOException { - SegmentReader reader = Lucene.segmentReader(context.reader()); - if (!(reader.getDocValuesReader() instanceof CompositeIndexReader)) { - return null; - } - CompositeIndexReader starTreeDocValuesReader = (CompositeIndexReader) reader.getDocValuesReader(); - return (StarTreeValues) starTreeDocValuesReader.getCompositeIndexValues(starTree); - } - - /** - * Get the star-tree leaf collector - * This collector computes the aggregation prematurely and invokes an early termination collector - */ - public static LeafBucketCollector getStarTreeLeafCollector( - SearchContext context, - ValuesSource.Numeric valuesSource, - LeafReaderContext ctx, - LeafBucketCollector sub, - CompositeIndexFieldInfo starTree, - String metric, - Consumer valueConsumer, - Runnable finalConsumer - ) throws IOException { - StarTreeValues starTreeValues = getStarTreeValues(ctx, starTree); - assert starTreeValues != null; - String fieldName = ((ValuesSource.Numeric.FieldData) valuesSource).getIndexFieldName(); - String metricName = StarTreeUtils.fullyQualifiedFieldNameForStarTreeMetricsDocValues(starTree.getField(), fieldName, metric); - - assert starTreeValues != null; - SortedNumericStarTreeValuesIterator valuesIterator = (SortedNumericStarTreeValuesIterator) starTreeValues.getMetricValuesIterator( - metricName - ); - // Obtain a FixedBitSet of matched star tree document IDs - FixedBitSet filteredValues = getStarTreeFilteredValues(context, ctx, starTreeValues); - assert filteredValues != null; - - int numBits = filteredValues.length(); // Get the number of the filtered values (matching docs) - if (numBits > 0) { - // Iterate over the filtered values - for (int bit = filteredValues.nextSetBit(0); bit != DocIdSetIterator.NO_MORE_DOCS; bit = (bit + 1 < numBits) - ? filteredValues.nextSetBit(bit + 1) - : DocIdSetIterator.NO_MORE_DOCS) { - // Advance to the entryId in the valuesIterator - if (valuesIterator.advanceExact(bit) == false) { - continue; // Skip if no more entries - } - - // Iterate over the values for the current entryId - for (int i = 0, count = valuesIterator.entryValueCount(); i < count; i++) { - long value = valuesIterator.nextValue(); - valueConsumer.accept(value); // Apply the consumer operation (e.g., max, sum) - } - } - } - - // Call the final consumer after processing all entries - finalConsumer.run(); - - // Terminate after pre-computing aggregation - throw new CollectionTerminatedException(); - } - - public static StarTreeBucketCollector getStarTreeBucketMetricCollector( - CompositeIndexFieldInfo starTree, - String metric, - ValuesSource.Numeric valuesSource, - StarTreeBucketCollector parentCollector, - Consumer growArrays, - BiConsumer updateBucket - ) throws IOException { - assert parentCollector != null; - return new StarTreeBucketCollector(parentCollector) { - String metricName = StarTreeUtils.fullyQualifiedFieldNameForStarTreeMetricsDocValues( - starTree.getField(), - ((ValuesSource.Numeric.FieldData) valuesSource).getIndexFieldName(), - metric - ); - SortedNumericStarTreeValuesIterator metricValuesIterator = (SortedNumericStarTreeValuesIterator) starTreeValues - .getMetricValuesIterator(metricName); - - @Override - public void collectStarTreeEntry(int starTreeEntryBit, long bucket) throws IOException { - growArrays.accept(bucket); - // Advance the valuesIterator to the current bit - if (!metricValuesIterator.advanceExact(starTreeEntryBit)) { - return; // Skip if no entries for this document - } - long metricValue = metricValuesIterator.nextValue(); - updateBucket.accept(bucket, metricValue); - } - }; - } - - /** - * Get the filtered values for the star-tree query - * Cache the results in case of multiple aggregations (if cache is initialized) - * @return FixedBitSet of matched document IDs - */ - public static FixedBitSet getStarTreeFilteredValues(SearchContext context, LeafReaderContext ctx, StarTreeValues starTreeValues) - throws IOException { - FixedBitSet result = context.getStarTreeQueryContext().getStarTreeValues(ctx); - if (result == null) { - result = StarTreeFilter.getStarTreeResult(starTreeValues, context.getStarTreeQueryContext().getQueryMap(), Set.of()); - context.getStarTreeQueryContext().setStarTreeValues(ctx, result); - } - return result; - } -} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/iterator/SortedNumericStarTreeValuesIterator.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/iterator/SortedNumericStarTreeValuesIterator.java index 4b4bfa6a915eb..595965c98ea07 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/iterator/SortedNumericStarTreeValuesIterator.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/iterator/SortedNumericStarTreeValuesIterator.java @@ -26,6 +26,11 @@ public SortedNumericStarTreeValuesIterator(DocIdSetIterator docIdSetIterator) { super(docIdSetIterator); } + @Override + public long value() throws IOException { + return nextValue(); + } + public long nextValue() throws IOException { return ((SortedNumericDocValues) docIdSetIterator).nextValue(); } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/iterator/SortedSetStarTreeValuesIterator.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/iterator/SortedSetStarTreeValuesIterator.java index 0cddffe5877e9..1605bd9cfc014 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/iterator/SortedSetStarTreeValuesIterator.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/iterator/SortedSetStarTreeValuesIterator.java @@ -29,6 +29,17 @@ public SortedSetStarTreeValuesIterator(DocIdSetIterator docIdSetIterator) { super(docIdSetIterator); } + @Override + public long value() throws IOException { + return nextOrd(); + } + + @Override + public boolean advanceExact(int target) throws IOException { + return ((SortedSetDocValues) docIdSetIterator).advanceExact(target); + } + + // TODO : Remove this and merge @org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator to use value() public long nextOrd() throws IOException { return ((SortedSetDocValues) docIdSetIterator).nextOrd(); } @@ -56,4 +67,5 @@ public TermsEnum termsEnum() throws IOException { public TermsEnum intersect(CompiledAutomaton automaton) throws IOException { return ((SortedSetDocValues) docIdSetIterator).intersect(automaton); } + } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/iterator/StarTreeValuesIterator.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/iterator/StarTreeValuesIterator.java index 32866f3e50092..d9ee67fe1b0d5 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/iterator/StarTreeValuesIterator.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/iterator/StarTreeValuesIterator.java @@ -45,4 +45,9 @@ public int advance(int target) throws IOException { public long cost() { return docIdSetIterator.cost(); } + + public abstract long value() throws IOException; + + public abstract boolean advanceExact(int target) throws IOException; + } diff --git a/server/src/main/java/org/opensearch/index/mapper/NumberFieldMapper.java b/server/src/main/java/org/opensearch/index/mapper/NumberFieldMapper.java index f3fc3f4b2aa95..faf3f1bb654c8 100644 --- a/server/src/main/java/org/opensearch/index/mapper/NumberFieldMapper.java +++ b/server/src/main/java/org/opensearch/index/mapper/NumberFieldMapper.java @@ -1305,7 +1305,7 @@ public static boolean hasDecimalPart(Object number) { /** * Returns -1, 0, or 1 if the value is lower than, equal to, or greater than 0 */ - static double signum(Object value) { + public static double signum(Object value) { if (value instanceof Number) { double doubleValue = ((Number) value).doubleValue(); return Math.signum(doubleValue); diff --git a/server/src/main/java/org/opensearch/index/query/QueryShardContext.java b/server/src/main/java/org/opensearch/index/query/QueryShardContext.java index 0610752e532e7..62c8c0e25596c 100644 --- a/server/src/main/java/org/opensearch/index/query/QueryShardContext.java +++ b/server/src/main/java/org/opensearch/index/query/QueryShardContext.java @@ -76,6 +76,7 @@ import org.opensearch.search.aggregations.support.AggregationUsageService; import org.opensearch.search.aggregations.support.ValuesSourceRegistry; import org.opensearch.search.lookup.SearchLookup; +import org.opensearch.search.startree.StarTreeQueryContext; import org.opensearch.transport.RemoteClusterAware; import java.io.IOException; @@ -128,6 +129,8 @@ public class QueryShardContext extends BaseQueryRewriteContext { private boolean keywordIndexOrDocValuesEnabled; private boolean isInnerHitQuery; + private StarTreeQueryContext starTreeQueryContext; + public QueryShardContext( int shardId, IndexSettings indexSettings, @@ -379,6 +382,14 @@ public > IFD getForField(MappedFieldType fieldType ); } + public StarTreeQueryContext getStarTreeQueryContext() { + return starTreeQueryContext; + } + + public void setStarTreeQueryContext(StarTreeQueryContext starTreeQueryContext) { + this.starTreeQueryContext = starTreeQueryContext; + } + public void addNamedQuery(String name, Query query) { if (query != null) { namedQueries.put(name, query); diff --git a/server/src/main/java/org/opensearch/search/SearchService.java b/server/src/main/java/org/opensearch/search/SearchService.java index d4380eb09e360..866cf0d62b033 100644 --- a/server/src/main/java/org/opensearch/search/SearchService.java +++ b/server/src/main/java/org/opensearch/search/SearchService.java @@ -77,7 +77,6 @@ import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; -import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeQueryHelper; import org.opensearch.index.engine.Engine; import org.opensearch.index.mapper.DerivedFieldResolver; import org.opensearch.index.mapper.DerivedFieldResolverFactory; @@ -141,6 +140,7 @@ import org.opensearch.search.sort.SortBuilder; import org.opensearch.search.sort.SortOrder; import org.opensearch.search.startree.StarTreeQueryContext; +import org.opensearch.search.startree.StarTreeQueryHelper; import org.opensearch.search.suggest.Suggest; import org.opensearch.search.suggest.completion.CompletionSuggestion; import org.opensearch.tasks.TaskResourceTrackingService; @@ -1548,15 +1548,11 @@ private void parseSource(DefaultSearchContext context, SearchSourceBuilder sourc if (this.indicesService.getCompositeIndexSettings() != null && this.indicesService.getCompositeIndexSettings().isStarTreeIndexCreationEnabled() && StarTreeQueryHelper.isStarTreeSupported(context)) { - try { - StarTreeQueryContext starTreeQueryContext = StarTreeQueryHelper.getStarTreeQueryContext(context, source); - if (starTreeQueryContext != null) { - context.starTreeQueryContext(starTreeQueryContext); - logger.debug("can use star tree"); - } else { - logger.debug("cannot use star tree"); - } - } catch (IOException ignored) {} + StarTreeQueryContext starTreeQueryContext = new StarTreeQueryContext(context, source.query()); + boolean consolidated = starTreeQueryContext.consolidateAllFilters(context); + if (consolidated) { + queryShardContext.setStarTreeQueryContext(starTreeQueryContext); + } } } diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java index 49672831625e4..451b96dc3cf9c 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java @@ -45,7 +45,6 @@ import org.opensearch.index.compositeindex.datacube.DateDimension; import org.opensearch.index.compositeindex.datacube.MetricStat; import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; -import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeQueryHelper; import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils; import org.opensearch.index.compositeindex.datacube.startree.utils.date.DateTimeUnitAdapter; import org.opensearch.index.compositeindex.datacube.startree.utils.date.DateTimeUnitRounding; @@ -68,18 +67,20 @@ import org.opensearch.search.aggregations.support.ValuesSource; import org.opensearch.search.aggregations.support.ValuesSourceConfig; import org.opensearch.search.internal.SearchContext; -import org.opensearch.search.startree.StarTreeFilter; +import org.opensearch.search.startree.StarTreeQueryHelper; +import org.opensearch.search.startree.StarTreeTraversalUtil; +import org.opensearch.search.startree.filter.DimensionFilter; import java.io.IOException; import java.util.Collections; +import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.Set; import java.util.function.BiConsumer; import java.util.function.Function; -import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeQueryHelper.getSupportedStarTree; import static org.opensearch.search.aggregations.bucket.filterrewrite.DateHistogramAggregatorBridge.segmentMatchAll; +import static org.opensearch.search.startree.StarTreeQueryHelper.getSupportedStarTree; /** * An aggregator for date values. Every date is rounded down using a configured @@ -172,7 +173,9 @@ protected Function bucketOrdProducer() { } }; filterRewriteOptimizationContext = new FilterRewriteOptimizationContext(bridge, parent, subAggregators.length, context); - this.starTreeDateDimension = (context.getStarTreeQueryContext() != null) ? fetchStarTreeCalendarUnit() : null; + this.starTreeDateDimension = (context.getQueryShardContext().getStarTreeQueryContext() != null) + ? fetchStarTreeCalendarUnit() + : null; } @Override @@ -193,7 +196,7 @@ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCol if (optimized) throw new CollectionTerminatedException(); SortedNumericDocValues values = valuesSource.longValues(ctx); - CompositeIndexFieldInfo supportedStarTree = getSupportedStarTree(this.context); + CompositeIndexFieldInfo supportedStarTree = getSupportedStarTree(this.context.getQueryShardContext()); if (supportedStarTree != null) { if (preComputeWithStarTree(ctx, supportedStarTree) == true) { throw new CollectionTerminatedException(); @@ -264,7 +267,15 @@ public StarTreeBucketCollector getStarTreeBucketCollector( StarTreeValues starTreeValues = StarTreeQueryHelper.getStarTreeValues(ctx, starTree); return new StarTreeBucketCollector( starTreeValues, - StarTreeFilter.getStarTreeResult(starTreeValues, context.getStarTreeQueryContext().getQueryMap(), Set.of(starTreeDateDimension)) + StarTreeTraversalUtil.getStarTreeResult( + starTreeValues, + StarTreeQueryHelper.mergeDimensionFilterIfNotExists( + context.getQueryShardContext().getStarTreeQueryContext().getBaseQueryStarTreeFilter(), + starTreeDateDimension, + List.of(DimensionFilter.MATCH_ALL_DEFAULT) + ), + context + ) ) { @Override public void setSubCollectors() throws IOException { diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/AvgAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/AvgAggregator.java index c9f5bb7f3534b..f71b6679a7c4d 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/AvgAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/AvgAggregator.java @@ -44,7 +44,6 @@ import org.opensearch.index.codec.composite.CompositeIndexFieldInfo; import org.opensearch.index.compositeindex.datacube.MetricStat; import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; -import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeQueryHelper; import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils; import org.opensearch.index.compositeindex.datacube.startree.utils.iterator.SortedNumericStarTreeValuesIterator; import org.opensearch.index.fielddata.SortedNumericDoubleValues; @@ -58,12 +57,13 @@ import org.opensearch.search.aggregations.support.ValuesSource; import org.opensearch.search.aggregations.support.ValuesSourceConfig; import org.opensearch.search.internal.SearchContext; +import org.opensearch.search.startree.StarTreeQueryHelper; import java.io.IOException; import java.util.Map; -import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeQueryHelper.getStarTreeFilteredValues; -import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeQueryHelper.getSupportedStarTree; +import static org.opensearch.search.startree.StarTreeQueryHelper.getStarTreeFilteredValues; +import static org.opensearch.search.startree.StarTreeQueryHelper.getSupportedStarTree; /** * Aggregate all docs into an average @@ -108,7 +108,7 @@ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBuc if (valuesSource == null) { return LeafBucketCollector.NO_OP_COLLECTOR; } - CompositeIndexFieldInfo supportedStarTree = getSupportedStarTree(this.context); + CompositeIndexFieldInfo supportedStarTree = getSupportedStarTree(this.context.getQueryShardContext()); if (supportedStarTree != null) { if (parent != null && subAggregators.length == 0) { // If this a child aggregator, then the parent will trigger star-tree pre-computation. diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/MaxAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/MaxAggregator.java index 49aaf5e0670bb..c64a6cf29fb63 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/MaxAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/MaxAggregator.java @@ -43,7 +43,6 @@ import org.opensearch.common.util.DoubleArray; import org.opensearch.index.codec.composite.CompositeIndexFieldInfo; import org.opensearch.index.compositeindex.datacube.MetricStat; -import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeQueryHelper; import org.opensearch.index.fielddata.NumericDoubleValues; import org.opensearch.index.fielddata.SortedNumericDoubleValues; import org.opensearch.search.DocValueFormat; @@ -57,6 +56,7 @@ import org.opensearch.search.aggregations.support.ValuesSource; import org.opensearch.search.aggregations.support.ValuesSourceConfig; import org.opensearch.search.internal.SearchContext; +import org.opensearch.search.startree.StarTreeQueryHelper; import java.io.IOException; import java.util.Arrays; @@ -64,7 +64,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; -import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeQueryHelper.getSupportedStarTree; +import static org.opensearch.search.startree.StarTreeQueryHelper.getSupportedStarTree; /** * Aggregate all docs into a max value @@ -130,14 +130,14 @@ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBuc } } - CompositeIndexFieldInfo supportedStarTree = getSupportedStarTree(this.context); + CompositeIndexFieldInfo supportedStarTree = getSupportedStarTree(this.context.getQueryShardContext()); if (supportedStarTree != null) { if (parent != null && subAggregators.length == 0) { // If this a child aggregator, then the parent will trigger star-tree pre-computation. // Returning NO_OP_COLLECTOR explicitly because the getLeafCollector() are invoked starting from innermost aggregators return LeafBucketCollector.NO_OP_COLLECTOR; } - return getStarTreeCollector(ctx, sub, supportedStarTree); + getStarTreeCollector(ctx, sub, supportedStarTree); } return getDefaultLeafCollector(ctx, sub); } @@ -167,21 +167,11 @@ public void collect(int doc, long bucket) throws IOException { }; } - public LeafBucketCollector getStarTreeCollector(LeafReaderContext ctx, LeafBucketCollector sub, CompositeIndexFieldInfo starTree) - throws IOException { + public void getStarTreeCollector(LeafReaderContext ctx, LeafBucketCollector sub, CompositeIndexFieldInfo starTree) throws IOException { AtomicReference max = new AtomicReference<>(maxes.get(0)); - return StarTreeQueryHelper.getStarTreeLeafCollector( - context, - valuesSource, - ctx, - sub, - starTree, - MetricStat.MAX.getTypeName(), - value -> { - max.set(Math.max(max.get(), (NumericUtils.sortableLongToDouble(value)))); - }, - () -> maxes.set(0, max.get()) - ); + StarTreeQueryHelper.getStarTreeLeafCollector(context, valuesSource, ctx, sub, starTree, MetricStat.MAX.getTypeName(), value -> { + max.set(Math.max(max.get(), (NumericUtils.sortableLongToDouble(value)))); + }, () -> maxes.set(0, max.get())); } @Override diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/MinAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/MinAggregator.java index febb227dd4e2a..5cdee536cde19 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/MinAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/MinAggregator.java @@ -43,7 +43,6 @@ import org.opensearch.common.util.DoubleArray; import org.opensearch.index.codec.composite.CompositeIndexFieldInfo; import org.opensearch.index.compositeindex.datacube.MetricStat; -import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeQueryHelper; import org.opensearch.index.fielddata.NumericDoubleValues; import org.opensearch.index.fielddata.SortedNumericDoubleValues; import org.opensearch.search.DocValueFormat; @@ -57,13 +56,14 @@ import org.opensearch.search.aggregations.support.ValuesSource; import org.opensearch.search.aggregations.support.ValuesSourceConfig; import org.opensearch.search.internal.SearchContext; +import org.opensearch.search.startree.StarTreeQueryHelper; import java.io.IOException; import java.util.Map; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; -import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeQueryHelper.getSupportedStarTree; +import static org.opensearch.search.startree.StarTreeQueryHelper.getSupportedStarTree; /** * Aggregate all docs into a min value @@ -129,14 +129,14 @@ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBuc } } - CompositeIndexFieldInfo supportedStarTree = getSupportedStarTree(this.context); + CompositeIndexFieldInfo supportedStarTree = getSupportedStarTree(this.context.getQueryShardContext()); if (supportedStarTree != null) { if (parent != null && subAggregators.length == 0) { // If this a child aggregator, then the parent will trigger star-tree pre-computation. // Returning NO_OP_COLLECTOR explicitly because the getLeafCollector() are invoked starting from innermost aggregators return LeafBucketCollector.NO_OP_COLLECTOR; } - return getStarTreeCollector(ctx, sub, supportedStarTree); + getStarTreeCollector(ctx, sub, supportedStarTree); } return getDefaultLeafCollector(ctx, sub); } @@ -164,21 +164,11 @@ public void collect(int doc, long bucket) throws IOException { }; } - public LeafBucketCollector getStarTreeCollector(LeafReaderContext ctx, LeafBucketCollector sub, CompositeIndexFieldInfo starTree) - throws IOException { + public void getStarTreeCollector(LeafReaderContext ctx, LeafBucketCollector sub, CompositeIndexFieldInfo starTree) throws IOException { AtomicReference min = new AtomicReference<>(mins.get(0)); - return StarTreeQueryHelper.getStarTreeLeafCollector( - context, - valuesSource, - ctx, - sub, - starTree, - MetricStat.MIN.getTypeName(), - value -> { - min.set(Math.min(min.get(), (NumericUtils.sortableLongToDouble(value)))); - }, - () -> mins.set(0, min.get()) - ); + StarTreeQueryHelper.getStarTreeLeafCollector(context, valuesSource, ctx, sub, starTree, MetricStat.MIN.getTypeName(), value -> { + min.set(Math.min(min.get(), (NumericUtils.sortableLongToDouble(value)))); + }, () -> mins.set(0, min.get())); } @Override diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/SumAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/SumAggregator.java index 7376cc1e93b41..edcfb61263fc1 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/SumAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/SumAggregator.java @@ -39,7 +39,6 @@ import org.opensearch.common.util.DoubleArray; import org.opensearch.index.codec.composite.CompositeIndexFieldInfo; import org.opensearch.index.compositeindex.datacube.MetricStat; -import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeQueryHelper; import org.opensearch.index.fielddata.SortedNumericDoubleValues; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.Aggregator; @@ -51,11 +50,12 @@ import org.opensearch.search.aggregations.support.ValuesSource; import org.opensearch.search.aggregations.support.ValuesSourceConfig; import org.opensearch.search.internal.SearchContext; +import org.opensearch.search.startree.StarTreeQueryHelper; import java.io.IOException; import java.util.Map; -import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeQueryHelper.getSupportedStarTree; +import static org.opensearch.search.startree.StarTreeQueryHelper.getSupportedStarTree; /** * Aggregate all docs into a single sum value @@ -98,14 +98,14 @@ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBuc return LeafBucketCollector.NO_OP_COLLECTOR; } - CompositeIndexFieldInfo supportedStarTree = getSupportedStarTree(this.context); + CompositeIndexFieldInfo supportedStarTree = getSupportedStarTree(this.context.getQueryShardContext()); if (supportedStarTree != null) { if (parent != null && subAggregators.length == 0) { // If this a child aggregator, then the parent will trigger star-tree pre-computation. // Returning NO_OP_COLLECTOR explicitly because the getLeafCollector() are invoked starting from innermost aggregators return LeafBucketCollector.NO_OP_COLLECTOR; } - return getStarTreeCollector(ctx, sub, supportedStarTree); + getStarTreeCollector(ctx, sub, supportedStarTree); } return getDefaultLeafCollector(ctx, sub); } @@ -140,11 +140,10 @@ public void collect(int doc, long bucket) throws IOException { }; } - public LeafBucketCollector getStarTreeCollector(LeafReaderContext ctx, LeafBucketCollector sub, CompositeIndexFieldInfo starTree) - throws IOException { + public void getStarTreeCollector(LeafReaderContext ctx, LeafBucketCollector sub, CompositeIndexFieldInfo starTree) throws IOException { final CompensatedSum kahanSummation = new CompensatedSum(sums.get(0), compensations.get(0)); - return StarTreeQueryHelper.getStarTreeLeafCollector( + StarTreeQueryHelper.getStarTreeLeafCollector( context, valuesSource, ctx, diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/ValueCountAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/ValueCountAggregator.java index f6f4a8a56eddc..d298361391ad9 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/ValueCountAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/ValueCountAggregator.java @@ -39,7 +39,6 @@ import org.opensearch.common.util.LongArray; import org.opensearch.index.codec.composite.CompositeIndexFieldInfo; import org.opensearch.index.compositeindex.datacube.MetricStat; -import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeQueryHelper; import org.opensearch.index.fielddata.MultiGeoPointValues; import org.opensearch.index.fielddata.SortedBinaryDocValues; import org.opensearch.search.aggregations.Aggregator; @@ -51,11 +50,12 @@ import org.opensearch.search.aggregations.support.ValuesSource; import org.opensearch.search.aggregations.support.ValuesSourceConfig; import org.opensearch.search.internal.SearchContext; +import org.opensearch.search.startree.StarTreeQueryHelper; import java.io.IOException; import java.util.Map; -import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeQueryHelper.getSupportedStarTree; +import static org.opensearch.search.startree.StarTreeQueryHelper.getSupportedStarTree; /** * A field data based aggregator that counts the number of values a specific field has within the aggregation context. @@ -96,14 +96,14 @@ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBuc if (valuesSource instanceof ValuesSource.Numeric) { - CompositeIndexFieldInfo supportedStarTree = getSupportedStarTree(this.context); + CompositeIndexFieldInfo supportedStarTree = getSupportedStarTree(this.context.getQueryShardContext()); if (supportedStarTree != null) { if (parent != null && subAggregators.length == 0) { // If this a child aggregator, then the parent will trigger star-tree pre-computation. // Returning NO_OP_COLLECTOR explicitly because the getLeafCollector() are invoked starting from innermost aggregators return LeafBucketCollector.NO_OP_COLLECTOR; } - return getStarTreeCollector(ctx, sub, supportedStarTree); + getStarTreeCollector(ctx, sub, supportedStarTree); } final SortedNumericDocValues values = ((ValuesSource.Numeric) valuesSource).longValues(ctx); @@ -145,9 +145,8 @@ public void collect(int doc, long bucket) throws IOException { }; } - public LeafBucketCollector getStarTreeCollector(LeafReaderContext ctx, LeafBucketCollector sub, CompositeIndexFieldInfo starTree) - throws IOException { - return StarTreeQueryHelper.getStarTreeLeafCollector( + public void getStarTreeCollector(LeafReaderContext ctx, LeafBucketCollector sub, CompositeIndexFieldInfo starTree) throws IOException { + StarTreeQueryHelper.getStarTreeLeafCollector( context, (ValuesSource.Numeric) valuesSource, ctx, diff --git a/server/src/main/java/org/opensearch/search/internal/SearchContext.java b/server/src/main/java/org/opensearch/search/internal/SearchContext.java index b7ea06d2989e5..b539981da4ebd 100644 --- a/server/src/main/java/org/opensearch/search/internal/SearchContext.java +++ b/server/src/main/java/org/opensearch/search/internal/SearchContext.java @@ -76,7 +76,6 @@ import org.opensearch.search.query.ReduceableSearchResult; import org.opensearch.search.rescore.RescoreContext; import org.opensearch.search.sort.SortAndFormats; -import org.opensearch.search.startree.StarTreeQueryContext; import org.opensearch.search.suggest.SuggestionSearchContext; import java.util.Collection; @@ -126,7 +125,6 @@ public List toInternalAggregations(Collection co private final AtomicBoolean closed = new AtomicBoolean(false); private InnerHitsContext innerHitsContext; private volatile boolean searchTimedOut; - private StarTreeQueryContext starTreeQueryContext; protected SearchContext() {} @@ -531,13 +529,4 @@ public int cardinalityAggregationPruningThreshold() { public boolean keywordIndexOrDocValuesEnabled() { return false; } - - public SearchContext starTreeQueryContext(StarTreeQueryContext starTreeQueryContext) { - this.starTreeQueryContext = starTreeQueryContext; - return this; - } - - public StarTreeQueryContext getStarTreeQueryContext() { - return this.starTreeQueryContext; - } } diff --git a/server/src/main/java/org/opensearch/search/startree/StarTreeNodeCollector.java b/server/src/main/java/org/opensearch/search/startree/StarTreeNodeCollector.java new file mode 100644 index 0000000000000..9d3dcd6824874 --- /dev/null +++ b/server/src/main/java/org/opensearch/search/startree/StarTreeNodeCollector.java @@ -0,0 +1,25 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.startree; + +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNode; + +/** + * Collects one or more @{@link StarTreeNode}'s + */ +@ExperimentalApi +public interface StarTreeNodeCollector { + /** + * Called to collect a @{@link StarTreeNode} + * @param node : Node to collect + */ + void collectStarTreeNode(StarTreeNode node); + +} diff --git a/server/src/main/java/org/opensearch/search/startree/StarTreeQueryContext.java b/server/src/main/java/org/opensearch/search/startree/StarTreeQueryContext.java index cda3a25b30e53..ca0ab9ce52f6e 100644 --- a/server/src/main/java/org/opensearch/search/startree/StarTreeQueryContext.java +++ b/server/src/main/java/org/opensearch/search/startree/StarTreeQueryContext.java @@ -8,72 +8,209 @@ package org.opensearch.search.startree; -import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.util.FixedBitSet; import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.index.codec.composite.CompositeIndexFieldInfo; +import org.opensearch.index.compositeindex.datacube.DateDimension; +import org.opensearch.index.compositeindex.datacube.Dimension; +import org.opensearch.index.compositeindex.datacube.Metric; +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.startree.utils.date.DateTimeUnitAdapter; +import org.opensearch.index.compositeindex.datacube.startree.utils.date.DateTimeUnitRounding; +import org.opensearch.index.mapper.CompositeDataCubeFieldType; +import org.opensearch.index.query.QueryBuilder; +import org.opensearch.search.aggregations.AggregatorFactory; +import org.opensearch.search.aggregations.bucket.histogram.DateHistogramAggregatorFactory; +import org.opensearch.search.aggregations.metrics.MetricAggregatorFactory; +import org.opensearch.search.internal.SearchContext; +import org.opensearch.search.startree.filter.StarTreeFilter; +import org.opensearch.search.startree.filter.provider.StarTreeFilterProvider; +import java.io.IOException; +import java.util.Collections; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; /** - * Query class for querying star tree data structure. - * - * @opensearch.experimental + * Stores the star tree related context of a search request. */ @ExperimentalApi public class StarTreeQueryContext { - /** - * Star tree field info - * This is used to get the star tree data structure - */ - private final CompositeIndexFieldInfo starTree; + private final CompositeDataCubeFieldType compositeMappedFieldType; /** - * Map of field name to a value to be queried for that field - * This is used to filter the data based on the query + * Cache for leaf results + * This is used to cache the results for each leaf reader context + * to avoid reading the filtered values from the leaf reader context multiple times */ - private final Map queryMap; + // TODO : Change caching to be based on aggregation specific filters. + private final FixedBitSet[] perSegmentNodeIdsCache; - /** - * Cache for leaf results - * This is used to cache the results for each leaf reader context - * to avoid reading the filtered values from the leaf reader context multiple times - */ - private final FixedBitSet[] starTreeValues; - - public StarTreeQueryContext(CompositeIndexFieldInfo starTree, Map queryMap, int numSegmentsCache) { - this.starTree = starTree; - this.queryMap = queryMap; - if (numSegmentsCache > -1) { - starTreeValues = new FixedBitSet[numSegmentsCache]; + private final QueryBuilder baseQueryBuilder; + private StarTreeFilter baseStarTreeFilter; + + // TODO : Implement storing and aggregating aggregation specific filters. + + public StarTreeQueryContext(SearchContext context, QueryBuilder baseQueryBuilder) { + this.baseQueryBuilder = baseQueryBuilder; + // TODO : We need to select the most appropriate one from multiple star tree field types. + compositeMappedFieldType = (CompositeDataCubeFieldType) context.mapperService().getCompositeFieldTypes().iterator().next(); + // need to cache star tree values only for multiple aggregations + boolean cacheStarTreeValues = context.aggregations().factories().getFactories().length > 1; + int cacheSize = cacheStarTreeValues ? context.indexShard().segments(false).size() : -1; + if (cacheSize > -1) { + perSegmentNodeIdsCache = new FixedBitSet[cacheSize]; } else { - starTreeValues = null; + perSegmentNodeIdsCache = null; + } + } + + // TODO : Make changes to change visibility into package private. Handle the same in @org.opensearch.search.SearchServiceStarTreeTests + public StarTreeQueryContext(CompositeDataCubeFieldType compositeMappedFieldType, QueryBuilder baseQueryBuilder, int cacheSize) { + this.compositeMappedFieldType = compositeMappedFieldType; + this.baseQueryBuilder = baseQueryBuilder; + if (cacheSize > -1) { + perSegmentNodeIdsCache = new FixedBitSet[cacheSize]; + } else { + perSegmentNodeIdsCache = null; } } public CompositeIndexFieldInfo getStarTree() { - return starTree; + return new CompositeIndexFieldInfo(compositeMappedFieldType.name(), compositeMappedFieldType.getCompositeIndexType()); } - public Map getQueryMap() { - return queryMap; + public FixedBitSet maybeGetCachedNodeIdsForSegment(int ordinal) { + return perSegmentNodeIdsCache != null ? perSegmentNodeIdsCache[ordinal] : null; } - public FixedBitSet[] getStarTreeValues() { - return starTreeValues; + public FixedBitSet[] getAllCachedValues() { + return perSegmentNodeIdsCache; } - public FixedBitSet getStarTreeValues(LeafReaderContext ctx) { - if (starTreeValues != null) { - return starTreeValues[ctx.ord]; + public void maybeSetCachedNodeIdsForSegment(int key, FixedBitSet values) { + if (perSegmentNodeIdsCache != null) { + perSegmentNodeIdsCache[key] = values; } - return null; } - public void setStarTreeValues(LeafReaderContext ctx, FixedBitSet values) { - if (starTreeValues != null) { - starTreeValues[ctx.ord] = values; + /** + * Generates the Base StarTreeFilter and then recursively merges + * any aggregation specific STF. + * @return true if recursively all filters were consolidated, else false. + */ + public boolean consolidateAllFilters(SearchContext context) { + // Validate the fields and metrics required by aggregations are supported in star tree + for (AggregatorFactory aggregatorFactory : context.aggregations().factories().getFactories()) { + // first check for aggregation is a metric aggregation + if (validateStarTreeMetricSupport(compositeMappedFieldType, aggregatorFactory)) { + continue; + } + + // if not a metric aggregation, check for applicable date histogram shape + if (validateDateHistogramSupport(compositeMappedFieldType, aggregatorFactory)) { + continue; + } + return false; + } + + // Generate the base Star Tree Filter + if (baseQueryBuilder != null) { + baseStarTreeFilter = getStarTreeFilter(context, baseQueryBuilder, compositeMappedFieldType); + return baseStarTreeFilter != null; // Base Query is not supported by star tree filter. + } + // TODO : Generate StarTreeFilter specific to aggregations by merging base and their parents. + return true; + } + + public StarTreeFilter getBaseQueryStarTreeFilter() { + if (baseStarTreeFilter == null) { + return new StarTreeFilter(Collections.emptyMap()); } + return baseStarTreeFilter; } + + // TODO : Push this validation down to a common method in AggregatorFactory or an equivalent place. + private static boolean validateStarTreeMetricSupport( + CompositeDataCubeFieldType compositeIndexFieldInfo, + AggregatorFactory aggregatorFactory + ) { + if (aggregatorFactory instanceof MetricAggregatorFactory && aggregatorFactory.getSubFactories().getFactories().length == 0) { + String field; + Map> supportedMetrics = compositeIndexFieldInfo.getMetrics() + .stream() + .collect(Collectors.toMap(Metric::getField, Metric::getMetrics)); + + MetricStat metricStat = ((MetricAggregatorFactory) aggregatorFactory).getMetricStat(); + field = ((MetricAggregatorFactory) aggregatorFactory).getField(); + + return field != null && supportedMetrics.containsKey(field) && supportedMetrics.get(field).contains(metricStat); + } + return false; + } + + private StarTreeFilter getStarTreeFilter( + SearchContext context, + QueryBuilder queryBuilder, + CompositeDataCubeFieldType compositeMappedFieldType + ) { + StarTreeFilterProvider starTreeFilterProvider = StarTreeFilterProvider.SingletonFactory.getProvider(queryBuilder); + // The query builder's support is not implemented. + if (starTreeFilterProvider == null) { + return null; + } + try { + return starTreeFilterProvider.getFilter(context, queryBuilder, compositeMappedFieldType); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private static boolean validateDateHistogramSupport( + CompositeDataCubeFieldType compositeIndexFieldInfo, + AggregatorFactory aggregatorFactory + ) { + if (!(aggregatorFactory instanceof DateHistogramAggregatorFactory dateHistogramAggregatorFactory) + || aggregatorFactory.getSubFactories().getFactories().length < 1) { + return false; + } + + // Find the DateDimension in the dimensions list + DateDimension starTreeDateDimension = null; + for (Dimension dimension : compositeIndexFieldInfo.getDimensions()) { + if (dimension instanceof DateDimension) { + starTreeDateDimension = (DateDimension) dimension; + break; + } + } + + // If no DateDimension is found, validation fails + if (starTreeDateDimension == null) { + return false; + } + + // Ensure the rounding is not null + if (dateHistogramAggregatorFactory.getRounding() == null) { + return false; + } + + // Find the closest valid interval in the DateTimeUnitRounding class associated with star tree + DateTimeUnitRounding rounding = starTreeDateDimension.findClosestValidInterval( + new DateTimeUnitAdapter(dateHistogramAggregatorFactory.getRounding()) + ); + if (rounding == null) { + return false; + } + + // Validate all sub-factories + for (AggregatorFactory subFactory : aggregatorFactory.getSubFactories().getFactories()) { + if (!validateStarTreeMetricSupport(compositeIndexFieldInfo, subFactory)) { + return false; + } + } + return true; + } + } diff --git a/server/src/main/java/org/opensearch/search/startree/StarTreeQueryHelper.java b/server/src/main/java/org/opensearch/search/startree/StarTreeQueryHelper.java new file mode 100644 index 0000000000000..edbccb53853d5 --- /dev/null +++ b/server/src/main/java/org/opensearch/search/startree/StarTreeQueryHelper.java @@ -0,0 +1,203 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.startree; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SegmentReader; +import org.apache.lucene.search.CollectionTerminatedException; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.util.FixedBitSet; +import org.opensearch.common.lucene.Lucene; +import org.opensearch.index.codec.composite.CompositeIndexFieldInfo; +import org.opensearch.index.codec.composite.CompositeIndexReader; +import org.opensearch.index.compositeindex.datacube.Dimension; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; +import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils; +import org.opensearch.index.compositeindex.datacube.startree.utils.iterator.SortedNumericStarTreeValuesIterator; +import org.opensearch.index.query.QueryShardContext; +import org.opensearch.search.aggregations.LeafBucketCollector; +import org.opensearch.search.aggregations.StarTreeBucketCollector; +import org.opensearch.search.aggregations.support.ValuesSource; +import org.opensearch.search.internal.SearchContext; +import org.opensearch.search.startree.filter.DimensionFilter; +import org.opensearch.search.startree.filter.StarTreeFilter; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.BiConsumer; +import java.util.function.Consumer; + +/** + * Helper class for building star-tree query + * + * @opensearch.internal + * @opensearch.experimental + */ +public class StarTreeQueryHelper { + + private static StarTreeValues starTreeValues; + + /** + * Checks if the search context can be supported by star-tree + */ + public static boolean isStarTreeSupported(SearchContext context) { + return context.aggregations() != null && context.mapperService().isCompositeIndexPresent() && context.parsedPostFilter() == null; + } + + public static CompositeIndexFieldInfo getSupportedStarTree(QueryShardContext context) { + StarTreeQueryContext starTreeQueryContext = context.getStarTreeQueryContext(); + return (starTreeQueryContext != null) ? starTreeQueryContext.getStarTree() : null; + } + + public static StarTreeValues getStarTreeValues(LeafReaderContext context, CompositeIndexFieldInfo starTree) throws IOException { + SegmentReader reader = Lucene.segmentReader(context.reader()); + if (!(reader.getDocValuesReader() instanceof CompositeIndexReader)) { + return null; + } + CompositeIndexReader starTreeDocValuesReader = (CompositeIndexReader) reader.getDocValuesReader(); + return (StarTreeValues) starTreeDocValuesReader.getCompositeIndexValues(starTree); + } + + /** + * Get the star-tree leaf collector + * This collector computes the aggregation prematurely and invokes an early termination collector + */ + public static void getStarTreeLeafCollector( + SearchContext context, + ValuesSource.Numeric valuesSource, + LeafReaderContext ctx, + LeafBucketCollector sub, + CompositeIndexFieldInfo starTree, + String metric, + Consumer valueConsumer, + Runnable finalConsumer + ) throws IOException { + StarTreeValues starTreeValues = getStarTreeValues(ctx, starTree); + assert starTreeValues != null; + String fieldName = ((ValuesSource.Numeric.FieldData) valuesSource).getIndexFieldName(); + String metricName = StarTreeUtils.fullyQualifiedFieldNameForStarTreeMetricsDocValues(starTree.getField(), fieldName, metric); + + SortedNumericStarTreeValuesIterator valuesIterator = (SortedNumericStarTreeValuesIterator) starTreeValues.getMetricValuesIterator( + metricName + ); + // Obtain a FixedBitSet of matched star tree document IDs + FixedBitSet filteredValues = getStarTreeFilteredValues(context, ctx, starTreeValues); + + int numBits = filteredValues.length(); // Get the number of the filtered values (matching docs) + if (numBits > 0) { + // Iterate over the filtered values + for (int bit = filteredValues.nextSetBit(0); bit != DocIdSetIterator.NO_MORE_DOCS; bit = (bit + 1 < numBits) + ? filteredValues.nextSetBit(bit + 1) + : DocIdSetIterator.NO_MORE_DOCS) { + // Advance to the entryId in the valuesIterator + if (valuesIterator.advanceExact(bit) == false) { + continue; // Skip if no more entries + } + + // Iterate over the values for the current entryId + for (int i = 0, count = valuesIterator.entryValueCount(); i < count; i++) { + long value = valuesIterator.nextValue(); + valueConsumer.accept(value); // Apply the consumer operation (e.g., max, sum) + } + } + } + + // Call the final consumer after processing all entries + finalConsumer.run(); + + // FIXME : Remove after @msfroh PR for precompute + // Terminate after pre-computing aggregation + throw new CollectionTerminatedException(); + } + + /** + * Get the filtered values for the star-tree query + * Cache the results in case of multiple aggregations (if cache is initialized) + * @return FixedBitSet of matched document IDs + */ + public static FixedBitSet getStarTreeFilteredValues(SearchContext context, LeafReaderContext ctx, StarTreeValues starTreeValues) + throws IOException { + FixedBitSet result = context.getQueryShardContext().getStarTreeQueryContext().maybeGetCachedNodeIdsForSegment(ctx.ord); + if (result == null) { + result = StarTreeTraversalUtil.getStarTreeResult( + starTreeValues, + context.getQueryShardContext().getStarTreeQueryContext().getBaseQueryStarTreeFilter(), + context + ); + } + context.getQueryShardContext().getStarTreeQueryContext().maybeSetCachedNodeIdsForSegment(ctx.ord, result); + return result; + } + + public static Dimension getMatchingDimensionOrThrow(String dimensionName, List orderedDimensions) { + Dimension matchingDimension = getMatchingDimensionOrNull(dimensionName, orderedDimensions); + if (matchingDimension == null) { + throw new IllegalStateException("No matching dimension found for [" + dimensionName + "]"); + } + return matchingDimension; + } + + public static Dimension getMatchingDimensionOrNull(String dimensionName, List orderedDimensions) { + List matchingDimensions = orderedDimensions.stream().filter(x -> x.getField().equals(dimensionName)).toList(); + if (matchingDimensions.size() != 1) { + return null; + } + return matchingDimensions.get(0); + } + + public static StarTreeBucketCollector getStarTreeBucketMetricCollector( + CompositeIndexFieldInfo starTree, + String metric, + ValuesSource.Numeric valuesSource, + StarTreeBucketCollector parentCollector, + Consumer growArrays, + BiConsumer updateBucket + ) throws IOException { + assert parentCollector != null; + return new StarTreeBucketCollector(parentCollector) { + String metricName = StarTreeUtils.fullyQualifiedFieldNameForStarTreeMetricsDocValues( + starTree.getField(), + ((ValuesSource.Numeric.FieldData) valuesSource).getIndexFieldName(), + metric + ); + SortedNumericStarTreeValuesIterator metricValuesIterator = (SortedNumericStarTreeValuesIterator) starTreeValues + .getMetricValuesIterator(metricName); + + @Override + public void collectStarTreeEntry(int starTreeEntryBit, long bucket) throws IOException { + growArrays.accept(bucket); + // Advance the valuesIterator to the current bit + if (!metricValuesIterator.advanceExact(starTreeEntryBit)) { + return; // Skip if no entries for this document + } + long metricValue = metricValuesIterator.nextValue(); + updateBucket.accept(bucket, metricValue); + } + }; + } + + public static StarTreeFilter mergeDimensionFilterIfNotExists( + StarTreeFilter baseStarTreeFilter, + String dimensionToMerge, + List dimensionFiltersToMerge + ) { + Map> dimensionFilterMap = new HashMap<>(baseStarTreeFilter.getDimensions().size()); + for (String baseDimension : baseStarTreeFilter.getDimensions()) { + dimensionFilterMap.put(baseDimension, baseStarTreeFilter.getFiltersForDimension(baseDimension)); + } + // Don't add groupBy when already present in base filter. + if (!dimensionFilterMap.containsKey(dimensionToMerge)) { + dimensionFilterMap.put(dimensionToMerge, dimensionFiltersToMerge); + } + return new StarTreeFilter(dimensionFilterMap); + } + +} diff --git a/server/src/main/java/org/opensearch/search/startree/StarTreeFilter.java b/server/src/main/java/org/opensearch/search/startree/StarTreeTraversalUtil.java similarity index 76% rename from server/src/main/java/org/opensearch/search/startree/StarTreeFilter.java rename to server/src/main/java/org/opensearch/search/startree/StarTreeTraversalUtil.java index 1629b9d0c1db4..cf9c125e84b79 100644 --- a/server/src/main/java/org/opensearch/search/startree/StarTreeFilter.java +++ b/server/src/main/java/org/opensearch/search/startree/StarTreeTraversalUtil.java @@ -19,6 +19,9 @@ import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNodeType; import org.opensearch.index.compositeindex.datacube.startree.utils.iterator.SortedNumericStarTreeValuesIterator; import org.opensearch.index.compositeindex.datacube.startree.utils.iterator.StarTreeValuesIterator; +import org.opensearch.search.internal.SearchContext; +import org.opensearch.search.startree.filter.DimensionFilter; +import org.opensearch.search.startree.filter.StarTreeFilter; import java.io.IOException; import java.util.ArrayDeque; @@ -27,7 +30,6 @@ import java.util.HashSet; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.Queue; import java.util.Set; import java.util.stream.Collectors; @@ -40,20 +42,24 @@ * @opensearch.experimental * @opensearch.internal */ -public class StarTreeFilter { - private static final Logger logger = LogManager.getLogger(StarTreeFilter.class); +public class StarTreeTraversalUtil { + private static final Logger logger = LogManager.getLogger(StarTreeTraversalUtil.class); /** - * First go over the star tree and try to match as many dimensions as possible - * For the remaining columns, use star-tree doc values to match them + * First go over the star tree and try to match as many dimensions as possible + * For the remaining columns, use star-tree doc values to match them */ - public static FixedBitSet getStarTreeResult( - StarTreeValues starTreeValues, - Map predicateEvaluators, - Set groupByField - ) throws IOException { - Map queryMap = predicateEvaluators != null ? predicateEvaluators : Collections.emptyMap(); - StarTreeResult starTreeResult = traverseStarTree(starTreeValues, queryMap, groupByField); + public static FixedBitSet getStarTreeResult(StarTreeValues starTreeValues, StarTreeFilter starTreeFilter, SearchContext searchContext) + throws IOException { + + // Initialising all dimension filters for this segment + for (String dimension : starTreeFilter.getDimensions()) { + for (DimensionFilter dimensionFilter : starTreeFilter.getFiltersForDimension(dimension)) { + dimensionFilter.initialiseForSegment(starTreeValues, searchContext); + } + } + + StarTreeResult starTreeResult = traverseStarTree(starTreeValues, starTreeFilter); // Initialize FixedBitSet with size maxMatchedDoc + 1 FixedBitSet bitSet = new FixedBitSet(starTreeResult.maxMatchedDoc + 1); @@ -78,11 +84,9 @@ public static FixedBitSet getStarTreeResult( for (String remainingPredicateColumn : starTreeResult.remainingPredicateColumns) { logger.debug("remainingPredicateColumn : {}, maxMatchedDoc : {} ", remainingPredicateColumn, starTreeResult.maxMatchedDoc); - SortedNumericStarTreeValuesIterator ndv = (SortedNumericStarTreeValuesIterator) starTreeValues.getDimensionValuesIterator( - remainingPredicateColumn - ); - - long queryValue = queryMap.get(remainingPredicateColumn); // Get the query value directly + StarTreeValuesIterator valuesIterator = starTreeValues.getDimensionValuesIterator(remainingPredicateColumn); + // Get the query value directly + List dimensionFilters = starTreeFilter.getFiltersForDimension(remainingPredicateColumn); // Clear the temporary bit set before reuse tempBitSet.clear(0, starTreeResult.maxMatchedDoc + 1); @@ -92,14 +96,12 @@ public static FixedBitSet getStarTreeResult( for (int entryId = bitSet.nextSetBit(0); entryId != DocIdSetIterator.NO_MORE_DOCS; entryId = (entryId + 1 < bitSet.length()) ? bitSet.nextSetBit(entryId + 1) : DocIdSetIterator.NO_MORE_DOCS) { - if (ndv.advance(entryId) != StarTreeValuesIterator.NO_MORE_ENTRIES) { - final int valuesCount = ndv.entryValueCount(); - for (int i = 0; i < valuesCount; i++) { - long value = ndv.nextValue(); - // Compare the value with the query value - if (value == queryValue) { - tempBitSet.set(entryId); // Set bit for the matching entryId - break; // No need to check other values for this entryId + if (valuesIterator.advanceExact(entryId)) { + long value = valuesIterator.value(); + for (DimensionFilter dimensionFilter : dimensionFilters) { + if (dimensionFilter.matchDimValue(value, starTreeValues)) { + tempBitSet.set(entryId);// Set bit for the matching entryId + break; } } } @@ -117,8 +119,7 @@ public static FixedBitSet getStarTreeResult( * Helper method to traverse the star tree, get matching documents and keep track of all the * predicate dimensions that are not matched. */ - private static StarTreeResult traverseStarTree(StarTreeValues starTreeValues, Map queryMap, Set groupbyField) - throws IOException { + private static StarTreeResult traverseStarTree(StarTreeValues starTreeValues, StarTreeFilter starTreeFilter) throws IOException { DocIdSetBuilder docsWithField = new DocIdSetBuilder(starTreeValues.getStarTreeDocumentCount()); DocIdSetBuilder.BulkAdder adder; Set globalRemainingPredicateColumns = null; @@ -130,8 +131,7 @@ private static StarTreeResult traverseStarTree(StarTreeValues starTreeValues, Ma Queue queue = new ArrayDeque<>(); queue.add(starTree); int currentDimensionId = -1; - Set remainingPredicateColumns = new HashSet<>(queryMap.keySet()); - Set remainingGroupByColumns = new HashSet<>(groupbyField); + Set remainingPredicateColumns = new HashSet<>(starTreeFilter.getDimensions()); int matchedDocsCountInStarTree = 0; int maxDocNum = -1; StarTreeNode starTreeNode; @@ -142,14 +142,13 @@ private static StarTreeResult traverseStarTree(StarTreeValues starTreeValues, Ma if (dimensionId > currentDimensionId) { String dimension = dimensionNames.get(dimensionId); remainingPredicateColumns.remove(dimension); - remainingGroupByColumns.remove(dimension); if (foundLeafNode && globalRemainingPredicateColumns == null) { globalRemainingPredicateColumns = new HashSet<>(remainingPredicateColumns); } currentDimensionId = dimensionId; } - if (remainingPredicateColumns.isEmpty() && remainingGroupByColumns.isEmpty()) { + if (remainingPredicateColumns.isEmpty()) { int docId = starTreeNode.getAggregatedDocId(); docIds.add(docId); matchedDocsCountInStarTree++; @@ -168,18 +167,20 @@ private static StarTreeResult traverseStarTree(StarTreeValues starTreeValues, Ma String childDimension = dimensionNames.get(dimensionId + 1); StarTreeNode starNode = null; - if (((globalRemainingPredicateColumns == null || !globalRemainingPredicateColumns.contains(childDimension)) - && !remainingGroupByColumns.contains(childDimension))) { + if (globalRemainingPredicateColumns == null || !globalRemainingPredicateColumns.contains(childDimension)) { starNode = starTreeNode.getChildStarNode(); } if (remainingPredicateColumns.contains(childDimension)) { - long queryValue = queryMap.get(childDimension); // Get the query value directly from the map - StarTreeNode matchingChild = starTreeNode.getChildForDimensionValue(queryValue); - if (matchingChild != null) { - queue.add(matchingChild); - foundLeafNode |= matchingChild.isLeaf(); + List dimensionFilters = starTreeFilter.getFiltersForDimension(childDimension); + final boolean[] tempFoundLeafNodes = new boolean[1]; + for (DimensionFilter dimensionFilter : dimensionFilters) { + dimensionFilter.matchStarTreeNodes(starTreeNode, starTreeValues, node -> { + queue.add(node); + tempFoundLeafNodes[0] |= node.isLeaf(); + }); } + foundLeafNode |= tempFoundLeafNodes[0]; } else { if (starNode != null) { queue.add(starNode); diff --git a/server/src/main/java/org/opensearch/search/startree/filter/DimensionFilter.java b/server/src/main/java/org/opensearch/search/startree/filter/DimensionFilter.java new file mode 100644 index 0000000000000..64f971a58f216 --- /dev/null +++ b/server/src/main/java/org/opensearch/search/startree/filter/DimensionFilter.java @@ -0,0 +1,89 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.startree.filter; + +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNode; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNodeType; +import org.opensearch.search.internal.SearchContext; +import org.opensearch.search.startree.StarTreeNodeCollector; + +import java.io.IOException; +import java.util.Iterator; + +/** + * Contains the logic to filter over a dimension either in StarTree Index or it's Dimension DocValues + */ +@ExperimentalApi +public interface DimensionFilter { + + DimensionFilter MATCH_ALL_DEFAULT = new DimensionFilter() { + @Override + public void initialiseForSegment(StarTreeValues starTreeValues, SearchContext searchContext) throws IOException { + + } + + @Override + public void matchStarTreeNodes(StarTreeNode parentNode, StarTreeValues starTreeValues, StarTreeNodeCollector collector) + throws IOException { + if (parentNode != null) { + for (Iterator it = parentNode.getChildrenIterator(); it.hasNext();) { + StarTreeNode starTreeNode = it.next(); + if (starTreeNode.getStarTreeNodeType() == StarTreeNodeType.DEFAULT.getValue()) { + collector.collectStarTreeNode(starTreeNode); + } + } + } + } + + @Override + public boolean matchDimValue(long ordinal, StarTreeValues starTreeValues) { + return true; + } + }; + + /** + * Converts parsed user values to ordinals based on segment and other init actions can be performed. + * @param starTreeValues : Segment specific star tree root node and other metadata + * @param searchContext : Search context + * @throws IOException : + */ + void initialiseForSegment(StarTreeValues starTreeValues, SearchContext searchContext) throws IOException; + + /** + * Called when matching a dimension values in the star tree index. + * @param parentNode : StarTreeNode below which the dimension to be filtered is present. + * @param starTreeValues : Segment specific star tree root node and other metadata + * @param collector : Collector which collates the matched StarTreeNode's + * @throws IOException : + */ + void matchStarTreeNodes(StarTreeNode parentNode, StarTreeValues starTreeValues, StarTreeNodeCollector collector) throws IOException; + + /** + * Called when a dimension is not found in star tree index and needs to matched by iterating its docValues + * @param ordinal : Value to Match + * @param starTreeValues : Segment specific star tree root node and other metadata + * @return : true if matches, else false. + */ + boolean matchDimValue(long ordinal, StarTreeValues starTreeValues); + + /** + * Represents how to match a value when comparing during StarTreeTraversal + */ + @ExperimentalApi + enum MatchType { + GT, + LT, + GTE, + LTE, + EXACT + } + +} diff --git a/server/src/main/java/org/opensearch/search/startree/filter/ExactMatchDimFilter.java b/server/src/main/java/org/opensearch/search/startree/filter/ExactMatchDimFilter.java new file mode 100644 index 0000000000000..28ea261ca1e56 --- /dev/null +++ b/server/src/main/java/org/opensearch/search/startree/filter/ExactMatchDimFilter.java @@ -0,0 +1,84 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.startree.filter; + +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.compositeindex.datacube.Dimension; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNode; +import org.opensearch.search.internal.SearchContext; +import org.opensearch.search.startree.StarTreeNodeCollector; +import org.opensearch.search.startree.StarTreeQueryHelper; +import org.opensearch.search.startree.filter.provider.DimensionFilterMapper; + +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import java.util.TreeSet; + +/** + * Handles Term and Terms query like search in StarTree Dimension filtering. + */ +@ExperimentalApi +public class ExactMatchDimFilter implements DimensionFilter { + + private final String dimensionName; + + private final List rawValues; + + // Order is essential for successive binary search + private TreeSet convertedOrdinals; + + public ExactMatchDimFilter(String dimensionName, List valuesToMatch) { + this.dimensionName = dimensionName; + this.rawValues = valuesToMatch; + } + + @Override + public void initialiseForSegment(StarTreeValues starTreeValues, SearchContext searchContext) { + convertedOrdinals = new TreeSet<>(); + Dimension matchedDim = StarTreeQueryHelper.getMatchingDimensionOrThrow( + dimensionName, + starTreeValues.getStarTreeField().getDimensionsOrder() + ); + DimensionFilterMapper dimensionFilterMapper = DimensionFilterMapper.Factory.fromMappedFieldType( + searchContext.mapperService().fieldType(dimensionName) + ); + for (Object rawValue : rawValues) { + Optional ordinal = dimensionFilterMapper.getMatchingOrdinal( + matchedDim.getField(), + rawValue, + starTreeValues, + MatchType.EXACT + ); + // Numeric type returning negative ordinal ( same as their value ) is valid + // Whereas Keyword type returning -ve ordinal indicates it doesn't exist in Star Tree Dimension values. + ordinal.ifPresent(aLong -> convertedOrdinals.add(aLong)); + } + } + + @Override + public void matchStarTreeNodes(StarTreeNode parentNode, StarTreeValues starTreeValues, StarTreeNodeCollector collector) + throws IOException { + if (parentNode != null) { + StarTreeNode lastMatchedNode = null; + for (long ordinal : convertedOrdinals) { + lastMatchedNode = parentNode.getChildForDimensionValue(ordinal, lastMatchedNode); + if (lastMatchedNode != null) { + collector.collectStarTreeNode(lastMatchedNode); + } + } + } + } + + @Override + public boolean matchDimValue(long ordinal, StarTreeValues starTreeValues) { + return convertedOrdinals.contains(ordinal); + } +} diff --git a/server/src/main/java/org/opensearch/search/startree/filter/MatchNoneFilter.java b/server/src/main/java/org/opensearch/search/startree/filter/MatchNoneFilter.java new file mode 100644 index 0000000000000..3066b4d7a8a3f --- /dev/null +++ b/server/src/main/java/org/opensearch/search/startree/filter/MatchNoneFilter.java @@ -0,0 +1,36 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.startree.filter; + +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNode; +import org.opensearch.search.internal.SearchContext; +import org.opensearch.search.startree.StarTreeNodeCollector; + +/** + * Filter which matches no StarTreeNodes. + */ +@ExperimentalApi +public class MatchNoneFilter implements DimensionFilter { + @Override + public void initialiseForSegment(StarTreeValues starTreeValues, SearchContext searchContext) { + // Nothing to do as we won't match anything. + } + + @Override + public void matchStarTreeNodes(StarTreeNode parentNode, StarTreeValues starTreeValues, StarTreeNodeCollector collector) { + // Don't match any star tree node. + } + + @Override + public boolean matchDimValue(long ordinal, StarTreeValues starTreeValues) { + return false; + } +} diff --git a/server/src/main/java/org/opensearch/search/startree/filter/RangeMatchDimFilter.java b/server/src/main/java/org/opensearch/search/startree/filter/RangeMatchDimFilter.java new file mode 100644 index 0000000000000..fecf1a9ebf76b --- /dev/null +++ b/server/src/main/java/org/opensearch/search/startree/filter/RangeMatchDimFilter.java @@ -0,0 +1,89 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.startree.filter; + +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNode; +import org.opensearch.search.internal.SearchContext; +import org.opensearch.search.startree.StarTreeNodeCollector; +import org.opensearch.search.startree.filter.provider.DimensionFilterMapper; + +import java.io.IOException; +import java.util.Optional; + +/** + * Performs range match based on the params of @{@link org.opensearch.index.query.RangeQueryBuilder} + * Also, contains logic to skip performing range search if it's sure that it won't be found in Star Tree. + */ +@ExperimentalApi +public class RangeMatchDimFilter implements DimensionFilter { + + private final String dimensionName; + + private final Object low; + private final Object high; + private final boolean includeLow; + private final boolean includeHigh; + + private Long lowOrdinal; + private Long highOrdinal; + + private boolean skipRangeCollection = false; + + public RangeMatchDimFilter(String dimensionName, Object low, Object high, boolean includeLow, boolean includeHigh) { + this.dimensionName = dimensionName; + this.low = low; + this.high = high; + this.includeLow = includeLow; + this.includeHigh = includeHigh; + } + + @Override + public void initialiseForSegment(StarTreeValues starTreeValues, SearchContext searchContext) { + skipRangeCollection = false; + DimensionFilterMapper dimensionFilterMapper = DimensionFilterMapper.Factory.fromMappedFieldType( + searchContext.mapperService().fieldType(dimensionName) + ); + lowOrdinal = 0L; + if (low != null) { + MatchType lowMatchType = includeLow ? MatchType.GTE : MatchType.GT; + Optional lowOrdinalFound = dimensionFilterMapper.getMatchingOrdinal(dimensionName, low, starTreeValues, lowMatchType); + if (lowOrdinalFound.isPresent()) { + lowOrdinal = lowOrdinalFound.get(); + } else { + // This is only valid for Non-numeric fields. + // High can't be found since nothing >= low exists. + lowOrdinal = highOrdinal = Long.MAX_VALUE; + skipRangeCollection = true; + return; + } + } + highOrdinal = Long.MAX_VALUE; + if (high != null) { + MatchType highMatchType = includeHigh ? MatchType.LTE : MatchType.LT; + Optional highOrdinalFound = dimensionFilterMapper.getMatchingOrdinal(dimensionName, high, starTreeValues, highMatchType); + highOrdinalFound.ifPresent(ord -> highOrdinal = ord); + } + } + + @Override + public void matchStarTreeNodes(StarTreeNode parentNode, StarTreeValues starTreeValues, StarTreeNodeCollector collector) + throws IOException { + if (parentNode != null && !skipRangeCollection) { + parentNode.collectChildrenInRange(lowOrdinal, highOrdinal, collector); + } + } + + @Override + public boolean matchDimValue(long ordinal, StarTreeValues starTreeValues) { + return lowOrdinal <= ordinal && ordinal <= highOrdinal; + } + +} diff --git a/server/src/main/java/org/opensearch/search/startree/filter/StarTreeFilter.java b/server/src/main/java/org/opensearch/search/startree/filter/StarTreeFilter.java new file mode 100644 index 0000000000000..38a1f092adc6f --- /dev/null +++ b/server/src/main/java/org/opensearch/search/startree/filter/StarTreeFilter.java @@ -0,0 +1,42 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.startree.filter; + +import org.opensearch.common.annotation.ExperimentalApi; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Container for intermediate/consolidated dimension filters that will be applied for a query in star tree traversal. + */ +@ExperimentalApi +public class StarTreeFilter { + + private final Map> dimensionFilterMap; + + public StarTreeFilter(Map> dimensionFilterMap) { + this.dimensionFilterMap = dimensionFilterMap; + } + + public List getFiltersForDimension(String dimension) { + return dimensionFilterMap.get(dimension); + } + + public Set getDimensions() { + return dimensionFilterMap.keySet(); + } + // TODO : Implement Merging of 2 Star Tree Filters + // This would also involve merging 2 different types of dimension filters. + // It also brings in the challenge of sorting input values in user query for efficient merging. + // Merging Range with Term and Range with Range and so on. + // All these will be implemented post OS 2.19 + +} diff --git a/server/src/main/java/org/opensearch/search/startree/filter/package-info.java b/server/src/main/java/org/opensearch/search/startree/filter/package-info.java new file mode 100644 index 0000000000000..565996d9bfb21 --- /dev/null +++ b/server/src/main/java/org/opensearch/search/startree/filter/package-info.java @@ -0,0 +1,10 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/** Star Tree Dimension Filters */ +package org.opensearch.search.startree.filter; diff --git a/server/src/main/java/org/opensearch/search/startree/filter/provider/DimensionFilterMapper.java b/server/src/main/java/org/opensearch/search/startree/filter/provider/DimensionFilterMapper.java new file mode 100644 index 0000000000000..8afdb00864b22 --- /dev/null +++ b/server/src/main/java/org/opensearch/search/startree/filter/provider/DimensionFilterMapper.java @@ -0,0 +1,410 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.startree.filter.provider; + +import org.apache.lucene.document.DoublePoint; +import org.apache.lucene.document.FloatPoint; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.sandbox.document.HalfFloatPoint; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.NumericUtils; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.lucene.BytesRefs; +import org.opensearch.common.lucene.Lucene; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; +import org.opensearch.index.compositeindex.datacube.startree.utils.iterator.SortedSetStarTreeValuesIterator; +import org.opensearch.index.mapper.KeywordFieldMapper.KeywordFieldType; +import org.opensearch.index.mapper.MappedFieldType; +import org.opensearch.index.mapper.NumberFieldMapper.NumberFieldType; +import org.opensearch.search.startree.filter.DimensionFilter; +import org.opensearch.search.startree.filter.ExactMatchDimFilter; +import org.opensearch.search.startree.filter.MatchNoneFilter; +import org.opensearch.search.startree.filter.RangeMatchDimFilter; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.opensearch.index.mapper.NumberFieldMapper.NumberType.BYTE; +import static org.opensearch.index.mapper.NumberFieldMapper.NumberType.DOUBLE; +import static org.opensearch.index.mapper.NumberFieldMapper.NumberType.FLOAT; +import static org.opensearch.index.mapper.NumberFieldMapper.NumberType.HALF_FLOAT; +import static org.opensearch.index.mapper.NumberFieldMapper.NumberType.INTEGER; +import static org.opensearch.index.mapper.NumberFieldMapper.NumberType.LONG; +import static org.opensearch.index.mapper.NumberFieldMapper.NumberType.SHORT; +import static org.opensearch.index.mapper.NumberFieldMapper.NumberType.hasDecimalPart; +import static org.opensearch.index.mapper.NumberFieldMapper.NumberType.signum; + +/** + * Generates the @{@link DimensionFilter} raw values and the @{@link MappedFieldType} of the dimension. + */ +@ExperimentalApi +public interface DimensionFilterMapper { + /** + * Generates @{@link ExactMatchDimFilter} from Term/Terms query input. + * @param mappedFieldType: + * @param rawValues: + * @return : + */ + DimensionFilter getExactMatchFilter(MappedFieldType mappedFieldType, List rawValues); + + /** + * Generates @{@link RangeMatchDimFilter} from Range query input. + * @param mappedFieldType: + * @param rawLow: + * @param rawHigh: + * @param includeLow: + * @param includeHigh: + * @return : + */ + DimensionFilter getRangeMatchFilter( + MappedFieldType mappedFieldType, + Object rawLow, + Object rawHigh, + boolean includeLow, + boolean includeHigh + ); + + /** + * Called during conversion from parsedUserInput to segmentOrdinal for every segment. + * @param dimensionName: + * @param value: + * @param starTreeValues: + * @param matchType: + * @return : + */ + Optional getMatchingOrdinal( + String dimensionName, + Object value, + StarTreeValues starTreeValues, + DimensionFilter.MatchType matchType + ); + + /** + * Singleton Factory for @{@link DimensionFilterMapper} + */ + class Factory { + + private static final Map DIMENSION_FILTER_MAPPINGS = Map.of( + BYTE.typeName(), + new IntegerFieldMapperNumeric(), + SHORT.typeName(), + new IntegerFieldMapperNumeric(), + INTEGER.typeName(), + new IntegerFieldMapperNumeric(), + LONG.typeName(), + new SignedLongFieldMapperNumeric(), + HALF_FLOAT.typeName(), + new HalfFloatFieldMapperNumeric(), + FLOAT.typeName(), + new FloatFieldMapperNumeric(), + DOUBLE.typeName(), + new DoubleFieldMapperNumeric(), + org.opensearch.index.mapper.KeywordFieldMapper.CONTENT_TYPE, + new KeywordFieldMapper() + ); + + public static DimensionFilterMapper fromMappedFieldType(MappedFieldType mappedFieldType) { + if (mappedFieldType != null) { + return DIMENSION_FILTER_MAPPINGS.get(mappedFieldType.typeName()); + } + return null; + } + } + +} + +abstract class NumericMapper implements DimensionFilterMapper { + + @Override + public Optional getMatchingOrdinal( + String dimensionName, + Object value, + StarTreeValues starTreeValues, + DimensionFilter.MatchType matchType + ) { + // Casting to long ensures that all numeric fields have been converted to equivalent long at request parsing time. + return Optional.of((long) value); + } +} + +abstract class NumericNonDecimalMapper extends NumericMapper { + + @Override + public DimensionFilter getExactMatchFilter(MappedFieldType mappedFieldType, List rawValues) { + NumberFieldType numberFieldType = (NumberFieldType) mappedFieldType; + List convertedValues = new ArrayList<>(rawValues.size()); + for (Object rawValue : rawValues) { + convertedValues.add(numberFieldType.numberType().parse(rawValue, true).longValue()); + } + return new ExactMatchDimFilter(mappedFieldType.name(), convertedValues); + } + + @Override + public DimensionFilter getRangeMatchFilter( + MappedFieldType mappedFieldType, + Object rawLow, + Object rawHigh, + boolean includeLow, + boolean includeHigh + ) { + NumberFieldType numberFieldType = (NumberFieldType) mappedFieldType; + + Long parsedLow = rawLow == null ? defaultMinimum() : numberFieldType.numberType().parse(rawLow, true).longValue(); + Long parsedHigh = rawHigh == null ? defaultMaximum() : numberFieldType.numberType().parse(rawHigh, true).longValue(); + + boolean lowerTermHasDecimalPart = hasDecimalPart(parsedLow); + if ((lowerTermHasDecimalPart == false && includeLow == false) || (lowerTermHasDecimalPart && signum(parsedLow) > 0)) { + if (parsedLow.equals(defaultMaximum())) { + return new MatchNoneFilter(); + } + ++parsedLow; + } + boolean upperTermHasDecimalPart = hasDecimalPart(parsedHigh); + if ((upperTermHasDecimalPart == false && includeHigh == false) || (upperTermHasDecimalPart && signum(parsedHigh) < 0)) { + if (parsedHigh.equals(defaultMinimum())) { + return new MatchNoneFilter(); + } + --parsedHigh; + } + return new RangeMatchDimFilter(mappedFieldType.name(), parsedLow, parsedHigh, true, true); + } + + abstract Long defaultMinimum(); + + abstract Long defaultMaximum(); + +} + +class IntegerFieldMapperNumeric extends NumericNonDecimalMapper { + @Override + Long defaultMinimum() { + return (long) Integer.MIN_VALUE; + } + + @Override + Long defaultMaximum() { + return (long) Integer.MAX_VALUE; + } +} + +class SignedLongFieldMapperNumeric extends NumericNonDecimalMapper { + @Override + Long defaultMinimum() { + return Long.MIN_VALUE; + } + + @Override + Long defaultMaximum() { + return Long.MAX_VALUE; + } +} + +abstract class NumericDecimalFieldMapper extends NumericMapper { + + @Override + public DimensionFilter getExactMatchFilter(MappedFieldType mappedFieldType, List rawValues) { + NumberFieldType numberFieldType = (NumberFieldType) mappedFieldType; + List convertedValues = new ArrayList<>(rawValues.size()); + for (Object rawValue : rawValues) { + convertedValues.add(convertToDocValues(numberFieldType.numberType().parse(rawValue, true))); + } + return new ExactMatchDimFilter(mappedFieldType.name(), convertedValues); + } + + @Override + public DimensionFilter getRangeMatchFilter( + MappedFieldType mappedFieldType, + Object rawLow, + Object rawHigh, + boolean includeLow, + boolean includeHigh + ) { + NumberFieldType numberFieldType = (NumberFieldType) mappedFieldType; + Number l = Long.MIN_VALUE; + Number u = Long.MAX_VALUE; + if (rawLow != null) { + l = numberFieldType.numberType().parse(rawLow, false); + if (includeLow == false) { + l = getNextHigh(l); + } + l = convertToDocValues(l); + } + if (rawHigh != null) { + u = numberFieldType.numberType().parse(rawHigh, false); + if (includeHigh == false) { + u = getNextLow(u); + } + u = convertToDocValues(u); + } + return new RangeMatchDimFilter(numberFieldType.name(), l, u, true, true); + } + + abstract long convertToDocValues(Number parsedValue); + + abstract Number getNextLow(Number parsedValue); + + abstract Number getNextHigh(Number parsedValue); + +} + +class HalfFloatFieldMapperNumeric extends NumericDecimalFieldMapper { + @Override + long convertToDocValues(Number parsedValue) { + return HalfFloatPoint.halfFloatToSortableShort((Float) parsedValue); + } + + @Override + Number getNextLow(Number parsedValue) { + return HalfFloatPoint.nextDown((Float) parsedValue); + } + + @Override + Number getNextHigh(Number parsedValue) { + return HalfFloatPoint.nextUp((Float) parsedValue); + } +} + +class FloatFieldMapperNumeric extends NumericDecimalFieldMapper { + @Override + long convertToDocValues(Number parsedValue) { + return NumericUtils.floatToSortableInt((Float) parsedValue); + } + + @Override + Number getNextLow(Number parsedValue) { + return FloatPoint.nextDown((Float) parsedValue); + } + + @Override + Number getNextHigh(Number parsedValue) { + return FloatPoint.nextUp((Float) parsedValue); + } +} + +class DoubleFieldMapperNumeric extends NumericDecimalFieldMapper { + @Override + long convertToDocValues(Number parsedValue) { + return NumericUtils.doubleToSortableLong((Double) parsedValue); + } + + @Override + Number getNextLow(Number parsedValue) { + return DoublePoint.nextDown((Double) parsedValue); + } + + @Override + Number getNextHigh(Number parsedValue) { + return DoublePoint.nextUp((Double) parsedValue); + } +} + +class KeywordFieldMapper implements DimensionFilterMapper { + + @Override + public DimensionFilter getExactMatchFilter(MappedFieldType mappedFieldType, List rawValues) { + KeywordFieldType keywordFieldType = (KeywordFieldType) mappedFieldType; + List convertedValues = new ArrayList<>(rawValues.size()); + for (Object rawValue : rawValues) { + convertedValues.add(parseRawKeyword(mappedFieldType.name(), rawValue, keywordFieldType)); + } + return new ExactMatchDimFilter(mappedFieldType.name(), convertedValues); + } + + @Override + public DimensionFilter getRangeMatchFilter( + MappedFieldType mappedFieldType, + Object rawLow, + Object rawHigh, + boolean includeLow, + boolean includeHigh + ) { + KeywordFieldType keywordFieldType = (KeywordFieldType) mappedFieldType; + return new RangeMatchDimFilter( + mappedFieldType.name(), + parseRawKeyword(mappedFieldType.name(), rawLow, keywordFieldType), + parseRawKeyword(mappedFieldType.name(), rawHigh, keywordFieldType), + includeLow, + includeHigh + ); + } + + @Override + public Optional getMatchingOrdinal( + String dimensionName, + Object value, + StarTreeValues starTreeValues, + DimensionFilter.MatchType matchType + ) { + SortedSetStarTreeValuesIterator sortedSetIterator = (SortedSetStarTreeValuesIterator) starTreeValues.getDimensionValuesIterator( + dimensionName + ); + try { + if (matchType == DimensionFilter.MatchType.EXACT) { + long ordinal = sortedSetIterator.lookupTerm((BytesRef) value); + return ordinal >= 0 ? Optional.of(ordinal) : Optional.empty(); + } else { + TermsEnum termsEnum = sortedSetIterator.termsEnum(); + TermsEnum.SeekStatus seekStatus = termsEnum.seekCeil((BytesRef) value); + // We reached the end and couldn't match anything, else we found a term which matches. + // LT || LTE + // If we found a term just greater, then return ordinal of the term just before it. + // Checking if we are in bounds for satisfying LT + // Checking if we are in bounds for satisfying LT + switch (matchType) { + case GTE: + return seekStatus == TermsEnum.SeekStatus.END ? Optional.empty() : Optional.of(termsEnum.ord()); + case GT: + return switch (seekStatus) { + case END -> Optional.empty(); + case FOUND -> ((termsEnum.ord() + 1) < sortedSetIterator.getValueCount()) + ? Optional.of(termsEnum.ord() + 1) + : Optional.empty(); + case NOT_FOUND -> Optional.of(termsEnum.ord()); + }; + case LTE: + if (seekStatus == TermsEnum.SeekStatus.NOT_FOUND) { + return ((termsEnum.ord() - 1) >= 0) ? Optional.of(termsEnum.ord() - 1) : Optional.empty(); + } else { + return Optional.of(termsEnum.ord()); + } + case LT: + if (seekStatus == TermsEnum.SeekStatus.END) { + return Optional.of(termsEnum.ord()); + } else { + return ((termsEnum.ord() - 1) >= 0) ? Optional.of(termsEnum.ord() - 1) : Optional.empty(); + } + default: + throw new IllegalStateException("unexpected matchType " + matchType); + } + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + // TODO : Think around making TermBasedFT#indexedValueForSearch() accessor public for reuse here. + private Object parseRawKeyword(String field, Object rawValue, KeywordFieldType keywordFieldType) { + Object parsedValue = null; + if (rawValue != null) { + if (keywordFieldType.getTextSearchInfo().getSearchAnalyzer() == Lucene.KEYWORD_ANALYZER) { + parsedValue = BytesRefs.toBytesRef(rawValue); + } else { + if (rawValue instanceof BytesRef) { + rawValue = ((BytesRef) rawValue).utf8ToString(); + } + parsedValue = keywordFieldType.getTextSearchInfo().getSearchAnalyzer().normalize(field, rawValue.toString()); + } + } + return parsedValue; + } + +} diff --git a/server/src/main/java/org/opensearch/search/startree/filter/provider/StarTreeFilterProvider.java b/server/src/main/java/org/opensearch/search/startree/filter/provider/StarTreeFilterProvider.java new file mode 100644 index 0000000000000..61f44ba3f163e --- /dev/null +++ b/server/src/main/java/org/opensearch/search/startree/filter/provider/StarTreeFilterProvider.java @@ -0,0 +1,159 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.startree.filter.provider; + +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.compositeindex.datacube.Dimension; +import org.opensearch.index.mapper.CompositeDataCubeFieldType; +import org.opensearch.index.mapper.MappedFieldType; +import org.opensearch.index.query.MatchAllQueryBuilder; +import org.opensearch.index.query.QueryBuilder; +import org.opensearch.index.query.RangeQueryBuilder; +import org.opensearch.index.query.TermQueryBuilder; +import org.opensearch.index.query.TermsQueryBuilder; +import org.opensearch.search.internal.SearchContext; +import org.opensearch.search.startree.StarTreeQueryHelper; +import org.opensearch.search.startree.filter.StarTreeFilter; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Converts a {@link QueryBuilder} into a {@link StarTreeFilter} by generating the appropriate @{@link org.opensearch.search.startree.filter.DimensionFilter} + * for the fields provided in the user query. + */ +@ExperimentalApi +public interface StarTreeFilterProvider { + + /** + * Returns the {@link StarTreeFilter} generated from the {@link QueryBuilder} + * @param context: + * @param rawFilter: + * @param compositeFieldType: + * @return : {@link StarTreeFilter} if the query shape is supported, else null. + * @throws IOException : + */ + StarTreeFilter getFilter(SearchContext context, QueryBuilder rawFilter, CompositeDataCubeFieldType compositeFieldType) + throws IOException; + + StarTreeFilterProvider MATCH_ALL_PROVIDER = (context, rawFilter, compositeFieldType) -> new StarTreeFilter(Collections.emptyMap()); + + /** + * Singleton instances for most {@link StarTreeFilterProvider} + */ + class SingletonFactory { + + private static final Map QUERY_BUILDERS_TO_STF_PROVIDER = Map.of( + MatchAllQueryBuilder.NAME, + MATCH_ALL_PROVIDER, + TermQueryBuilder.NAME, + new TermStarTreeFilterProvider(), + TermsQueryBuilder.NAME, + new TermsStarTreeFilterProvider(), + RangeQueryBuilder.NAME, + new RangeStarTreeFilterProvider() + ); + + public static StarTreeFilterProvider getProvider(QueryBuilder query) { + if (query != null) { + return QUERY_BUILDERS_TO_STF_PROVIDER.get(query.getName()); + } + return MATCH_ALL_PROVIDER; + } + + } + + /** + * Converts @{@link TermQueryBuilder} into @{@link org.opensearch.search.startree.filter.ExactMatchDimFilter} + */ + class TermStarTreeFilterProvider implements StarTreeFilterProvider { + @Override + public StarTreeFilter getFilter(SearchContext context, QueryBuilder rawFilter, CompositeDataCubeFieldType compositeFieldType) + throws IOException { + TermQueryBuilder termQueryBuilder = (TermQueryBuilder) rawFilter; + String field = termQueryBuilder.fieldName(); + MappedFieldType mappedFieldType = context.mapperService().fieldType(field); + DimensionFilterMapper dimensionFilterMapper = mappedFieldType != null + ? DimensionFilterMapper.Factory.fromMappedFieldType(mappedFieldType) + : null; + Dimension matchedDimension = StarTreeQueryHelper.getMatchingDimensionOrNull(field, compositeFieldType.getDimensions()); + if (matchedDimension == null || mappedFieldType == null || dimensionFilterMapper == null) { + return null; // Indicates Aggregators to fallback to default implementation. + } else { + return new StarTreeFilter( + Map.of(field, List.of(dimensionFilterMapper.getExactMatchFilter(mappedFieldType, List.of(termQueryBuilder.value())))) + ); + } + } + } + + /** + * Converts @{@link TermsQueryBuilder} into @{@link org.opensearch.search.startree.filter.ExactMatchDimFilter} + */ + class TermsStarTreeFilterProvider implements StarTreeFilterProvider { + @Override + public StarTreeFilter getFilter(SearchContext context, QueryBuilder rawFilter, CompositeDataCubeFieldType compositeFieldType) + throws IOException { + TermsQueryBuilder termsQueryBuilder = (TermsQueryBuilder) rawFilter; + String field = termsQueryBuilder.fieldName(); + Dimension matchedDimension = StarTreeQueryHelper.getMatchingDimensionOrNull(field, compositeFieldType.getDimensions()); + MappedFieldType mappedFieldType = context.mapperService().fieldType(field); + DimensionFilterMapper dimensionFilterMapper = mappedFieldType != null + ? DimensionFilterMapper.Factory.fromMappedFieldType(mappedFieldType) + : null; + if (matchedDimension == null || mappedFieldType == null || dimensionFilterMapper == null) { + return null; // Indicates Aggregators to fallback to default implementation. + } else { + return new StarTreeFilter( + Map.of(field, List.of(dimensionFilterMapper.getExactMatchFilter(mappedFieldType, termsQueryBuilder.values()))) + ); + } + } + } + + /** + * Converts @{@link RangeQueryBuilder} into @{@link org.opensearch.search.startree.filter.RangeMatchDimFilter} + */ + class RangeStarTreeFilterProvider implements StarTreeFilterProvider { + + @Override + public StarTreeFilter getFilter(SearchContext context, QueryBuilder rawFilter, CompositeDataCubeFieldType compositeFieldType) + throws IOException { + RangeQueryBuilder rangeQueryBuilder = (RangeQueryBuilder) rawFilter; + String field = rangeQueryBuilder.fieldName(); + Dimension matchedDimension = StarTreeQueryHelper.getMatchingDimensionOrNull(field, compositeFieldType.getDimensions()); + MappedFieldType mappedFieldType = context.mapperService().fieldType(field); + DimensionFilterMapper dimensionFilterMapper = mappedFieldType == null + ? null + : DimensionFilterMapper.Factory.fromMappedFieldType(mappedFieldType); + if (matchedDimension == null || mappedFieldType == null || dimensionFilterMapper == null) { + return null; + } else { + return new StarTreeFilter( + Map.of( + field, + List.of( + dimensionFilterMapper.getRangeMatchFilter( + mappedFieldType, + rangeQueryBuilder.from(), + rangeQueryBuilder.to(), + rangeQueryBuilder.includeLower(), + rangeQueryBuilder.includeUpper() + ) + ) + ) + ); + } + } + + } + +} diff --git a/server/src/main/java/org/opensearch/search/startree/filter/provider/package-info.java b/server/src/main/java/org/opensearch/search/startree/filter/provider/package-info.java new file mode 100644 index 0000000000000..b6ff423e1f140 --- /dev/null +++ b/server/src/main/java/org/opensearch/search/startree/filter/provider/package-info.java @@ -0,0 +1,10 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/** Star Tree Filter and Dimension Filter Providers */ +package org.opensearch.search.startree.filter.provider; diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNodeSearchTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNodeSearchTests.java new file mode 100644 index 0000000000000..4d95034d80bb7 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNodeSearchTests.java @@ -0,0 +1,338 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.compositeindex.datacube.startree.fileformats.node; + +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.opensearch.index.compositeindex.datacube.startree.fileformats.StarTreeWriter; +import org.opensearch.index.compositeindex.datacube.startree.fileformats.meta.StarTreeMetadata; +import org.opensearch.index.compositeindex.datacube.startree.node.InMemoryTreeNode; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeFactory; +import org.opensearch.search.aggregations.startree.ArrayBasedCollector; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.TreeSet; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.ALL; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class FixedLengthStarTreeNodeSearchTests extends OpenSearchTestCase { + + public void testExactMatch() { + long[] randomSorted = random().longs(100, Long.MIN_VALUE, Long.MAX_VALUE).toArray(); + Arrays.sort(randomSorted); + for (boolean createStarNode : new boolean[] { true, false }) { + for (boolean createNullNode : new boolean[] { true, false }) { + createStarTreeForDimension(new long[] { -1, 1, 2, 5 }, createStarNode, createNullNode, List.of(fixedLengthStarTreeNode -> { + try { + boolean result = true; + FixedLengthStarTreeNode lastMatchedNode; + lastMatchedNode = (FixedLengthStarTreeNode) fixedLengthStarTreeNode.getChildForDimensionValue(-1L); + result &= -1 == lastMatchedNode.getDimensionValue(); + // Leaf Node should return null + result &= null == lastMatchedNode.getChildForDimensionValue(5L); + result &= null == lastMatchedNode.getChildForDimensionValue(5L, lastMatchedNode); + // Asserting Last Matched Node works as expected + lastMatchedNode = (FixedLengthStarTreeNode) fixedLengthStarTreeNode.getChildForDimensionValue(1L, lastMatchedNode); + result &= 1 == lastMatchedNode.getDimensionValue(); + lastMatchedNode = (FixedLengthStarTreeNode) fixedLengthStarTreeNode.getChildForDimensionValue(5L, lastMatchedNode); + result &= 5 == lastMatchedNode.getDimensionValue(); + // Asserting null is returned when last matched node is after the value to search. + lastMatchedNode = (FixedLengthStarTreeNode) fixedLengthStarTreeNode.getChildForDimensionValue(2L, lastMatchedNode); + result &= null == lastMatchedNode; + // When dimension value is null + result &= null == fixedLengthStarTreeNode.getChildForDimensionValue(null); + result &= null == fixedLengthStarTreeNode.getChildForDimensionValue(null, null); + // non-existing dimensionValue + result &= null == fixedLengthStarTreeNode.getChildForDimensionValue(4L); + result &= null == fixedLengthStarTreeNode.getChildForDimensionValue(randomLongBetween(6, Long.MAX_VALUE)); + result &= null == fixedLengthStarTreeNode.getChildForDimensionValue(randomLongBetween(Long.MIN_VALUE, -2)); + return result; + } catch (IOException e) { + throw new RuntimeException(e); + } + })); + createStarTreeForDimension(new long[] { 1 }, createStarNode, createNullNode, List.of(fixedLengthStarTreeNode -> { + try { + boolean result = true; + result &= 1 == fixedLengthStarTreeNode.getChildForDimensionValue(1L).getDimensionValue(); + result &= null == fixedLengthStarTreeNode.getChildForDimensionValue(2L); + result &= null == fixedLengthStarTreeNode.getChildForDimensionValue(randomLongBetween(2, Long.MAX_VALUE)); + result &= null == fixedLengthStarTreeNode.getChildForDimensionValue(randomLongBetween(Long.MIN_VALUE, 0)); + return result; + } catch (IOException e) { + throw new RuntimeException(e); + } + })); + createStarTreeForDimension(new long[] {}, createStarNode, createNullNode, List.of(fixedLengthStarTreeNode -> { + try { + boolean result = true; + result &= null == fixedLengthStarTreeNode.getChildForDimensionValue(1L); + result &= null == fixedLengthStarTreeNode.getChildForDimensionValue(randomLongBetween(0, Long.MAX_VALUE)); + result &= null == fixedLengthStarTreeNode.getChildForDimensionValue(randomLongBetween(Long.MIN_VALUE, 0)); + return result; + } catch (IOException e) { + throw new RuntimeException(e); + } + })); + createStarTreeForDimension(randomSorted, createStarNode, createNullNode, List.of(fixedLengthStarTreeNode -> { + boolean result = true; + for (int i = 1; i <= 100; i++) { + try { + ArrayBasedCollector collector = new ArrayBasedCollector(); + long key = randomLong(); + FixedLengthStarTreeNode node = (FixedLengthStarTreeNode) fixedLengthStarTreeNode.getChildForDimensionValue(key); + long match = Arrays.binarySearch(randomSorted, key); + if (match >= 0) { + assertNotNull(node); + assertEquals(key, node.getDimensionValue()); + } else { + assertEquals(0, collector.collectedNodeCount()); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return result; + })); + } + } + } + + public void testRangeMatch() { + long[] randomSorted = random().longs(100, Long.MIN_VALUE, Long.MAX_VALUE).toArray(); + Arrays.sort(randomSorted); + for (boolean createStarNode : new boolean[] { true, false }) { + for (boolean createNullNode : new boolean[] { true, false }) { + createStarTreeForDimension( + new long[] { -10, -1, 1, 2, 5, 9, 25 }, + createStarNode, + createNullNode, + List.of(fixedLengthStarTreeNode -> { + try { + boolean result = true; + ArrayBasedCollector collector; + // Whole range + collector = new ArrayBasedCollector(); + fixedLengthStarTreeNode.collectChildrenInRange(-20, 26, collector); + result &= collector.matchAllCollectedValues(new long[] { -10, -1, 1, 2, 5, 9, 25 }); + // Subset matched from left + collector = new ArrayBasedCollector(); + fixedLengthStarTreeNode.collectChildrenInRange(-2, 1, collector); + result &= collector.matchAllCollectedValues(new long[] { -1, 1 }); + // Subset matched from right + collector = new ArrayBasedCollector(); + fixedLengthStarTreeNode.collectChildrenInRange(6, 100, collector); + result &= collector.matchAllCollectedValues(new long[] { 9, 25 }); + // No match on left + collector = new ArrayBasedCollector(); + fixedLengthStarTreeNode.collectChildrenInRange(-30, -20, collector); + result &= collector.collectedNodeCount() == 0; + // No match on right + collector = new ArrayBasedCollector(); + fixedLengthStarTreeNode.collectChildrenInRange(30, 50, collector); + result &= collector.collectedNodeCount() == 0; + // Low > High + collector = new ArrayBasedCollector(); + fixedLengthStarTreeNode.collectChildrenInRange(50, 10, collector); + result &= collector.collectedNodeCount() == 0; + // Match leftmost + collector = new ArrayBasedCollector(); + fixedLengthStarTreeNode.collectChildrenInRange(-30, -10, collector); + result &= collector.matchAllCollectedValues(new long[] { -10 }); + // Match rightmost + collector = new ArrayBasedCollector(); + fixedLengthStarTreeNode.collectChildrenInRange(10, 25, collector); + result &= collector.matchAllCollectedValues(new long[] { 25 }); + // Match contains interval which has nothing + collector = new ArrayBasedCollector(); + fixedLengthStarTreeNode.collectChildrenInRange(10, 24, collector); + result &= collector.collectedNodeCount() == 0; + // Match contains interval which has nothing + collector = new ArrayBasedCollector(); + fixedLengthStarTreeNode.collectChildrenInRange(6, 24, collector); + result &= collector.matchAllCollectedValues(new long[] { 9 }); + return result; + } catch (IOException e) { + throw new RuntimeException(e); + } + }) + ); + createStarTreeForDimension(randomSorted, createStarNode, createNullNode, List.of(fixedLengthStarTreeNode -> { + boolean result = true; + TreeSet treeSet = Arrays.stream(randomSorted).boxed().collect(Collectors.toCollection(TreeSet::new)); + for (int i = 1; i <= 100; i++) { + try { + ArrayBasedCollector collector = new ArrayBasedCollector(); + long low = randomLong(), high = randomLong(); + fixedLengthStarTreeNode.collectChildrenInRange(low, high, collector); + if (low < high) { + Long lowValue = treeSet.ceiling(low); + if (lowValue != null) { + Long highValue = treeSet.floor(high); + if (highValue != null && highValue >= lowValue) { + collector.matchAllCollectedValues( + Arrays.copyOfRange( + randomSorted, + Arrays.binarySearch(randomSorted, lowValue), + Arrays.binarySearch(randomSorted, highValue) + ) + ); + } else if (lowValue <= high) { + collector.matchAllCollectedValues(new long[] { lowValue }); + } else { + assertEquals(0, collector.collectedNodeCount()); + } + } else { + assertEquals(0, collector.collectedNodeCount()); + } + } else if (low == high) { + collector.matchAllCollectedValues(new long[] { low }); + } else { + assertEquals(0, collector.collectedNodeCount()); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return result; + })); + } + } + } + + private void createStarTreeForDimension( + long[] dimensionValues, + boolean createStarNode, + boolean createNullNode, + List> predicates + ) { + + try (Directory directory = newFSDirectory(createTempDir())) { + + long starTreeDataLength; + + try (IndexOutput dataOut = directory.createOutput("star-tree-data", IOContext.DEFAULT)) { + StarTreeWriter starTreeWriter = new StarTreeWriter(); + int starNodeLengthContribution = 0; + + InMemoryTreeNode rootNode = new InMemoryTreeNode( + 0, + randomInt(), + randomInt(), + randomFrom((byte) 0, (byte) -1, (byte) 1), + -1 + ); + rootNode.setChildDimensionId(1); + rootNode.setAggregatedDocId(randomInt()); + + if (createStarNode && dimensionValues.length > 1) { + InMemoryTreeNode starChild = new InMemoryTreeNode( + rootNode.getDimensionId() + 1, + randomInt(), + randomInt(), + (byte) -1, + -1 + ); + starChild.setChildDimensionId(-1); + starChild.setAggregatedDocId(randomInt()); + rootNode.addChildNode(starChild, (long) ALL); + starNodeLengthContribution++; + } + + for (long dimensionValue : dimensionValues) { + InMemoryTreeNode defaultNode = new InMemoryTreeNode( + rootNode.getDimensionId() + 1, + randomInt(), + randomInt(), + (byte) 0, + dimensionValue + ); + defaultNode.setChildDimensionId(-1); + defaultNode.setAggregatedDocId(randomInt()); + rootNode.addChildNode(defaultNode, dimensionValue); + } + + if (createNullNode) { + InMemoryTreeNode nullNode = new InMemoryTreeNode(rootNode.getDimensionId() + 1, randomInt(), randomInt(), (byte) 1, -1); + nullNode.setChildDimensionId(-1); + nullNode.setAggregatedDocId(randomInt()); + rootNode.addChildNode(nullNode, null); + } + + starTreeDataLength = starTreeWriter.writeStarTree( + dataOut, + rootNode, + starNodeLengthContribution + rootNode.getChildren().size() + 1, + "star-tree" + ); + + // asserting on the actual length of the star tree data file + assertEquals(starTreeDataLength, (33L * rootNode.getChildren().size()) + (starNodeLengthContribution * 33L) + 33L); + } + + for (Predicate predicate : predicates) { + try (IndexInput dataIn = directory.openInput("star-tree-data", IOContext.READONCE)) { + StarTreeMetadata starTreeMetadata = mock(StarTreeMetadata.class); + when(starTreeMetadata.getDataLength()).thenReturn(starTreeDataLength); + when(starTreeMetadata.getDataStartFilePointer()).thenReturn(0L); + FixedLengthStarTreeNode effectiveRoot = (FixedLengthStarTreeNode) StarTreeFactory.createStarTree( + dataIn, + starTreeMetadata + ); + assertTrue(predicate.test(effectiveRoot)); + } + } + + } catch (IOException e) { + throw new RuntimeException(e); + } + + } + + // private static class ArrayBasedCollector implements StarTreeNodeCollector { + // + // private final List nodes = new ArrayList<>(); + // + // @Override + // public void collectStarTreeNode(StarTreeNode node) { + // nodes.add(node); + // } + // + // public boolean matchAllCollectedValues(long[] values) throws IOException { + // boolean matches = true; + // for (int i = 0; i < values.length; i++) { + // matches &= nodes.get(i).getDimensionValue() == values[i]; + // } + // return matches; + // } + // + // public int collectedNodeCount() { + // return nodes.size(); + // } + // + // } + +} diff --git a/server/src/test/java/org/opensearch/search/SearchServiceStarTreeTests.java b/server/src/test/java/org/opensearch/search/SearchServiceStarTreeTests.java index 1beec828e849e..b548c844b2476 100644 --- a/server/src/test/java/org/opensearch/search/SearchServiceStarTreeTests.java +++ b/server/src/test/java/org/opensearch/search/SearchServiceStarTreeTests.java @@ -8,23 +8,43 @@ package org.opensearch.search; +import org.apache.lucene.util.FixedBitSet; import org.opensearch.action.OriginalIndices; import org.opensearch.action.admin.indices.create.CreateIndexRequestBuilder; import org.opensearch.action.search.SearchRequest; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.Rounding; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; import org.opensearch.core.common.Strings; import org.opensearch.index.IndexService; import org.opensearch.index.codec.composite.CompositeIndexFieldInfo; import org.opensearch.index.compositeindex.CompositeIndexSettings; +import org.opensearch.index.compositeindex.datacube.DateDimension; +import org.opensearch.index.compositeindex.datacube.Dimension; +import org.opensearch.index.compositeindex.datacube.Metric; +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.NumericDimension; +import org.opensearch.index.compositeindex.datacube.OrdinalDimension; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; import org.opensearch.index.compositeindex.datacube.startree.StarTreeIndexSettings; +import org.opensearch.index.compositeindex.datacube.startree.utils.date.DateTimeUnitAdapter; +import org.opensearch.index.engine.Segment; +import org.opensearch.index.mapper.CompositeDataCubeFieldType; import org.opensearch.index.mapper.CompositeMappedFieldType; +import org.opensearch.index.mapper.DateFieldMapper; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.mapper.StarTreeMapper; import org.opensearch.index.query.MatchAllQueryBuilder; +import org.opensearch.index.query.QueryBuilder; import org.opensearch.index.query.TermQueryBuilder; import org.opensearch.index.shard.IndexShard; import org.opensearch.indices.IndicesService; import org.opensearch.search.aggregations.AggregationBuilders; +import org.opensearch.search.aggregations.AggregatorFactories; +import org.opensearch.search.aggregations.AggregatorFactory; +import org.opensearch.search.aggregations.SearchContextAggregations; import org.opensearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; import org.opensearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.opensearch.search.aggregations.metrics.MaxAggregationBuilder; @@ -32,6 +52,7 @@ import org.opensearch.search.aggregations.metrics.SumAggregationBuilder; import org.opensearch.search.aggregations.startree.DateHistogramAggregatorTests; import org.opensearch.search.aggregations.startree.StarTreeFilterTests; +import org.opensearch.search.aggregations.support.ValuesSourceAggregationBuilder; import org.opensearch.search.builder.SearchSourceBuilder; import org.opensearch.search.internal.AliasFilter; import org.opensearch.search.internal.ReaderContext; @@ -41,7 +62,9 @@ import org.opensearch.test.OpenSearchSingleNodeTestCase; import java.io.IOException; -import java.util.Map; +import java.util.Collections; +import java.util.List; +import java.util.Set; import static org.opensearch.search.aggregations.AggregationBuilders.dateHistogram; import static org.opensearch.search.aggregations.AggregationBuilders.max; @@ -49,6 +72,8 @@ import static org.opensearch.search.aggregations.AggregationBuilders.sum; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.nullValue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; /** * Tests for validating query shapes which can be resolved using star-tree index @@ -76,7 +101,14 @@ public void testQueryParsingForMetricAggregations() throws IOException { .indices() .prepareCreate("test") .setSettings(settings) - .setMapping(StarTreeFilterTests.getExpandedMapping(1, false)); + .setMapping( + StarTreeFilterTests.getExpandedMapping( + 1, + false, + StarTreeFilterTests.DIMENSION_TYPE_MAP, + StarTreeFilterTests.METRIC_TYPE_MAP + ) + ); createIndex("test", builder); IndicesService indicesService = getInstanceFromNode(IndicesService.class); @@ -94,6 +126,14 @@ public void testQueryParsingForMetricAggregations() throws IOException { null ); + QueryBuilder baseQuery; + SearchContext searchContext = createSearchContext(indexService); + StarTreeFieldConfiguration starTreeFieldConfiguration = new StarTreeFieldConfiguration( + 1, + Collections.emptySet(), + StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP + ); + // Case 1: No query or aggregations, should not use star tree SearchSourceBuilder sourceBuilder = new SearchSourceBuilder(); assertStarTreeContext(request, sourceBuilder, null, -1); @@ -102,14 +142,25 @@ public void testQueryParsingForMetricAggregations() throws IOException { sourceBuilder = new SearchSourceBuilder().query(new MatchAllQueryBuilder()); assertStarTreeContext(request, sourceBuilder, null, -1); - // Case 3: MatchAllQuery and metric aggregations present, should use star tree - sourceBuilder = new SearchSourceBuilder().size(0).query(new MatchAllQueryBuilder()).aggregation(max("test").field("field")); - CompositeIndexFieldInfo expectedStarTree = new CompositeIndexFieldInfo( - "startree", - CompositeMappedFieldType.CompositeFieldType.STAR_TREE + // Case 3: MatchAllQuery and aggregations present, should use star tree + baseQuery = new MatchAllQueryBuilder(); + sourceBuilder = new SearchSourceBuilder().size(0).query(baseQuery).aggregation(AggregationBuilders.max("test").field("field")); + assertStarTreeContext( + request, + sourceBuilder, + getStarTreeQueryContext( + searchContext, + starTreeFieldConfiguration, + "startree", + -1, + List.of(new NumericDimension(FIELD_NAME)), + List.of(new Metric("field", List.of(MetricStat.MAX))), + baseQuery, + sourceBuilder, + true + ), + -1 ); - Map expectedQueryMap = null; - assertStarTreeContext(request, sourceBuilder, new StarTreeQueryContext(expectedStarTree, expectedQueryMap, -1), -1); // Case 4: MatchAllQuery and metric aggregations present, but postFilter specified, should not use star tree sourceBuilder = new SearchSourceBuilder().size(0) @@ -118,24 +169,70 @@ public void testQueryParsingForMetricAggregations() throws IOException { .postFilter(new MatchAllQueryBuilder()); assertStarTreeContext(request, sourceBuilder, null, -1); - // Case 5: TermQuery and single metric aggregation, should use star tree, but not initialize query cache - sourceBuilder = new SearchSourceBuilder().size(0).query(new TermQueryBuilder("sndv", 1)).aggregation(max("test").field("field")); - expectedQueryMap = Map.of("sndv", 1L); - assertStarTreeContext(request, sourceBuilder, new StarTreeQueryContext(expectedStarTree, expectedQueryMap, -1), -1); + // Case 5: TermQuery and single aggregation, should use star tree, but not initialize query cache + baseQuery = new TermQueryBuilder("sndv", 1); + sourceBuilder = new SearchSourceBuilder().size(0).query(baseQuery).aggregation(AggregationBuilders.max("test").field("field")); + assertStarTreeContext( + request, + sourceBuilder, + getStarTreeQueryContext( + searchContext, + starTreeFieldConfiguration, + "startree", + -1, + List.of(new OrdinalDimension("sndv")), + List.of(new Metric("field", List.of(MetricStat.MAX))), + baseQuery, + sourceBuilder, + true + ), + -1 + ); - // Case 6: TermQuery and multiple metric aggregations present, should use star tree & initialize cache + // Case 6: TermQuery and multiple aggregations present, should use star tree & initialize cache + baseQuery = new TermQueryBuilder("sndv", 1); sourceBuilder = new SearchSourceBuilder().size(0) - .query(new TermQueryBuilder("sndv", 1)) - .aggregation(max("test").field("field")) + .query(baseQuery) + .aggregation(AggregationBuilders.max("test").field("field")) .aggregation(AggregationBuilders.sum("test2").field("field")); - expectedQueryMap = Map.of("sndv", 1L); - assertStarTreeContext(request, sourceBuilder, new StarTreeQueryContext(expectedStarTree, expectedQueryMap, 0), 0); + assertStarTreeContext( + request, + sourceBuilder, + getStarTreeQueryContext( + searchContext, + starTreeFieldConfiguration, + "startree", + -1, + List.of(new OrdinalDimension("sndv")), + List.of(new Metric("field", List.of(MetricStat.MAX, MetricStat.SUM))), + baseQuery, + sourceBuilder, + true + ), + 0 + ); // Case 7: No query, metric aggregations present, should use star tree - sourceBuilder = new SearchSourceBuilder().size(0).aggregation(max("test").field("field")); - assertStarTreeContext(request, sourceBuilder, new StarTreeQueryContext(expectedStarTree, null, -1), -1); + sourceBuilder = new SearchSourceBuilder().size(0).aggregation(AggregationBuilders.max("test").field("field")); + assertStarTreeContext( + request, + sourceBuilder, + getStarTreeQueryContext( + searchContext, + starTreeFieldConfiguration, + "startree", + -1, + List.of(new OrdinalDimension("sndv")), + List.of(new Metric("field", List.of(MetricStat.MAX))), + null, + sourceBuilder, + true + ), + -1 + ); setStarTreeIndexSetting(null); + searchContext.close(); } /** @@ -177,49 +274,102 @@ public void testQueryParsingForDateHistogramAggregations() throws IOException { SumAggregationBuilder sumAggSub = sum("sum").field(FIELD_NAME).subAggregation(maxAggNoSub); MedianAbsoluteDeviationAggregationBuilder medianAgg = medianAbsoluteDeviation("median").field(FIELD_NAME); + StarTreeFieldConfiguration starTreeFieldConfiguration = new StarTreeFieldConfiguration( + 1, + Collections.emptySet(), + StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP + ); + + QueryBuilder baseQuery; + SearchContext searchContext = createSearchContext(indexService); // Case 1: No query or aggregations, should not use star tree SearchSourceBuilder sourceBuilder = new SearchSourceBuilder(); assertStarTreeContext(request, sourceBuilder, null, -1); // Case 2: MatchAllQuery present but no aggregations, should not use star tree - sourceBuilder = new SearchSourceBuilder().query(new MatchAllQueryBuilder()); + baseQuery = new MatchAllQueryBuilder(); + sourceBuilder = new SearchSourceBuilder().query(baseQuery); assertStarTreeContext(request, sourceBuilder, null, -1); // Case 3: MatchAllQuery and non-nested metric aggregations is nested within date-histogram aggregation, should use star tree DateHistogramAggregationBuilder dateHistogramAggregationBuilder = dateHistogram("by_day").field(TIMESTAMP_FIELD) .calendarInterval(DateHistogramInterval.DAY) .subAggregation(maxAggNoSub); - sourceBuilder = new SearchSourceBuilder().size(0).query(new MatchAllQueryBuilder()).aggregation(dateHistogramAggregationBuilder); + baseQuery = new MatchAllQueryBuilder(); + sourceBuilder = new SearchSourceBuilder().size(0).query(baseQuery).aggregation(dateHistogramAggregationBuilder); CompositeIndexFieldInfo expectedStarTree = new CompositeIndexFieldInfo( "startree1", CompositeMappedFieldType.CompositeFieldType.STAR_TREE ); - Map expectedQueryMap = null; - assertStarTreeContext(request, sourceBuilder, new StarTreeQueryContext(expectedStarTree, expectedQueryMap, -1), -1); + assertStarTreeContext( + request, + sourceBuilder, + getStarTreeQueryContext( + searchContext, + starTreeFieldConfiguration, + "startree1", + -1, + List.of( + new DateDimension( + TIMESTAMP_FIELD, + List.of(new DateTimeUnitAdapter(Rounding.DateTimeUnit.DAY_OF_MONTH)), + DateFieldMapper.Resolution.MILLISECONDS + ), + new NumericDimension(FIELD_NAME) + ), + List.of(new Metric(FIELD_NAME, List.of(MetricStat.SUM, MetricStat.MAX))), + baseQuery, + sourceBuilder, + true + ), + -1 + ); // Case 4: MatchAllQuery and nested-metric aggregations is nested within date-histogram aggregation, should not use star tree dateHistogramAggregationBuilder = dateHistogram("by_day").field(TIMESTAMP_FIELD) .calendarInterval(DateHistogramInterval.DAY) .subAggregation(sumAggSub); - sourceBuilder = new SearchSourceBuilder().size(0).query(new MatchAllQueryBuilder()).aggregation(dateHistogramAggregationBuilder); + baseQuery = new MatchAllQueryBuilder(); + sourceBuilder = new SearchSourceBuilder().size(0).query(baseQuery).aggregation(dateHistogramAggregationBuilder); assertStarTreeContext(request, sourceBuilder, null, -1); // Case 5: MatchAllQuery and non-startree supported aggregation nested within date-histogram aggregation, should not use star tree dateHistogramAggregationBuilder = dateHistogram("by_day").field(TIMESTAMP_FIELD) .calendarInterval(DateHistogramInterval.DAY) .subAggregation(medianAgg); - sourceBuilder = new SearchSourceBuilder().size(0).query(new MatchAllQueryBuilder()).aggregation(dateHistogramAggregationBuilder); + baseQuery = new MatchAllQueryBuilder(); + sourceBuilder = new SearchSourceBuilder().size(0).query(baseQuery).aggregation(dateHistogramAggregationBuilder); assertStarTreeContext(request, sourceBuilder, null, -1); // Case 6: NumericTermQuery and date-histogram aggregation present, should use star tree dateHistogramAggregationBuilder = dateHistogram("by_day").field(TIMESTAMP_FIELD) .calendarInterval(DateHistogramInterval.DAY) .subAggregation(maxAggNoSub); - sourceBuilder = new SearchSourceBuilder().size(0) - .query(new TermQueryBuilder(FIELD_NAME, 1)) - .aggregation(dateHistogramAggregationBuilder); - expectedQueryMap = Map.of(FIELD_NAME, 1L); - assertStarTreeContext(request, sourceBuilder, new StarTreeQueryContext(expectedStarTree, expectedQueryMap, -1), -1); + baseQuery = new TermQueryBuilder(FIELD_NAME, 1L); + sourceBuilder = new SearchSourceBuilder().size(0).query(baseQuery).aggregation(dateHistogramAggregationBuilder); + assertStarTreeContext( + request, + sourceBuilder, + getStarTreeQueryContext( + searchContext, + starTreeFieldConfiguration, + "startree1", + -1, + List.of( + new DateDimension( + TIMESTAMP_FIELD, + List.of(new DateTimeUnitAdapter(Rounding.DateTimeUnit.DAY_OF_MONTH)), + DateFieldMapper.Resolution.MILLISECONDS + ), + new NumericDimension(FIELD_NAME) + ), + List.of(new Metric(FIELD_NAME, List.of(MetricStat.SUM, MetricStat.MAX))), + baseQuery, + sourceBuilder, + true + ), + -1 + ); // Case 7: Date histogram with non calendar interval: rounding is null for DateHistogramFactory - cannot use star-tree dateHistogramAggregationBuilder = dateHistogram("non_cal").field(TIMESTAMP_FIELD) @@ -245,13 +395,94 @@ public void testQueryParsingForDateHistogramAggregations() throws IOException { .calendarInterval(DateHistogramInterval.DAY) .subAggregation(maxAggNoSub) .subAggregation(sumAggNoSub); - expectedQueryMap = null; + baseQuery = null; sourceBuilder = new SearchSourceBuilder().size(0).aggregation(dateHistogramAggregationBuilder); - assertStarTreeContext(request, sourceBuilder, new StarTreeQueryContext(expectedStarTree, expectedQueryMap, -1), -1); + assertStarTreeContext( + request, + sourceBuilder, + getStarTreeQueryContext( + searchContext, + starTreeFieldConfiguration, + "startree1", + -1, + List.of( + new DateDimension( + TIMESTAMP_FIELD, + List.of(new DateTimeUnitAdapter(Rounding.DateTimeUnit.DAY_OF_MONTH)), + DateFieldMapper.Resolution.MILLISECONDS + ), + new NumericDimension(FIELD_NAME) + ), + List.of( + new Metric(TIMESTAMP_FIELD, List.of(MetricStat.SUM, MetricStat.MAX)), + new Metric(FIELD_NAME, List.of(MetricStat.MAX, MetricStat.SUM)) + ), + baseQuery, + sourceBuilder, + true + ), + -1 + ); setStarTreeIndexSetting(null); } + public void testCacheCreationInStarTreeQueryContext() throws IOException { + StarTreeFieldConfiguration starTreeFieldConfiguration = new StarTreeFieldConfiguration( + 1, + Collections.emptySet(), + StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP + ); + CompositeDataCubeFieldType compositeDataCubeFieldType = new StarTreeMapper.StarTreeFieldType( + "star_tree", + new StarTreeField( + "star_tree", + List.of(new OrdinalDimension("field")), + List.of(new Metric("metricField", List.of(MetricStat.SUM, MetricStat.MAX))), + starTreeFieldConfiguration + ) + ); + + QueryBuilder baseQuery = new MatchAllQueryBuilder(); + SearchContext searchContext = mock(SearchContext.class); + MapperService mapperService = mock(MapperService.class); + IndexShard indexShard = mock(IndexShard.class); + Segment segment = mock(Segment.class); + SearchContextAggregations searchContextAggregations = mock(SearchContextAggregations.class); + AggregatorFactories aggregatorFactories = mock(AggregatorFactories.class); + + when(mapperService.getCompositeFieldTypes()).thenReturn(Set.of(compositeDataCubeFieldType)); + when(searchContext.mapperService()).thenReturn(mapperService); + when(searchContext.indexShard()).thenReturn(indexShard); + when(indexShard.segments(false)).thenReturn(List.of(segment, segment)); + when(searchContext.aggregations()).thenReturn(searchContextAggregations); + when(searchContextAggregations.factories()).thenReturn(aggregatorFactories); + when(aggregatorFactories.getFactories()).thenReturn(new AggregatorFactory[] { null, null }); + StarTreeQueryContext starTreeQueryContext = new StarTreeQueryContext(searchContext, baseQuery); + + assertEquals(2, starTreeQueryContext.getAllCachedValues().length); + + // Asserting null values are ignored + when(aggregatorFactories.getFactories()).thenReturn(new AggregatorFactory[] {}); + starTreeQueryContext = new StarTreeQueryContext(searchContext, baseQuery); + starTreeQueryContext.maybeSetCachedNodeIdsForSegment(-1, null); + assertNull(starTreeQueryContext.getAllCachedValues()); + assertNull(starTreeQueryContext.maybeGetCachedNodeIdsForSegment(0)); + + // Assert correct cached value is returned + when(aggregatorFactories.getFactories()).thenReturn(new AggregatorFactory[] { null, null }); + starTreeQueryContext = new StarTreeQueryContext(searchContext, baseQuery); + FixedBitSet cachedValues = new FixedBitSet(22); + starTreeQueryContext.maybeSetCachedNodeIdsForSegment(0, cachedValues); + assertEquals(2, starTreeQueryContext.getAllCachedValues().length); + assertEquals(22, starTreeQueryContext.maybeGetCachedNodeIdsForSegment(0).length()); + + starTreeQueryContext = new StarTreeQueryContext(compositeDataCubeFieldType, new MatchAllQueryBuilder(), 2); + assertEquals(2, starTreeQueryContext.getAllCachedValues().length); + + mapperService.close(); + } + /** * Test query parsing for date histogram aggregations on star-tree index when @timestamp field does not exist */ @@ -268,7 +499,14 @@ public void testInvalidQueryParsingForDateHistogramAggregations() throws IOExcep .indices() .prepareCreate("test") .setSettings(settings) - .setMapping(StarTreeFilterTests.getExpandedMapping(1, false)); + .setMapping( + StarTreeFilterTests.getExpandedMapping( + 1, + false, + StarTreeFilterTests.DIMENSION_TYPE_MAP, + StarTreeFilterTests.METRIC_TYPE_MAP + ) + ); createIndex("test", builder); IndicesService indicesService = getInstanceFromNode(IndicesService.class); @@ -293,10 +531,6 @@ public void testInvalidQueryParsingForDateHistogramAggregations() throws IOExcep SearchSourceBuilder sourceBuilder = new SearchSourceBuilder().size(0) .query(new MatchAllQueryBuilder()) .aggregation(dateHistogramAggregationBuilder); - CompositeIndexFieldInfo expectedStarTree = new CompositeIndexFieldInfo( - "startree1", - CompositeMappedFieldType.CompositeFieldType.STAR_TREE - ); assertStarTreeContext(request, sourceBuilder, null, -1); setStarTreeIndexSetting(null); @@ -320,22 +554,61 @@ private void assertStarTreeContext( SearchService searchService = getInstanceFromNode(SearchService.class); try (ReaderContext reader = searchService.createOrGetReaderContext(request, false)) { SearchContext context = searchService.createContext(reader, request, null, true); - StarTreeQueryContext actualContext = context.getStarTreeQueryContext(); + StarTreeQueryContext actualContext = context.getQueryShardContext().getStarTreeQueryContext(); if (expectedContext == null) { - assertThat(context.getStarTreeQueryContext(), nullValue()); + assertThat(context.getQueryShardContext().getStarTreeQueryContext(), nullValue()); } else { assertThat(actualContext, notNullValue()); assertEquals(expectedContext.getStarTree().getType(), actualContext.getStarTree().getType()); assertEquals(expectedContext.getStarTree().getField(), actualContext.getStarTree().getField()); - assertEquals(expectedContext.getQueryMap(), actualContext.getQueryMap()); + assertEquals( + expectedContext.getBaseQueryStarTreeFilter().getDimensions(), + actualContext.getBaseQueryStarTreeFilter().getDimensions() + ); if (expectedCacheUsage > -1) { - assertEquals(expectedCacheUsage, actualContext.getStarTreeValues().length); + assertEquals(expectedCacheUsage, actualContext.getAllCachedValues().length); } else { - assertNull(actualContext.getStarTreeValues()); + assertNull(actualContext.getAllCachedValues()); } } searchService.doStop(); } } + + private StarTreeQueryContext getStarTreeQueryContext( + SearchContext searchContext, + StarTreeFieldConfiguration starTreeFieldConfiguration, + String compositeFieldName, + int cacheSize, + List dimensions, + List metrics, + QueryBuilder baseQuery, + SearchSourceBuilder sourceBuilder, + boolean assertConsolidation + ) { + AggregatorFactories aggregatorFactories = mock(AggregatorFactories.class); + AggregatorFactory[] aggregatorFactoriesArray = sourceBuilder.aggregations().getAggregatorFactories().stream().map(af -> { + try { + return ((ValuesSourceAggregationBuilder) af).build(searchContext.getQueryShardContext(), null); + } catch (IOException e) { + throw new RuntimeException(e); + } + }).toArray(AggregatorFactory[]::new); + when(aggregatorFactories.getFactories()).thenReturn(aggregatorFactoriesArray); + SearchContextAggregations mockAggregations = mock(SearchContextAggregations.class); + when(mockAggregations.factories()).thenReturn(aggregatorFactories); + searchContext.aggregations(mockAggregations); + CompositeDataCubeFieldType compositeDataCubeFieldType = new StarTreeMapper.StarTreeFieldType( + compositeFieldName, + new StarTreeField(compositeFieldName, dimensions, metrics, starTreeFieldConfiguration) + ); + StarTreeQueryContext starTreeQueryContext = new StarTreeQueryContext(compositeDataCubeFieldType, baseQuery, cacheSize); + boolean consolidated = starTreeQueryContext.consolidateAllFilters(searchContext); + if (assertConsolidation) { + assertTrue(consolidated); + searchContext.getQueryShardContext().setStarTreeQueryContext(starTreeQueryContext); + } + return starTreeQueryContext; + } } diff --git a/server/src/test/java/org/opensearch/search/aggregations/startree/ArrayBasedCollector.java b/server/src/test/java/org/opensearch/search/aggregations/startree/ArrayBasedCollector.java new file mode 100644 index 0000000000000..6aad0fbaafd2c --- /dev/null +++ b/server/src/test/java/org/opensearch/search/aggregations/startree/ArrayBasedCollector.java @@ -0,0 +1,42 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.aggregations.startree; + +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNode; +import org.opensearch.search.startree.StarTreeNodeCollector; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Set; + +public class ArrayBasedCollector implements StarTreeNodeCollector { + + private final Set nodeDimensionValues = new HashSet<>(); + + @Override + public void collectStarTreeNode(StarTreeNode node) { + try { + nodeDimensionValues.add(node.getDimensionValue()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public boolean matchAllCollectedValues(long... values) throws IOException { + for (long value : values) { + if (!nodeDimensionValues.contains(value)) return false; + } + return true; + } + + public int collectedNodeCount() { + return nodeDimensionValues.size(); + } + +} diff --git a/server/src/test/java/org/opensearch/search/aggregations/startree/DateHistogramAggregatorTests.java b/server/src/test/java/org/opensearch/search/aggregations/startree/DateHistogramAggregatorTests.java index 564a86deff1af..a374e2f5653b9 100644 --- a/server/src/test/java/org/opensearch/search/aggregations/startree/DateHistogramAggregatorTests.java +++ b/server/src/test/java/org/opensearch/search/aggregations/startree/DateHistogramAggregatorTests.java @@ -55,7 +55,7 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.LinkedList; +import java.util.LinkedHashMap; import java.util.List; import java.util.Random; @@ -155,10 +155,16 @@ public void testStarTreeDateHistogram() throws IOException { count("_name").field(FIELD_NAME), avg("_name").field(FIELD_NAME) }; - List supportedDimensions = new LinkedList<>(); - supportedDimensions.add(new NumericDimension(STATUS)); - supportedDimensions.add(new NumericDimension(SIZE)); - supportedDimensions.add( + LinkedHashMap supportedDimensions = new LinkedHashMap<>(); + supportedDimensions.put( + new NumericDimension(STATUS), + new NumberFieldMapper.NumberFieldType(STATUS, NumberFieldMapper.NumberType.INTEGER) + ); + supportedDimensions.put( + new NumericDimension(SIZE), + new NumberFieldMapper.NumberFieldType(STATUS, NumberFieldMapper.NumberType.INTEGER) + ); + supportedDimensions.put( new DateDimension( TIMESTAMP_FIELD, List.of( @@ -166,7 +172,8 @@ public void testStarTreeDateHistogram() throws IOException { new DateTimeUnitAdapter(Rounding.DateTimeUnit.DAY_OF_MONTH) ), DateFieldMapper.Resolution.MILLISECONDS - ) + ), + new DateFieldMapper.DateFieldType(TIMESTAMP_FIELD) ); for (ValuesSourceAggregationBuilder aggregationBuilder : agggBuilders) { @@ -224,7 +231,7 @@ private void testCase( QueryBuilder queryBuilder, DateHistogramAggregationBuilder dateHistogramAggregationBuilder, CompositeIndexFieldInfo starTree, - List supportedDimensions + LinkedHashMap supportedDimensions ) throws IOException { InternalDateHistogram starTreeAggregation = searchAndReduceStarTree( createIndexSettings(), diff --git a/server/src/test/java/org/opensearch/search/aggregations/startree/DimensionFilterAndMapperTests.java b/server/src/test/java/org/opensearch/search/aggregations/startree/DimensionFilterAndMapperTests.java new file mode 100644 index 0000000000000..e89bc8e60e9da --- /dev/null +++ b/server/src/test/java/org/opensearch/search/aggregations/startree/DimensionFilterAndMapperTests.java @@ -0,0 +1,193 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.aggregations.startree; + +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.util.BytesRef; +import org.opensearch.index.compositeindex.datacube.Metric; +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.OrdinalDimension; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; +import org.opensearch.index.compositeindex.datacube.startree.utils.iterator.SortedSetStarTreeValuesIterator; +import org.opensearch.index.mapper.CompositeDataCubeFieldType; +import org.opensearch.index.mapper.KeywordFieldMapper; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.mapper.StarTreeMapper; +import org.opensearch.index.mapper.WildcardFieldMapper; +import org.opensearch.index.query.QueryBuilder; +import org.opensearch.index.query.RangeQueryBuilder; +import org.opensearch.index.query.TermQueryBuilder; +import org.opensearch.index.query.TermsQueryBuilder; +import org.opensearch.search.internal.SearchContext; +import org.opensearch.search.startree.filter.DimensionFilter; +import org.opensearch.search.startree.filter.DimensionFilter.MatchType; +import org.opensearch.search.startree.filter.MatchNoneFilter; +import org.opensearch.search.startree.filter.provider.DimensionFilterMapper; +import org.opensearch.search.startree.filter.provider.StarTreeFilterProvider; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class DimensionFilterAndMapperTests extends OpenSearchTestCase { + + public void testKeywordOrdinalMapping() throws IOException { + DimensionFilterMapper dimensionFilterMapper = DimensionFilterMapper.Factory.fromMappedFieldType( + new KeywordFieldMapper.KeywordFieldType("keyword") + ); + StarTreeValues starTreeValues = mock(StarTreeValues.class); + SortedSetStarTreeValuesIterator sortedSetStarTreeValuesIterator = mock(SortedSetStarTreeValuesIterator.class); + TermsEnum termsEnum = mock(TermsEnum.class); + when(sortedSetStarTreeValuesIterator.termsEnum()).thenReturn(termsEnum); + when(starTreeValues.getDimensionValuesIterator("field")).thenReturn(sortedSetStarTreeValuesIterator); + Optional matchingOrdinal; + + // Case Exact Match and found + BytesRef bytesRef = new BytesRef(new byte[] { 17, 29 }); + when(sortedSetStarTreeValuesIterator.lookupTerm(bytesRef)).thenReturn(1L); + matchingOrdinal = dimensionFilterMapper.getMatchingOrdinal("field", bytesRef, starTreeValues, MatchType.EXACT); + assertTrue(matchingOrdinal.isPresent()); + assertEquals(1, (long) matchingOrdinal.get()); + + // Case Exact Match and not found + when(sortedSetStarTreeValuesIterator.lookupTerm(bytesRef)).thenReturn(-10L); + matchingOrdinal = dimensionFilterMapper.getMatchingOrdinal("field", bytesRef, starTreeValues, MatchType.EXACT); + assertFalse(matchingOrdinal.isPresent()); + + // Case GTE -> FOUND and NOT_FOUND + for (TermsEnum.SeekStatus seekStatus : new TermsEnum.SeekStatus[] { TermsEnum.SeekStatus.FOUND, TermsEnum.SeekStatus.NOT_FOUND }) { + when(termsEnum.seekCeil(bytesRef)).thenReturn(seekStatus); + when(termsEnum.ord()).thenReturn(10L); + matchingOrdinal = dimensionFilterMapper.getMatchingOrdinal("field", bytesRef, starTreeValues, MatchType.GTE); + assertTrue(matchingOrdinal.isPresent()); + assertEquals(10L, (long) matchingOrdinal.get()); + } + + // Seek Status END is same for GTE, GT + for (MatchType matchType : new MatchType[] { MatchType.GT, MatchType.GTE }) { + when(termsEnum.seekCeil(bytesRef)).thenReturn(TermsEnum.SeekStatus.END); + when(termsEnum.ord()).thenReturn(10L); + matchingOrdinal = dimensionFilterMapper.getMatchingOrdinal("field", bytesRef, starTreeValues, matchType); + assertFalse(matchingOrdinal.isPresent()); + } + + // Case GT -> FOUND and matched + when(termsEnum.seekCeil(bytesRef)).thenReturn(TermsEnum.SeekStatus.FOUND); + when(sortedSetStarTreeValuesIterator.getValueCount()).thenReturn(2L); + when(termsEnum.ord()).thenReturn(0L); + matchingOrdinal = dimensionFilterMapper.getMatchingOrdinal("field", bytesRef, starTreeValues, MatchType.GT); + assertTrue(matchingOrdinal.isPresent()); + assertEquals(1L, (long) matchingOrdinal.get()); + // Case GT -> FOUND and unmatched + when(termsEnum.ord()).thenReturn(3L); + matchingOrdinal = dimensionFilterMapper.getMatchingOrdinal("field", bytesRef, starTreeValues, MatchType.GT); + assertFalse(matchingOrdinal.isPresent()); + + // Case GT -> NOT_FOUND + when(termsEnum.seekCeil(bytesRef)).thenReturn(TermsEnum.SeekStatus.NOT_FOUND); + when(termsEnum.ord()).thenReturn(10L); + matchingOrdinal = dimensionFilterMapper.getMatchingOrdinal("field", bytesRef, starTreeValues, MatchType.GT); + assertTrue(matchingOrdinal.isPresent()); + assertEquals(10L, (long) matchingOrdinal.get()); + + // Seek Status END is same for LTE, LT + for (MatchType matchType : new MatchType[] { MatchType.LT, MatchType.LTE }) { + when(termsEnum.seekCeil(bytesRef)).thenReturn(TermsEnum.SeekStatus.END); + when(termsEnum.ord()).thenReturn(10L); + matchingOrdinal = dimensionFilterMapper.getMatchingOrdinal("field", bytesRef, starTreeValues, matchType); + assertTrue(matchingOrdinal.isPresent()); + assertEquals(10L, (long) matchingOrdinal.get()); + } + + // Seek Status NOT_FOUND is same for LTE, LT + for (MatchType matchType : new MatchType[] { MatchType.LT, MatchType.LTE }) { + when(termsEnum.seekCeil(bytesRef)).thenReturn(TermsEnum.SeekStatus.NOT_FOUND); + when(sortedSetStarTreeValuesIterator.getValueCount()).thenReturn(2L); + when(termsEnum.ord()).thenReturn(1L); + matchingOrdinal = dimensionFilterMapper.getMatchingOrdinal("field", bytesRef, starTreeValues, matchType); + assertTrue(matchingOrdinal.isPresent()); + assertEquals(0L, (long) matchingOrdinal.get()); + // Case unmatched + when(termsEnum.ord()).thenReturn(0L); + matchingOrdinal = dimensionFilterMapper.getMatchingOrdinal("field", bytesRef, starTreeValues, matchType); + assertFalse(matchingOrdinal.isPresent()); + } + } + + public void testStarTreeFilterProviders() throws IOException { + CompositeDataCubeFieldType compositeDataCubeFieldType = new StarTreeMapper.StarTreeFieldType( + "star_tree", + new StarTreeField( + "star_tree", + List.of(new OrdinalDimension("keyword")), + List.of(new Metric("field", List.of(MetricStat.MAX))), + new StarTreeFieldConfiguration( + randomIntBetween(1, 10_000), + Collections.emptySet(), + StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP + ) + ) + ); + MapperService mapperService = mock(MapperService.class); + SearchContext searchContext = mock(SearchContext.class); + when(searchContext.mapperService()).thenReturn(mapperService); + + // Null returned when mapper doesn't exist + assertNull(DimensionFilterMapper.Factory.fromMappedFieldType(new WildcardFieldMapper.WildcardFieldType("field"))); + + // Null returned for no mapped field type + assertNull(DimensionFilterMapper.Factory.fromMappedFieldType(null)); + + // Provider for null Query builder + assertEquals(StarTreeFilterProvider.MATCH_ALL_PROVIDER, StarTreeFilterProvider.SingletonFactory.getProvider(null)); + + QueryBuilder[] queryBuilders = new QueryBuilder[] { + new TermQueryBuilder("field", "value"), + new TermsQueryBuilder("field", List.of("value")), + new RangeQueryBuilder("field") }; + + for (QueryBuilder queryBuilder : queryBuilders) { + // Dimension Not Found + StarTreeFilterProvider provider = StarTreeFilterProvider.SingletonFactory.getProvider(queryBuilder); + assertNull(provider.getFilter(searchContext, queryBuilder, compositeDataCubeFieldType)); + } + + queryBuilders = new QueryBuilder[] { + new TermQueryBuilder("keyword", "value"), + new TermsQueryBuilder("keyword", List.of("value")), + new RangeQueryBuilder("keyword") }; + + for (QueryBuilder queryBuilder : queryBuilders) { + // Mapped field type not supported + StarTreeFilterProvider provider = StarTreeFilterProvider.SingletonFactory.getProvider(queryBuilder); + when(mapperService.fieldType("keyword")).thenReturn(new WildcardFieldMapper.WildcardFieldType("keyword")); + assertNull(provider.getFilter(searchContext, queryBuilder, compositeDataCubeFieldType)); + + // Unsupported Mapped Type + when(mapperService.fieldType("keyword")).thenReturn(null); + assertNull(provider.getFilter(searchContext, queryBuilder, compositeDataCubeFieldType)); + } + + // Testing MatchNoneFilter + DimensionFilter dimensionFilter = new MatchNoneFilter(); + dimensionFilter.initialiseForSegment(null, null); + ArrayBasedCollector collector = new ArrayBasedCollector(); + assertFalse(dimensionFilter.matchDimValue(1, null)); + dimensionFilter.matchStarTreeNodes(null, null, collector); + assertEquals(0, collector.collectedNodeCount()); + } + +} diff --git a/server/src/test/java/org/opensearch/search/aggregations/startree/MetricAggregatorTests.java b/server/src/test/java/org/opensearch/search/aggregations/startree/MetricAggregatorTests.java index 21a4155a53ee5..6e10562c3a846 100644 --- a/server/src/test/java/org/opensearch/search/aggregations/startree/MetricAggregatorTests.java +++ b/server/src/test/java/org/opensearch/search/aggregations/startree/MetricAggregatorTests.java @@ -15,13 +15,20 @@ import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.lucene101.Lucene101Codec; import org.apache.lucene.document.Document; +import org.apache.lucene.document.DoubleField; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.FloatField; +import org.apache.lucene.document.IntField; +import org.apache.lucene.document.KeywordField; +import org.apache.lucene.document.LongField; import org.apache.lucene.document.SortedNumericDocValuesField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.SegmentReader; +import org.apache.lucene.sandbox.document.HalfFloatPoint; import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.RandomIndexWriter; @@ -40,12 +47,16 @@ import org.opensearch.index.compositeindex.datacube.Metric; import org.opensearch.index.compositeindex.datacube.MetricStat; import org.opensearch.index.compositeindex.datacube.NumericDimension; +import org.opensearch.index.compositeindex.datacube.OrdinalDimension; +import org.opensearch.index.mapper.KeywordFieldMapper; import org.opensearch.index.mapper.MappedFieldType; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.mapper.NumberFieldMapper; import org.opensearch.index.query.QueryBuilder; import org.opensearch.index.query.QueryShardContext; +import org.opensearch.index.query.RangeQueryBuilder; import org.opensearch.index.query.TermQueryBuilder; +import org.opensearch.index.query.TermsQueryBuilder; import org.opensearch.search.aggregations.AggregationBuilder; import org.opensearch.search.aggregations.AggregatorFactories; import org.opensearch.search.aggregations.AggregatorFactory; @@ -69,11 +80,16 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; -import java.util.LinkedList; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Locale; +import java.util.Map; import java.util.Random; import java.util.function.BiConsumer; import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.opensearch.search.aggregations.AggregationBuilders.avg; import static org.opensearch.search.aggregations.AggregationBuilders.count; @@ -100,11 +116,17 @@ public void teardown() throws IOException { FeatureFlags.initializeFeatureFlags(Settings.EMPTY); } - protected Codec getCodec() { + protected Codec getCodec( + Supplier maxLeafDocsSupplier, + LinkedHashMap dimensionAndType, + Map metricFieldAndType + ) { final Logger testLogger = LogManager.getLogger(MetricAggregatorTests.class); MapperService mapperService; try { - mapperService = StarTreeDocValuesFormatTests.createMapperService(StarTreeFilterTests.getExpandedMapping(1, false)); + mapperService = StarTreeDocValuesFormatTests.createMapperService( + StarTreeFilterTests.getExpandedMapping(maxLeafDocsSupplier.get(), false, dimensionAndType, metricFieldAndType) + ); } catch (IOException e) { throw new RuntimeException(e); } @@ -112,36 +134,65 @@ protected Codec getCodec() { } public void testStarTreeDocValues() throws IOException { + final List> MAX_LEAF_DOC_VARIATIONS = List.of( + () -> 1, + () -> randomIntBetween(2, 100), + () -> randomIntBetween(101, 10_000) + ); + final List dimensionFieldDatum = List.of( + new DimensionFieldData("sndv", () -> random().nextInt(10) - 5, DimensionTypes.INTEGER), + new DimensionFieldData("dv", () -> random().nextInt(20) - 10, DimensionTypes.INTEGER), + new DimensionFieldData("keyword_field", () -> random().nextInt(50), DimensionTypes.KEYWORD), + new DimensionFieldData("long_field", () -> random().nextInt(50), DimensionTypes.LONG), + new DimensionFieldData("half_float_field", () -> random().nextFloat(50), DimensionTypes.HALF_FLOAT), + new DimensionFieldData("float_field", () -> random().nextFloat(50), DimensionTypes.FLOAT), + new DimensionFieldData("double_field", () -> random().nextDouble(50), DimensionTypes.DOUBLE) + ); + for (Supplier maxLeafDocsSupplier : MAX_LEAF_DOC_VARIATIONS) { + testStarTreeDocValuesInternal( + getCodec( + maxLeafDocsSupplier, + dimensionFieldDatum.stream() + .collect( + Collectors.toMap( + df -> df.getDimension().getField(), + DimensionFieldData::getFieldType, + (v1, v2) -> v1, + LinkedHashMap::new + ) + ), + StarTreeFilterTests.METRIC_TYPE_MAP + ), + dimensionFieldDatum + ); + } + } + + private void testStarTreeDocValuesInternal(Codec codec, List dimensionFieldData) throws IOException { Directory directory = newDirectory(); IndexWriterConfig conf = newIndexWriterConfig(null); - conf.setCodec(getCodec()); + conf.setCodec(codec); conf.setMergePolicy(newLogMergePolicy()); RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf); Random random = RandomizedTest.getRandom(); int totalDocs = 100; - final String SNDV = "sndv"; - final String DV = "dv"; int val; - List docs = new ArrayList<>(); // Index 100 random documents for (int i = 0; i < totalDocs; i++) { Document doc = new Document(); - if (random.nextBoolean()) { - val = random.nextInt(10) - 5; // Random long between -5 and 4 - doc.add(new SortedNumericDocValuesField(SNDV, val)); - } - if (random.nextBoolean()) { - val = random.nextInt(20) - 10; // Random long between -10 and 9 - doc.add(new SortedNumericDocValuesField(DV, val)); + for (DimensionFieldData fieldData : dimensionFieldData) { + // FIXME: Reduce the frequency of nulls to be with at least some non-null docs like after every 1-2 ? + if (random.nextBoolean()) { + doc.add(fieldData.getField()); + } } if (random.nextBoolean()) { val = random.nextInt(50); // Random long between 0 and 49 doc.add(new SortedNumericDocValuesField(FIELD_NAME, val)); } iw.addDocument(doc); - docs.add(doc); } if (randomBoolean()) { @@ -157,6 +208,23 @@ public void testStarTreeDocValues() throws IOException { IndexSearcher indexSearcher = newSearcher(reader, false, false); CompositeIndexReader starTreeDocValuesReader = (CompositeIndexReader) reader.getDocValuesReader(); + MapperService mapperService = mapperServiceMock(); + CircuitBreakerService circuitBreakerService = new NoneCircuitBreakerService(); + for (DimensionFieldData fieldData : dimensionFieldData) { + when(mapperService.fieldType(fieldData.fieldName)).thenReturn(fieldData.getMappedField()); + } + QueryShardContext queryShardContext = queryShardContextMock( + indexSearcher, + mapperService, + createIndexSettings(), + circuitBreakerService, + new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), circuitBreakerService).withCircuitBreaking() + ); + for (DimensionFieldData fieldData : dimensionFieldData) { + when(mapperService.fieldType(fieldData.fieldName)).thenReturn(fieldData.getMappedField()); + when(queryShardContext.fieldMapper(fieldData.fieldName)).thenReturn(fieldData.getMappedField()); + } + List compositeIndexFields = starTreeDocValuesReader.getCompositeIndexFields(); CompositeIndexFieldInfo starTree = compositeIndexFields.get(0); @@ -166,131 +234,71 @@ public void testStarTreeDocValues() throws IOException { ValueCountAggregationBuilder valueCountAggregationBuilder = count("_name").field(FIELD_NAME); AvgAggregationBuilder avgAggregationBuilder = avg("_name").field(FIELD_NAME); - List supportedDimensions = new LinkedList<>(); - supportedDimensions.add(new NumericDimension(SNDV)); - supportedDimensions.add(new NumericDimension(DV)); - - Query query = new MatchAllDocsQuery(); - // match-all query - QueryBuilder queryBuilder = null; // no predicates - testCase( - indexSearcher, - query, - queryBuilder, - sumAggregationBuilder, - starTree, - supportedDimensions, - verifyAggregation(InternalSum::getValue) - ); - testCase( - indexSearcher, - query, - queryBuilder, - maxAggregationBuilder, - starTree, - supportedDimensions, - verifyAggregation(InternalMax::getValue) - ); - testCase( - indexSearcher, - query, - queryBuilder, - minAggregationBuilder, - starTree, - supportedDimensions, - verifyAggregation(InternalMin::getValue) - ); - testCase( - indexSearcher, - query, - queryBuilder, - valueCountAggregationBuilder, - starTree, - supportedDimensions, - verifyAggregation(InternalValueCount::getValue) - ); - testCase( - indexSearcher, - query, - queryBuilder, - avgAggregationBuilder, - starTree, - supportedDimensions, - verifyAggregation(InternalAvg::getValue) - ); + LinkedHashMap supportedDimensions = dimensionFieldData.stream() + .collect( + Collectors.toMap(DimensionFieldData::getDimension, DimensionFieldData::getMappedField, (v1, v2) -> v1, LinkedHashMap::new) + ); - // Numeric-terms query - for (int cases = 0; cases < 100; cases++) { - String queryField; - long queryValue; - if (randomBoolean()) { - queryField = SNDV; - queryValue = random.nextInt(10); - } else { - queryField = DV; - queryValue = random.nextInt(20) - 15; + Query query = null; + QueryBuilder queryBuilder = null; + + for (int cases = 0; cases < 15; cases++) { + // Get all types of queries (Term/Terms/Range) for all the given dimensions. + List allFieldQueries = dimensionFieldData.stream() + .flatMap(x -> Stream.of(x.getTermQueryBuilder(), x.getTermsQueryBuilder(), x.getRangeQueryBuilder())) + .toList(); + + for (QueryBuilder qb : allFieldQueries) { + query = qb.toQuery(queryShardContext); + queryBuilder = qb; + testCase( + indexSearcher, + query, + qb, + sumAggregationBuilder, + starTree, + supportedDimensions, + verifyAggregation(InternalSum::getValue) + ); + testCase( + indexSearcher, + query, + qb, + maxAggregationBuilder, + starTree, + supportedDimensions, + verifyAggregation(InternalMax::getValue) + ); + testCase( + indexSearcher, + query, + qb, + minAggregationBuilder, + starTree, + supportedDimensions, + verifyAggregation(InternalMin::getValue) + ); + testCase( + indexSearcher, + query, + qb, + valueCountAggregationBuilder, + starTree, + supportedDimensions, + verifyAggregation(InternalValueCount::getValue) + ); + testCase( + indexSearcher, + query, + qb, + avgAggregationBuilder, + starTree, + supportedDimensions, + verifyAggregation(InternalAvg::getValue) + ); } - - query = SortedNumericDocValuesField.newSlowExactQuery(queryField, queryValue); - queryBuilder = new TermQueryBuilder(queryField, queryValue); - - testCase( - indexSearcher, - query, - queryBuilder, - sumAggregationBuilder, - starTree, - supportedDimensions, - verifyAggregation(InternalSum::getValue) - ); - testCase( - indexSearcher, - query, - queryBuilder, - maxAggregationBuilder, - starTree, - supportedDimensions, - verifyAggregation(InternalMax::getValue) - ); - testCase( - indexSearcher, - query, - queryBuilder, - minAggregationBuilder, - starTree, - supportedDimensions, - verifyAggregation(InternalMin::getValue) - ); - testCase( - indexSearcher, - query, - queryBuilder, - valueCountAggregationBuilder, - starTree, - supportedDimensions, - verifyAggregation(InternalValueCount::getValue) - ); - testCase( - indexSearcher, - query, - queryBuilder, - avgAggregationBuilder, - starTree, - supportedDimensions, - verifyAggregation(InternalAvg::getValue) - ); } - CircuitBreakerService circuitBreakerService = new NoneCircuitBreakerService(); - - QueryShardContext queryShardContext = queryShardContextMock( - indexSearcher, - mapperServiceMock(), - createIndexSettings(), - circuitBreakerService, - new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), circuitBreakerService).withCircuitBreaking() - ); - MetricAggregatorFactory aggregatorFactory = mock(MetricAggregatorFactory.class); when(aggregatorFactory.getSubFactories()).thenReturn(AggregatorFactories.EMPTY); when(aggregatorFactory.getField()).thenReturn(FIELD_NAME); @@ -385,6 +393,22 @@ public void testStarTreeDocValues() throws IOException { false ); + // Keyword Range query with missing Low Ordinal + RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder("keyword_field"); + rangeQueryBuilder.from(Long.MAX_VALUE).includeLower(random().nextBoolean()); + testCase( + indexSearcher, + rangeQueryBuilder.toQuery(queryShardContext), + rangeQueryBuilder, + sumAggregationBuilder, + starTree, + supportedDimensions, + List.of(new Metric(FIELD_NAME, List.of(MetricStat.SUM, MetricStat.MAX, MetricStat.MIN, MetricStat.AVG))), + verifyAggregation(InternalSum::getValue), + null, + true + ); + ir.close(); directory.close(); } @@ -403,7 +427,7 @@ private void testC QueryBuilder queryBuilder, T aggBuilder, CompositeIndexFieldInfo starTree, - List supportedDimensions, + LinkedHashMap supportedDimensions, BiConsumer verify ) throws IOException { testCase(searcher, query, queryBuilder, aggBuilder, starTree, supportedDimensions, Collections.emptyList(), verify, null, true); @@ -415,7 +439,8 @@ private void testC QueryBuilder queryBuilder, T aggBuilder, CompositeIndexFieldInfo starTree, - List supportedDimensions, + LinkedHashMap supportedDimensions, // FIXME : Merge with the same input that goes to generating the + // codec. List supportedMetrics, BiConsumer verify, AggregatorFactory aggregatorFactory, @@ -453,4 +478,164 @@ private void testC ); verify.accept(expectedAggregation, starTreeAggregation); } + + private interface DimensionFieldDataSupplier { + IndexableField getField(String fieldName, Supplier valueSupplier); + + MappedFieldType getMappedField(String fieldName); + + Dimension getDimension(String fieldName); + } + + private abstract static class NumericDimensionFieldDataSupplier implements DimensionFieldDataSupplier { + + @Override + public Dimension getDimension(String fieldName) { + return new NumericDimension(fieldName); + } + + @Override + public MappedFieldType getMappedField(String fieldName) { + return new NumberFieldMapper.NumberFieldType(fieldName, numberType()); + } + + abstract NumberFieldMapper.NumberType numberType(); + } + + private static class DimensionFieldData { + private final String fieldName; + private final Supplier valueSupplier; + private final DimensionFieldDataSupplier dimensionFieldDataSupplier; + private final String fieldType; + + DimensionFieldData(String fieldName, Supplier valueSupplier, DimensionTypes dimensionType) { + this.fieldName = fieldName; + this.valueSupplier = valueSupplier; + this.dimensionFieldDataSupplier = dimensionType.getFieldDataSupplier(); + this.fieldType = dimensionType.name().toLowerCase(Locale.ROOT); + } + + public Dimension getDimension() { + return dimensionFieldDataSupplier.getDimension(fieldName); + } + + public MappedFieldType getMappedField() { + return dimensionFieldDataSupplier.getMappedField(fieldName); + } + + public IndexableField getField() { + return dimensionFieldDataSupplier.getField(fieldName, valueSupplier); + } + + public QueryBuilder getTermQueryBuilder() { + return new TermQueryBuilder(fieldName, valueSupplier.get()); + } + + public QueryBuilder getTermsQueryBuilder() { + int limit = randomIntBetween(1, 20); + List values = new ArrayList<>(limit); + for (int i = 0; i < limit; i++) { + values.add(valueSupplier.get()); + } + return new TermsQueryBuilder(fieldName, values); + } + + public QueryBuilder getRangeQueryBuilder() { + return new RangeQueryBuilder(fieldName).from(valueSupplier.get()) + .to(valueSupplier.get()) + .includeLower(randomBoolean()) + .includeUpper(randomBoolean()); + } + + public String getFieldType() { + return fieldType; + } + } + + private enum DimensionTypes { + + INTEGER(new NumericDimensionFieldDataSupplier() { + @Override + NumberFieldMapper.NumberType numberType() { + return NumberFieldMapper.NumberType.INTEGER; + } + + @Override + public IndexableField getField(String fieldName, Supplier valueSupplier) { + return new IntField(fieldName, (Integer) valueSupplier.get(), Field.Store.YES); + } + }), + LONG(new NumericDimensionFieldDataSupplier() { + @Override + NumberFieldMapper.NumberType numberType() { + return NumberFieldMapper.NumberType.LONG; + } + + @Override + public IndexableField getField(String fieldName, Supplier valueSupplier) { + return new LongField(fieldName, (Integer) valueSupplier.get(), Field.Store.YES); + } + }), + HALF_FLOAT(new NumericDimensionFieldDataSupplier() { + @Override + public IndexableField getField(String fieldName, Supplier valueSupplier) { + return new SortedNumericDocValuesField(fieldName, HalfFloatPoint.halfFloatToSortableShort((Float) valueSupplier.get())); + } + + @Override + NumberFieldMapper.NumberType numberType() { + return NumberFieldMapper.NumberType.HALF_FLOAT; + } + }), + FLOAT(new NumericDimensionFieldDataSupplier() { + @Override + public IndexableField getField(String fieldName, Supplier valueSupplier) { + return new FloatField(fieldName, (Float) valueSupplier.get(), Field.Store.YES); + } + + @Override + NumberFieldMapper.NumberType numberType() { + return NumberFieldMapper.NumberType.FLOAT; + } + }), + DOUBLE(new NumericDimensionFieldDataSupplier() { + @Override + public IndexableField getField(String fieldName, Supplier valueSupplier) { + return new DoubleField(fieldName, (Double) valueSupplier.get(), Field.Store.YES); + } + + @Override + NumberFieldMapper.NumberType numberType() { + return NumberFieldMapper.NumberType.DOUBLE; + } + }), + KEYWORD(new DimensionFieldDataSupplier() { + @Override + public IndexableField getField(String fieldName, Supplier valueSupplier) { + return new KeywordField(fieldName, String.valueOf(valueSupplier.get()), Field.Store.YES); + } + + @Override + public MappedFieldType getMappedField(String fieldName) { + return new KeywordFieldMapper.KeywordFieldType(fieldName, Lucene.STANDARD_ANALYZER); + } + + @Override + public Dimension getDimension(String fieldName) { + return new OrdinalDimension(fieldName); + } + }); + + private final DimensionFieldDataSupplier dimensionFieldDataSupplier; + + DimensionTypes(DimensionFieldDataSupplier dimensionFieldDataSupplier) { + this.dimensionFieldDataSupplier = dimensionFieldDataSupplier; + } + + public DimensionFieldDataSupplier getFieldDataSupplier() { + return dimensionFieldDataSupplier; + } + + } + } diff --git a/server/src/test/java/org/opensearch/search/aggregations/startree/StarTreeFilterTests.java b/server/src/test/java/org/opensearch/search/aggregations/startree/StarTreeFilterTests.java index ef8e858e3efe1..7282b0fafb8aa 100644 --- a/server/src/test/java/org/opensearch/search/aggregations/startree/StarTreeFilterTests.java +++ b/server/src/test/java/org/opensearch/search/aggregations/startree/StarTreeFilterTests.java @@ -33,20 +33,29 @@ import org.opensearch.index.codec.composite912.datacube.startree.StarTreeDocValuesFormatTests; import org.opensearch.index.compositeindex.datacube.MetricStat; import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; -import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeQueryHelper; import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils; import org.opensearch.index.compositeindex.datacube.startree.utils.iterator.SortedNumericStarTreeValuesIterator; import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.mapper.NumberFieldMapper; import org.opensearch.search.aggregations.AggregatorTestCase; -import org.opensearch.search.startree.StarTreeFilter; +import org.opensearch.search.internal.SearchContext; +import org.opensearch.search.startree.StarTreeQueryHelper; +import org.opensearch.search.startree.StarTreeTraversalUtil; +import org.opensearch.search.startree.filter.DimensionFilter; +import org.opensearch.search.startree.filter.ExactMatchDimFilter; +import org.opensearch.search.startree.filter.RangeMatchDimFilter; +import org.opensearch.search.startree.filter.StarTreeFilter; import org.junit.After; import org.junit.Before; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Set; + +import org.mockito.Mockito; import static org.opensearch.index.codec.composite912.datacube.startree.AbstractStarTreeDVFormatTests.topMapping; @@ -57,6 +66,16 @@ public class StarTreeFilterTests extends AggregatorTestCase { private static final String SDV = "sdv"; private static final String DV = "dv"; + public static final LinkedHashMap DIMENSION_TYPE_MAP = new LinkedHashMap<>(); + public static final Map METRIC_TYPE_MAP = Map.of(FIELD_NAME, "integer"); + + static { + // Ordered dimensions + DIMENSION_TYPE_MAP.put(SNDV, "integer"); + DIMENSION_TYPE_MAP.put(SDV, "integer"); + DIMENSION_TYPE_MAP.put(DV, "integer"); + } + @Before public void setup() { FeatureFlags.initializeFeatureFlags(Settings.builder().put(FeatureFlags.STAR_TREE_INDEX, true).build()); @@ -72,7 +91,7 @@ protected Codec getCodec(int maxLeafDoc, boolean skipStarNodeCreationForSDVDimen MapperService mapperService; try { mapperService = StarTreeDocValuesFormatTests.createMapperService( - getExpandedMapping(maxLeafDoc, skipStarNodeCreationForSDVDimension) + getExpandedMapping(maxLeafDoc, skipStarNodeCreationForSDVDimension, DIMENSION_TYPE_MAP, METRIC_TYPE_MAP) ); } catch (IOException e) { throw new RuntimeException(e); @@ -88,6 +107,64 @@ public void testStarTreeFilterWithDocsInSVDFieldButNoStarNode() throws IOExcepti testStarTreeFilter(10, false); } + public void testStarTreeFilterMerging() { + + StarTreeFilter mergedStarTreeFilter; + String dimensionToMerge = "dim"; + + DimensionFilter exactMatchDimFilter = new ExactMatchDimFilter(dimensionToMerge, Collections.emptyList()); + DimensionFilter rangeMatchDimFilter = new RangeMatchDimFilter(dimensionToMerge, null, null, true, true); + + // When Star Tree doesn't have the same dimension as @dimensionToMerge + StarTreeFilter starTreeFilter = new StarTreeFilter(Collections.emptyMap()); + mergedStarTreeFilter = StarTreeQueryHelper.mergeDimensionFilterIfNotExists( + starTreeFilter, + dimensionToMerge, + List.of(exactMatchDimFilter) + ); + assertEquals(1, mergedStarTreeFilter.getDimensions().size()); + DimensionFilter mergedDimensionFilter1 = mergedStarTreeFilter.getFiltersForDimension(dimensionToMerge).get(0); + assertEquals(ExactMatchDimFilter.class, mergedDimensionFilter1.getClass()); + + // When Star Tree has the same dimension as @dimensionToMerge + starTreeFilter = new StarTreeFilter(Map.of(dimensionToMerge, List.of(rangeMatchDimFilter))); + mergedStarTreeFilter = StarTreeQueryHelper.mergeDimensionFilterIfNotExists( + starTreeFilter, + dimensionToMerge, + List.of(exactMatchDimFilter) + ); + assertEquals(1, mergedStarTreeFilter.getDimensions().size()); + DimensionFilter mergedDimensionFilter2 = mergedStarTreeFilter.getFiltersForDimension(dimensionToMerge).get(0); + assertEquals(RangeMatchDimFilter.class, mergedDimensionFilter2.getClass()); + + // When Star Tree has the same dimension as @dimensionToMerge with other dimensions + starTreeFilter = new StarTreeFilter(Map.of(dimensionToMerge, List.of(rangeMatchDimFilter), "status", List.of(rangeMatchDimFilter))); + mergedStarTreeFilter = StarTreeQueryHelper.mergeDimensionFilterIfNotExists( + starTreeFilter, + dimensionToMerge, + List.of(exactMatchDimFilter) + ); + assertEquals(2, mergedStarTreeFilter.getDimensions().size()); + DimensionFilter mergedDimensionFilter3 = mergedStarTreeFilter.getFiltersForDimension(dimensionToMerge).get(0); + assertEquals(RangeMatchDimFilter.class, mergedDimensionFilter3.getClass()); + DimensionFilter mergedDimensionFilter4 = mergedStarTreeFilter.getFiltersForDimension("status").get(0); + assertEquals(RangeMatchDimFilter.class, mergedDimensionFilter4.getClass()); + + // When Star Tree doesn't have the same dimension as @dimensionToMerge but has other dimensions + starTreeFilter = new StarTreeFilter(Map.of("status", List.of(rangeMatchDimFilter))); + mergedStarTreeFilter = StarTreeQueryHelper.mergeDimensionFilterIfNotExists( + starTreeFilter, + dimensionToMerge, + List.of(exactMatchDimFilter) + ); + assertEquals(2, mergedStarTreeFilter.getDimensions().size()); + DimensionFilter mergedDimensionFilter5 = mergedStarTreeFilter.getFiltersForDimension(dimensionToMerge).get(0); + assertEquals(ExactMatchDimFilter.class, mergedDimensionFilter5.getClass()); + DimensionFilter mergedDimensionFilter6 = mergedStarTreeFilter.getFiltersForDimension("status").get(0); + assertEquals(RangeMatchDimFilter.class, mergedDimensionFilter6.getClass()); + + } + private Directory createStarTreeIndex(int maxLeafDoc, boolean skipStarNodeCreationForSDVDimension, List docs) throws IOException { Directory directory = newDirectory(); @@ -129,69 +206,141 @@ private void testStarTreeFilter(int maxLeafDoc, boolean skipStarNodeCreationForS long starTreeDocCount, docCount; + MapperService mapperService = Mockito.mock(MapperService.class); + SearchContext searchContext = Mockito.mock(SearchContext.class); + + Mockito.when(searchContext.mapperService()).thenReturn(mapperService); + Mockito.when(mapperService.fieldType(SNDV)) + .thenReturn(new NumberFieldMapper.NumberFieldType(SNDV, NumberFieldMapper.NumberType.INTEGER)); + Mockito.when(mapperService.fieldType(DV)) + .thenReturn(new NumberFieldMapper.NumberFieldType(DV, NumberFieldMapper.NumberType.INTEGER)); + Mockito.when(mapperService.fieldType(SDV)) + .thenReturn(new NumberFieldMapper.NumberFieldType(SDV, NumberFieldMapper.NumberType.INTEGER)); + // assert that all documents are included if no filters are given - starTreeDocCount = getDocCountFromStarTree(starTreeDocValuesReader, Map.of(), context); + starTreeDocCount = getDocCountFromStarTree( + starTreeDocValuesReader, + new StarTreeFilter(Collections.emptyMap()), + context, + searchContext + ); docCount = getDocCount(docs, Map.of()); assertEquals(totalDocs, starTreeDocCount); assertEquals(docCount, starTreeDocCount); // single filter - matches docs - starTreeDocCount = getDocCountFromStarTree(starTreeDocValuesReader, Map.of(SNDV, 0L), context); + starTreeDocCount = getDocCountFromStarTree( + starTreeDocValuesReader, + new StarTreeFilter(Map.of(SNDV, List.of(new ExactMatchDimFilter(SNDV, List.of(0L))))), + context, + searchContext + ); docCount = getDocCount(docs, Map.of(SNDV, 0L)); assertEquals(1, docCount); assertEquals(docCount, starTreeDocCount); // single filter on 3rd field in ordered dimension - matches docs - starTreeDocCount = getDocCountFromStarTree(starTreeDocValuesReader, Map.of(DV, 0L), context); + starTreeDocCount = getDocCountFromStarTree( + starTreeDocValuesReader, + new StarTreeFilter(Map.of(DV, List.of(new ExactMatchDimFilter(DV, List.of(0L))))), + context, + searchContext + ); docCount = getDocCount(docs, Map.of(DV, 0L)); assertEquals(1, docCount); assertEquals(docCount, starTreeDocCount); // single filter - does not match docs - starTreeDocCount = getDocCountFromStarTree(starTreeDocValuesReader, Map.of(SNDV, 101L), context); + starTreeDocCount = getDocCountFromStarTree( + starTreeDocValuesReader, + new StarTreeFilter(Map.of(SNDV, List.of(new ExactMatchDimFilter(SNDV, List.of(101L))))), + context, + searchContext + ); docCount = getDocCount(docs, Map.of(SNDV, 101L)); assertEquals(0, docCount); assertEquals(docCount, starTreeDocCount); // single filter on 3rd field in ordered dimension - does not match docs - starTreeDocCount = getDocCountFromStarTree(starTreeDocValuesReader, Map.of(DV, -101L), context); + starTreeDocCount = getDocCountFromStarTree( + starTreeDocValuesReader, + new StarTreeFilter(Map.of(SNDV, List.of(new ExactMatchDimFilter(SNDV, List.of(-101L))))), + context, + searchContext + ); docCount = getDocCount(docs, Map.of(SNDV, -101L)); assertEquals(0, docCount); assertEquals(docCount, starTreeDocCount); // multiple filters - matches docs - starTreeDocCount = getDocCountFromStarTree(starTreeDocValuesReader, Map.of(SNDV, 0L, DV, 0L), context); + starTreeDocCount = getDocCountFromStarTree( + starTreeDocValuesReader, + new StarTreeFilter( + Map.of(SNDV, List.of(new ExactMatchDimFilter(SNDV, List.of(0L))), DV, List.of(new ExactMatchDimFilter(DV, List.of(0L)))) + ), + context, + searchContext + ); docCount = getDocCount(docs, Map.of(SNDV, 0L, DV, 0L)); assertEquals(1, docCount); assertEquals(docCount, starTreeDocCount); // no document should match the filter - starTreeDocCount = getDocCountFromStarTree(starTreeDocValuesReader, Map.of(SNDV, 0L, DV, -11L), context); + starTreeDocCount = getDocCountFromStarTree( + starTreeDocValuesReader, + new StarTreeFilter( + Map.of(SNDV, List.of(new ExactMatchDimFilter(SNDV, List.of(0L))), DV, List.of(new ExactMatchDimFilter(DV, List.of(-11L)))) + ), + context, + searchContext + ); docCount = getDocCount(docs, Map.of(SNDV, 0L, DV, -11L)); assertEquals(0, docCount); assertEquals(docCount, starTreeDocCount); // Only the first filter should match some documents, second filter matches none - starTreeDocCount = getDocCountFromStarTree(starTreeDocValuesReader, Map.of(SNDV, 0L, DV, -100L), context); + starTreeDocCount = getDocCountFromStarTree( + starTreeDocValuesReader, + new StarTreeFilter( + Map.of(SNDV, List.of(new ExactMatchDimFilter(SNDV, List.of(0L))), DV, List.of(new ExactMatchDimFilter(DV, List.of(-100L)))) + ), + context, + searchContext + ); docCount = getDocCount(docs, Map.of(SNDV, 0L, DV, -100L)); assertEquals(0, docCount); assertEquals(docCount, starTreeDocCount); // non-dimension fields in filter - should throw IllegalArgumentException expectThrows( - IllegalArgumentException.class, - () -> getDocCountFromStarTree(starTreeDocValuesReader, Map.of(FIELD_NAME, 0L), context) + IllegalStateException.class, + () -> getDocCountFromStarTree( + starTreeDocValuesReader, + new StarTreeFilter(Map.of(FIELD_NAME, List.of(new ExactMatchDimFilter(FIELD_NAME, List.of(0L))))), + context, + searchContext + ) ); if (skipStarNodeCreationForSDVDimension == true) { // Documents are not indexed - starTreeDocCount = getDocCountFromStarTree(starTreeDocValuesReader, Map.of(SDV, 4L), context); + starTreeDocCount = getDocCountFromStarTree( + starTreeDocValuesReader, + new StarTreeFilter(Map.of(SDV, List.of(new ExactMatchDimFilter(SDV, List.of(4L))))), + context, + searchContext + ); docCount = getDocCount(docs, Map.of(SDV, 4L)); assertEquals(1, docCount); assertEquals(docCount, starTreeDocCount); } else { // Documents are indexed - starTreeDocCount = getDocCountFromStarTree(starTreeDocValuesReader, Map.of(SDV, 4L), context); + starTreeDocCount = getDocCountFromStarTree( + starTreeDocValuesReader, + new StarTreeFilter(Map.of(SDV, List.of(new ExactMatchDimFilter(SDV, List.of(4L))))), + context, + searchContext + ); docCount = getDocCount(docs, Map.of(SDV, 4L)); assertEquals(0, docCount); assertEquals(docCount, starTreeDocCount); @@ -225,12 +374,16 @@ private long getDocCount(List documents, Map filters) { } // Returns count of documents in the star tree having field SNDV & applied filters - private long getDocCountFromStarTree(CompositeIndexReader starTreeDocValuesReader, Map filters, LeafReaderContext context) - throws IOException { + private long getDocCountFromStarTree( + CompositeIndexReader starTreeDocValuesReader, + StarTreeFilter starTreeFilter, + LeafReaderContext context, + SearchContext searchContext + ) throws IOException { List compositeIndexFields = starTreeDocValuesReader.getCompositeIndexFields(); CompositeIndexFieldInfo starTree = compositeIndexFields.get(0); StarTreeValues starTreeValues = StarTreeQueryHelper.getStarTreeValues(context, starTree); - FixedBitSet filteredValues = StarTreeFilter.getStarTreeResult(starTreeValues, filters, Set.of()); + FixedBitSet filteredValues = StarTreeTraversalUtil.getStarTreeResult(starTreeValues, starTreeFilter, searchContext); SortedNumericStarTreeValuesIterator valuesIterator = (SortedNumericStarTreeValuesIterator) starTreeValues.getMetricValuesIterator( StarTreeUtils.fullyQualifiedFieldNameForStarTreeMetricsDocValues( @@ -262,7 +415,12 @@ private long getDocCountFromStarTree(CompositeIndexReader starTreeDocValuesReade return docCount; } - public static XContentBuilder getExpandedMapping(int maxLeafDocs, boolean skipStarNodeCreationForSDVDimension) throws IOException { + public static XContentBuilder getExpandedMapping( + int maxLeafDocs, + boolean skipStarNodeCreationForSDVDimension, + LinkedHashMap dimensionNameAndType, + Map metricFieldNameAndType + ) throws IOException { return topMapping(b -> { b.startObject("composite"); b.startObject("startree"); @@ -274,16 +432,14 @@ public static XContentBuilder getExpandedMapping(int maxLeafDocs, boolean skipSt b.value("sdv"); b.endArray(); } + // FIXME : Change to take dimension order and other inputs as method params. + // FIXME : Create default constants for the existing so other can call easily. b.startArray("ordered_dimensions"); - b.startObject(); - b.field("name", "sndv"); - b.endObject(); - b.startObject(); - b.field("name", "sdv"); - b.endObject(); - b.startObject(); - b.field("name", "dv"); - b.endObject(); + for (String dimension : dimensionNameAndType.keySet()) { + b.startObject(); + b.field("name", dimension); + b.endObject(); + } b.endArray(); b.startArray("metrics"); b.startObject(); @@ -311,18 +467,16 @@ public static XContentBuilder getExpandedMapping(int maxLeafDocs, boolean skipSt b.endObject(); b.endObject(); b.startObject("properties"); - b.startObject("sndv"); - b.field("type", "integer"); - b.endObject(); - b.startObject("sdv"); - b.field("type", "integer"); - b.endObject(); - b.startObject("dv"); - b.field("type", "integer"); - b.endObject(); - b.startObject("field"); - b.field("type", "integer"); - b.endObject(); + for (String dimension : dimensionNameAndType.keySet()) { + b.startObject(dimension); + b.field("type", dimensionNameAndType.get(dimension)); + b.endObject(); + } + for (String metricField : metricFieldNameAndType.keySet()) { + b.startObject(metricField); + b.field("type", metricFieldNameAndType.get(metricField)); + b.endObject(); + } b.endObject(); }); } diff --git a/test/framework/src/main/java/org/opensearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/opensearch/search/aggregations/AggregatorTestCase.java index b982665e01d8a..78e3d4f50a0d5 100644 --- a/test/framework/src/main/java/org/opensearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/opensearch/search/aggregations/AggregatorTestCase.java @@ -95,10 +95,8 @@ import org.opensearch.index.codec.composite.CompositeIndexFieldInfo; import org.opensearch.index.compositeindex.datacube.Dimension; import org.opensearch.index.compositeindex.datacube.Metric; -import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeQueryHelper; import org.opensearch.index.fielddata.IndexFieldData; import org.opensearch.index.fielddata.IndexFieldDataCache; -import org.opensearch.index.fielddata.IndexFieldDataService; import org.opensearch.index.mapper.BinaryFieldMapper; import org.opensearch.index.mapper.CompletionFieldMapper; import org.opensearch.index.mapper.CompositeDataCubeFieldType; @@ -129,7 +127,6 @@ import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.SearchOperationListener; import org.opensearch.indices.IndicesModule; -import org.opensearch.indices.fielddata.cache.IndicesFieldDataCache; import org.opensearch.indices.mapper.MapperRegistry; import org.opensearch.plugins.SearchPlugin; import org.opensearch.script.ScriptService; @@ -143,7 +140,6 @@ import org.opensearch.search.aggregations.support.CoreValuesSourceType; import org.opensearch.search.aggregations.support.ValuesSourceRegistry; import org.opensearch.search.aggregations.support.ValuesSourceType; -import org.opensearch.search.builder.SearchSourceBuilder; import org.opensearch.search.fetch.FetchPhase; import org.opensearch.search.fetch.subphase.FetchDocValuesPhase; import org.opensearch.search.fetch.subphase.FetchSourcePhase; @@ -162,6 +158,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -349,7 +346,7 @@ protected CountingAggregator createCountingAggregator( IndexSearcher indexSearcher, IndexSettings indexSettings, CompositeIndexFieldInfo starTree, - List supportedDimensions, + LinkedHashMap supportedDimensions, List supportedMetrics, MultiBucketConsumer bucketConsumer, AggregatorFactory aggregatorFactory, @@ -394,7 +391,7 @@ protected SearchContext createSearchContextWithStarTreeContext( Query query, QueryBuilder queryBuilder, CompositeIndexFieldInfo starTree, - List supportedDimensions, + LinkedHashMap supportedDimensions, List supportedMetrics, MultiBucketConsumer bucketConsumer, AggregatorFactory aggregatorFactory, @@ -427,16 +424,23 @@ protected SearchContext createSearchContextWithStarTreeContext( when(compositeMappedFieldType.getCompositeIndexType()).thenReturn(starTree.getType()); Set compositeFieldTypes = Set.of(compositeMappedFieldType); - when((compositeMappedFieldType).getDimensions()).thenReturn(supportedDimensions); + when((compositeMappedFieldType).getDimensions()).thenReturn(new ArrayList<>(supportedDimensions.keySet())); when((compositeMappedFieldType).getMetrics()).thenReturn(supportedMetrics); MapperService mapperService = mock(MapperService.class); when(mapperService.getCompositeFieldTypes()).thenReturn(compositeFieldTypes); when(searchContext.mapperService()).thenReturn(mapperService); - SearchSourceBuilder sb = new SearchSourceBuilder().query(queryBuilder); - StarTreeQueryContext starTreeQueryContext = StarTreeQueryHelper.getStarTreeQueryContext(searchContext, sb); + for (Dimension dimension : supportedDimensions.keySet()) { + when(mapperService.fieldType(dimension.getField())).thenReturn(supportedDimensions.get(dimension)); + when(searchContext.getQueryShardContext().fieldMapper(dimension.getField())).thenReturn(supportedDimensions.get(dimension)); + } + + StarTreeQueryContext starTreeQueryContext = new StarTreeQueryContext(searchContext, queryBuilder); + boolean consolidated = starTreeQueryContext.consolidateAllFilters(searchContext); + if (consolidated) { + searchContext.getQueryShardContext().setStarTreeQueryContext(starTreeQueryContext); + } - when(searchContext.getStarTreeQueryContext()).thenReturn(starTreeQueryContext); return searchContext; } @@ -496,13 +500,6 @@ public boolean shouldCache(Query query) { when(mapperService.getIndexSettings()).thenReturn(indexSettings); when(mapperService.hasNested()).thenReturn(false); when(searchContext.mapperService()).thenReturn(mapperService); - IndexFieldDataService ifds = new IndexFieldDataService( - indexSettings, - new IndicesFieldDataCache(Settings.EMPTY, new IndexFieldDataCache.Listener() { - }), - circuitBreakerService, - mapperService - ); QueryShardContext queryShardContext = queryShardContextMock( contextIndexSearcher, mapperService, @@ -768,7 +765,7 @@ protected A searchAndReduc QueryBuilder queryBuilder, AggregationBuilder builder, CompositeIndexFieldInfo compositeIndexFieldInfo, - List supportedDimensions, + LinkedHashMap supportedDimensions, List supportedMetrics, int maxBucket, boolean hasNested,