diff --git a/server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexReader.java b/server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexReader.java index a159b0619bcbb..9beb99e4a97cd 100644 --- a/server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexReader.java +++ b/server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexReader.java @@ -9,6 +9,7 @@ package org.opensearch.index.codec.composite; import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.compositeindex.datacube.startree.index.CompositeIndexValues; import java.io.IOException; import java.util.List; diff --git a/server/src/main/java/org/opensearch/index/codec/composite/composite99/Composite99DocValuesFormat.java b/server/src/main/java/org/opensearch/index/codec/composite/composite99/Composite99DocValuesFormat.java index e8c69b11b7c88..63fe5a6af76f7 100644 --- a/server/src/main/java/org/opensearch/index/codec/composite/composite99/Composite99DocValuesFormat.java +++ b/server/src/main/java/org/opensearch/index/codec/composite/composite99/Composite99DocValuesFormat.java @@ -37,6 +37,36 @@ public class Composite99DocValuesFormat extends DocValuesFormat { private final DocValuesFormat delegate; private final MapperService mapperService; + /** Data codec name for Composite Doc Values Format */ + public static final String DATA_CODEC_NAME = "Composite99FormatData"; + + /** Meta codec name for Composite Doc Values Format */ + public static final String META_CODEC_NAME = "Composite99FormatMeta"; + + /** Filename extension for the composite index data */ + public static final String DATA_EXTENSION = "cid"; + + /** Filename extension for the composite index meta */ + public static final String META_EXTENSION = "cim"; + + /** Data doc values codec name for Composite Doc Values Format */ + public static final String DATA_DOC_VALUES_CODEC = "Composite99DocValuesData"; + + /** Meta doc values codec name for Composite Doc Values Format */ + public static final String META_DOC_VALUES_CODEC = "Composite99DocValuesMetadata"; + + /** Filename extension for the composite index data doc values */ + public static final String DATA_DOC_VALUES_EXTENSION = "cidvd"; + + /** Filename extension for the composite index meta doc values */ + public static final String META_DOC_VALUES_EXTENSION = "cidvm"; + + /** Initial version for the Composite90DocValuesFormat */ + public static final int VERSION_START = 0; + + /** Current version for the Composite90DocValuesFormat */ + public static final int VERSION_CURRENT = VERSION_START; + // needed for SPI public Composite99DocValuesFormat() { this(new Lucene90DocValuesFormat(), null); diff --git a/server/src/main/java/org/opensearch/index/codec/composite/composite99/Composite99DocValuesReader.java b/server/src/main/java/org/opensearch/index/codec/composite/composite99/Composite99DocValuesReader.java index e3bfe01cfa2d5..ca55e8e4573ab 100644 --- a/server/src/main/java/org/opensearch/index/codec/composite/composite99/Composite99DocValuesReader.java +++ b/server/src/main/java/org/opensearch/index/codec/composite/composite99/Composite99DocValuesReader.java @@ -8,22 +8,47 @@ package org.opensearch.index.codec.composite.composite99; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.DocValues; import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SortedDocValues; import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.IndexInput; import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.util.io.IOUtils; import org.opensearch.index.codec.composite.CompositeIndexFieldInfo; import org.opensearch.index.codec.composite.CompositeIndexReader; -import org.opensearch.index.codec.composite.CompositeIndexValues; +import org.opensearch.index.codec.composite.LuceneDocValuesProducerFactory; +import org.opensearch.index.compositeindex.CompositeIndexMetadata; +import org.opensearch.index.compositeindex.datacube.Metric; +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.startree.fileformats.meta.StarTreeMetadata; +import org.opensearch.index.compositeindex.datacube.startree.index.CompositeIndexValues; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; +import org.opensearch.index.mapper.CompositeMappedFieldType; import java.io.IOException; import java.util.ArrayList; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; + +import static org.opensearch.index.compositeindex.CompositeIndexConstants.COMPOSITE_FIELD_MARKER; +import static org.opensearch.index.compositeindex.datacube.startree.fileformats.StarTreeWriter.VERSION_CURRENT; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.fullyQualifiedFieldNameForStarTreeDimensionsDocValues; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.fullyQualifiedFieldNameForStarTreeMetricsDocValues; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.getFieldInfoList; /** * Reader for star tree index and star tree doc values from the segments @@ -32,11 +57,158 @@ */ @ExperimentalApi public class Composite99DocValuesReader extends DocValuesProducer implements CompositeIndexReader { - private DocValuesProducer delegate; + private static final Logger logger = LogManager.getLogger(Composite99DocValuesReader.class); + + private final DocValuesProducer delegate; + private IndexInput dataIn; + private ChecksumIndexInput metaIn; + private final Map compositeIndexInputMap = new LinkedHashMap<>(); + private final Map compositeIndexMetadataMap = new LinkedHashMap<>(); + private final List fields; + private DocValuesProducer compositeDocValuesProducer; + private final List compositeFieldInfos = new ArrayList<>(); + private SegmentReadState readState; - public Composite99DocValuesReader(DocValuesProducer producer, SegmentReadState state) throws IOException { + public Composite99DocValuesReader(DocValuesProducer producer, SegmentReadState readState) throws IOException { this.delegate = producer; - // TODO : read star tree files + this.fields = new ArrayList<>(); + + String metaFileName = IndexFileNames.segmentFileName( + readState.segmentInfo.name, + readState.segmentSuffix, + Composite99DocValuesFormat.META_EXTENSION + ); + + String dataFileName = IndexFileNames.segmentFileName( + readState.segmentInfo.name, + readState.segmentSuffix, + Composite99DocValuesFormat.DATA_EXTENSION + ); + + boolean success = false; + try { + + // initialize meta input + dataIn = readState.directory.openInput(dataFileName, readState.context); + CodecUtil.checkIndexHeader( + dataIn, + Composite99DocValuesFormat.DATA_CODEC_NAME, + Composite99DocValuesFormat.VERSION_START, + Composite99DocValuesFormat.VERSION_CURRENT, + readState.segmentInfo.getId(), + readState.segmentSuffix + ); + + // initialize data input + metaIn = readState.directory.openChecksumInput(metaFileName, readState.context); + Throwable priorE = null; + try { + CodecUtil.checkIndexHeader( + metaIn, + Composite99DocValuesFormat.META_CODEC_NAME, + Composite99DocValuesFormat.VERSION_START, + Composite99DocValuesFormat.VERSION_CURRENT, + readState.segmentInfo.getId(), + readState.segmentSuffix + ); + + while (true) { + + // validate magic marker + long magicMarker = metaIn.readLong(); + if (magicMarker == -1) { + break; + } else if (magicMarker < 0) { + throw new CorruptIndexException("Unknown token encountered: " + magicMarker, metaIn); + } else if (COMPOSITE_FIELD_MARKER != magicMarker) { + logger.error("Invalid composite field magic marker"); + throw new IOException("Invalid composite field magic marker"); + } + + int version = metaIn.readVInt(); + if (VERSION_CURRENT != version) { + logger.error("Invalid composite field version"); + throw new IOException("Invalid composite field version"); + } + + // construct composite index metadata + String compositeFieldName = metaIn.readString(); + CompositeMappedFieldType.CompositeFieldType compositeFieldType = CompositeMappedFieldType.CompositeFieldType.fromName( + metaIn.readString() + ); + + switch (compositeFieldType) { + case STAR_TREE: + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + metaIn, + compositeFieldName, + compositeFieldType, + version + ); + compositeFieldInfos.add(new CompositeIndexFieldInfo(compositeFieldName, compositeFieldType)); + + IndexInput starTreeIndexInput = dataIn.slice( + "star-tree data slice for respective star-tree fields", + starTreeMetadata.getDataStartFilePointer(), + starTreeMetadata.getDataLength() + ); + compositeIndexInputMap.put(compositeFieldName, starTreeIndexInput); + compositeIndexMetadataMap.put(compositeFieldName, starTreeMetadata); + + List dimensionFields = starTreeMetadata.getDimensionFields(); + + // generating star tree unique fields (fully qualified name for dimension and metrics) + for (String dimensions : dimensionFields) { + fields.add(fullyQualifiedFieldNameForStarTreeDimensionsDocValues(compositeFieldName, dimensions)); + } + + // adding metric fields + for (Metric metric : starTreeMetadata.getMetrics()) { + for (MetricStat metricStat : metric.getMetrics()) { + fields.add( + fullyQualifiedFieldNameForStarTreeMetricsDocValues( + compositeFieldName, + metric.getField(), + metricStat.getTypeName() + ) + ); + + } + } + + break; + default: + throw new CorruptIndexException("Invalid composite field type found in the file", dataIn); + } + } + + // populates the dummy list of field infos to fetch doc id set iterators for respective fields. + // the dummy field info is used to fetch the doc id set iterators for respective fields based on field name + FieldInfos fieldInfos = new FieldInfos(getFieldInfoList(fields)); + this.readState = new SegmentReadState(readState.directory, readState.segmentInfo, fieldInfos, readState.context); + + // initialize star-tree doc values producer + + compositeDocValuesProducer = LuceneDocValuesProducerFactory.getDocValuesProducerForCompositeCodec( + Composite99Codec.COMPOSITE_INDEX_CODEC_NAME, + this.readState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + + } catch (Throwable t) { + priorE = t; + } finally { + CodecUtil.checkFooter(metaIn, priorE); + } + success = true; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); + } + } } @Override @@ -67,24 +239,63 @@ public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { @Override public void checkIntegrity() throws IOException { delegate.checkIntegrity(); - // Todo : check integrity of composite index related [star tree] files + CodecUtil.checksumEntireFile(dataIn); } @Override public void close() throws IOException { delegate.close(); - // Todo: close composite index related files [star tree] files + boolean success = false; + try { + IOUtils.close(metaIn, dataIn); + IOUtils.close(compositeDocValuesProducer); + success = true; + } finally { + if (!success) { + IOUtils.closeWhileHandlingException(metaIn, dataIn); + } + compositeIndexInputMap.clear(); + compositeIndexMetadataMap.clear(); + fields.clear(); + metaIn = null; + dataIn = null; + } } @Override public List getCompositeIndexFields() { - // todo : read from file formats and get the field names. - return new ArrayList<>(); + return compositeFieldInfos; } @Override public CompositeIndexValues getCompositeIndexValues(CompositeIndexFieldInfo compositeIndexFieldInfo) throws IOException { - // TODO : read compositeIndexValues [starTreeValues] from star tree files - throw new UnsupportedOperationException(); + + switch (compositeIndexFieldInfo.getType()) { + case STAR_TREE: + return new StarTreeValues( + compositeIndexMetadataMap.get(compositeIndexFieldInfo.getField()), + compositeIndexInputMap.get(compositeIndexFieldInfo.getField()), + compositeDocValuesProducer, + this.readState + ); + + default: + throw new CorruptIndexException("Unsupported composite index field type: ", compositeIndexFieldInfo.getType().getName()); + } + } + + /** + * Returns the sorted numeric doc values for the given sorted numeric field. + * If the sorted numeric field is null, it returns an empty doc id set iterator. + *

+ * Sorted numeric field can be null for cases where the segment doesn't hold a particular value. + * + * @param sortedNumeric the sorted numeric doc values for a field + * @return empty sorted numeric values if the field is not present, else sortedNumeric + */ + public static SortedNumericDocValues getSortedNumericDocValues(SortedNumericDocValues sortedNumeric) { + return sortedNumeric == null ? DocValues.emptySortedNumeric() : sortedNumeric; + } + } diff --git a/server/src/main/java/org/opensearch/index/codec/composite/composite99/Composite99DocValuesWriter.java b/server/src/main/java/org/opensearch/index/codec/composite/composite99/Composite99DocValuesWriter.java index 74ab7d423998e..0d4e35f7c3ab8 100644 --- a/server/src/main/java/org/opensearch/index/codec/composite/composite99/Composite99DocValuesWriter.java +++ b/server/src/main/java/org/opensearch/index/codec/composite/composite99/Composite99DocValuesWriter.java @@ -8,34 +8,44 @@ package org.opensearch.index.codec.composite.composite99; +import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.index.DocValues; import org.apache.lucene.index.DocValuesType; import org.apache.lucene.index.EmptyDocValuesProducer; import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.MergeState; import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexOutput; import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.util.io.IOUtils; import org.opensearch.index.codec.composite.CompositeIndexFieldInfo; import org.opensearch.index.codec.composite.CompositeIndexReader; -import org.opensearch.index.codec.composite.CompositeIndexValues; -import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; +import org.opensearch.index.codec.composite.LuceneDocValuesConsumerFactory; import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; import org.opensearch.index.compositeindex.datacube.startree.builder.StarTreesBuilder; +import org.opensearch.index.compositeindex.datacube.startree.index.CompositeIndexValues; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; import org.opensearch.index.mapper.CompositeMappedFieldType; import org.opensearch.index.mapper.DocCountFieldMapper; import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.mapper.StarTreeMapper; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; /** @@ -52,21 +62,28 @@ public class Composite99DocValuesWriter extends DocValuesConsumer { AtomicReference mergeState = new AtomicReference<>(); private final Set compositeMappedFieldTypes; private final Set compositeFieldSet; + private DocValuesConsumer composite99DocValuesConsumer; + + public IndexOutput dataOut; + public IndexOutput metaOut; private final Set segmentFieldSet; private final boolean segmentHasCompositeFields; + private final AtomicInteger fieldNumberAcrossCompositeFields; private final Map fieldProducerMap = new HashMap<>(); - public Composite99DocValuesWriter(DocValuesConsumer delegate, SegmentWriteState segmentWriteState, MapperService mapperService) { + public Composite99DocValuesWriter(DocValuesConsumer delegate, SegmentWriteState segmentWriteState, MapperService mapperService) + throws IOException { this.delegate = delegate; this.state = segmentWriteState; this.mapperService = mapperService; + this.fieldNumberAcrossCompositeFields = new AtomicInteger(); this.compositeMappedFieldTypes = mapperService.getCompositeFieldTypes(); compositeFieldSet = new HashSet<>(); segmentFieldSet = new HashSet<>(); // TODO : add integ test for this - for (FieldInfo fi : segmentWriteState.fieldInfos) { + for (FieldInfo fi : this.state.fieldInfos) { if (DocValuesType.SORTED_NUMERIC.equals(fi.getDocValuesType())) { segmentFieldSet.add(fi.name); } else if (fi.name.equals(DocCountFieldMapper.NAME)) { @@ -76,6 +93,56 @@ public Composite99DocValuesWriter(DocValuesConsumer delegate, SegmentWriteState for (CompositeMappedFieldType type : compositeMappedFieldTypes) { compositeFieldSet.addAll(type.fields()); } + + boolean success = false; + try { + + // Get consumer write state with DocIdSetIterator.NO_MORE_DOCS as segment doc count, + // so that all the fields are sparse numeric doc values and not dense numeric doc values + SegmentWriteState consumerWriteState = getSegmentWriteState(segmentWriteState); + + this.composite99DocValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + consumerWriteState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + + String dataFileName = IndexFileNames.segmentFileName( + this.state.segmentInfo.name, + this.state.segmentSuffix, + Composite99DocValuesFormat.DATA_EXTENSION + ); + dataOut = this.state.directory.createOutput(dataFileName, this.state.context); + CodecUtil.writeIndexHeader( + dataOut, + Composite99DocValuesFormat.DATA_CODEC_NAME, + Composite99DocValuesFormat.VERSION_CURRENT, + this.state.segmentInfo.getId(), + this.state.segmentSuffix + ); + + String metaFileName = IndexFileNames.segmentFileName( + this.state.segmentInfo.name, + this.state.segmentSuffix, + Composite99DocValuesFormat.META_EXTENSION + ); + metaOut = this.state.directory.createOutput(metaFileName, this.state.context); + CodecUtil.writeIndexHeader( + metaOut, + Composite99DocValuesFormat.META_CODEC_NAME, + Composite99DocValuesFormat.VERSION_CURRENT, + this.state.segmentInfo.getId(), + this.state.segmentSuffix + ); + + success = true; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); + } + } // check if there are any composite fields which are part of the segment // TODO : add integ test where there are no composite fields in a segment, test both flush and merge cases segmentHasCompositeFields = Collections.disjoint(segmentFieldSet, compositeFieldSet) == false; @@ -117,6 +184,26 @@ public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer) @Override public void close() throws IOException { delegate.close(); + boolean success = false; + try { + if (metaOut != null) { + metaOut.writeLong(-1); // write EOF marker + CodecUtil.writeFooter(metaOut); // write checksum + } + if (dataOut != null) { + CodecUtil.writeFooter(dataOut); // write checksum + } + + success = true; + } finally { + if (success) { + IOUtils.close(dataOut, metaOut, composite99DocValuesConsumer); + } else { + IOUtils.closeWhileHandlingException(dataOut, metaOut, composite99DocValuesConsumer); + } + metaOut = dataOut = null; + composite99DocValuesConsumer = null; + } } private void createCompositeIndicesIfPossible(DocValuesProducer valuesProducer, FieldInfo field) throws IOException { @@ -135,9 +222,9 @@ private void createCompositeIndicesIfPossible(DocValuesProducer valuesProducer, // we have all the required fields to build composite fields if (compositeFieldSet.isEmpty()) { for (CompositeMappedFieldType mappedType : compositeMappedFieldTypes) { - if (mappedType.getCompositeIndexType().equals(CompositeMappedFieldType.CompositeFieldType.STAR_TREE)) { - try (StarTreesBuilder starTreesBuilder = new StarTreesBuilder(state, mapperService)) { - starTreesBuilder.build(fieldProducerMap); + if (mappedType instanceof StarTreeMapper.StarTreeFieldType) { + try (StarTreesBuilder starTreesBuilder = new StarTreesBuilder(state, mapperService, fieldNumberAcrossCompositeFields)) { + starTreesBuilder.build(metaOut, dataOut, fieldProducerMap, composite99DocValuesConsumer); } } } @@ -175,6 +262,7 @@ public void merge(MergeState mergeState) throws IOException { /** * Merges composite fields from multiple segments + * * @param mergeState merge state */ private void mergeCompositeFields(MergeState mergeState) throws IOException { @@ -183,6 +271,7 @@ private void mergeCompositeFields(MergeState mergeState) throws IOException { /** * Merges star tree data fields from multiple segments + * * @param mergeState merge state */ private void mergeStarTreeFields(MergeState mergeState) throws IOException { @@ -205,7 +294,7 @@ private void mergeStarTreeFields(MergeState mergeState) throws IOException { CompositeIndexValues compositeIndexValues = reader.getCompositeIndexValues(fieldInfo); if (compositeIndexValues instanceof StarTreeValues) { StarTreeValues starTreeValues = (StarTreeValues) compositeIndexValues; - List fieldsList = starTreeSubsPerField.getOrDefault(fieldInfo.getField(), Collections.emptyList()); + List fieldsList = starTreeSubsPerField.getOrDefault(fieldInfo.getField(), new ArrayList<>()); if (starTreeField == null) { starTreeField = starTreeValues.getStarTreeField(); } @@ -223,8 +312,36 @@ private void mergeStarTreeFields(MergeState mergeState) throws IOException { } } } - try (StarTreesBuilder starTreesBuilder = new StarTreesBuilder(state, mapperService)) { - starTreesBuilder.buildDuringMerge(starTreeSubsPerField); + try (StarTreesBuilder starTreesBuilder = new StarTreesBuilder(state, mapperService, fieldNumberAcrossCompositeFields)) { + starTreesBuilder.buildDuringMerge(metaOut, dataOut, starTreeSubsPerField, composite99DocValuesConsumer); } } + + private static SegmentWriteState getSegmentWriteState(SegmentWriteState segmentWriteState) { + + SegmentInfo segmentInfo = new SegmentInfo( + segmentWriteState.segmentInfo.dir, + segmentWriteState.segmentInfo.getVersion(), + segmentWriteState.segmentInfo.getMinVersion(), + segmentWriteState.segmentInfo.name, + DocIdSetIterator.NO_MORE_DOCS, + segmentWriteState.segmentInfo.getUseCompoundFile(), + segmentWriteState.segmentInfo.getHasBlocks(), + segmentWriteState.segmentInfo.getCodec(), + segmentWriteState.segmentInfo.getDiagnostics(), + segmentWriteState.segmentInfo.getId(), + segmentWriteState.segmentInfo.getAttributes(), + segmentWriteState.segmentInfo.getIndexSort() + ); + + return new SegmentWriteState( + segmentWriteState.infoStream, + segmentWriteState.directory, + segmentInfo, + segmentWriteState.fieldInfos, + segmentWriteState.segUpdates, + segmentWriteState.context + ); + } + } diff --git a/server/src/main/java/org/opensearch/index/codec/composite/datacube/startree/StarTreeValues.java b/server/src/main/java/org/opensearch/index/codec/composite/datacube/startree/StarTreeValues.java deleted file mode 100644 index 8378a4063b7ca..0000000000000 --- a/server/src/main/java/org/opensearch/index/codec/composite/datacube/startree/StarTreeValues.java +++ /dev/null @@ -1,70 +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.codec.composite.datacube.startree; - -import org.apache.lucene.search.DocIdSetIterator; -import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.index.codec.composite.CompositeIndexValues; -import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; -import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNode; - -import java.util.Map; - -/** - * Concrete class that holds the star tree associated values from the segment - * - * @opensearch.experimental - */ -@ExperimentalApi -public class StarTreeValues implements CompositeIndexValues { - private final StarTreeField starTreeField; - private final StarTreeNode root; - private final Map dimensionDocValuesIteratorMap; - private final Map metricDocValuesIteratorMap; - private final Map attributes; - - public StarTreeValues( - StarTreeField starTreeField, - StarTreeNode root, - Map dimensionDocValuesIteratorMap, - Map metricDocValuesIteratorMap, - Map attributes - ) { - this.starTreeField = starTreeField; - this.root = root; - this.dimensionDocValuesIteratorMap = dimensionDocValuesIteratorMap; - this.metricDocValuesIteratorMap = metricDocValuesIteratorMap; - this.attributes = attributes; - } - - @Override - public CompositeIndexValues getValues() { - return this; - } - - public StarTreeField getStarTreeField() { - return starTreeField; - } - - public StarTreeNode getRoot() { - return root; - } - - public Map getDimensionDocValuesIteratorMap() { - return dimensionDocValuesIteratorMap; - } - - public Map getMetricDocValuesIteratorMap() { - return metricDocValuesIteratorMap; - } - - public Map getAttributes() { - return attributes; - } -} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexConstants.java b/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexConstants.java index 9402675ff39d9..81e19f6cdb1a3 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexConstants.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexConstants.java @@ -23,4 +23,9 @@ public class CompositeIndexConstants { */ public static final String SEGMENT_DOCS_COUNT = "segmentDocsCount"; + /** + * Represents the key to fetch number of total star tree documents in a segment. + */ + public static final String STAR_TREE_DOCS_COUNT = "starTreeDocsCount"; + } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java index d3105b4ae23c7..13d5059a241b1 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java @@ -9,15 +9,18 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.index.DocValues; import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.EmptyDocValuesProducer; import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.SegmentWriteState; -import org.apache.lucene.index.VectorEncoding; -import org.apache.lucene.index.VectorSimilarityFunction; -import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.index.SortedNumericDocValuesWriterWrapper; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.Counter; +import org.apache.lucene.util.NumericUtils; import org.opensearch.index.compositeindex.datacube.Dimension; import org.opensearch.index.compositeindex.datacube.Metric; import org.opensearch.index.compositeindex.datacube.MetricStat; @@ -26,6 +29,8 @@ import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricAggregatorInfo; import org.opensearch.index.compositeindex.datacube.startree.aggregators.ValueAggregator; +import org.opensearch.index.compositeindex.datacube.startree.fileformats.StarTreeWriter; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; import org.opensearch.index.compositeindex.datacube.startree.node.InMemoryTreeNode; import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNodeType; import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; @@ -38,16 +43,18 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.ALL; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.fullyQualifiedFieldNameForStarTreeDimensionsDocValues; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.fullyQualifiedFieldNameForStarTreeMetricsDocValues; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.getFieldInfo; /** * Builder for star tree. Defines the algorithm to construct star-tree @@ -76,19 +83,30 @@ public abstract class BaseStarTreeBuilder implements StarTreeBuilder { protected final InMemoryTreeNode rootNode = getNewNode(); protected final StarTreeField starTreeField; - private final SegmentWriteState state; - static String NUM_SEGMENT_DOCS = "numSegmentDocs"; + private final SegmentWriteState writeState; + + private final IndexOutput metaOut; + private final IndexOutput dataOut; /** * Reads all the configuration related to dimensions and metrics, builds a star-tree based on the different construction parameters. * * @param starTreeField holds the configuration for the star tree - * @param state stores the segment write state + * @param writeState stores the segment write writeState * @param mapperService helps to find the original type of the field */ - protected BaseStarTreeBuilder(StarTreeField starTreeField, SegmentWriteState state, MapperService mapperService) { + protected BaseStarTreeBuilder( + IndexOutput metaOut, + IndexOutput dataOut, + StarTreeField starTreeField, + SegmentWriteState writeState, + MapperService mapperService + ) { logger.debug("Building star tree : {}", starTreeField.getName()); + this.metaOut = metaOut; + this.dataOut = dataOut; + this.starTreeField = starTreeField; StarTreeFieldConfiguration starTreeFieldSpec = starTreeField.getStarTreeConfig(); @@ -96,8 +114,8 @@ protected BaseStarTreeBuilder(StarTreeField starTreeField, SegmentWriteState sta this.numDimensions = dimensionsSplitOrder.size(); this.skipStarNodeCreationForDimensions = new HashSet<>(); - this.totalSegmentDocs = state.segmentInfo.maxDoc(); - this.state = state; + this.totalSegmentDocs = writeState.segmentInfo.maxDoc(); + this.writeState = writeState; Set skipStarNodeCreationForDimensions = starTreeFieldSpec.getSkipStarNodeCreationInDims(); @@ -155,6 +173,227 @@ public List generateMetricAggregatorInfos(MapperService ma return metricAggregatorInfos; } + /** + * Generates the configuration required to perform aggregation for all the metrics on a field + * + * @return list of MetricAggregatorInfo + */ + public List getMetricReaders(SegmentWriteState state, Map fieldProducerMap) + throws IOException { + + List metricReaders = new ArrayList<>(); + for (Metric metric : this.starTreeField.getMetrics()) { + for (MetricStat metricStat : metric.getMetrics()) { + SequentialDocValuesIterator metricReader; + FieldInfo metricFieldInfo = state.fieldInfos.fieldInfo(metric.getField()); + if (metricStat.equals(MetricStat.DOC_COUNT)) { + // _doc_count is numeric field , so we convert to sortedNumericDocValues and get iterator + metricReader = getIteratorForNumericField(fieldProducerMap, metricFieldInfo, DocCountFieldMapper.NAME); + } else { + if (metricFieldInfo == null) { + metricFieldInfo = getFieldInfo(metric.getField(), DocValuesType.SORTED_NUMERIC); + } + metricReader = new SequentialDocValuesIterator( + fieldProducerMap.get(metricFieldInfo.name).getSortedNumeric(metricFieldInfo) + ); + } + metricReaders.add(metricReader); + } + } + return metricReaders; + } + + /** + * Builds the star tree from the original segment documents + * + * @param fieldProducerMap contain s the docValues producer to get docValues associated with each field + * @param fieldNumberAcrossStarTrees maintains a counter for the number of star-tree fields + * @param starTreeDocValuesConsumer consumes the generated star-tree docValues + * @throws IOException when we are unable to build star-tree + */ + public void build( + Map fieldProducerMap, + AtomicInteger fieldNumberAcrossStarTrees, + DocValuesConsumer starTreeDocValuesConsumer + ) throws IOException { + long startTime = System.currentTimeMillis(); + logger.debug("Star-tree build is a go with star tree field {}", starTreeField.getName()); + + List metricReaders = getMetricReaders(writeState, fieldProducerMap); + List dimensionsSplitOrder = starTreeField.getDimensionsOrder(); + SequentialDocValuesIterator[] dimensionReaders = new SequentialDocValuesIterator[dimensionsSplitOrder.size()]; + for (int i = 0; i < numDimensions; i++) { + String dimension = dimensionsSplitOrder.get(i).getField(); + FieldInfo dimensionFieldInfo = writeState.fieldInfos.fieldInfo(dimension); + if (dimensionFieldInfo == null) { + dimensionFieldInfo = getFieldInfo(dimension, DocValuesType.SORTED_NUMERIC); + } + dimensionReaders[i] = new SequentialDocValuesIterator( + fieldProducerMap.get(dimensionFieldInfo.name).getSortedNumeric(dimensionFieldInfo) + ); + } + Iterator starTreeDocumentIterator = sortAndAggregateSegmentDocuments(dimensionReaders, metricReaders); + logger.debug("Sorting and aggregating star-tree in ms : {}", (System.currentTimeMillis() - startTime)); + build(starTreeDocumentIterator, fieldNumberAcrossStarTrees, starTreeDocValuesConsumer); + logger.debug("Finished Building star-tree in ms : {}", (System.currentTimeMillis() - startTime)); + } + + /** + * Builds the star tree using sorted and aggregated star-tree Documents + * + * @param starTreeDocumentIterator contains the sorted and aggregated documents + * @param fieldNumberAcrossStarTrees maintains a counter for the number of star-tree fields + * @param starTreeDocValuesConsumer consumes the generated star-tree docValues + * @throws IOException when we are unable to build star-tree + */ + public void build( + Iterator starTreeDocumentIterator, + AtomicInteger fieldNumberAcrossStarTrees, + DocValuesConsumer starTreeDocValuesConsumer + ) throws IOException { + int numSegmentStarTreeDocument = totalSegmentDocs; + + appendDocumentsToStarTree(starTreeDocumentIterator); + int numStarTreeDocument = numStarTreeDocs; + logger.debug("Generated star tree docs : [{}] from segment docs : [{}]", numStarTreeDocument, numSegmentStarTreeDocument); + + if (numStarTreeDocs == 0) { + // serialize the star tree data + serializeStarTree(numStarTreeDocument, numStarTreeDocs); + return; + } + + constructStarTree(rootNode, 0, numStarTreeDocs); + int numStarTreeDocumentUnderStarNode = numStarTreeDocs - numStarTreeDocument; + logger.debug( + "Finished constructing star-tree, got [ {} ] tree nodes and [ {} ] starTreeDocument under star-node", + numStarTreeNodes, + numStarTreeDocumentUnderStarNode + ); + + createAggregatedDocs(rootNode); + int numAggregatedStarTreeDocument = numStarTreeDocs - numStarTreeDocument - numStarTreeDocumentUnderStarNode; + logger.debug("Finished creating aggregated documents : {}", numAggregatedStarTreeDocument); + + // Create doc values indices in disk + createSortedDocValuesIndices(starTreeDocValuesConsumer, fieldNumberAcrossStarTrees); + + // serialize star-tree + serializeStarTree(numStarTreeDocument, numStarTreeDocs); + } + + void appendDocumentsToStarTree(Iterator starTreeDocumentIterator) throws IOException { + while (starTreeDocumentIterator.hasNext()) { + appendToStarTree(starTreeDocumentIterator.next()); + } + } + + private void serializeStarTree(int numSegmentStarTreeDocument, int numStarTreeDocs) throws IOException { + // serialize the star tree data + long dataFilePointer = dataOut.getFilePointer(); + StarTreeWriter starTreeWriter = new StarTreeWriter(); + long totalStarTreeDataLength = starTreeWriter.writeStarTree(dataOut, rootNode, numStarTreeNodes, starTreeField.getName()); + + // serialize the star tree meta + starTreeWriter.writeStarTreeMetadata( + metaOut, + starTreeField, + metricAggregatorInfos, + numStarTreeNodes, + numSegmentStarTreeDocument, + numStarTreeDocs, + dataFilePointer, + totalStarTreeDataLength + ); + } + + private void createSortedDocValuesIndices(DocValuesConsumer docValuesConsumer, AtomicInteger fieldNumberAcrossStarTrees) + throws IOException { + List dimensionWriters = new ArrayList<>(); + List metricWriters = new ArrayList<>(); + FieldInfo[] dimensionFieldInfoList = new FieldInfo[starTreeField.getDimensionsOrder().size()]; + FieldInfo[] metricFieldInfoList = new FieldInfo[metricAggregatorInfos.size()]; + for (int i = 0; i < dimensionFieldInfoList.length; i++) { + final FieldInfo fi = getFieldInfo( + fullyQualifiedFieldNameForStarTreeDimensionsDocValues( + starTreeField.getName(), + starTreeField.getDimensionsOrder().get(i).getField() + ), + DocValuesType.SORTED_NUMERIC, + fieldNumberAcrossStarTrees.getAndIncrement() + ); + dimensionFieldInfoList[i] = fi; + dimensionWriters.add(new SortedNumericDocValuesWriterWrapper(fi, Counter.newCounter())); + } + for (int i = 0; i < metricAggregatorInfos.size(); i++) { + + final FieldInfo fi = getFieldInfo( + fullyQualifiedFieldNameForStarTreeMetricsDocValues( + starTreeField.getName(), + metricAggregatorInfos.get(i).getField(), + metricAggregatorInfos.get(i).getMetricStat().getTypeName() + ), + DocValuesType.SORTED_NUMERIC, + fieldNumberAcrossStarTrees.getAndIncrement() + ); + + metricFieldInfoList[i] = fi; + metricWriters.add(new SortedNumericDocValuesWriterWrapper(fi, Counter.newCounter())); + } + + for (int docId = 0; docId < numStarTreeDocs; docId++) { + StarTreeDocument starTreeDocument = getStarTreeDocument(docId); + for (int i = 0; i < starTreeDocument.dimensions.length; i++) { + if (starTreeDocument.dimensions[i] != null) { + dimensionWriters.get(i).addValue(docId, starTreeDocument.dimensions[i]); + } + } + + for (int i = 0; i < starTreeDocument.metrics.length; i++) { + try { + switch (metricAggregatorInfos.get(i).getValueAggregators().getAggregatedValueType()) { + case LONG: + if (starTreeDocument.metrics[i] != null) { + metricWriters.get(i).addValue(docId, (long) starTreeDocument.metrics[i]); + } + break; + case DOUBLE: + if (starTreeDocument.metrics[i] != null) { + metricWriters.get(i) + .addValue(docId, NumericUtils.doubleToSortableLong((Double) starTreeDocument.metrics[i])); + } + break; + default: + throw new IllegalStateException("Unknown metric doc value type"); + } + } catch (IllegalArgumentException e) { + logger.error("could not parse the value, exiting creation of star tree"); + } + } + } + + addStarTreeDocValueFields(docValuesConsumer, dimensionWriters, dimensionFieldInfoList, starTreeField.getDimensionsOrder().size()); + addStarTreeDocValueFields(docValuesConsumer, metricWriters, metricFieldInfoList, metricAggregatorInfos.size()); + } + + private void addStarTreeDocValueFields( + DocValuesConsumer docValuesConsumer, + List docValuesWriters, + FieldInfo[] fieldInfoList, + int fieldCount + ) throws IOException { + for (int i = 0; i < fieldCount; i++) { + final int writerIndex = i; + DocValuesProducer docValuesProducer = new EmptyDocValuesProducer() { + @Override + public SortedNumericDocValues getSortedNumeric(FieldInfo field) { + return docValuesWriters.get(writerIndex).getDocValues(); + } + }; + docValuesConsumer.addSortedNumericField(fieldInfoList[i], docValuesProducer); + } + } + /** * Get star tree document from the segment for the current docId with the dimensionReaders and metricReaders */ @@ -376,6 +615,7 @@ protected StarTreeDocument reduceSegmentStarTreeDocuments( */ private static Long getLong(Object metric) { Long metricValue = null; + if (metric instanceof Long) { metricValue = (long) metric; } @@ -422,89 +662,6 @@ public StarTreeDocument reduceStarTreeDocuments(StarTreeDocument aggregatedDocum } } - /** - * Builds the star tree from the original segment documents - * - * @param fieldProducerMap contain s the docValues producer to get docValues associated with each field - * @throws IOException when we are unable to build star-tree - */ - public void build(Map fieldProducerMap) throws IOException { - long startTime = System.currentTimeMillis(); - logger.debug("Star-tree build is a go with star tree field {}", starTreeField.getName()); - if (totalSegmentDocs == 0) { - logger.debug("No documents found in the segment"); - return; - } - List metricReaders = getMetricReaders(state, fieldProducerMap); - List dimensionsSplitOrder = starTreeField.getDimensionsOrder(); - SequentialDocValuesIterator[] dimensionReaders = new SequentialDocValuesIterator[dimensionsSplitOrder.size()]; - for (int i = 0; i < numDimensions; i++) { - String dimension = dimensionsSplitOrder.get(i).getField(); - FieldInfo dimensionFieldInfo = state.fieldInfos.fieldInfo(dimension); - if (dimensionFieldInfo == null) { - dimensionFieldInfo = getFieldInfo(dimension, DocValuesType.SORTED_NUMERIC); - } - dimensionReaders[i] = new SequentialDocValuesIterator( - fieldProducerMap.get(dimensionFieldInfo.name).getSortedNumeric(dimensionFieldInfo) - ); - } - Iterator starTreeDocumentIterator = sortAndAggregateSegmentDocuments(dimensionReaders, metricReaders); - logger.debug("Sorting and aggregating star-tree in ms : {}", (System.currentTimeMillis() - startTime)); - build(starTreeDocumentIterator); - logger.debug("Finished Building star-tree in ms : {}", (System.currentTimeMillis() - startTime)); - } - - private static FieldInfo getFieldInfo(String field, DocValuesType docValuesType) { - return new FieldInfo( - field, - 1, // This is filled as part of doc values creation and is not used otherwise - false, - false, - false, - IndexOptions.NONE, - docValuesType, - -1, - Collections.emptyMap(), - 0, - 0, - 0, - 0, - VectorEncoding.FLOAT32, - VectorSimilarityFunction.EUCLIDEAN, - false, - false - ); - } - - /** - * Generates the configuration required to perform aggregation for all the metrics on a field - * - * @return list of MetricAggregatorInfo - */ - public List getMetricReaders(SegmentWriteState state, Map fieldProducerMap) - throws IOException { - List metricReaders = new ArrayList<>(); - for (Metric metric : this.starTreeField.getMetrics()) { - for (MetricStat metricStat : metric.getMetrics()) { - SequentialDocValuesIterator metricReader = null; - FieldInfo metricFieldInfo = state.fieldInfos.fieldInfo(metric.getField()); - if (metricStat.equals(MetricStat.DOC_COUNT)) { - // _doc_count is numeric field , so we convert to sortedNumericDocValues and get iterator - metricReader = getIteratorForNumericField(fieldProducerMap, metricFieldInfo, DocCountFieldMapper.NAME); - } else { - if (metricFieldInfo == null) { - metricFieldInfo = getFieldInfo(metric.getField(), DocValuesType.SORTED_NUMERIC); - } - metricReader = new SequentialDocValuesIterator( - fieldProducerMap.get(metricFieldInfo.name).getSortedNumeric(metricFieldInfo) - ); - } - metricReaders.add(metricReader); - } - } - return metricReaders; - } - /** * Converts numericDocValues to sortedNumericDocValues and returns SequentialDocValuesIterator */ @@ -524,45 +681,6 @@ private SequentialDocValuesIterator getIteratorForNumericField( return sequentialDocValuesIterator; } - /** - * Builds the star tree using Star-Tree Document - * - * @param starTreeDocumentIterator contains the sorted and aggregated documents - * @throws IOException when we are unable to build star-tree - */ - void build(Iterator starTreeDocumentIterator) throws IOException { - int numSegmentStarTreeDocument = totalSegmentDocs; - - while (starTreeDocumentIterator.hasNext()) { - appendToStarTree(starTreeDocumentIterator.next()); - } - int numStarTreeDocument = numStarTreeDocs; - logger.debug("Generated star tree docs : [{}] from segment docs : [{}]", numStarTreeDocument, numSegmentStarTreeDocument); - - if (numStarTreeDocs == 0) { - // TODO: Uncomment when segment codec and file formats is ready - // StarTreeBuilderUtils.serializeTree(indexOutput, rootNode, dimensionsSplitOrder, numNodes); - return; - } - - constructStarTree(rootNode, 0, numStarTreeDocs); - int numStarTreeDocumentUnderStarNode = numStarTreeDocs - numStarTreeDocument; - logger.debug( - "Finished constructing star-tree, got [ {} ] tree nodes and [ {} ] starTreeDocument under star-node", - numStarTreeNodes, - numStarTreeDocumentUnderStarNode - ); - - createAggregatedDocs(rootNode); - int numAggregatedStarTreeDocument = numStarTreeDocs - numStarTreeDocument - numStarTreeDocumentUnderStarNode; - logger.debug("Finished creating aggregated documents : {}", numAggregatedStarTreeDocument); - - // TODO: When StarTreeFactory Codec is ready - // Create doc values indices in disk - // Serialize and save in disk - // Write star tree metadata for off heap implementation - } - /** * Adds a document to star-tree * @@ -584,6 +702,20 @@ private InMemoryTreeNode getNewNode() { return new InMemoryTreeNode(); } + /** + * Returns a new star-tree node + * @param dimensionId dimension id of the star-tree node + * @param startDocId start doc id of the star-tree node + * @param endDocId end doc id of the star-tree node + * @param nodeType node type of the star-tree node + * @param dimensionValue dimension value of the star-tree node + * @return + */ + private InMemoryTreeNode getNewNode(int dimensionId, int startDocId, int endDocId, byte nodeType, long dimensionValue) { + numStarTreeNodes++; + return new InMemoryTreeNode(dimensionId, startDocId, endDocId, nodeType, dimensionValue); + } + /** * Implements the algorithm to construct a star-tree * @@ -594,63 +726,73 @@ private InMemoryTreeNode getNewNode() { */ private void constructStarTree(InMemoryTreeNode node, int startDocId, int endDocId) throws IOException { - int childDimensionId = node.dimensionId + 1; + int childDimensionId = node.getDimensionId() + 1; if (childDimensionId == numDimensions) { return; } // Construct all non-star children nodes - node.childDimensionId = childDimensionId; - Map children = constructNonStarNodes(startDocId, endDocId, childDimensionId); - node.children = children; + node.setChildDimensionId(childDimensionId); + constructNonStarNodes(node, startDocId, endDocId, childDimensionId); // Construct star-node if required - if (!skipStarNodeCreationForDimensions.contains(childDimensionId) && children.size() > 1) { - children.put((long) ALL, constructStarNode(startDocId, endDocId, childDimensionId)); + if (!skipStarNodeCreationForDimensions.contains(childDimensionId) && node.getChildren().size() > 1) { + node.addChildNode(constructStarNode(startDocId, endDocId, childDimensionId), (long) ALL); + } + + // Further split star node if needed + if (node.getChildStarNode() != null + && (node.getChildStarNode().getEndDocId() - node.getChildStarNode().getStartDocId() > maxLeafDocuments)) { + constructStarTree(node.getChildStarNode(), node.getChildStarNode().getStartDocId(), node.getChildStarNode().getEndDocId()); } // Further split on child nodes if required - for (InMemoryTreeNode child : children.values()) { - if (child.endDocId - child.startDocId > maxLeafDocuments) { - constructStarTree(child, child.startDocId, child.endDocId); + for (InMemoryTreeNode child : node.getChildren().values()) { + if (child.getEndDocId() - child.getStartDocId() > maxLeafDocuments) { + constructStarTree(child, child.getStartDocId(), child.getEndDocId()); } } + } /** * Constructs non star tree nodes * + * @param node parent node * @param startDocId start document id (inclusive) * @param endDocId end document id (exclusive) * @param dimensionId id of the dimension in the star tree - * @return root node with non-star nodes constructed + * * @throws IOException throws an exception if we are unable to construct non-star nodes */ - private Map constructNonStarNodes(int startDocId, int endDocId, int dimensionId) throws IOException { - Map nodes = new HashMap<>(); + private void constructNonStarNodes(InMemoryTreeNode node, int startDocId, int endDocId, int dimensionId) throws IOException { int nodeStartDocId = startDocId; Long nodeDimensionValue = getDimensionValue(startDocId, dimensionId); for (int i = startDocId + 1; i < endDocId; i++) { Long dimensionValue = getDimensionValue(i, dimensionId); if (Objects.equals(dimensionValue, nodeDimensionValue) == false) { - InMemoryTreeNode child = getNewNode(); - child.dimensionId = dimensionId; - child.dimensionValue = nodeDimensionValue != null ? nodeDimensionValue : ALL; - child.startDocId = nodeStartDocId; - child.endDocId = i; - nodes.put(nodeDimensionValue, child); + addChildNode(node, i, dimensionId, nodeStartDocId, nodeDimensionValue); nodeStartDocId = i; nodeDimensionValue = dimensionValue; } } - InMemoryTreeNode lastNode = getNewNode(); - lastNode.dimensionId = dimensionId; - lastNode.dimensionValue = nodeDimensionValue != null ? nodeDimensionValue : ALL; - lastNode.startDocId = nodeStartDocId; - lastNode.endDocId = endDocId; - nodes.put(nodeDimensionValue, lastNode); - return nodes; + addChildNode(node, endDocId, dimensionId, nodeStartDocId, nodeDimensionValue); + } + + private void addChildNode(InMemoryTreeNode node, int endDocId, int dimensionId, int nodeStartDocId, Long nodeDimensionValue) { + long childNodeDimensionValue; + byte childNodeType; + if (nodeDimensionValue == null) { + childNodeDimensionValue = ALL; + childNodeType = StarTreeNodeType.NULL.getValue(); + } else { + childNodeDimensionValue = nodeDimensionValue; + childNodeType = StarTreeNodeType.DEFAULT.getValue(); + } + + InMemoryTreeNode lastNode = getNewNode(dimensionId, nodeStartDocId, endDocId, childNodeType, childNodeDimensionValue); + node.addChildNode(lastNode, nodeDimensionValue); } /** @@ -663,17 +805,10 @@ private Map constructNonStarNodes(int startDocId, int en * @throws IOException throws an exception if we are unable to construct non-star nodes */ private InMemoryTreeNode constructStarNode(int startDocId, int endDocId, int dimensionId) throws IOException { - InMemoryTreeNode starNode = getNewNode(); - starNode.dimensionId = dimensionId; - starNode.dimensionValue = ALL; - starNode.nodeType = StarTreeNodeType.STAR.getValue(); - starNode.startDocId = numStarTreeDocs; + int starNodeStartDocId = numStarTreeDocs; Iterator starTreeDocumentIterator = generateStarTreeDocumentsForStarNode(startDocId, endDocId, dimensionId); - while (starTreeDocumentIterator.hasNext()) { - appendToStarTree(starTreeDocumentIterator.next()); - } - starNode.endDocId = numStarTreeDocs; - return starNode; + appendDocumentsToStarTree(starTreeDocumentIterator); + return getNewNode(dimensionId, starNodeStartDocId, numStarTreeDocs, StarTreeNodeType.STAR.getValue(), ALL); } /** @@ -685,57 +820,56 @@ private InMemoryTreeNode constructStarNode(int startDocId, int endDocId, int dim */ private StarTreeDocument createAggregatedDocs(InMemoryTreeNode node) throws IOException { StarTreeDocument aggregatedStarTreeDocument = null; - if (node.children == null) { - // For leaf node - if (node.startDocId == node.endDocId - 1) { + // For leaf node + if (!node.hasChild()) { + + if (node.getStartDocId() == node.getEndDocId() - 1) { // If it has only one document, use it as the aggregated document - aggregatedStarTreeDocument = getStarTreeDocument(node.startDocId); - node.aggregatedDocId = node.startDocId; + aggregatedStarTreeDocument = getStarTreeDocument(node.getStartDocId()); + node.setAggregatedDocId(node.getStartDocId()); } else { // If it has multiple documents, aggregate all of them - for (int i = node.startDocId; i < node.endDocId; i++) { + for (int i = node.getStartDocId(); i < node.getEndDocId(); i++) { aggregatedStarTreeDocument = reduceStarTreeDocuments(aggregatedStarTreeDocument, getStarTreeDocument(i)); } if (null == aggregatedStarTreeDocument) { throw new IllegalStateException("aggregated star-tree document is null after reducing the documents"); } - for (int i = node.dimensionId + 1; i < numDimensions; i++) { + for (int i = node.getDimensionId() + 1; i < numDimensions; i++) { aggregatedStarTreeDocument.dimensions[i] = STAR_IN_DOC_VALUES_INDEX; } - node.aggregatedDocId = numStarTreeDocs; + node.setAggregatedDocId(numStarTreeDocs); appendToStarTree(aggregatedStarTreeDocument); } } else { // For non-leaf node - if (node.children.containsKey((long) ALL)) { + if (node.getChildStarNode() != null) { // If it has star child, use the star child aggregated document directly - for (InMemoryTreeNode child : node.children.values()) { - if (child.nodeType == StarTreeNodeType.STAR.getValue()) { - aggregatedStarTreeDocument = createAggregatedDocs(child); - node.aggregatedDocId = child.aggregatedDocId; - } else { - createAggregatedDocs(child); - } + aggregatedStarTreeDocument = createAggregatedDocs(node.getChildStarNode()); + node.setAggregatedDocId(node.getChildStarNode().getAggregatedDocId()); + + for (InMemoryTreeNode child : node.getChildren().values()) { + createAggregatedDocs(child); } } else { // If no star child exists, aggregate all aggregated documents from non-star children - if (node.children.values().size() == 1) { - for (InMemoryTreeNode child : node.children.values()) { + if (node.getChildren().values().size() == 1) { + for (InMemoryTreeNode child : node.getChildren().values()) { aggregatedStarTreeDocument = reduceStarTreeDocuments(aggregatedStarTreeDocument, createAggregatedDocs(child)); - node.aggregatedDocId = child.aggregatedDocId; + node.setAggregatedDocId(child.getAggregatedDocId()); } } else { - for (InMemoryTreeNode child : node.children.values()) { + for (InMemoryTreeNode child : node.getChildren().values()) { aggregatedStarTreeDocument = reduceStarTreeDocuments(aggregatedStarTreeDocument, createAggregatedDocs(child)); } if (null == aggregatedStarTreeDocument) { throw new IllegalStateException("aggregated star-tree document is null after reducing the documents"); } - for (int i = node.dimensionId + 1; i < numDimensions; i++) { + for (int i = node.getDimensionId() + 1; i < numDimensions; i++) { aggregatedStarTreeDocument.dimensions[i] = STAR_IN_DOC_VALUES_INDEX; } - node.aggregatedDocId = numStarTreeDocs; + node.setAggregatedDocId(numStarTreeDocs); appendToStarTree(aggregatedStarTreeDocument); } } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java index f63b0cb0cc77d..62fda3e56d289 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java @@ -10,14 +10,18 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexOutput; import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; 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.StarTreeDocument; import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeDocumentsSorter; import org.opensearch.index.mapper.MapperService; @@ -29,11 +33,15 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.Objects; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.opensearch.index.compositeindex.CompositeIndexConstants.SEGMENT_DOCS_COUNT; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.fullyQualifiedFieldNameForStarTreeMetricsDocValues; /** * Off-heap implementation of the star tree builder. + * * @opensearch.experimental */ @ExperimentalApi @@ -46,12 +54,20 @@ public class OffHeapStarTreeBuilder extends BaseStarTreeBuilder { * Builds star tree based on star tree field configuration consisting of dimensions, metrics and star tree index * specific configuration. * + * @param metaOut an index output to write star-tree metadata + * @param dataOut an index output to write star-tree data * @param starTreeField holds the configuration for the star tree * @param state stores the segment write state * @param mapperService helps to find the original type of the field */ - protected OffHeapStarTreeBuilder(StarTreeField starTreeField, SegmentWriteState state, MapperService mapperService) throws IOException { - super(starTreeField, state, mapperService); + protected OffHeapStarTreeBuilder( + IndexOutput metaOut, + IndexOutput dataOut, + StarTreeField starTreeField, + SegmentWriteState state, + MapperService mapperService + ) throws IOException { + super(metaOut, dataOut, starTreeField, state, mapperService); segmentDocumentFileManager = new SegmentDocsFileManager(state, starTreeField, metricAggregatorInfos); try { starTreeDocumentFileManager = new StarTreeDocsFileManager(state, starTreeField, metricAggregatorInfos); @@ -73,10 +89,14 @@ public void appendStarTreeDocument(StarTreeDocument starTreeDocument) throws IOE * @param starTreeValuesSubs contains the star tree values from multiple segments */ @Override - public void build(List starTreeValuesSubs) throws IOException { + public void build( + List starTreeValuesSubs, + AtomicInteger fieldNumberAcrossStarTrees, + DocValuesConsumer starTreeDocValuesConsumer + ) throws IOException { boolean success = false; try { - build(mergeStarTrees(starTreeValuesSubs)); + build(mergeStarTrees(starTreeValuesSubs), fieldNumberAcrossStarTrees, starTreeDocValuesConsumer); success = true; } finally { starTreeDocumentFileManager.deleteFiles(success); @@ -133,15 +153,23 @@ Iterator mergeStarTrees(List starTreeValuesSub .size()]; for (int i = 0; i < dimensionsSplitOrder.size(); i++) { String dimension = dimensionsSplitOrder.get(i).getField(); - dimensionReaders[i] = new SequentialDocValuesIterator(starTreeValues.getDimensionDocValuesIteratorMap().get(dimension)); + dimensionReaders[i] = new SequentialDocValuesIterator(starTreeValues.getDimensionDocIdSetIterator(dimension)); } List metricReaders = new ArrayList<>(); - for (Map.Entry metricDocValuesEntry : starTreeValues.getMetricDocValuesIteratorMap().entrySet()) { - metricReaders.add(new SequentialDocValuesIterator(metricDocValuesEntry.getValue())); + // get doc id set iterators for metrics + for (Metric metric : starTreeValues.getStarTreeField().getMetrics()) { + for (MetricStat metricStat : metric.getMetrics()) { + String metricFullName = fullyQualifiedFieldNameForStarTreeMetricsDocValues( + starTreeValues.getStarTreeField().getName(), + metric.getField(), + metricStat.getTypeName() + ); + metricReaders.add(new SequentialDocValuesIterator(starTreeValues.getMetricDocIdSetIterator(metricFullName))); + } } int currentDocId = 0; int numSegmentDocs = Integer.parseInt( - starTreeValues.getAttributes().getOrDefault(NUM_SEGMENT_DOCS, String.valueOf(DocIdSetIterator.NO_MORE_DOCS)) + starTreeValues.getAttributes().getOrDefault(SEGMENT_DOCS_COUNT, String.valueOf(DocIdSetIterator.NO_MORE_DOCS)) ); while (currentDocId < numSegmentDocs) { StarTreeDocument starTreeDocument = getStarTreeDocument(currentDocId, dimensionReaders, metricReaders); diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilder.java index 8ff111d3b41d9..ec9475caf7d6d 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilder.java @@ -7,13 +7,17 @@ */ package org.opensearch.index.compositeindex.datacube.startree.builder; +import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexOutput; import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; 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.StarTreeDocument; import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; import org.opensearch.index.mapper.MapperService; @@ -22,8 +26,11 @@ import java.util.Arrays; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.Objects; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.opensearch.index.compositeindex.CompositeIndexConstants.SEGMENT_DOCS_COUNT; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.fullyQualifiedFieldNameForStarTreeMetricsDocValues; /** * On heap single tree builder @@ -38,12 +45,20 @@ public class OnHeapStarTreeBuilder extends BaseStarTreeBuilder { /** * Constructor for OnHeapStarTreeBuilder * + * @param metaOut an index output to write star-tree metadata + * @param dataOut an index output to write star-tree data * @param starTreeField star-tree field * @param segmentWriteState segment write state * @param mapperService helps with the numeric type of field */ - public OnHeapStarTreeBuilder(StarTreeField starTreeField, SegmentWriteState segmentWriteState, MapperService mapperService) { - super(starTreeField, segmentWriteState, mapperService); + public OnHeapStarTreeBuilder( + IndexOutput metaOut, + IndexOutput dataOut, + StarTreeField starTreeField, + SegmentWriteState segmentWriteState, + MapperService mapperService + ) throws IOException { + super(metaOut, dataOut, starTreeField, segmentWriteState, mapperService); } @Override @@ -84,12 +99,16 @@ public Iterator sortAndAggregateSegmentDocuments( // TODO : we can save empty iterator for dimensions which are not part of segment starTreeDocuments[currentDocId] = getSegmentStarTreeDocument(currentDocId, dimensionReaders, metricReaders); } - return sortAndAggregateStarTreeDocuments(starTreeDocuments); + return sortAndAggregateStarTreeDocuments(starTreeDocuments, false); } @Override - public void build(List starTreeValuesSubs) throws IOException { - build(mergeStarTrees(starTreeValuesSubs)); + public void build( + List starTreeValuesSubs, + AtomicInteger fieldNumberAcrossStarTrees, + DocValuesConsumer starTreeDocValuesConsumer + ) throws IOException { + build(mergeStarTrees(starTreeValuesSubs), fieldNumberAcrossStarTrees, starTreeDocValuesConsumer); } /** @@ -119,17 +138,26 @@ StarTreeDocument[] getSegmentsStarTreeDocuments(List starTreeVal for (int i = 0; i < dimensionsSplitOrder.size(); i++) { String dimension = dimensionsSplitOrder.get(i).getField(); - dimensionReaders[i] = new SequentialDocValuesIterator(starTreeValues.getDimensionDocValuesIteratorMap().get(dimension)); + dimensionReaders[i] = new SequentialDocValuesIterator(starTreeValues.getDimensionDocIdSetIterator(dimension)); } List metricReaders = new ArrayList<>(); - for (Map.Entry metricDocValuesEntry : starTreeValues.getMetricDocValuesIteratorMap().entrySet()) { - metricReaders.add(new SequentialDocValuesIterator(metricDocValuesEntry.getValue())); + // get doc id set iterators for metrics + for (Metric metric : starTreeValues.getStarTreeField().getMetrics()) { + for (MetricStat metricStat : metric.getMetrics()) { + String metricFullName = fullyQualifiedFieldNameForStarTreeMetricsDocValues( + starTreeValues.getStarTreeField().getName(), + metric.getField(), + metricStat.getTypeName() + ); + metricReaders.add(new SequentialDocValuesIterator(starTreeValues.getMetricDocIdSetIterator(metricFullName))); + + } } int currentDocId = 0; int numSegmentDocs = Integer.parseInt( - starTreeValues.getAttributes().getOrDefault(NUM_SEGMENT_DOCS, String.valueOf(DocIdSetIterator.NO_MORE_DOCS)) + starTreeValues.getAttributes().getOrDefault(SEGMENT_DOCS_COUNT, String.valueOf(DocIdSetIterator.NO_MORE_DOCS)) ); while (currentDocId < numSegmentDocs) { starTreeDocuments.add(getStarTreeDocument(currentDocId, dimensionReaders, metricReaders)); @@ -140,10 +168,6 @@ StarTreeDocument[] getSegmentsStarTreeDocuments(List starTreeVal return starTreeDocuments.toArray(starTreeDocumentsArr); } - Iterator sortAndAggregateStarTreeDocuments(StarTreeDocument[] starTreeDocuments) { - return sortAndAggregateStarTreeDocuments(starTreeDocuments, false); - } - /** * Sort, aggregates and merges the star-tree documents * diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeBuilder.java index 357f48c0cc726..23415ddf29132 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeBuilder.java @@ -8,14 +8,16 @@ package org.opensearch.index.compositeindex.datacube.startree.builder; +import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.codecs.DocValuesProducer; import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; import java.io.Closeable; import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; /** * A star-tree builder that builds a single star-tree. @@ -27,17 +29,29 @@ public interface StarTreeBuilder extends Closeable { /** * Builds the star tree from the original segment documents * - * @param fieldProducerMap contains the docValues producer to get docValues associated with each field + * @param fieldProducerMap contains the docValues producer to get docValues associated with each field + * @param fieldNumberAcrossStarTrees maintains the unique field number across the fields in the star tree + * @param starTreeDocValuesConsumer consumer of star-tree doc values * @throws IOException when we are unable to build star-tree */ - void build(Map fieldProducerMap) throws IOException; + void build( + Map fieldProducerMap, + AtomicInteger fieldNumberAcrossStarTrees, + DocValuesConsumer starTreeDocValuesConsumer + ) throws IOException; /** - * Builds the star tree using StarTreeFactory values from multiple segments + * Builds the star tree using Star Tree values from multiple segments * * @param starTreeValuesSubs contains the star tree values from multiple segments + * @param fieldNumberAcrossStarTrees maintains the unique field number across the fields in the star tree + * @param starTreeDocValuesConsumer consumer of star-tree doc values * @throws IOException when we are unable to build star-tree */ - void build(List starTreeValuesSubs) throws IOException; + void build( + List starTreeValuesSubs, + AtomicInteger fieldNumberAcrossStarTrees, + DocValuesConsumer starTreeDocValuesConsumer + ) throws IOException; } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilder.java index 3b376d7c34351..bc598c9aeab7c 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilder.java @@ -10,11 +10,13 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.IndexOutput; import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; import org.opensearch.index.mapper.CompositeMappedFieldType; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.mapper.StarTreeMapper; @@ -25,6 +27,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; /** * Builder to construct star-trees based on multiple star-tree fields. @@ -39,8 +42,9 @@ public class StarTreesBuilder implements Closeable { private final List starTreeFields; private final SegmentWriteState state; private final MapperService mapperService; + private AtomicInteger fieldNumberAcrossStarTrees; - public StarTreesBuilder(SegmentWriteState segmentWriteState, MapperService mapperService) { + public StarTreesBuilder(SegmentWriteState segmentWriteState, MapperService mapperService, AtomicInteger fieldNumberAcrossStarTrees) { List starTreeFields = new ArrayList<>(); for (CompositeMappedFieldType compositeMappedFieldType : mapperService.getCompositeFieldTypes()) { if (compositeMappedFieldType instanceof StarTreeMapper.StarTreeFieldType) { @@ -58,12 +62,24 @@ public StarTreesBuilder(SegmentWriteState segmentWriteState, MapperService mappe this.starTreeFields = starTreeFields; this.state = segmentWriteState; this.mapperService = mapperService; + this.fieldNumberAcrossStarTrees = fieldNumberAcrossStarTrees; } /** - * Builds the star-trees. + * Builds all star-trees for given star-tree fields. + * + * @param metaOut an IndexInput for star-tree metadata + * @param dataOut an IndexInput for star-tree data + * @param fieldProducerMap fetches iterators for the fields (dimensions and metrics) + * @param starTreeDocValuesConsumer a consumer to write star-tree doc values + * @throws IOException when an error occurs while building the star-trees */ - public void build(Map fieldProducerMap) throws IOException { + public void build( + IndexOutput metaOut, + IndexOutput dataOut, + Map fieldProducerMap, + DocValuesConsumer starTreeDocValuesConsumer + ) throws IOException { if (starTreeFields.isEmpty()) { logger.debug("no star-tree fields found, returning from star-tree builder"); return; @@ -75,8 +91,8 @@ public void build(Map fieldProducerMap) throws IOExce // Build all star-trees for (StarTreeField starTreeField : starTreeFields) { - try (StarTreeBuilder starTreeBuilder = getStarTreeBuilder(starTreeField, state, mapperService)) { - starTreeBuilder.build(fieldProducerMap); + try (StarTreeBuilder starTreeBuilder = getStarTreeBuilder(metaOut, dataOut, starTreeField, state, mapperService)) { + starTreeBuilder.build(fieldProducerMap, fieldNumberAcrossStarTrees, starTreeDocValuesConsumer); } } logger.debug("Took {} ms to build {} star-trees with star-tree fields", System.currentTimeMillis() - startTime, numStarTrees); @@ -90,9 +106,17 @@ public void close() throws IOException { /** * Merges star tree fields from multiple segments * + * @param metaOut an IndexInput for star-tree metadata + * @param dataOut an IndexInput for star-tree data * @param starTreeValuesSubsPerField starTreeValuesSubs per field + * @param starTreeDocValuesConsumer a consumer to write star-tree doc values */ - public void buildDuringMerge(final Map> starTreeValuesSubsPerField) throws IOException { + public void buildDuringMerge( + IndexOutput metaOut, + IndexOutput dataOut, + final Map> starTreeValuesSubsPerField, + DocValuesConsumer starTreeDocValuesConsumer + ) throws IOException { logger.debug("Starting merge of {} star-trees with star-tree fields", starTreeValuesSubsPerField.size()); long startTime = System.currentTimeMillis(); for (Map.Entry> entry : starTreeValuesSubsPerField.entrySet()) { @@ -102,8 +126,8 @@ public void buildDuringMerge(final Map> starTreeVal continue; } StarTreeField starTreeField = starTreeValuesList.get(0).getStarTreeField(); - try (StarTreeBuilder builder = getStarTreeBuilder(starTreeField, state, mapperService)) { - builder.build(starTreeValuesList); + try (StarTreeBuilder builder = getStarTreeBuilder(metaOut, dataOut, starTreeField, state, mapperService)) { + builder.build(starTreeValuesList, fieldNumberAcrossStarTrees, starTreeDocValuesConsumer); } } logger.debug( @@ -116,13 +140,18 @@ public void buildDuringMerge(final Map> starTreeVal /** * Get star-tree builder based on build mode. */ - StarTreeBuilder getStarTreeBuilder(StarTreeField starTreeField, SegmentWriteState state, MapperService mapperService) - throws IOException { + StarTreeBuilder getStarTreeBuilder( + IndexOutput metaOut, + IndexOutput dataOut, + StarTreeField starTreeField, + SegmentWriteState state, + MapperService mapperService + ) throws IOException { switch (starTreeField.getStarTreeConfig().getBuildMode()) { case ON_HEAP: - return new OnHeapStarTreeBuilder(starTreeField, state, mapperService); + return new OnHeapStarTreeBuilder(metaOut, dataOut, starTreeField, state, mapperService); case OFF_HEAP: - return new OffHeapStarTreeBuilder(starTreeField, state, mapperService); + return new OffHeapStarTreeBuilder(metaOut, dataOut, starTreeField, state, mapperService); default: throw new IllegalArgumentException( String.format( diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/StarTreeWriter.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/StarTreeWriter.java index 7f1839024eea7..e5890be3ccb5b 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/StarTreeWriter.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/StarTreeWriter.java @@ -11,7 +11,7 @@ import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricAggregatorInfo; import org.opensearch.index.compositeindex.datacube.startree.fileformats.data.StarTreeDataWriter; -import org.opensearch.index.compositeindex.datacube.startree.fileformats.meta.StarTreeMetaWriter; +import org.opensearch.index.compositeindex.datacube.startree.fileformats.meta.StarTreeMetadataWriter; import org.opensearch.index.compositeindex.datacube.startree.node.InMemoryTreeNode; import java.io.IOException; @@ -54,6 +54,7 @@ public long writeStarTree(IndexOutput dataOut, InMemoryTreeNode rootNode, int nu * @param metricAggregatorInfos metric aggregator infos * @param numNodes number of nodes in the star tree * @param segmentAggregatedCount segment aggregated count + * @param numStarTreeDocs the total number of star tree documents for the segment * @param dataFilePointer data file pointer * @param dataFileLength data file length * @throws IOException when star-tree data serialization fails @@ -64,15 +65,17 @@ public void writeStarTreeMetadata( List metricAggregatorInfos, Integer numNodes, Integer segmentAggregatedCount, + Integer numStarTreeDocs, long dataFilePointer, long dataFileLength ) throws IOException { - StarTreeMetaWriter.writeStarTreeMetadata( + StarTreeMetadataWriter.writeStarTreeMetadata( metaOut, starTreeField, metricAggregatorInfos, numNodes, segmentAggregatedCount, + numStarTreeDocs, dataFilePointer, dataFileLength ); diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/StarTreeDataWriter.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/StarTreeDataWriter.java index 32feb78a4db3d..5fe70d3c075cf 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/StarTreeDataWriter.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/StarTreeDataWriter.java @@ -14,10 +14,7 @@ import org.opensearch.index.compositeindex.datacube.startree.node.InMemoryTreeNode; import java.io.IOException; -import java.util.ArrayList; -import java.util.Comparator; import java.util.LinkedList; -import java.util.List; import java.util.Queue; import static org.opensearch.index.compositeindex.datacube.startree.fileformats.node.FixedLengthStarTreeNode.SERIALIZABLE_DATA_SIZE_IN_BYTES; @@ -66,21 +63,26 @@ private static void writeStarTreeNodes(IndexOutput output, InMemoryTreeNode root while (!queue.isEmpty()) { InMemoryTreeNode node = queue.remove(); - if (node.children == null || node.children.isEmpty()) { + if (!node.hasChild()) { writeStarTreeNode(output, node, ALL, ALL); } else { - // Sort all children nodes based on dimension value - List sortedChildren = new ArrayList<>(node.children.values()); - sortedChildren.sort( - Comparator.comparingInt(InMemoryTreeNode::getNodeType).thenComparingLong(InMemoryTreeNode::getDimensionValue) - ); - + int totalNumberOfChildren = 0; int firstChildId = currentNodeId + queue.size() + 1; - int lastChildId = firstChildId + sortedChildren.size() - 1; + + if (node.getChildStarNode() != null) { + totalNumberOfChildren++; + queue.add(node.getChildStarNode()); + } + + if (node.getChildren() != null) { + totalNumberOfChildren = totalNumberOfChildren + node.getChildren().values().size(); + queue.addAll(node.getChildren().values()); + } + + int lastChildId = firstChildId + totalNumberOfChildren - 1; writeStarTreeNode(output, node, firstChildId, lastChildId); - queue.addAll(sortedChildren); } currentNodeId++; @@ -97,12 +99,12 @@ private static void writeStarTreeNodes(IndexOutput output, InMemoryTreeNode root * @throws IOException if an I/O error occurs while writing the node */ private static void writeStarTreeNode(IndexOutput output, InMemoryTreeNode node, int firstChildId, int lastChildId) throws IOException { - output.writeInt(node.dimensionId); - output.writeLong(node.dimensionValue); - output.writeInt(node.startDocId); - output.writeInt(node.endDocId); - output.writeInt(node.aggregatedDocId); - output.writeByte(node.nodeType); + output.writeInt(node.getDimensionId()); + output.writeLong(node.getDimensionValue()); + output.writeInt(node.getStartDocId()); + output.writeInt(node.getEndDocId()); + output.writeInt(node.getAggregatedDocId()); + output.writeByte(node.getNodeType()); output.writeInt(firstChildId); output.writeInt(lastChildId); } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadata.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadata.java index 7519c85562a8c..2e20c9b9e0737 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadata.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadata.java @@ -72,12 +72,12 @@ public class StarTreeMetadata extends CompositeIndexMetadata { /** * The total number of documents aggregated in this star-tree segment. */ - private final Integer segmentAggregatedDocCount; + private final int segmentAggregatedDocCount; /** * The maximum number of documents allowed in a leaf node. */ - private final Integer maxLeafDocs; + private final int maxLeafDocs; /** * Set of dimensions for which star node creation should be skipped. @@ -99,6 +99,11 @@ public class StarTreeMetadata extends CompositeIndexMetadata { */ private final long dataLength; + /** + * The number of star tree documents in the star tree. + */ + private final int starTreeDocCount; + /** * A star tree metadata constructor to initialize star tree metadata from the segment file (.cim) using index input. * @@ -124,6 +129,7 @@ public StarTreeMetadata( this.dimensionFields = readStarTreeDimensions(); this.metrics = readMetricEntries(); this.segmentAggregatedDocCount = readSegmentAggregatedDocCount(); + this.starTreeDocCount = readStarTreeDocCount(); this.maxLeafDocs = readMaxLeafDocs(); this.skipStarNodeCreationInDims = readSkipStarNodeCreationInDims(); this.starTreeBuildMode = readBuildMode(); @@ -146,6 +152,7 @@ public StarTreeMetadata( * @param dimensionFields list of dimension fields * @param metrics list of metric entries * @param segmentAggregatedDocCount segment aggregated doc count + * @param starTreeDocCount the total number of star tree documents for the segment * @param maxLeafDocs max leaf docs * @param skipStarNodeCreationInDims set of dimensions to skip star node creation * @param starTreeBuildMode star tree build mode @@ -161,6 +168,7 @@ public StarTreeMetadata( List dimensionFields, List metrics, Integer segmentAggregatedDocCount, + Integer starTreeDocCount, Integer maxLeafDocs, Set skipStarNodeCreationInDims, StarTreeFieldConfiguration.StarTreeBuildMode starTreeBuildMode, @@ -176,6 +184,7 @@ public StarTreeMetadata( this.dimensionFields = dimensionFields; this.metrics = metrics; this.segmentAggregatedDocCount = segmentAggregatedDocCount; + this.starTreeDocCount = starTreeDocCount; this.maxLeafDocs = maxLeafDocs; this.skipStarNodeCreationInDims = skipStarNodeCreationInDims; this.starTreeBuildMode = starTreeBuildMode; @@ -184,7 +193,7 @@ public StarTreeMetadata( } private int readNumberOfNodes() throws IOException { - return meta.readInt(); + return meta.readVInt(); } private int readDimensionsCount() throws IOException { @@ -225,6 +234,10 @@ private int readSegmentAggregatedDocCount() throws IOException { return meta.readVInt(); } + private Integer readStarTreeDocCount() throws IOException { + return meta.readVInt(); + } + private int readMaxLeafDocs() throws IOException { return meta.readVInt(); } @@ -296,16 +309,25 @@ public List getMetrics() { * * @return the aggregated document count for the star-tree. */ - public Integer getSegmentAggregatedDocCount() { + public int getSegmentAggregatedDocCount() { return segmentAggregatedDocCount; } + /** + * Returns the total number of star tree documents in the segment + * + * @return the number of star tree documents in the segment + */ + public int getStarTreeDocCount() { + return starTreeDocCount; + } + /** * Returns the max leaf docs for the star-tree. * * @return the max leaf docs. */ - public Integer getMaxLeafDocs() { + public int getMaxLeafDocs() { return maxLeafDocs; } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetaWriter.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadataWriter.java similarity index 90% rename from server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetaWriter.java rename to server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadataWriter.java index 2515c1efc3aed..1c04350e25047 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetaWriter.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadataWriter.java @@ -29,9 +29,9 @@ * * @opensearch.experimental */ -public class StarTreeMetaWriter { +public class StarTreeMetadataWriter { - private static final Logger logger = LogManager.getLogger(StarTreeMetaWriter.class); + private static final Logger logger = LogManager.getLogger(StarTreeMetadataWriter.class); /** * Writes the star-tree metadata. @@ -51,6 +51,7 @@ public static void writeStarTreeMetadata( List metricAggregatorInfos, Integer numNodes, Integer segmentAggregatedCount, + Integer numStarTreeDocs, long dataFilePointer, long dataFileLength ) throws IOException { @@ -60,7 +61,16 @@ public static void writeStarTreeMetadata( writeMetaHeader(metaOut); // TODO: Replace the parameters with StarTreeMetadata class object - writeMeta(metaOut, metricAggregatorInfos, starTreeField, numNodes, segmentAggregatedCount, dataFilePointer, dataFileLength); + writeMeta( + metaOut, + metricAggregatorInfos, + starTreeField, + numNodes, + segmentAggregatedCount, + numStarTreeDocs, + dataFilePointer, + dataFileLength + ); logger.debug( "Star tree meta size in bytes : {} for star-tree field {}", @@ -81,6 +91,7 @@ private static void writeMetaHeader(IndexOutput metaOut) throws IOException { // version metaOut.writeVInt(VERSION_CURRENT); + } /** @@ -91,6 +102,7 @@ private static void writeMetaHeader(IndexOutput metaOut) throws IOException { * @param starTreeField the star tree field * @param numNodes number of nodes in the star tree * @param segmentAggregatedDocCount the aggregated document count for the segment + * @param numStarTreeDocs the total number of star tree documents for the segment * @param dataFilePointer the file pointer to the start of the star-tree data * @param dataFileLength the length of the star-tree data file * @throws IOException if an I/O error occurs while writing the metadata @@ -101,6 +113,7 @@ private static void writeMeta( StarTreeField starTreeField, int numNodes, Integer segmentAggregatedDocCount, + Integer numStarTreeDocs, long dataFilePointer, long dataFileLength ) throws IOException { @@ -112,7 +125,7 @@ private static void writeMeta( metaOut.writeString(CompositeMappedFieldType.CompositeFieldType.STAR_TREE.getName()); // number of nodes - metaOut.writeInt(numNodes); + metaOut.writeVInt(numNodes); // number of dimensions // TODO: Revisit the number of dimensions for timestamps (as we will split timestamp into min, hour, etc.) @@ -137,6 +150,9 @@ private static void writeMeta( // segment aggregated document count metaOut.writeVInt(segmentAggregatedDocCount); + // segment star tree document count + metaOut.writeVInt(numStarTreeDocs); + // max leaf docs metaOut.writeVInt(starTreeField.getStarTreeConfig().maxLeafDocs()); 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 89ac4af51e221..72c08d96d26ff 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 @@ -250,13 +250,12 @@ private FixedLengthStarTreeNode binarySearchChild(long dimensionValue) throws IO low++; } + int high = getInt(LAST_CHILD_ID_OFFSET); // if the current node is null node, increment the low to reduce the search space - if (matchStarTreeNodeTypeOrNull(new FixedLengthStarTreeNode(in, low), StarTreeNodeType.NULL) != null) { - low++; + if (matchStarTreeNodeTypeOrNull(new FixedLengthStarTreeNode(in, high), StarTreeNodeType.NULL) != null) { + high--; } - int high = getInt(LAST_CHILD_ID_OFFSET); - while (low <= high) { int mid = low + (high - low) / 2; FixedLengthStarTreeNode midNode = new FixedLengthStarTreeNode(in, mid); diff --git a/server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexValues.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/index/CompositeIndexValues.java similarity index 86% rename from server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexValues.java rename to server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/index/CompositeIndexValues.java index f8848aceab343..3c5c6b71849f2 100644 --- a/server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexValues.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/index/CompositeIndexValues.java @@ -6,7 +6,7 @@ * compatible open source license. */ -package org.opensearch.index.codec.composite; +package org.opensearch.index.compositeindex.datacube.startree.index; import org.opensearch.common.annotation.ExperimentalApi; diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/index/StarTreeValues.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/index/StarTreeValues.java new file mode 100644 index 0000000000000..8181a69f49bd4 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/index/StarTreeValues.java @@ -0,0 +1,265 @@ +/* + * 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.index; + +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.SegmentReadState; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexInput; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.compositeindex.CompositeIndexMetadata; +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.ReadDimension; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; +import org.opensearch.index.compositeindex.datacube.startree.fileformats.meta.StarTreeMetadata; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeFactory; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNode; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Supplier; + +import static org.opensearch.index.codec.composite.composite99.Composite99DocValuesReader.getSortedNumericDocValues; +import static org.opensearch.index.compositeindex.CompositeIndexConstants.SEGMENT_DOCS_COUNT; +import static org.opensearch.index.compositeindex.CompositeIndexConstants.STAR_TREE_DOCS_COUNT; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.fullyQualifiedFieldNameForStarTreeDimensionsDocValues; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.fullyQualifiedFieldNameForStarTreeMetricsDocValues; + +/** + * Concrete class that holds the star tree associated values from the segment + * + * @opensearch.experimental + */ +@ExperimentalApi +public class StarTreeValues implements CompositeIndexValues { + + /** + * Representing the star tree field configuration. + */ + private final StarTreeField starTreeField; + + /** + * The root node of the star tree. + */ + private final StarTreeNode root; + + /** + * A map containing suppliers for DocIdSetIterators for dimensions. + */ + private final Map> dimensionDocValuesIteratorMap; + + /** + * A map containing suppliers for DocIdSetIterators for metrics. + */ + private final Map> metricDocValuesIteratorMap; + + /** + * A map containing attributes associated with the star tree values. + */ + private final Map attributes; + + /** + * Constructs a new StarTreeValues object with the provided parameters. + * Used for testing. + * + * @param starTreeField The StarTreeField object representing the star tree field configuration. + * @param root The root node of the star tree. + * @param dimensionDocValuesIteratorMap A map containing suppliers for DocIdSetIterators for dimensions. + * @param metricDocValuesIteratorMap A map containing suppliers for DocIdSetIterators for metrics. + * @param attributes A map containing attributes associated with the star tree values. + */ + public StarTreeValues( + StarTreeField starTreeField, + StarTreeNode root, + Map> dimensionDocValuesIteratorMap, + Map> metricDocValuesIteratorMap, + Map attributes + ) { + this.starTreeField = starTreeField; + this.root = root; + this.dimensionDocValuesIteratorMap = dimensionDocValuesIteratorMap; + this.metricDocValuesIteratorMap = metricDocValuesIteratorMap; + this.attributes = attributes; + } + + /** + * Constructs a new StarTreeValues object by reading the data from the segment + * + * @param compositeIndexMetadata The CompositeIndexMetadata object containing metadata for the composite index. + * @param compositeIndexDataIn The IndexInput object for reading the composite index data. + * @param compositeDocValuesProducer The DocValuesProducer object for producing doc values. + * @param readState The SegmentReadState object representing the state of the segment being read. + * @throws IOException If an I/O error occurs while reading the data. + */ + public StarTreeValues( + CompositeIndexMetadata compositeIndexMetadata, + IndexInput compositeIndexDataIn, + DocValuesProducer compositeDocValuesProducer, + SegmentReadState readState + ) throws IOException { + + StarTreeMetadata starTreeMetadata = (StarTreeMetadata) compositeIndexMetadata; + + // build skip star node dimensions + Set skipStarNodeCreationInDims = starTreeMetadata.getSkipStarNodeCreationInDims(); + + // build dimensions + List readDimensions = new ArrayList<>(); + for (String dimension : starTreeMetadata.getDimensionFields()) { + readDimensions.add(new ReadDimension(dimension)); + } + + // star-tree field + this.starTreeField = new StarTreeField( + starTreeMetadata.getCompositeFieldName(), + readDimensions, + starTreeMetadata.getMetrics(), + new StarTreeFieldConfiguration( + starTreeMetadata.getMaxLeafDocs(), + skipStarNodeCreationInDims, + starTreeMetadata.getStarTreeBuildMode() + ) + ); + + this.root = StarTreeFactory.createStarTree(compositeIndexDataIn, starTreeMetadata); + + // get doc id set iterators for metrics and dimensions + dimensionDocValuesIteratorMap = new LinkedHashMap<>(); + metricDocValuesIteratorMap = new LinkedHashMap<>(); + + // get doc id set iterators for dimensions + for (String dimension : starTreeMetadata.getDimensionFields()) { + dimensionDocValuesIteratorMap.put(dimension, () -> { + try { + SortedNumericDocValues dimensionSortedNumericDocValues = null; + if (readState != null) { + FieldInfo dimensionfieldInfo = readState.fieldInfos.fieldInfo( + fullyQualifiedFieldNameForStarTreeDimensionsDocValues(starTreeField.getName(), dimension) + ); + if (dimensionfieldInfo != null) { + dimensionSortedNumericDocValues = compositeDocValuesProducer.getSortedNumeric(dimensionfieldInfo); + } + } + return getSortedNumericDocValues(dimensionSortedNumericDocValues); + } catch (IOException e) { + throw new RuntimeException("Error loading dimension DocIdSetIterator", e); + } + }); + } + + // get doc id set iterators for metrics + for (Metric metric : starTreeMetadata.getMetrics()) { + for (MetricStat metricStat : metric.getMetrics()) { + String metricFullName = fullyQualifiedFieldNameForStarTreeMetricsDocValues( + starTreeField.getName(), + metric.getField(), + metricStat.getTypeName() + ); + metricDocValuesIteratorMap.put(metricFullName, () -> { + try { + SortedNumericDocValues metricSortedNumericDocValues = null; + if (readState != null) { + FieldInfo metricFieldInfo = readState.fieldInfos.fieldInfo(metricFullName); + if (metricFieldInfo != null) { + metricSortedNumericDocValues = compositeDocValuesProducer.getSortedNumeric(metricFieldInfo); + } + } + return getSortedNumericDocValues(metricSortedNumericDocValues); + } catch (IOException e) { + throw new RuntimeException("Error loading metric DocIdSetIterator", e); + } + }); + } + } + + // create star-tree attributes map + + // Create an unmodifiable view of the map + attributes = Map.of( + SEGMENT_DOCS_COUNT, + String.valueOf(starTreeMetadata.getSegmentAggregatedDocCount()), + STAR_TREE_DOCS_COUNT, + String.valueOf(starTreeMetadata.getStarTreeDocCount()) + ); + + } + + @Override + public CompositeIndexValues getValues() { + return this; + } + + /** + * Returns an object representing the star tree field configuration. + * + * @return The StarTreeField object representing the star tree field configuration. + */ + public StarTreeField getStarTreeField() { + return starTreeField; + } + + /** + * Returns the root node of the star tree. + * + * @return The root node of the star tree. + */ + public StarTreeNode getRoot() { + return root; + } + + /** + * Returns the map containing attributes associated with the star tree values. + * + * @return The map containing attributes associated with the star tree values. + */ + public Map getAttributes() { + return attributes; + } + + /** + * Returns the DocIdSetIterator for the specified dimension. + * + * @param dimension The name of the dimension. + * @return The DocIdSetIterator for the specified dimension. + */ + public DocIdSetIterator getDimensionDocIdSetIterator(String dimension) { + + if (dimensionDocValuesIteratorMap.containsKey(dimension)) { + return dimensionDocValuesIteratorMap.get(dimension).get(); + } + + return DocValues.emptySortedNumeric(); + } + + /** + * Returns the DocIdSetIterator for the specified fully qualified metric name. + * + * @param fullyQualifiedMetricName The fully qualified name of the metric. + * @return The DocIdSetIterator for the specified fully qualified metric name. + */ + public DocIdSetIterator getMetricDocIdSetIterator(String fullyQualifiedMetricName) { + + if (metricDocValuesIteratorMap.containsKey(fullyQualifiedMetricName)) { + return metricDocValuesIteratorMap.get(fullyQualifiedMetricName).get(); + } + + return DocValues.emptySortedNumeric(); + } + +} diff --git a/server/src/main/java/org/opensearch/index/codec/composite/datacube/startree/package-info.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/index/package-info.java similarity index 57% rename from server/src/main/java/org/opensearch/index/codec/composite/datacube/startree/package-info.java rename to server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/index/package-info.java index 67808ad51289a..06029042ab407 100644 --- a/server/src/main/java/org/opensearch/index/codec/composite/datacube/startree/package-info.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/index/package-info.java @@ -7,6 +7,6 @@ */ /** - * classes responsible for handling all star tree structures and operations as part of codec + * Classes responsible for handling all star tree values from the segment */ -package org.opensearch.index.codec.composite.datacube.startree; +package org.opensearch.index.compositeindex.datacube.startree.index; diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/InMemoryTreeNode.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/InMemoryTreeNode.java index 20f7dcf184391..c3bf4475f75c2 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/InMemoryTreeNode.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/InMemoryTreeNode.java @@ -7,14 +7,15 @@ */ package org.opensearch.index.compositeindex.datacube.startree.node; +import org.opensearch.common.SetOnce; import org.opensearch.common.annotation.ExperimentalApi; +import java.util.LinkedHashMap; import java.util.Map; import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.ALL; /** - * /** * Represents a node in a tree data structure, specifically designed for a star-tree implementation. * A star-tree node will represent both star and non-star nodes. * @@ -23,45 +24,70 @@ @ExperimentalApi public class InMemoryTreeNode { + public InMemoryTreeNode() { + this.dimensionId = ALL; + this.startDocId = ALL; + this.endDocId = ALL; + this.nodeType = (byte) 0; + this.dimensionValue = ALL; + this.childStarNode = new SetOnce<>(); + this.children = new LinkedHashMap<>(); + } + + public InMemoryTreeNode(int dimensionId, int startDocId, int endDocId, byte nodeType, long dimensionValue) { + this.dimensionId = dimensionId; + this.startDocId = startDocId; + this.endDocId = endDocId; + this.nodeType = nodeType; + this.dimensionValue = dimensionValue; + this.childStarNode = new SetOnce<>(); + this.children = new LinkedHashMap<>(); + } + /** * The dimension id for the dimension (field) associated with this star-tree node. */ - public int dimensionId = ALL; + private final int dimensionId; /** * The starting document id (inclusive) associated with this star-tree node. */ - public int startDocId = ALL; + private final int startDocId; /** * The ending document id (exclusive) associated with this star-tree node. */ - public int endDocId = ALL; + private final int endDocId; /** * The aggregated document id associated with this star-tree node. */ - public int aggregatedDocId = ALL; + private int aggregatedDocId = ALL; /** * The child dimension identifier associated with this star-tree node. */ - public int childDimensionId = ALL; + private int childDimensionId = ALL; /** * The value of the dimension associated with this star-tree node. */ - public long dimensionValue = ALL; + private final long dimensionValue; /** * A byte indicating whether the node is star node, null node or default node (with dimension value present). */ - public byte nodeType = 0; + private byte nodeType; /** * A map containing the child nodes of this star-tree node, keyed by their dimension id. */ - public Map children; + private final Map children; + + /** + * A map containing the child star node of this star-tree node. + */ + private final SetOnce childStarNode; public long getDimensionValue() { return dimensionValue; @@ -71,4 +97,74 @@ public byte getNodeType() { return nodeType; } + public boolean hasChild() { + return !(this.children.isEmpty() && this.childStarNode.get() == null); + } + + public int getDimensionId() { + return dimensionId; + } + + public int getStartDocId() { + return startDocId; + } + + public int getEndDocId() { + return endDocId; + } + + public void setNodeType(byte nodeType) { + this.nodeType = nodeType; + } + + public void addChildNode(InMemoryTreeNode childNode, Long dimensionValue) { + if (childNode.getNodeType() == StarTreeNodeType.STAR.getValue()) { + this.childStarNode.set(childNode); + } else { + this.children.put(dimensionValue, childNode); + assert assertStarTreeChildOrder(childNode); + } + } + + public Map getChildren() { + return children; + } + + public InMemoryTreeNode getChildStarNode() { + return childStarNode.get(); + } + + public int getChildDimensionId() { + return childDimensionId; + } + + public void setChildDimensionId(int childDimensionId) { + this.childDimensionId = childDimensionId; + } + + public int getAggregatedDocId() { + return aggregatedDocId; + } + + public void setAggregatedDocId(int aggregatedDocId) { + this.aggregatedDocId = aggregatedDocId; + } + + private boolean assertStarTreeChildOrder(InMemoryTreeNode childNode) { + if (childNode.nodeType != StarTreeNodeType.NULL.getValue() && !this.children.isEmpty()) { + InMemoryTreeNode lastNode = null; + for (Map.Entry entry : this.children.entrySet()) { + lastNode = entry.getValue(); + } + assert lastNode.dimensionValue <= childNode.dimensionValue; + } else if (childNode.nodeType == StarTreeNodeType.NULL.getValue() && !this.children.isEmpty()) { + InMemoryTreeNode lastNode = null; + for (Map.Entry entry : this.children.entrySet()) { + lastNode = entry.getValue(); + } + assert lastNode.nodeType == StarTreeNodeType.NULL.getValue(); + } + return true; + } + } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNodeType.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNodeType.java index 4c4725e78ff15..9d7a31bda4deb 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNodeType.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNodeType.java @@ -9,7 +9,7 @@ package org.opensearch.index.compositeindex.datacube.startree.node; /** - * Represents the different types of nodes in a StarTreeFactory data structure. + * Represents the different types of nodes in a Star Tree data structure. * *

* In order to handle different node types, we use a byte value to represent the node type. @@ -23,11 +23,11 @@ *

* The node type can be one of the following: *

    - *
  • Star Node: Represented by the value -2. A star node is a special node that represents + *
  • Star Node: Represented by the value -1. A star node is a special node that represents * all possible values for a dimension.
  • - *
  • Null Node: Represented by the value -1. A null node indicates the absence of any value + *
  • Null Node: Represented by the value 0. A null node indicates the absence of any value * for a dimension.
  • - *
  • Default Node: Represented by the value 0. A default node represents a node with an + *
  • Default Node: Represented by the value -1. A default node represents a node with an * actual dimension value.
  • *
* @@ -42,17 +42,17 @@ public enum StarTreeNodeType { * Represents a star node type. * */ - STAR("star", (byte) -2), + STAR("star", (byte) -1), /** - * Represents a null node type. + * Represents a default node type. */ - NULL("null", (byte) -1), + DEFAULT("default", (byte) 0), /** - * Represents a default node type. + * Represents a null node type. */ - DEFAULT("default", (byte) 0); + NULL("null", (byte) 1); private final String name; private final byte value; diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/package-info.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/package-info.java index 516d5b5a012ab..19d12bc6318d7 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/package-info.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/package-info.java @@ -8,5 +8,7 @@ /** * Holds classes associated with star tree node + * + * @opensearch.experimental */ package org.opensearch.index.compositeindex.datacube.startree.node; diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/SequentialDocValuesIterator.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/SequentialDocValuesIterator.java index 400d7a1c00104..061841d3e140a 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/SequentialDocValuesIterator.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/SequentialDocValuesIterator.java @@ -28,6 +28,11 @@ public class SequentialDocValuesIterator { */ private final DocIdSetIterator docIdSetIterator; + /** + * The value associated with the latest document. + */ + private Long docValue; + /** * The id of the latest document. */ @@ -47,10 +52,19 @@ public SequentialDocValuesIterator(DocIdSetIterator docIdSetIterator) { * * @return the id of the latest document */ - int getDocId() { + public int getDocId() { return docId; } + /** + * Sets the id of the latest document. + * + * @param docId the ID of the latest document + */ + private void setDocId(int docId) { + this.docId = docId; + } + /** * Returns the DocIdSetIterator associated with this instance. * @@ -65,7 +79,7 @@ public int nextDoc(int currentDocId) throws IOException { if (docId >= currentDocId) { return docId; } - docId = this.docIdSetIterator.nextDoc(); + setDocId(this.docIdSetIterator.nextDoc()); return docId; } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeUtils.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeUtils.java index dc155df4eafca..2aae0d4ca7e29 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeUtils.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeUtils.java @@ -74,7 +74,7 @@ public static FieldInfo[] getFieldInfoList(List fields) { int fieldNumber = 0; for (String fieldName : fields) { - fieldInfoList[fieldNumber] = getFieldInfo(fieldName, fieldNumber); + fieldInfoList[fieldNumber] = getFieldInfo(fieldName, DocValuesType.SORTED_NUMERIC, fieldNumber); fieldNumber++; } return fieldInfoList; @@ -83,10 +83,11 @@ public static FieldInfo[] getFieldInfoList(List fields) { /** * Get new field info instance for a given field name and field number * @param fieldName name of the field + * @param docValuesType doc value type of the field * @param fieldNumber number of the field * @return new field info instance */ - public static FieldInfo getFieldInfo(String fieldName, int fieldNumber) { + public static FieldInfo getFieldInfo(String fieldName, DocValuesType docValuesType, int fieldNumber) { return new FieldInfo( fieldName, fieldNumber, @@ -94,7 +95,40 @@ public static FieldInfo getFieldInfo(String fieldName, int fieldNumber) { false, true, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, - DocValuesType.SORTED_NUMERIC, + docValuesType, + -1, + Collections.emptyMap(), + 0, + 0, + 0, + 0, + VectorEncoding.FLOAT32, + VectorSimilarityFunction.EUCLIDEAN, + false, + false + ); + } + + /** + * Get new field info instance for a given field name and field number. + * It's a dummy field info to fetch doc id set iterators based on field name. + *

+ * Actual field infos uses fieldNumberAcrossStarTrees parameter to achieve consistent + * and unique field numbers across fields and across multiple star trees + * + * @param fieldName name of the field + * @param docValuesType doc value type of the field + * @return new field info instance + */ + public static FieldInfo getFieldInfo(String fieldName, DocValuesType docValuesType) { + return new FieldInfo( + fieldName, + 0, + false, + false, + true, + IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, + docValuesType, -1, Collections.emptyMap(), 0, diff --git a/server/src/main/java/org/opensearch/index/engine/SegmentsStats.java b/server/src/main/java/org/opensearch/index/engine/SegmentsStats.java index 34aecfc62b8b2..8a5a4a5a94ce6 100644 --- a/server/src/main/java/org/opensearch/index/engine/SegmentsStats.java +++ b/server/src/main/java/org/opensearch/index/engine/SegmentsStats.java @@ -41,6 +41,7 @@ import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.ReplicationStats; +import org.opensearch.index.codec.composite.composite99.Composite99DocValuesFormat; import org.opensearch.index.codec.fuzzy.FuzzyFilterPostingsFormat; import org.opensearch.index.remote.RemoteSegmentStats; @@ -97,6 +98,10 @@ public class SegmentsStats implements Writeable, ToXContentFragment { Map.entry("tvd", "Term Vector Documents"), Map.entry("tvf", "Term Vector Fields"), Map.entry("liv", "Live Documents"), + Map.entry(Composite99DocValuesFormat.DATA_EXTENSION, "Composite Index"), + Map.entry(Composite99DocValuesFormat.META_EXTENSION, "Composite Index"), + Map.entry(Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, "Composite Index DocValues"), + Map.entry(Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION, "Composite Index DocValues"), Map.entry(FuzzyFilterPostingsFormat.FUZZY_FILTER_FILE_EXTENSION, "Fuzzy Filter") ); diff --git a/server/src/test/java/org/opensearch/index/codec/composite/datacube/startree/StarTreeDocValuesFormatTests.java b/server/src/test/java/org/opensearch/index/codec/composite99/datacube/startree/StarTreeDocValuesFormatTests.java similarity index 63% rename from server/src/test/java/org/opensearch/index/codec/composite/datacube/startree/StarTreeDocValuesFormatTests.java rename to server/src/test/java/org/opensearch/index/codec/composite99/datacube/startree/StarTreeDocValuesFormatTests.java index 54a9cc035d7a9..fa492e1adec0a 100644 --- a/server/src/test/java/org/opensearch/index/codec/composite/datacube/startree/StarTreeDocValuesFormatTests.java +++ b/server/src/test/java/org/opensearch/index/codec/composite99/datacube/startree/StarTreeDocValuesFormatTests.java @@ -6,7 +6,9 @@ * compatible open source license. */ -package org.opensearch.index.codec.composite.datacube.startree; +package org.opensearch.index.codec.composite99.datacube.startree; + +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -14,22 +16,34 @@ import org.apache.lucene.codecs.lucene99.Lucene99Codec; import org.apache.lucene.document.Document; import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SegmentReader; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.BaseDocValuesFormatTestCase; import org.apache.lucene.tests.index.RandomIndexWriter; import org.apache.lucene.tests.util.LuceneTestCase; +import org.apache.lucene.tests.util.TestUtil; import org.opensearch.Version; import org.opensearch.cluster.ClusterModule; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.CheckedConsumer; +import org.opensearch.common.lucene.Lucene; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.MapperTestUtils; +import org.opensearch.index.codec.composite.CompositeIndexFieldInfo; +import org.opensearch.index.codec.composite.CompositeIndexReader; import org.opensearch.index.codec.composite.composite99.Composite99Codec; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeDocument; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeTestUtils; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; import org.opensearch.index.mapper.MapperService; import org.opensearch.indices.IndicesModule; import org.junit.After; @@ -37,9 +51,13 @@ import org.junit.BeforeClass; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; +import java.util.List; import static org.opensearch.common.util.FeatureFlags.STAR_TREE_INDEX; +import static org.opensearch.index.compositeindex.datacube.startree.StarTreeTestUtils.assertStarTreeDocuments; /** * Star tree doc values Lucene tests @@ -47,6 +65,19 @@ @LuceneTestCase.SuppressSysoutChecks(bugUrl = "we log a lot on purpose") public class StarTreeDocValuesFormatTests extends BaseDocValuesFormatTestCase { MapperService mapperService = null; + StarTreeFieldConfiguration.StarTreeBuildMode buildMode; + + public StarTreeDocValuesFormatTests(StarTreeFieldConfiguration.StarTreeBuildMode buildMode) { + this.buildMode = buildMode; + } + + @ParametersFactory + public static Collection parameters() { + List parameters = new ArrayList<>(); + parameters.add(new Object[] { StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP }); + parameters.add(new Object[] { StarTreeFieldConfiguration.StarTreeBuildMode.OFF_HEAP }); + return parameters; + } @BeforeClass public static void createMapper() throws Exception { @@ -68,7 +99,7 @@ protected Codec getCodec() { final Logger testLogger = LogManager.getLogger(StarTreeDocValuesFormatTests.class); try { - createMapperService(getExpandedMapping("status", "size")); + createMapperService(getExpandedMapping()); } catch (IOException e) { throw new RuntimeException(e); } @@ -86,15 +117,18 @@ public void testStarTreeDocValues() throws IOException { doc.add(new SortedNumericDocValuesField("dv", 1)); doc.add(new SortedNumericDocValuesField("field", 1)); iw.addDocument(doc); + doc = new Document(); doc.add(new SortedNumericDocValuesField("sndv", 1)); doc.add(new SortedNumericDocValuesField("dv", 1)); doc.add(new SortedNumericDocValuesField("field", 1)); iw.addDocument(doc); + doc = new Document(); iw.forceMerge(1); doc.add(new SortedNumericDocValuesField("sndv", 2)); doc.add(new SortedNumericDocValuesField("dv", 2)); doc.add(new SortedNumericDocValuesField("field", 2)); iw.addDocument(doc); + doc = new Document(); doc.add(new SortedNumericDocValuesField("sndv", 2)); doc.add(new SortedNumericDocValuesField("dv", 2)); doc.add(new SortedNumericDocValuesField("field", 2)); @@ -102,17 +136,42 @@ public void testStarTreeDocValues() throws IOException { iw.forceMerge(1); iw.close(); - // TODO : validate star tree structures that got created + DirectoryReader ir = maybeWrapWithMergingReader(DirectoryReader.open(directory)); + TestUtil.checkReader(ir); + assertEquals(1, ir.leaves().size()); + + StarTreeDocument[] expectedStarTreeDocuments = new StarTreeDocument[4]; + expectedStarTreeDocuments[0] = new StarTreeDocument(new Long[] { 1L, 1L }, new Double[] { 2.0, 2.0, 2.0 }); + expectedStarTreeDocuments[1] = new StarTreeDocument(new Long[] { 2L, 2L }, new Double[] { 4.0, 2.0, 4.0 }); + expectedStarTreeDocuments[2] = new StarTreeDocument(new Long[] { null, 1L }, new Double[] { 2.0, 2.0, 2.0 }); + expectedStarTreeDocuments[3] = new StarTreeDocument(new Long[] { null, 2L }, new Double[] { 4.0, 2.0, 4.0 }); + + for (LeafReaderContext context : ir.leaves()) { + SegmentReader reader = Lucene.segmentReader(context.reader()); + CompositeIndexReader starTreeDocValuesReader = (CompositeIndexReader) reader.getDocValuesReader(); + List compositeIndexFields = starTreeDocValuesReader.getCompositeIndexFields(); + + for (CompositeIndexFieldInfo compositeIndexFieldInfo : compositeIndexFields) { + StarTreeValues starTreeValues = (StarTreeValues) starTreeDocValuesReader.getCompositeIndexValues(compositeIndexFieldInfo); + StarTreeDocument[] starTreeDocuments = StarTreeTestUtils.getSegmentsStarTreeDocuments( + List.of(starTreeValues), + List.of(StarTreeNumericType.DOUBLE, StarTreeNumericType.LONG, StarTreeNumericType.LONG), + reader.maxDoc() + ); + assertStarTreeDocuments(starTreeDocuments, expectedStarTreeDocuments); + } + } + ir.close(); directory.close(); } - private XContentBuilder getExpandedMapping(String dim, String metric) throws IOException { + private XContentBuilder getExpandedMapping() throws IOException { return topMapping(b -> { b.startObject("composite"); b.startObject("startree"); b.field("type", "star_tree"); b.startObject("config"); - b.field("max_leaf_docs", 100); + b.field("max_leaf_docs", 1); b.startArray("ordered_dimensions"); b.startObject(); b.field("name", "sndv"); diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeTestUtils.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeTestUtils.java new file mode 100644 index 0000000000000..53c9e05ccb490 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeTestUtils.java @@ -0,0 +1,297 @@ +/* + * 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; + +import org.apache.lucene.store.IndexInput; +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.aggregators.numerictype.StarTreeNumericType; +import org.opensearch.index.compositeindex.datacube.startree.fileformats.meta.StarTreeMetadata; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; +import org.opensearch.index.compositeindex.datacube.startree.node.InMemoryTreeNode; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeFactory; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNode; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNodeType; +import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; +import org.opensearch.index.mapper.CompositeMappedFieldType; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Queue; + +import static org.opensearch.index.compositeindex.CompositeIndexConstants.COMPOSITE_FIELD_MARKER; +import static org.opensearch.index.compositeindex.datacube.startree.fileformats.StarTreeWriter.VERSION_CURRENT; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.fullyQualifiedFieldNameForStarTreeMetricsDocValues; +import static org.opensearch.index.mapper.CompositeMappedFieldType.CompositeFieldType.STAR_TREE; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +public class StarTreeTestUtils { + + public static StarTreeDocument[] getSegmentsStarTreeDocuments( + List starTreeValuesSubs, + List starTreeNumericTypes, + int numDocs + ) throws IOException { + List starTreeDocuments = new ArrayList<>(); + for (StarTreeValues starTreeValues : starTreeValuesSubs) { + List dimensionsSplitOrder = starTreeValues.getStarTreeField().getDimensionsOrder(); + SequentialDocValuesIterator[] dimensionReaders = new SequentialDocValuesIterator[dimensionsSplitOrder.size()]; + + for (int i = 0; i < dimensionsSplitOrder.size(); i++) { + String dimension = dimensionsSplitOrder.get(i).getField(); + dimensionReaders[i] = new SequentialDocValuesIterator(starTreeValues.getDimensionDocIdSetIterator(dimension)); + } + + List metricReaders = new ArrayList<>(); + // get doc id set iterators for metrics + for (Metric metric : starTreeValues.getStarTreeField().getMetrics()) { + for (MetricStat metricStat : metric.getMetrics()) { + String metricFullName = fullyQualifiedFieldNameForStarTreeMetricsDocValues( + starTreeValues.getStarTreeField().getName(), + metric.getField(), + metricStat.getTypeName() + ); + metricReaders.add(new SequentialDocValuesIterator(starTreeValues.getMetricDocIdSetIterator(metricFullName))); + + } + } + int currentDocId = 0; + while (currentDocId < numDocs) { + starTreeDocuments.add(getStarTreeDocument(currentDocId, dimensionReaders, metricReaders, starTreeNumericTypes)); + currentDocId++; + } + } + StarTreeDocument[] starTreeDocumentsArr = new StarTreeDocument[starTreeDocuments.size()]; + return starTreeDocuments.toArray(starTreeDocumentsArr); + } + + public static StarTreeDocument getStarTreeDocument( + int currentDocId, + SequentialDocValuesIterator[] dimensionReaders, + List metricReaders, + List starTreeNumericTypes + ) throws IOException { + Long[] dims = new Long[dimensionReaders.length]; + int i = 0; + for (SequentialDocValuesIterator dimensionDocValueIterator : dimensionReaders) { + dimensionDocValueIterator.nextDoc(currentDocId); + Long val = dimensionDocValueIterator.value(currentDocId); + dims[i] = val; + i++; + } + i = 0; + Object[] metrics = new Object[metricReaders.size()]; + for (SequentialDocValuesIterator metricDocValuesIterator : metricReaders) { + metricDocValuesIterator.nextDoc(currentDocId); + metrics[i] = toStarTreeNumericTypeValue(metricDocValuesIterator.value(currentDocId), starTreeNumericTypes.get(i)); + i++; + } + return new StarTreeDocument(dims, metrics); + } + + public static Double toStarTreeNumericTypeValue(Long value, StarTreeNumericType starTreeNumericType) { + try { + return starTreeNumericType.getDoubleValue(value); + } catch (Exception e) { + throw new IllegalStateException("Cannot convert " + value + " to sortable aggregation type", e); + } + } + + public static void assertStarTreeDocuments(StarTreeDocument[] starTreeDocuments, StarTreeDocument[] expectedStarTreeDocuments) { + + assertNotNull(starTreeDocuments); + assertEquals(starTreeDocuments.length, expectedStarTreeDocuments.length); + + for (int i = 0; i < starTreeDocuments.length; i++) { + + StarTreeDocument resultStarTreeDocument = starTreeDocuments[i]; + StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocuments[i]; + + assertNotNull(resultStarTreeDocument.dimensions); + assertNotNull(resultStarTreeDocument.metrics); + + assertEquals(resultStarTreeDocument.dimensions.length, expectedStarTreeDocument.dimensions.length); + assertEquals(resultStarTreeDocument.metrics.length, expectedStarTreeDocument.metrics.length); + + for (int di = 0; di < resultStarTreeDocument.dimensions.length; di++) { + assertEquals(resultStarTreeDocument.dimensions[di], expectedStarTreeDocument.dimensions[di]); + } + + for (int mi = 0; mi < resultStarTreeDocument.metrics.length; mi++) { + if (expectedStarTreeDocument.metrics[mi] instanceof Long) { + assertEquals(resultStarTreeDocument.metrics[mi], ((Long) expectedStarTreeDocument.metrics[mi]).doubleValue()); + } else { + assertEquals(resultStarTreeDocument.metrics[mi], expectedStarTreeDocument.metrics[mi]); + } + } + } + } + + public static void validateFileFormats( + IndexInput dataIn, + IndexInput metaIn, + InMemoryTreeNode rootNode, + StarTreeMetadata expectedStarTreeMetadata + ) throws IOException { + long magicMarker = metaIn.readLong(); + assertEquals(COMPOSITE_FIELD_MARKER, magicMarker); + int version = metaIn.readVInt(); + assertEquals(VERSION_CURRENT, version); + + String compositeFieldName = metaIn.readString(); + assertEquals(expectedStarTreeMetadata.getStarTreeFieldName(), compositeFieldName); + CompositeMappedFieldType.CompositeFieldType compositeFieldType = CompositeMappedFieldType.CompositeFieldType.fromName( + metaIn.readString() + ); + assertEquals(STAR_TREE, compositeFieldType); + StarTreeMetadata resultStarTreeMetadata = new StarTreeMetadata(metaIn, compositeFieldName, compositeFieldType, version); + assertStarTreeMetadata(expectedStarTreeMetadata, resultStarTreeMetadata); + + IndexInput starTreeIndexInput = dataIn.slice( + "star-tree data slice for respective star-tree fields", + resultStarTreeMetadata.getDataStartFilePointer(), + resultStarTreeMetadata.getDataLength() + ); + + StarTreeNode starTreeNode = StarTreeFactory.createStarTree(starTreeIndexInput, resultStarTreeMetadata); + Queue expectedTreeNodeQueue = new ArrayDeque<>(); + Queue resultTreeNodeQueue = new ArrayDeque<>(); + + expectedTreeNodeQueue.add(starTreeNode); + resultTreeNodeQueue.add(rootNode); + + while ((starTreeNode = expectedTreeNodeQueue.poll()) != null && (rootNode = resultTreeNodeQueue.poll()) != null) { + + // verify the star node + assertStarTreeNode(starTreeNode, rootNode); + + Iterator expectedChildrenIterator = starTreeNode.getChildrenIterator(); + + List sortedChildren = new ArrayList<>(); + if (rootNode.getChildren() != null) { + sortedChildren = new ArrayList<>(rootNode.getChildren().values()); + } + + if (starTreeNode.getChildDimensionId() != -1) { + assertFalse(sortedChildren.isEmpty()); + int childCount = 0; + boolean childStarNodeAsserted = false; + while (expectedChildrenIterator.hasNext()) { + StarTreeNode child = expectedChildrenIterator.next(); + InMemoryTreeNode resultChildNode = null; + if (!childStarNodeAsserted && rootNode.getChildStarNode() != null) { + // check if star tree node exists + resultChildNode = rootNode.getChildStarNode(); + assertNotNull(child); + assertNotNull(starTreeNode.getChildStarNode()); + assertStarTreeNode(child, resultChildNode); + childStarNodeAsserted = true; + } else { + resultChildNode = sortedChildren.get(childCount); + assertNotNull(child); + assertNotNull(resultChildNode); + if (child.getStarTreeNodeType() != StarTreeNodeType.NULL.getValue()) { + assertNotNull(starTreeNode.getChildForDimensionValue(child.getDimensionValue())); + } else { + StarTreeNode finalStarTreeNode = starTreeNode; + assertThrows( + AssertionError.class, + () -> finalStarTreeNode.getChildForDimensionValue(child.getDimensionValue()) + ); + } + assertStarTreeNode(child, resultChildNode); + assertNotEquals(child.getStarTreeNodeType(), StarTreeNodeType.STAR.getValue()); + childCount++; + } + + expectedTreeNodeQueue.add(child); + resultTreeNodeQueue.add(resultChildNode); + } + + assertEquals(childCount, rootNode.getChildren().size()); + } else { + assertTrue(rootNode.getChildren().isEmpty()); + } + } + + assertTrue(expectedTreeNodeQueue.isEmpty()); + assertTrue(resultTreeNodeQueue.isEmpty()); + + } + + public static void assertStarTreeNode(StarTreeNode starTreeNode, InMemoryTreeNode treeNode) throws IOException { + assertEquals(starTreeNode.getDimensionId(), treeNode.getDimensionId()); + assertEquals(starTreeNode.getDimensionValue(), treeNode.getDimensionValue()); + assertEquals(starTreeNode.getStartDocId(), treeNode.getStartDocId()); + assertEquals(starTreeNode.getEndDocId(), treeNode.getEndDocId()); + assertEquals(starTreeNode.getChildDimensionId(), treeNode.getChildDimensionId()); + assertEquals(starTreeNode.getAggregatedDocId(), treeNode.getAggregatedDocId()); + + if (starTreeNode.getChildDimensionId() != -1) { + assertFalse(starTreeNode.isLeaf()); + if (treeNode.getChildren() != null) { + assertEquals( + starTreeNode.getNumChildren(), + treeNode.getChildren().values().size() + (treeNode.getChildStarNode() != null ? 1 : 0) + ); + } + } else { + assertTrue(starTreeNode.isLeaf()); + } + + } + + public static void assertStarTreeMetadata(StarTreeMetadata expectedStarTreeMetadata, StarTreeMetadata resultStarTreeMetadata) { + + assertEquals(expectedStarTreeMetadata.getCompositeFieldName(), resultStarTreeMetadata.getCompositeFieldName()); + assertEquals(expectedStarTreeMetadata.getCompositeFieldType(), resultStarTreeMetadata.getCompositeFieldType()); + assertEquals(expectedStarTreeMetadata.getDimensionFields().size(), resultStarTreeMetadata.getDimensionFields().size()); + for (int i = 0; i < expectedStarTreeMetadata.getDimensionFields().size(); i++) { + assertEquals(expectedStarTreeMetadata.getDimensionFields().get(i), resultStarTreeMetadata.getDimensionFields().get(i)); + } + assertEquals(expectedStarTreeMetadata.getMetrics().size(), resultStarTreeMetadata.getMetrics().size()); + + for (int i = 0; i < expectedStarTreeMetadata.getMetrics().size(); i++) { + + Metric expectedMetric = expectedStarTreeMetadata.getMetrics().get(i); + Metric resultMetric = resultStarTreeMetadata.getMetrics().get(i); + assertEquals(expectedMetric.getField(), resultMetric.getField()); + + for (int j = 0; j < expectedMetric.getMetrics().size(); j++) { + assertEquals(expectedMetric.getMetrics().get(j), resultMetric.getMetrics().get(j)); + } + + } + + assertEquals(expectedStarTreeMetadata.getSegmentAggregatedDocCount(), resultStarTreeMetadata.getSegmentAggregatedDocCount()); + assertEquals(expectedStarTreeMetadata.getStarTreeDocCount(), resultStarTreeMetadata.getStarTreeDocCount()); + assertEquals(expectedStarTreeMetadata.getMaxLeafDocs(), resultStarTreeMetadata.getMaxLeafDocs()); + assertEquals( + expectedStarTreeMetadata.getSkipStarNodeCreationInDims().size(), + resultStarTreeMetadata.getSkipStarNodeCreationInDims().size() + ); + for (String skipDimension : expectedStarTreeMetadata.getSkipStarNodeCreationInDims()) { + assertTrue(resultStarTreeMetadata.getSkipStarNodeCreationInDims().contains(skipDimension)); + } + assertEquals(expectedStarTreeMetadata.getStarTreeBuildMode(), resultStarTreeMetadata.getStarTreeBuildMode()); + assertEquals(expectedStarTreeMetadata.getDataStartFilePointer(), resultStarTreeMetadata.getDataStartFilePointer()); + assertEquals(expectedStarTreeMetadata.getDataLength(), resultStarTreeMetadata.getDataLength()); + assertEquals(0, (resultStarTreeMetadata.getDataLength()) % 33); + } + +} diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java index 33088e8ccbcb3..79d1b0e4b9785 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java @@ -8,6 +8,7 @@ package org.opensearch.index.compositeindex.datacube.startree.builder; +import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.lucene99.Lucene99Codec; import org.apache.lucene.index.DocValues; @@ -15,8 +16,10 @@ import org.apache.lucene.index.EmptyDocValuesProducer; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.index.SegmentReadState; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.index.VectorEncoding; @@ -24,11 +27,18 @@ import org.apache.lucene.sandbox.document.HalfFloatPoint; import org.apache.lucene.search.DocIdSetIterator; 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.apache.lucene.util.InfoStream; import org.apache.lucene.util.NumericUtils; import org.apache.lucene.util.Version; import org.opensearch.common.settings.Settings; -import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; +import org.opensearch.index.codec.composite.LuceneDocValuesConsumerFactory; +import org.opensearch.index.codec.composite.LuceneDocValuesProducerFactory; +import org.opensearch.index.codec.composite.composite99.Composite99Codec; +import org.opensearch.index.codec.composite.composite99.Composite99DocValuesFormat; +import org.opensearch.index.compositeindex.CompositeIndexConstants; import org.opensearch.index.compositeindex.datacube.Dimension; import org.opensearch.index.compositeindex.datacube.Metric; import org.opensearch.index.compositeindex.datacube.MetricStat; @@ -36,6 +46,10 @@ import org.opensearch.index.compositeindex.datacube.startree.StarTreeDocument; import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeTestUtils; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; +import org.opensearch.index.compositeindex.datacube.startree.fileformats.meta.StarTreeMetadata; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; import org.opensearch.index.compositeindex.datacube.startree.node.InMemoryTreeNode; import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNodeType; import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; @@ -65,8 +79,14 @@ import java.util.Queue; import java.util.Set; import java.util.UUID; - -import static org.opensearch.index.compositeindex.datacube.startree.builder.BaseStarTreeBuilder.NUM_SEGMENT_DOCS; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; + +import static org.opensearch.index.compositeindex.datacube.startree.StarTreeTestUtils.validateFileFormats; +import static org.opensearch.index.compositeindex.datacube.startree.fileformats.StarTreeWriter.VERSION_CURRENT; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.fullyQualifiedFieldNameForStarTreeDimensionsDocValues; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.fullyQualifiedFieldNameForStarTreeMetricsDocValues; +import static org.opensearch.index.mapper.CompositeMappedFieldType.CompositeFieldType.STAR_TREE; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -80,7 +100,12 @@ public abstract class AbstractStarTreeBuilderTests extends OpenSearchTestCase { protected StarTreeField compositeField; protected Map fieldProducerMap; protected SegmentWriteState writeState; - private BaseStarTreeBuilder builder; + protected BaseStarTreeBuilder builder; + protected IndexOutput dataOut; + protected IndexOutput metaOut; + protected DocValuesConsumer docValuesConsumer; + protected String dataFileName; + protected String metaFileName; @Before public void setup() throws IOException { @@ -107,7 +132,7 @@ public void setup() throws IOException { "test", dimensionsOrder, metrics, - new StarTreeFieldConfiguration(1, Set.of("field8"), StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP) + new StarTreeFieldConfiguration(1, Set.of("field8"), getBuildMode()) ); directory = newFSDirectory(createTempDir()); @@ -135,7 +160,21 @@ public void setup() throws IOException { ); fieldProducerMap.put(fields.get(i), docValuesProducer); } - writeState = getWriteState(5); + writeState = getWriteState(5, UUID.randomUUID().toString().substring(0, 16).getBytes(StandardCharsets.UTF_8)); + + dataFileName = IndexFileNames.segmentFileName( + writeState.segmentInfo.name, + writeState.segmentSuffix, + Composite99DocValuesFormat.DATA_EXTENSION + ); + dataOut = writeState.directory.createOutput(dataFileName, writeState.context); + + metaFileName = IndexFileNames.segmentFileName( + writeState.segmentInfo.name, + writeState.segmentSuffix, + Composite99DocValuesFormat.META_EXTENSION + ); + metaOut = writeState.directory.createOutput(metaFileName, writeState.context); mapperService = mock(MapperService.class); DocumentMapper documentMapper = mock(DocumentMapper.class); @@ -159,9 +198,91 @@ public void setup() throws IOException { null ); when(documentMapper.mappers()).thenReturn(fieldMappers); + docValuesConsumer = mock(DocValuesConsumer.class); + } + + private SegmentReadState getReadState(int numDocs, List dimensionFields, List metrics) { + + int numMetrics = 0; + for (Metric metric : metrics) { + for (MetricStat metricStat : metric.getMetrics()) { + numMetrics++; + } + } + + FieldInfo[] fields = new FieldInfo[dimensionFields.size() + numMetrics]; + + int i = 0; + for (String dimension : dimensionFields) { + fields[i] = new FieldInfo( + fullyQualifiedFieldNameForStarTreeDimensionsDocValues(compositeField.getName(), dimension), + i, + false, + false, + true, + IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, + DocValuesType.SORTED_NUMERIC, + -1, + Collections.emptyMap(), + 0, + 0, + 0, + 0, + VectorEncoding.FLOAT32, + VectorSimilarityFunction.EUCLIDEAN, + false, + false + ); + i++; + } + + for (Metric metric : metrics) { + for (MetricStat metricStat : metric.getMetrics()) { + fields[i] = new FieldInfo( + fullyQualifiedFieldNameForStarTreeMetricsDocValues( + compositeField.getName(), + metric.getField(), + metricStat.getTypeName() + ), + i, + false, + false, + true, + IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, + DocValuesType.SORTED_NUMERIC, + -1, + Collections.emptyMap(), + 0, + 0, + 0, + 0, + VectorEncoding.FLOAT32, + VectorSimilarityFunction.EUCLIDEAN, + false, + false + ); + i++; + } + } + + SegmentInfo segmentInfo = new SegmentInfo( + directory, + Version.LATEST, + Version.LUCENE_9_11_0, + "test_segment", + numDocs, + false, + false, + new Lucene99Codec(), + new HashMap<>(), + writeState.segmentInfo.getId(), + new HashMap<>(), + null + ); + return new SegmentReadState(segmentInfo.dir, segmentInfo, new FieldInfos(fields), writeState.context); } - private SegmentWriteState getWriteState(int numDocs) { + private SegmentWriteState getWriteState(int numDocs, byte[] id) { FieldInfos fieldInfos = new FieldInfos(fieldsInfo); SegmentInfo segmentInfo = new SegmentInfo( directory, @@ -173,7 +294,7 @@ private SegmentWriteState getWriteState(int numDocs) { false, new Lucene99Codec(), new HashMap<>(), - UUID.randomUUID().toString().substring(0, 16).getBytes(StandardCharsets.UTF_8), + id, new HashMap<>(), null ); @@ -181,6 +302,8 @@ private SegmentWriteState getWriteState(int numDocs) { } public abstract BaseStarTreeBuilder getStarTreeBuilder( + IndexOutput metaOut, + IndexOutput dataOut, StarTreeField starTreeField, SegmentWriteState segmentWriteState, MapperService mapperService @@ -233,7 +356,7 @@ public void test_sortAndAggregateStarTreeDocuments() throws IOException { SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); List metricsIterators = getMetricIterators(segmentStarTreeDocuments); - builder = getStarTreeBuilder(compositeField, writeState, mapperService); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( dimsIterators, metricsIterators @@ -329,7 +452,7 @@ public void test_sortAndAggregateStarTreeDocuments_nullMetric() throws IOExcepti } SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); List metricsIterators = getMetricIterators(segmentStarTreeDocuments); - builder = getStarTreeBuilder(compositeField, writeState, mapperService); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( dimsIterators, metricsIterators @@ -400,7 +523,7 @@ public void test_sortAndAggregateStarTreeDocuments_nullMetricField() throws IOEx } SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); List metricsIterators = getMetricIterators(segmentStarTreeDocuments); - builder = getStarTreeBuilder(compositeField, writeState, mapperService); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( dimsIterators, metricsIterators @@ -422,30 +545,35 @@ public void test_sortAndAggregateStarTreeDocuments_nullMetricField() throws IOEx } } - @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/14813") public void test_sortAndAggregateStarTreeDocuments_nullAndMinusOneInDimensionField() throws IOException { int noOfStarTreeDocuments = 5; StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; // Setting second metric iterator as empty sorted numeric , indicating a metric field is null starTreeDocuments[0] = new StarTreeDocument( new Long[] { 2L, null, 3L, 4L }, - new Object[] { 12.0, null, randomDouble(), 8.0, 20.0 } + new Object[] { 12.0, null, randomDouble(), 8.0, 20.0, null } ); starTreeDocuments[1] = new StarTreeDocument( new Long[] { null, 4L, 2L, 1L }, - new Object[] { 10.0, null, randomDouble(), 12.0, 10.0 } + new Object[] { 10.0, null, randomDouble(), 12.0, 10.0, null } ); starTreeDocuments[2] = new StarTreeDocument( new Long[] { null, 4L, 2L, 1L }, - new Object[] { 14.0, null, randomDouble(), 6.0, 24.0 } + new Object[] { 14.0, null, randomDouble(), 6.0, 24.0, null } + ); + starTreeDocuments[3] = new StarTreeDocument( + new Long[] { 2L, null, 3L, 4L }, + new Object[] { 9.0, null, randomDouble(), 9.0, 12.0, 10L } + ); + starTreeDocuments[4] = new StarTreeDocument( + new Long[] { -1L, 4L, 2L, 1L }, + new Object[] { 11.0, null, randomDouble(), 8.0, 13.0, null } ); - starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, null, 3L, 4L }, new Object[] { 9.0, null, randomDouble(), 9.0, 12.0 }); - starTreeDocuments[4] = new StarTreeDocument(new Long[] { -1L, 4L, 2L, 1L }, new Object[] { 11.0, null, randomDouble(), 8.0, 13.0 }); List inorderStarTreeDocuments = List.of( - new StarTreeDocument(new Long[] { 2L, null, 3L, 4L }, new Object[] { 21.0, 0.0, 2L }), - new StarTreeDocument(new Long[] { null, 4L, 2L, 1L }, new Object[] { 24.0, 0.0, 2L }), - new StarTreeDocument(new Long[] { -1L, 4L, 2L, 1L }, new Object[] { 11.0, 0.0, 1L }) + new StarTreeDocument(new Long[] { -1L, 4L, 2L, 1L }, new Object[] { 11.0, 0.0, 1L, 8.0, 13.0, 1L }), + new StarTreeDocument(new Long[] { 2L, null, 3L, 4L }, new Object[] { 21.0, 0.0, 2L, 8.0, 20.0, 11L }), + new StarTreeDocument(new Long[] { null, 4L, 2L, 1L }, new Object[] { 24.0, 0.0, 2L, 6.0, 24.0, 2L }) ); Iterator expectedStarTreeDocumentIterator = inorderStarTreeDocuments.iterator(); @@ -458,20 +586,22 @@ public void test_sortAndAggregateStarTreeDocuments_nullAndMinusOneInDimensionFie long metric3 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[2]); long metric4 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[3]); long metric5 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[4]); + Long metric6 = starTreeDocuments[i].metrics[5] != null ? (long) starTreeDocuments[i].metrics[5] : null; segmentStarTreeDocuments[i] = new StarTreeDocument( starTreeDocuments[i].dimensions, - new Object[] { metric1, metric2, metric3, metric4, metric5 } + new Object[] { metric1, metric2, metric3, metric4, metric5, metric6 } ); } SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); List metricsIterators = getMetricIterators(segmentStarTreeDocuments); - builder = getStarTreeBuilder(compositeField, writeState, mapperService); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( dimsIterators, metricsIterators ); - for (StarTreeDocument resultStarTreeDocument : builder.getStarTreeDocuments()) { + while (segmentStarTreeDocumentIterator.hasNext()) { + StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); @@ -484,7 +614,10 @@ public void test_sortAndAggregateStarTreeDocuments_nullAndMinusOneInDimensionFie assertEquals(expectedStarTreeDocument.metrics[4], resultStarTreeDocument.metrics[4]); assertEquals(expectedStarTreeDocument.metrics[5], resultStarTreeDocument.metrics[5]); } - builder.build(segmentStarTreeDocumentIterator); + + assertFalse(expectedStarTreeDocumentIterator.hasNext()); + + builder.build(segmentStarTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); validateStarTree(builder.getRootNode(), 4, 1, builder.getStarTreeDocuments()); } @@ -542,7 +675,7 @@ public void test_sortAndAggregateStarTreeDocuments_nullDimensionsAndNullMetrics( } SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); List metricsIterators = getMetricIterators(segmentStarTreeDocuments); - builder = getStarTreeBuilder(compositeField, writeState, mapperService); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( dimsIterators, metricsIterators @@ -562,7 +695,7 @@ public void test_sortAndAggregateStarTreeDocuments_nullDimensionsAndNullMetrics( assertEquals(expectedStarTreeDocument.metrics[4], resultStarTreeDocument.metrics[4]); assertEquals(expectedStarTreeDocument.metrics[5], resultStarTreeDocument.metrics[5]); } - builder.build(segmentStarTreeDocumentIterator); + builder.build(segmentStarTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); validateStarTree(builder.getRootNode(), 4, 1, builder.getStarTreeDocuments()); } @@ -619,7 +752,7 @@ public void test_sortAndAggregateStarTreeDocuments_nullDimensionsAndFewNullMetri } SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); List metricsIterators = getMetricIterators(segmentStarTreeDocuments); - builder = getStarTreeBuilder(compositeField, writeState, mapperService); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( dimsIterators, metricsIterators @@ -639,7 +772,7 @@ public void test_sortAndAggregateStarTreeDocuments_nullDimensionsAndFewNullMetri assertEquals(expectedStarTreeDocument.metrics[4], resultStarTreeDocument.metrics[4]); assertEquals(expectedStarTreeDocument.metrics[5], resultStarTreeDocument.metrics[5]); } - builder.build(segmentStarTreeDocumentIterator); + builder.build(segmentStarTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); validateStarTree(builder.getRootNode(), 4, 1, builder.getStarTreeDocuments()); } @@ -691,7 +824,7 @@ public void test_sortAndAggregateStarTreeDocuments_emptyDimensions() throws IOEx } SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); List metricsIterators = getMetricIterators(segmentStarTreeDocuments); - builder = getStarTreeBuilder(compositeField, writeState, mapperService); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( dimsIterators, metricsIterators @@ -760,7 +893,7 @@ public void test_sortAndAggregateStarTreeDocument_longMaxAndLongMinDimensions() SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); List metricsIterators = getMetricIterators(segmentStarTreeDocuments); - builder = getStarTreeBuilder(compositeField, writeState, mapperService); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( dimsIterators, metricsIterators @@ -836,7 +969,7 @@ public void test_sortAndAggregateStarTreeDocument_DoubleMaxAndDoubleMinMetrics() SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); List metricsIterators = getMetricIterators(segmentStarTreeDocuments); - builder = getStarTreeBuilder(compositeField, writeState, mapperService); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( dimsIterators, metricsIterators @@ -860,7 +993,7 @@ public void test_sortAndAggregateStarTreeDocument_DoubleMaxAndDoubleMinMetrics() } assertEquals(inorderStarTreeDocuments.size(), numOfAggregatedDocuments); - builder.build(segmentStarTreeDocumentIterator); + builder.build(segmentStarTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); validateStarTree(builder.getRootNode(), 3, 1, builder.getStarTreeDocuments()); } @@ -968,19 +1101,53 @@ public void test_build_halfFloatMetrics() throws IOException { SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); List metricsIterators = getMetricIterators(segmentStarTreeDocuments); - builder = getStarTreeBuilder(compositeField, writeState, mapperService); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( dimsIterators, metricsIterators ); - builder.build(segmentStarTreeDocumentIterator); + builder.build(segmentStarTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); List resultStarTreeDocuments = builder.getStarTreeDocuments(); assertEquals(7, resultStarTreeDocuments.size()); - Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator(); + Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator().iterator(); assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); - builder.build(expectedStarTreeDocumentIterator); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "test", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + getStarTreeDimensionNames(compositeField.getDimensionsOrder()), + compositeField.getMetrics(), + 2, + getExpectedStarTreeDocumentIterator().size(), + 1, + Set.of("field8"), + getBuildMode(), + 0, + 330 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + getExpectedStarTreeDocumentIterator().size(), + starTreeMetadata, + getExpectedStarTreeDocumentIterator() + ); } public void test_build_floatMetrics() throws IOException { @@ -1048,20 +1215,57 @@ public void test_build_floatMetrics() throws IOException { SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); List metricsIterators = getMetricIterators(segmentStarTreeDocuments); - builder = getStarTreeBuilder(compositeField, writeState, mapperService); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( dimsIterators, metricsIterators ); - builder.build(segmentStarTreeDocumentIterator); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder.build(segmentStarTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); List resultStarTreeDocuments = builder.getStarTreeDocuments(); assertEquals(7, resultStarTreeDocuments.size()); - Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator(); + Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator().iterator(); assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "test", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + getStarTreeDimensionNames(compositeField.getDimensionsOrder()), + compositeField.getMetrics(), + 2, + getExpectedStarTreeDocumentIterator().size(), + 1, + Set.of("field8"), + getBuildMode(), + 0, + 330 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + getExpectedStarTreeDocumentIterator().size(), + starTreeMetadata, + getExpectedStarTreeDocumentIterator() + ); } + abstract StarTreeFieldConfiguration.StarTreeBuildMode getBuildMode(); + public void test_build_longMetrics() throws IOException { mapperService = mock(MapperService.class); @@ -1111,22 +1315,57 @@ public void test_build_longMetrics() throws IOException { SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); List metricsIterators = getMetricIterators(segmentStarTreeDocuments); - builder = getStarTreeBuilder(compositeField, writeState, mapperService); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( dimsIterators, metricsIterators ); - builder.build(segmentStarTreeDocumentIterator); + builder.build(segmentStarTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); List resultStarTreeDocuments = builder.getStarTreeDocuments(); assertEquals(7, resultStarTreeDocuments.size()); - Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator(); + Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator().iterator(); assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "test", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + getStarTreeDimensionNames(compositeField.getDimensionsOrder()), + compositeField.getMetrics(), + 2, + getExpectedStarTreeDocumentIterator().size(), + 1, + Set.of("field8"), + getBuildMode(), + 0, + 330 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + getExpectedStarTreeDocumentIterator().size(), + starTreeMetadata, + getExpectedStarTreeDocumentIterator() + ); } - private static Iterator getExpectedStarTreeDocumentIterator() { - List expectedStarTreeDocuments = List.of( + private static List getExpectedStarTreeDocumentIterator() { + return List.of( new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Object[] { 21.0, 14.0, 2L, 8.0, 20.0, 2L }), new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Object[] { 35.0, 34.0, 3L, 6.0, 24.0, 3L }), new StarTreeDocument(new Long[] { null, 4L, 2L, 1L }, new Object[] { 35.0, 34.0, 3L, 6.0, 24.0, 3L }), @@ -1135,7 +1374,220 @@ private static Iterator getExpectedStarTreeDocumentIterator() new StarTreeDocument(new Long[] { null, 4L, null, 4L }, new Object[] { 21.0, 14.0, 2L, 8.0, 20.0, 2L }), new StarTreeDocument(new Long[] { null, 4L, null, null }, new Object[] { 56.0, 48.0, 5L, 6.0, 24.0, 5L }) ); - return expectedStarTreeDocuments.iterator(); + } + + public void test_build_multipleStarTrees() throws IOException { + + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + + starTreeDocuments[0] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 12.0, 10.0, randomDouble(), 8.0, 20.0 }); + starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 10.0, 6.0, randomDouble(), 12.0, 10.0 }); + starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 14.0, 12.0, randomDouble(), 6.0, 24.0 }); + starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 9.0, 4.0, randomDouble(), 9.0, 12.0 }); + starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 11.0, 16.0, randomDouble(), 8.0, 13.0 }); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + long metric1 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[0]); + long metric2 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[1]); + long metric3 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[2]); + long metric4 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[3]); + long metric5 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[4]); + segmentStarTreeDocuments[i] = new StarTreeDocument( + starTreeDocuments[i].dimensions, + new Long[] { metric1, metric2, metric3, metric4, metric5 } + ); + } + + SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); + List metricsIterators = getMetricIterators(segmentStarTreeDocuments); + + metrics = List.of( + new Metric("field2", List.of(MetricStat.SUM)), + new Metric("field4", List.of(MetricStat.SUM)), + new Metric("field6", List.of(MetricStat.VALUE_COUNT)), + new Metric("field9", List.of(MetricStat.MIN)), + new Metric("field10", List.of(MetricStat.MAX)) + ); + + compositeField = new StarTreeField( + "test", + dimensionsOrder, + metrics, + new StarTreeFieldConfiguration(1, Set.of("field8"), getBuildMode()) + ); + + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( + dimsIterators, + metricsIterators + ); + builder.build(segmentStarTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); + + List resultStarTreeDocuments = builder.getStarTreeDocuments(); + assertEquals(7, resultStarTreeDocuments.size()); + + Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator().iterator(); + assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); + builder.close(); + + // building another tree in the same file + fields = List.of("fieldC", "fieldB", "fieldL", "fieldI"); + + dimensionsOrder = List.of(new NumericDimension("fieldC"), new NumericDimension("fieldB"), new NumericDimension("fieldL")); + metrics = List.of(new Metric("fieldI", List.of(MetricStat.SUM))); + + DocValuesProducer docValuesProducer = mock(DocValuesProducer.class); + + compositeField = new StarTreeField("test", dimensionsOrder, metrics, new StarTreeFieldConfiguration(1, Set.of(), getBuildMode())); + SegmentInfo segmentInfo = new SegmentInfo( + directory, + Version.LATEST, + Version.LUCENE_9_11_0, + "test_segment", + 7, + false, + false, + new Lucene99Codec(), + new HashMap<>(), + UUID.randomUUID().toString().substring(0, 16).getBytes(StandardCharsets.UTF_8), + new HashMap<>(), + null + ); + + fieldsInfo = new FieldInfo[fields.size()]; + fieldProducerMap = new HashMap<>(); + for (int i = 0; i < fieldsInfo.length; i++) { + fieldsInfo[i] = new FieldInfo( + fields.get(i), + i, + false, + false, + true, + IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, + DocValuesType.SORTED_NUMERIC, + -1, + Collections.emptyMap(), + 0, + 0, + 0, + 0, + VectorEncoding.FLOAT32, + VectorSimilarityFunction.EUCLIDEAN, + false, + false + ); + fieldProducerMap.put(fields.get(i), docValuesProducer); + } + FieldInfos fieldInfos = new FieldInfos(fieldsInfo); + writeState = new SegmentWriteState(InfoStream.getDefault(), segmentInfo.dir, segmentInfo, fieldInfos, null, newIOContext(random())); + + mapperService = mock(MapperService.class); + DocumentMapper documentMapper = mock(DocumentMapper.class); + when(mapperService.documentMapper()).thenReturn(documentMapper); + Settings settings = Settings.builder().put(settings(org.opensearch.Version.CURRENT).build()).build(); + NumberFieldMapper numberFieldMapper1 = new NumberFieldMapper.Builder("fieldI", NumberFieldMapper.NumberType.DOUBLE, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + MappingLookup fieldMappers = new MappingLookup( + Set.of(numberFieldMapper1), + Collections.emptyList(), + Collections.emptyList(), + 0, + null + ); + when(documentMapper.mappers()).thenReturn(fieldMappers); + + InMemoryTreeNode rootNode1 = builder.getRootNode(); + + int noOfStarTreeDocuments2 = 7; + StarTreeDocument[] starTreeDocuments2 = new StarTreeDocument[noOfStarTreeDocuments2]; + starTreeDocuments2[0] = new StarTreeDocument(new Long[] { 1L, 11L, 21L }, new Double[] { 400.0 }); + starTreeDocuments2[1] = new StarTreeDocument(new Long[] { 1L, 12L, 22L }, new Double[] { 200.0 }); + starTreeDocuments2[2] = new StarTreeDocument(new Long[] { 2L, 13L, 23L }, new Double[] { 300.0 }); + starTreeDocuments2[3] = new StarTreeDocument(new Long[] { 2L, 13L, 21L }, new Double[] { 100.0 }); + starTreeDocuments2[4] = new StarTreeDocument(new Long[] { 3L, 11L, 21L }, new Double[] { 600.0 }); + starTreeDocuments2[5] = new StarTreeDocument(new Long[] { 3L, 12L, 23L }, new Double[] { 200.0 }); + starTreeDocuments2[6] = new StarTreeDocument(new Long[] { 3L, 12L, 21L }, new Double[] { 400.0 }); + + StarTreeDocument[] segmentStarTreeDocuments2 = new StarTreeDocument[noOfStarTreeDocuments2]; + for (int i = 0; i < noOfStarTreeDocuments2; i++) { + long metric1 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments2[i].metrics[0]); + segmentStarTreeDocuments2[i] = new StarTreeDocument(starTreeDocuments2[i].dimensions, new Long[] { metric1 }); + } + + SequentialDocValuesIterator[] dimsIterators2 = getDimensionIterators(segmentStarTreeDocuments2); + List metricsIterators2 = getMetricIterators(segmentStarTreeDocuments2); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); + Iterator segmentStarTreeDocumentIterator2 = builder.sortAndAggregateSegmentDocuments( + dimsIterators2, + metricsIterators2 + ); + builder.build(segmentStarTreeDocumentIterator2, new AtomicInteger(), mock(DocValuesConsumer.class)); + InMemoryTreeNode rootNode2 = builder.getRootNode(); + + metaOut.close(); + dataOut.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "test", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + List.of("field1", "field3", "field5", "field8"), + List.of( + new Metric("field2", List.of(MetricStat.SUM)), + new Metric("field4", List.of(MetricStat.SUM)), + new Metric("field6", List.of(MetricStat.VALUE_COUNT)), + new Metric("field9", List.of(MetricStat.MIN)), + new Metric("field10", List.of(MetricStat.MAX)) + ), + 2, + getExpectedStarTreeDocumentIterator().size(), + 1, + Set.of("field8"), + getBuildMode(), + 0, + 330 + ); + + StarTreeMetadata starTreeMetadata2 = new StarTreeMetadata( + "test", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + List.of("fieldC", "fieldB", "fieldL"), + List.of(new Metric("fieldI", List.of(MetricStat.SUM))), + 7, + 27, + 1, + Set.of(), + getBuildMode(), + 330, + 1287 + ); + + List totalDimensionFields = new ArrayList<>(); + totalDimensionFields.addAll(starTreeMetadata.getDimensionFields()); + totalDimensionFields.addAll(starTreeMetadata2.getDimensionFields()); + + List metrics = new ArrayList<>(); + metrics.addAll(starTreeMetadata.getMetrics()); + metrics.addAll(starTreeMetadata2.getMetrics()); + + SegmentReadState readState = getReadState(3, totalDimensionFields, metrics); + + IndexInput dataIn = readState.directory.openInput(dataFileName, IOContext.DEFAULT); + IndexInput metaIn = readState.directory.openInput(metaFileName, IOContext.DEFAULT); + + validateFileFormats(dataIn, metaIn, rootNode1, starTreeMetadata); + validateFileFormats(dataIn, metaIn, rootNode2, starTreeMetadata2); + + dataIn.close(); + metaIn.close(); + } public void test_build() throws IOException { @@ -1180,18 +1632,53 @@ public void test_build() throws IOException { SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); List metricsIterators = getMetricIterators(segmentStarTreeDocuments); - builder = getStarTreeBuilder(compositeField, writeState, mapperService); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( dimsIterators, metricsIterators ); - builder.build(segmentStarTreeDocumentIterator); + docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder.build(segmentStarTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); List resultStarTreeDocuments = builder.getStarTreeDocuments(); assertEquals(7, resultStarTreeDocuments.size()); - Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator(); + Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator().iterator(); assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "test", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + getStarTreeDimensionNames(compositeField.getDimensionsOrder()), + compositeField.getMetrics(), + 2, + getExpectedStarTreeDocumentIterator().size(), + 1, + Set.of("field8"), + getBuildMode(), + 0, + 330 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + getExpectedStarTreeDocumentIterator().size(), + starTreeMetadata, + getExpectedStarTreeDocumentIterator() + ); } private void assertStarTreeDocuments( @@ -1224,12 +1711,7 @@ public void test_build_starTreeDataset() throws IOException { DocValuesProducer docValuesProducer = mock(DocValuesProducer.class); - compositeField = new StarTreeField( - "test", - dimensionsOrder, - metrics, - new StarTreeFieldConfiguration(1, Set.of(), StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP) - ); + compositeField = new StarTreeField("test", dimensionsOrder, metrics, new StarTreeFieldConfiguration(1, Set.of(), getBuildMode())); SegmentInfo segmentInfo = new SegmentInfo( directory, Version.LATEST, @@ -1271,7 +1753,13 @@ public void test_build_starTreeDataset() throws IOException { } FieldInfos fieldInfos = new FieldInfos(fieldsInfo); writeState = new SegmentWriteState(InfoStream.getDefault(), segmentInfo.dir, segmentInfo, fieldInfos, null, newIOContext(random())); - + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); mapperService = mock(MapperService.class); DocumentMapper documentMapper = mock(DocumentMapper.class); when(mapperService.documentMapper()).thenReturn(documentMapper); @@ -1305,18 +1793,18 @@ public void test_build_starTreeDataset() throws IOException { SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); List metricsIterators = getMetricIterators(segmentStarTreeDocuments); - builder = getStarTreeBuilder(compositeField, writeState, mapperService); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( dimsIterators, metricsIterators ); - builder.build(segmentStarTreeDocumentIterator); + builder.build(segmentStarTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); List resultStarTreeDocuments = builder.getStarTreeDocuments(); - Iterator expectedStarTreeDocumentIterator = expectedStarTreeDocuments(); + Iterator expectedStarTreeDocumentIterator = expectedStarTreeDocuments().iterator(); Iterator resultStarTreeDocumentIterator = resultStarTreeDocuments.iterator(); Map> dimValueToDocIdMap = new HashMap<>(); - builder.rootNode.nodeType = StarTreeNodeType.STAR.getValue(); + builder.rootNode.setNodeType(StarTreeNodeType.STAR.getValue()); traverseStarTree(builder.rootNode, dimValueToDocIdMap, true); Map> expectedDimToValueMap = getExpectedDimToValueMap(); @@ -1342,7 +1830,88 @@ public void test_build_starTreeDataset() throws IOException { assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); } + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + validateStarTree(builder.getRootNode(), 3, 1, builder.getStarTreeDocuments()); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "test", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + getStarTreeDimensionNames(compositeField.getDimensionsOrder()), + compositeField.getMetrics(), + 7, + 27, + 1, + Set.of(), + getBuildMode(), + 0, + 1287 + ); + validateStarTreeFileFormats(builder.getRootNode(), 27, starTreeMetadata, expectedStarTreeDocuments()); + } + + private List getStarTreeDimensionNames(List dimensionsOrder) { + + List dimensionNames = new ArrayList<>(); + for (Dimension dimension : dimensionsOrder) { + dimensionNames.add(dimension.getField()); + } + return dimensionNames; + + } + + private void validateStarTreeFileFormats( + InMemoryTreeNode rootNode, + int numDocs, + StarTreeMetadata expectedStarTreeMetadata, + List expectedStarTreeDocuments + ) throws IOException { + + assertNotNull(rootNode.getChildren()); + assertFalse(rootNode.getChildren().isEmpty()); + SegmentReadState readState = getReadState( + numDocs, + expectedStarTreeMetadata.getDimensionFields(), + expectedStarTreeMetadata.getMetrics() + ); + + DocValuesProducer compositeDocValuesProducer = LuceneDocValuesProducerFactory.getDocValuesProducerForCompositeCodec( + Composite99Codec.COMPOSITE_INDEX_CODEC_NAME, + readState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + + IndexInput dataIn = readState.directory.openInput(dataFileName, IOContext.DEFAULT); + IndexInput metaIn = readState.directory.openInput(metaFileName, IOContext.DEFAULT); + + StarTreeValues starTreeValues = new StarTreeValues(expectedStarTreeMetadata, dataIn, compositeDocValuesProducer, readState); + List starTreeNumericTypes = new ArrayList<>(); + builder.metricAggregatorInfos.forEach( + metricAggregatorInfo -> starTreeNumericTypes.add(metricAggregatorInfo.getValueAggregators().getAggregatedValueType()) + ); + StarTreeDocument[] starTreeDocuments = StarTreeTestUtils.getSegmentsStarTreeDocuments( + List.of(starTreeValues), + starTreeNumericTypes, + readState.segmentInfo.maxDoc() + ); + + StarTreeDocument[] expectedStarTreeDocumentsArray = expectedStarTreeDocuments.toArray(new StarTreeDocument[0]); + StarTreeTestUtils.assertStarTreeDocuments(starTreeDocuments, expectedStarTreeDocumentsArray); + + validateFileFormats(dataIn, metaIn, rootNode, expectedStarTreeMetadata); + + dataIn.close(); + metaIn.close(); + compositeDocValuesProducer.close(); } private static Map> getExpectedDimToValueMap() { @@ -1367,8 +1936,8 @@ private static Map> getExpectedDimToValueMap() { return expectedDimToValueMap; } - private Iterator expectedStarTreeDocuments() { - List expectedStarTreeDocuments = List.of( + private List expectedStarTreeDocuments() { + return List.of( new StarTreeDocument(new Long[] { 1L, 11L, 21L }, new Object[] { 400.0, 1L }), new StarTreeDocument(new Long[] { 1L, 12L, 22L }, new Object[] { 200.0, 1L }), new StarTreeDocument(new Long[] { 2L, 13L, 21L }, new Object[] { 100.0, 1L }), @@ -1398,7 +1967,6 @@ private Iterator expectedStarTreeDocuments() { new StarTreeDocument(new Long[] { 3L, 12L, null }, new Object[] { 600.0, 2L }) ); - return expectedStarTreeDocuments.iterator(); } public void testFlushFlow() throws IOException { @@ -1417,13 +1985,14 @@ public void testFlushFlow() throws IOException { ); List metricsWithField = List.of(0, 1, 2, 3, 4, 5); - StarTreeField sf = getStarTreeFieldWithMultipleMetrics(); + compositeField = getStarTreeFieldWithMultipleMetrics(); SortedNumericDocValues d1sndv = getSortedNumericMock(dimList, docsWithField); SortedNumericDocValues d2sndv = getSortedNumericMock(dimList2, docsWithField2); SortedNumericDocValues m1sndv = getSortedNumericMock(metricsList, metricsWithField); SortedNumericDocValues m2sndv = getSortedNumericMock(metricsList, metricsWithField); - builder = getStarTreeBuilder(sf, getWriteState(6), mapperService); + writeState = getWriteState(6, writeState.segmentInfo.getId()); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); SequentialDocValuesIterator[] dimDvs = { new SequentialDocValuesIterator(d1sndv), new SequentialDocValuesIterator(d2sndv) }; Iterator starTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( dimDvs, @@ -1438,19 +2007,62 @@ public void testFlushFlow() throws IOException { [5, 5] | [50.0, 1] [null, 2] | [20.0, 1] */ - int count = 0; - while (starTreeDocumentIterator.hasNext()) { - count++; - StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); - assertEquals( - starTreeDocument.dimensions[0] != null ? starTreeDocument.dimensions[0] * 1 * 10.0 : 20.0, - starTreeDocument.metrics[0] - ); - assertEquals(1L, starTreeDocument.metrics[1]); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder.build(starTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); + List starTreeDocuments = builder.getStarTreeDocuments(); + int count = 0; + for (StarTreeDocument starTreeDocument : starTreeDocuments) { + count++; + if (starTreeDocument.dimensions[1] != null) { + assertEquals( + starTreeDocument.dimensions[0] == null + ? starTreeDocument.dimensions[1] * 1 * 10.0 + : starTreeDocument.dimensions[0] * 10, + starTreeDocument.metrics[0] + ); + assertEquals(1L, starTreeDocument.metrics[1]); + } else { + assertEquals(150D, starTreeDocument.metrics[0]); + assertEquals(6L, starTreeDocument.metrics[1]); + } } - assertEquals(6, count); - builder.build(starTreeDocumentIterator); - validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); + assertEquals(13, count); + validateStarTree(builder.getRootNode(), 2, 1000, builder.getStarTreeDocuments()); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "sf", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + List.of("field1", "field3"), + List.of(new Metric("field2", List.of(MetricStat.SUM, MetricStat.VALUE_COUNT))), + 6, + builder.numStarTreeDocs, + 1000, + Set.of(), + getBuildMode(), + 0, + 264 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + builder.getStarTreeDocuments().size(), + starTreeMetadata, + builder.getStarTreeDocuments() + ); + } public void testFlushFlowDimsReverse() throws IOException { @@ -1469,13 +2081,21 @@ public void testFlushFlowDimsReverse() throws IOException { ); List metricsWithField = List.of(0, 1, 2, 3, 4, 5); - StarTreeField sf = getStarTreeFieldWithMultipleMetrics(); + compositeField = getStarTreeFieldWithMultipleMetrics(); SortedNumericDocValues d1sndv = getSortedNumericMock(dimList, docsWithField); SortedNumericDocValues d2sndv = getSortedNumericMock(dimList2, docsWithField2); SortedNumericDocValues m1sndv = getSortedNumericMock(metricsList, metricsWithField); SortedNumericDocValues m2sndv = getSortedNumericMock(metricsList, metricsWithField); - builder = getStarTreeBuilder(sf, getWriteState(6), mapperService); + writeState = getWriteState(6, writeState.segmentInfo.getId()); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); SequentialDocValuesIterator[] dimDvs = { new SequentialDocValuesIterator(d1sndv), new SequentialDocValuesIterator(d2sndv) }; Iterator starTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( dimDvs, @@ -1490,21 +2110,49 @@ public void testFlushFlowDimsReverse() throws IOException { [5, 5] | [50.0, 1] [null, 0] | [0.0, 1] */ + builder.appendDocumentsToStarTree(starTreeDocumentIterator); + assertEquals(6, builder.getStarTreeDocuments().size()); + builder.build(starTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); int count = 0; - while (starTreeDocumentIterator.hasNext()) { - count++; - StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); - if (starTreeDocument.dimensions[0] != null) { - assertEquals(count, (long) starTreeDocument.dimensions[0]); - } else { - assertEquals(6, count); + for (StarTreeDocument starTreeDocument : builder.getStarTreeDocuments()) { + if (count <= 6) { + count++; + if (starTreeDocument.dimensions[0] != null) { + assertEquals(count, (long) starTreeDocument.dimensions[0]); + } + assertEquals(starTreeDocument.dimensions[1] * 10.0, starTreeDocument.metrics[0]); + assertEquals(1L, starTreeDocument.metrics[1]); } - assertEquals(starTreeDocument.dimensions[1] * 10.0, starTreeDocument.metrics[0]); - assertEquals(1L, starTreeDocument.metrics[1]); } - assertEquals(6, count); - builder.build(starTreeDocumentIterator); - validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); + validateStarTree(builder.getRootNode(), 2, 1000, builder.getStarTreeDocuments()); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "sf", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + List.of("field1", "field3"), + List.of(new Metric("field2", List.of(MetricStat.SUM, MetricStat.VALUE_COUNT))), + 6, + builder.numStarTreeDocs, + 1000, + Set.of(), + getBuildMode(), + 0, + 264 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + builder.getStarTreeDocuments().size(), + starTreeMetadata, + builder.getStarTreeDocuments() + ); } public void testFlushFlowBuild() throws IOException { @@ -1534,23 +2182,28 @@ public void testFlushFlowBuild() throws IOException { Metric m1 = new Metric("field2", List.of(MetricStat.SUM)); List dims = List.of(d1, d2); List metrics = List.of(m1); - StarTreeFieldConfiguration c = new StarTreeFieldConfiguration( - 1, - new HashSet<>(), - StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP - ); - StarTreeField sf = new StarTreeField("sf", dims, metrics, c); + StarTreeFieldConfiguration c = new StarTreeFieldConfiguration(1, new HashSet<>(), getBuildMode()); + compositeField = new StarTreeField("sf", dims, metrics, c); SortedNumericDocValues d1sndv = getSortedNumericMock(dimList, docsWithField); SortedNumericDocValues d2sndv = getSortedNumericMock(dimList2, docsWithField2); SortedNumericDocValues m1sndv = getSortedNumericMock(metricsList, metricsWithField); - builder = getStarTreeBuilder(sf, getWriteState(100), mapperService); + writeState = getWriteState(100, writeState.segmentInfo.getId()); + SegmentWriteState consumerWriteState = getWriteState(DocIdSetIterator.NO_MORE_DOCS, writeState.segmentInfo.getId()); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + consumerWriteState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); DocValuesProducer d1vp = getDocValuesProducer(d1sndv); DocValuesProducer d2vp = getDocValuesProducer(d2sndv); DocValuesProducer m1vp = getDocValuesProducer(m1sndv); Map fieldProducerMap = Map.of("field1", d1vp, "field3", d2vp, "field2", m1vp); - builder.build(fieldProducerMap); + builder.build(fieldProducerMap, new AtomicInteger(), docValuesConsumer); /** * Asserting following dim / metrics [ dim1, dim2 / Sum [ metric] ] [0, 0] | [0.0] @@ -1572,6 +2225,34 @@ public void testFlushFlowBuild() throws IOException { ); } validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "sf", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + List.of("field1", "field3"), + compositeField.getMetrics(), + 100, + builder.numStarTreeDocs, + 1, + Set.of(), + getBuildMode(), + 0, + 6699 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + builder.getStarTreeDocuments().size(), + starTreeMetadata, + builder.getStarTreeDocuments() + ); } private static DocValuesProducer getDocValuesProducer(SortedNumericDocValues sndv) { @@ -1583,20 +2264,15 @@ public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOExcepti }; } - private static StarTreeField getStarTreeFieldWithMultipleMetrics() { + private StarTreeField getStarTreeFieldWithMultipleMetrics() { Dimension d1 = new NumericDimension("field1"); Dimension d2 = new NumericDimension("field3"); Metric m1 = new Metric("field2", List.of(MetricStat.SUM)); Metric m2 = new Metric("field2", List.of(MetricStat.VALUE_COUNT)); List dims = List.of(d1, d2); List metrics = List.of(m1, m2); - StarTreeFieldConfiguration c = new StarTreeFieldConfiguration( - 1000, - new HashSet<>(), - StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP - ); - StarTreeField sf = new StarTreeField("sf", dims, metrics, c); - return sf; + StarTreeFieldConfiguration c = new StarTreeFieldConfiguration(1000, new HashSet<>(), getBuildMode()); + return new StarTreeField("sf", dims, metrics, c); } public void testMergeFlow_randomNumberTypes() throws Exception { @@ -1631,8 +2307,6 @@ public void testMergeFlow_randomNumberTypes() throws Exception { ); when(documentMapper.mappers()).thenReturn(fieldMappers); testMergeFlowWithSum(); - builder.close(); - testMergeFlowWithCount(); } public void testMergeFlowWithSum() throws IOException { @@ -1653,12 +2327,12 @@ public void testMergeFlowWithSum() throws IOException { ); List metricsWithField = List.of(0, 1, 2, 3, 4, 5, 6); - StarTreeField sf = getStarTreeField(MetricStat.SUM); + compositeField = getStarTreeField(MetricStat.SUM); StarTreeValues starTreeValues = getStarTreeValues( getSortedNumericMock(dimList, docsWithField), getSortedNumericMock(dimList2, docsWithField2), getSortedNumericMock(metricsList, metricsWithField), - sf, + compositeField, "6" ); @@ -1666,10 +2340,18 @@ public void testMergeFlowWithSum() throws IOException { getSortedNumericMock(dimList, docsWithField), getSortedNumericMock(dimList2, docsWithField2), getSortedNumericMock(metricsList, metricsWithField), - sf, + compositeField, "6" ); - builder = getStarTreeBuilder(sf, getWriteState(6), mapperService); + writeState = getWriteState(6, writeState.segmentInfo.getId()); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** * Asserting following dim / metrics [ dim1, dim2 / Sum [ metric] ] @@ -1682,18 +2364,49 @@ public void testMergeFlowWithSum() throws IOException { * ------------------ We only take non star docs * [6,-1] | [120.0] */ + builder.appendDocumentsToStarTree(starTreeDocumentIterator); + assertEquals(6, builder.getStarTreeDocuments().size()); + builder.build(starTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); int count = 0; - while (starTreeDocumentIterator.hasNext()) { + for (StarTreeDocument starTreeDocument : builder.getStarTreeDocuments()) { count++; - StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); - assertEquals( - starTreeDocument.dimensions[0] != null ? starTreeDocument.dimensions[0] * 2 * 10.0 : 40.0, - starTreeDocument.metrics[0] - ); + if (count <= 6) { + assertEquals( + starTreeDocument.dimensions[0] != null ? starTreeDocument.dimensions[0] * 2 * 10.0 : 40.0, + starTreeDocument.metrics[0] + ); + } } - assertEquals(6, count); - builder.build(starTreeDocumentIterator); - validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); + + validateStarTree(builder.getRootNode(), 2, 1000, builder.getStarTreeDocuments()); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "sf", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + List.of("field1", "field3"), + compositeField.getMetrics(), + 6, + builder.numStarTreeDocs, + 1000, + Set.of(), + getBuildMode(), + 0, + 264 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + builder.getStarTreeDocuments().size(), + starTreeMetadata, + builder.getStarTreeDocuments() + ); } public void testMergeFlowWithCount() throws IOException { @@ -1705,12 +2418,12 @@ public void testMergeFlowWithCount() throws IOException { List metricsList = List.of(0L, 1L, 2L, 3L, 4L, 5L, 6L); List metricsWithField = List.of(0, 1, 2, 3, 4, 5, 6); - StarTreeField sf = getStarTreeField(MetricStat.VALUE_COUNT); + compositeField = getStarTreeField(MetricStat.VALUE_COUNT); StarTreeValues starTreeValues = getStarTreeValues( getSortedNumericMock(dimList, docsWithField), getSortedNumericMock(dimList2, docsWithField2), getSortedNumericMock(metricsList, metricsWithField), - sf, + compositeField, "6" ); @@ -1718,10 +2431,18 @@ public void testMergeFlowWithCount() throws IOException { getSortedNumericMock(dimList, docsWithField), getSortedNumericMock(dimList2, docsWithField2), getSortedNumericMock(metricsList, metricsWithField), - sf, + compositeField, "6" ); - builder = getStarTreeBuilder(sf, getWriteState(6), mapperService); + writeState = getWriteState(6, writeState.segmentInfo.getId()); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] @@ -1734,15 +2455,46 @@ public void testMergeFlowWithCount() throws IOException { --------------- [6,-1] | [12] */ + builder.appendDocumentsToStarTree(starTreeDocumentIterator); + assertEquals(6, builder.getStarTreeDocuments().size()); + builder.build(starTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); int count = 0; - while (starTreeDocumentIterator.hasNext()) { + for (StarTreeDocument starTreeDocument : builder.getStarTreeDocuments()) { count++; - StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); - assertEquals(starTreeDocument.dimensions[0] != null ? starTreeDocument.dimensions[0] * 2 : 4, starTreeDocument.metrics[0]); + if (count <= 6) { + assertEquals(starTreeDocument.dimensions[0] != null ? starTreeDocument.dimensions[0] * 2 : 4, starTreeDocument.metrics[0]); + } } - assertEquals(6, count); - builder.build(starTreeDocumentIterator); - validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); + + validateStarTree(builder.getRootNode(), 2, 1000, builder.getStarTreeDocuments()); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "sf", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + List.of("field1", "field3"), + compositeField.getMetrics(), + 6, + builder.numStarTreeDocs, + 1000, + Set.of(), + getBuildMode(), + 0, + 264 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + builder.getStarTreeDocuments().size(), + starTreeMetadata, + builder.getStarTreeDocuments() + ); } @@ -1756,14 +2508,26 @@ private StarTreeValues getStarTreeValues( SortedNumericDocValues d1sndv = dimList; SortedNumericDocValues d2sndv = dimList2; SortedNumericDocValues m1sndv = metricsList; - Map dimDocIdSetIterators = Map.of("field1", d1sndv, "field3", d2sndv); - Map metricDocIdSetIterators = Map.of("field2", m1sndv); + Map> dimDocIdSetIterators = Map.of("field1", () -> d1sndv, "field3", () -> d2sndv); + + Map> metricDocIdSetIterators = new LinkedHashMap<>(); + for (Metric metric : sf.getMetrics()) { + for (MetricStat metricStat : metric.getMetrics()) { + String metricFullName = fullyQualifiedFieldNameForStarTreeMetricsDocValues( + sf.getName(), + metric.getField(), + metricStat.getTypeName() + ); + metricDocIdSetIterators.put(metricFullName, () -> m1sndv); + } + } + StarTreeValues starTreeValues = new StarTreeValues( sf, null, dimDocIdSetIterators, metricDocIdSetIterators, - Map.of("numSegmentDocs", number) + Map.of(CompositeIndexConstants.SEGMENT_DOCS_COUNT, number) ); return starTreeValues; } @@ -1785,12 +2549,12 @@ public void testMergeFlowWithDifferentDocsFromSegments() throws IOException { List metricsList2 = List.of(5L, 6L, 7L, 8L, 9L); List metricsWithField2 = List.of(0, 1, 2, 3, 4); - StarTreeField sf = getStarTreeField(MetricStat.VALUE_COUNT); + compositeField = getStarTreeField(MetricStat.VALUE_COUNT); StarTreeValues starTreeValues = getStarTreeValues( getSortedNumericMock(dimList, docsWithField), getSortedNumericMock(dimList2, docsWithField2), getSortedNumericMock(metricsList, metricsWithField), - sf, + compositeField, "6" ); @@ -1798,10 +2562,18 @@ public void testMergeFlowWithDifferentDocsFromSegments() throws IOException { getSortedNumericMock(dimList3, docsWithField3), getSortedNumericMock(dimList4, docsWithField4), getSortedNumericMock(metricsList2, metricsWithField2), - sf, + compositeField, "4" ); - builder = getStarTreeBuilder(sf, getWriteState(4), mapperService); + writeState = getWriteState(4, writeState.segmentInfo.getId()); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] @@ -1815,19 +2587,49 @@ public void testMergeFlowWithDifferentDocsFromSegments() throws IOException { [null, 2] | [2] [null, 7] | [7] */ + builder.appendDocumentsToStarTree(starTreeDocumentIterator); + assertEquals(9, builder.getStarTreeDocuments().size()); + builder.build(starTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); int count = 0; - while (starTreeDocumentIterator.hasNext()) { + for (StarTreeDocument starTreeDocument : builder.getStarTreeDocuments()) { count++; - StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); - if (Objects.equals(starTreeDocument.dimensions[0], 5L)) { - assertEquals(starTreeDocument.dimensions[0] * 2, starTreeDocument.metrics[0]); - } else { - assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); + if (count <= 9) { + if (Objects.equals(starTreeDocument.dimensions[0], 5L)) { + assertEquals(starTreeDocument.dimensions[0] * 2, starTreeDocument.metrics[0]); + } else { + assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); + } } } - assertEquals(9, count); - builder.build(starTreeDocumentIterator); - validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); + validateStarTree(builder.getRootNode(), 2, 1000, builder.getStarTreeDocuments()); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "sf", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + List.of("field1", "field3"), + compositeField.getMetrics(), + 9, + builder.numStarTreeDocs, + 1000, + Set.of(), + getBuildMode(), + 0, + 330 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + builder.getStarTreeDocuments().size(), + starTreeMetadata, + builder.getStarTreeDocuments() + ); } public void testMergeFlowNumSegmentsDocs() throws IOException { @@ -1863,7 +2665,7 @@ public void testMergeFlowNumSegmentsDocs() throws IOException { sf, "4" ); - builder = getStarTreeBuilder(sf, getWriteState(4), mapperService); + builder = getStarTreeBuilder(metaOut, dataOut, sf, getWriteState(4, writeState.segmentInfo.getId()), mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] @@ -1907,12 +2709,12 @@ public void testMergeFlowWithMissingDocs() throws IOException { List metricsList2 = List.of(5L, 6L, 7L, 8L, 9L); List metricsWithField2 = List.of(0, 1, 2, 3, 4); - StarTreeField sf = getStarTreeField(MetricStat.VALUE_COUNT); + compositeField = getStarTreeField(MetricStat.VALUE_COUNT); StarTreeValues starTreeValues = getStarTreeValues( getSortedNumericMock(dimList, docsWithField), getSortedNumericMock(dimList2, docsWithField2), getSortedNumericMock(metricsList, metricsWithField), - sf, + compositeField, "6" ); @@ -1920,10 +2722,18 @@ public void testMergeFlowWithMissingDocs() throws IOException { getSortedNumericMock(dimList3, docsWithField3), getSortedNumericMock(dimList4, docsWithField4), getSortedNumericMock(metricsList2, metricsWithField2), - sf, + compositeField, "4" ); - builder = getStarTreeBuilder(sf, getWriteState(4), mapperService); + writeState = getWriteState(4, writeState.segmentInfo.getId()); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] @@ -1938,18 +2748,49 @@ public void testMergeFlowWithMissingDocs() throws IOException { [null, 5] | [5] [null, 7] | [7] */ + builder.appendDocumentsToStarTree(starTreeDocumentIterator); + assertEquals(10, builder.getStarTreeDocuments().size()); + builder.build(starTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); int count = 0; - while (starTreeDocumentIterator.hasNext()) { + for (StarTreeDocument starTreeDocument : builder.getStarTreeDocuments()) { count++; - StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); - if (starTreeDocument.dimensions[0] == null) { - assertTrue(List.of(5L, 7L).contains(starTreeDocument.dimensions[1])); + if (count <= 10) { + if (starTreeDocument.dimensions[0] == null) { + assertTrue(List.of(5L, 7L).contains(starTreeDocument.dimensions[1])); + } + assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); } - assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); } - assertEquals(10, count); - builder.build(starTreeDocumentIterator); - validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); + + validateStarTree(builder.getRootNode(), 2, 1000, builder.getStarTreeDocuments()); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "sf", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + List.of("field1", "field3"), + compositeField.getMetrics(), + 10, + builder.numStarTreeDocs, + 1000, + Set.of(), + getBuildMode(), + 0, + 363 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + builder.getStarTreeDocuments().size(), + starTreeMetadata, + builder.getStarTreeDocuments() + ); } public void testMergeFlowWithMissingDocsWithZero() throws IOException { @@ -1969,12 +2810,12 @@ public void testMergeFlowWithMissingDocsWithZero() throws IOException { List metricsList2 = List.of(5L, 6L, 7L, 8L, 9L); List metricsWithField2 = List.of(0, 1, 2, 3, 4); - StarTreeField sf = getStarTreeField(MetricStat.VALUE_COUNT); + compositeField = getStarTreeField(MetricStat.VALUE_COUNT); StarTreeValues starTreeValues = getStarTreeValues( getSortedNumericMock(dimList, docsWithField), getSortedNumericMock(dimList2, docsWithField2), getSortedNumericMock(metricsList, metricsWithField), - sf, + compositeField, "7" ); @@ -1982,10 +2823,19 @@ public void testMergeFlowWithMissingDocsWithZero() throws IOException { getSortedNumericMock(dimList3, docsWithField3), getSortedNumericMock(dimList4, docsWithField4), getSortedNumericMock(metricsList2, metricsWithField2), - sf, + compositeField, "4" ); - builder = getStarTreeBuilder(sf, getWriteState(4), mapperService); + writeState = getWriteState(4, writeState.segmentInfo.getId()); + SegmentWriteState consumerWriteState = getWriteState(DocIdSetIterator.NO_MORE_DOCS, writeState.segmentInfo.getId()); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + consumerWriteState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] @@ -1996,23 +2846,54 @@ public void testMergeFlowWithMissingDocsWithZero() throws IOException { [null, 7] | [7] [null, null] | [12] */ + builder.appendDocumentsToStarTree(starTreeDocumentIterator); + assertEquals(6, builder.getStarTreeDocuments().size()); + builder.build(starTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); int count = 0; - while (starTreeDocumentIterator.hasNext()) { + for (StarTreeDocument starTreeDocument : builder.getStarTreeDocuments()) { count++; - StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); - if (starTreeDocument.dimensions[0] == null && starTreeDocument.dimensions[1] == null) { - assertEquals(12L, (long) starTreeDocument.metrics[0]); - } else if (starTreeDocument.dimensions[0] == null) { - assertEquals(7L, starTreeDocument.metrics[0]); - } else if (starTreeDocument.dimensions[0] == 0) { - assertEquals(9L, starTreeDocument.metrics[0]); - } else { - assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); + if (count <= 6) { + if (starTreeDocument.dimensions[0] == null && starTreeDocument.dimensions[1] == null) { + assertEquals(12L, (long) starTreeDocument.metrics[0]); + } else if (starTreeDocument.dimensions[0] == null) { + assertEquals(7L, starTreeDocument.metrics[0]); + } else if (starTreeDocument.dimensions[0] == 0) { + assertEquals(9L, starTreeDocument.metrics[0]); + } else { + assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); + } } } - assertEquals(6, count); - builder.build(starTreeDocumentIterator); - validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); + + validateStarTree(builder.getRootNode(), 2, 1000, builder.getStarTreeDocuments()); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "sf", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + List.of("field1", "field3"), + compositeField.getMetrics(), + 6, + builder.numStarTreeDocs, + 1000, + Set.of(), + getBuildMode(), + 0, + 231 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + builder.getStarTreeDocuments().size(), + starTreeMetadata, + builder.getStarTreeDocuments() + ); } public void testMergeFlowWithMissingDocsWithZeroComplexCase() throws IOException { @@ -2032,12 +2913,12 @@ public void testMergeFlowWithMissingDocsWithZeroComplexCase() throws IOException List metricsList2 = List.of(5L, 6L, 7L, 8L, 9L); List metricsWithField2 = List.of(0, 1, 2, 3, 4); - StarTreeField sf = getStarTreeField(MetricStat.VALUE_COUNT); + compositeField = getStarTreeField(MetricStat.VALUE_COUNT); StarTreeValues starTreeValues = getStarTreeValues( getSortedNumericMock(dimList, docsWithField), getSortedNumericMock(dimList2, docsWithField2), getSortedNumericMock(metricsList, metricsWithField), - sf, + compositeField, "9" ); @@ -2045,10 +2926,18 @@ public void testMergeFlowWithMissingDocsWithZeroComplexCase() throws IOException getSortedNumericMock(dimList3, docsWithField3), getSortedNumericMock(dimList4, docsWithField4), getSortedNumericMock(metricsList2, metricsWithField2), - sf, + compositeField, "4" ); - builder = getStarTreeBuilder(sf, getWriteState(4), mapperService); + writeState = getWriteState(4, writeState.segmentInfo.getId()); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] @@ -2060,26 +2949,57 @@ public void testMergeFlowWithMissingDocsWithZeroComplexCase() throws IOException [null, 7] | [7] [null, null] | [19] */ + builder.appendDocumentsToStarTree(starTreeDocumentIterator); + assertEquals(7, builder.getStarTreeDocuments().size()); + builder.build(starTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); int count = 0; - while (starTreeDocumentIterator.hasNext()) { + for (StarTreeDocument starTreeDocument : builder.getStarTreeDocuments()) { count++; - StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); - if (starTreeDocument.dimensions[0] == null && starTreeDocument.dimensions[1] == null) { - assertEquals(19L, (long) starTreeDocument.metrics[0]); - assertEquals(7, count); - } else if (starTreeDocument.dimensions[0] == null) { - assertEquals(7L, starTreeDocument.metrics[0]); - } else if (starTreeDocument.dimensions[1] == null) { - assertEquals(8L, starTreeDocument.metrics[0]); - } else if (starTreeDocument.dimensions[0] == 0) { - assertEquals(9L, starTreeDocument.metrics[0]); - } else { - assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); + if (count <= 7) { + if (starTreeDocument.dimensions[0] == null && starTreeDocument.dimensions[1] == null) { + assertEquals(19L, (long) starTreeDocument.metrics[0]); + assertEquals(7, count); + } else if (starTreeDocument.dimensions[0] == null) { + assertEquals(7L, starTreeDocument.metrics[0]); + } else if (starTreeDocument.dimensions[1] == null) { + assertEquals(8L, starTreeDocument.metrics[0]); + } else if (starTreeDocument.dimensions[0] == 0) { + assertEquals(9L, starTreeDocument.metrics[0]); + } else { + assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); + } } } - assertEquals(7, count); - builder.build(starTreeDocumentIterator); - validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); + + validateStarTree(builder.getRootNode(), 2, 1000, builder.getStarTreeDocuments()); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "sf", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + List.of("field1", "field3"), + compositeField.getMetrics(), + 7, + builder.numStarTreeDocs, + 1000, + Set.of(), + getBuildMode(), + 0, + 231 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + builder.getStarTreeDocuments().size(), + starTreeMetadata, + builder.getStarTreeDocuments() + ); } public void testMergeFlowWithMissingDocsInSecondDim() throws IOException { @@ -2099,12 +3019,12 @@ public void testMergeFlowWithMissingDocsInSecondDim() throws IOException { List metricsList2 = List.of(5L, 6L, 7L, 8L, 9L); List metricsWithField2 = List.of(0, 1, 2, 3, 4); - StarTreeField sf = getStarTreeField(MetricStat.VALUE_COUNT); + compositeField = getStarTreeField(MetricStat.VALUE_COUNT); StarTreeValues starTreeValues = getStarTreeValues( getSortedNumericMock(dimList, docsWithField), getSortedNumericMock(dimList2, docsWithField2), getSortedNumericMock(metricsList, metricsWithField), - sf, + compositeField, "6" ); @@ -2112,10 +3032,18 @@ public void testMergeFlowWithMissingDocsInSecondDim() throws IOException { getSortedNumericMock(dimList3, docsWithField3), getSortedNumericMock(dimList4, docsWithField4), getSortedNumericMock(metricsList2, metricsWithField2), - sf, + compositeField, "4" ); - builder = getStarTreeBuilder(sf, getWriteState(4), mapperService); + writeState = getWriteState(4, writeState.segmentInfo.getId()); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] @@ -2130,19 +3058,50 @@ public void testMergeFlowWithMissingDocsInSecondDim() throws IOException { [8, 8] | [8] [null, 7] | [7] */ + builder.appendDocumentsToStarTree(starTreeDocumentIterator); + assertEquals(10, builder.getStarTreeDocuments().size()); + builder.build(starTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); int count = 0; - while (starTreeDocumentIterator.hasNext()) { + for (StarTreeDocument starTreeDocument : builder.getStarTreeDocuments()) { count++; - StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); - if (starTreeDocument.dimensions[0] != null && starTreeDocument.dimensions[0] == 5) { - assertEquals(starTreeDocument.dimensions[0], starTreeDocument.metrics[0]); - } else { - assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); + if (count <= 10) { + if (starTreeDocument.dimensions[0] != null && starTreeDocument.dimensions[0] == 5) { + assertEquals(starTreeDocument.dimensions[0], starTreeDocument.metrics[0]); + } else { + assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); + } } } - assertEquals(10, count); - builder.build(starTreeDocumentIterator); - validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); + + validateStarTree(builder.getRootNode(), 2, 1000, builder.getStarTreeDocuments()); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "sf", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + List.of("field1", "field3"), + compositeField.getMetrics(), + 10, + builder.numStarTreeDocs, + 1000, + Set.of(), + getBuildMode(), + 0, + 363 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + builder.getStarTreeDocuments().size(), + starTreeMetadata, + builder.getStarTreeDocuments() + ); } public void testMergeFlowWithDocsMissingAtTheEnd() throws IOException { @@ -2162,12 +3121,12 @@ public void testMergeFlowWithDocsMissingAtTheEnd() throws IOException { List metricsList2 = List.of(5L, 6L, 7L, 8L, 9L); List metricsWithField2 = List.of(0, 1, 2, 3, 4); - StarTreeField sf = getStarTreeField(MetricStat.VALUE_COUNT); + compositeField = getStarTreeField(MetricStat.VALUE_COUNT); StarTreeValues starTreeValues = getStarTreeValues( getSortedNumericMock(dimList, docsWithField), getSortedNumericMock(dimList2, docsWithField2), getSortedNumericMock(metricsList, metricsWithField), - sf, + compositeField, "6" ); @@ -2175,10 +3134,17 @@ public void testMergeFlowWithDocsMissingAtTheEnd() throws IOException { getSortedNumericMock(dimList3, docsWithField3), getSortedNumericMock(dimList4, docsWithField4), getSortedNumericMock(metricsList2, metricsWithField2), - sf, + compositeField, "4" ); - builder = getStarTreeBuilder(sf, writeState, mapperService); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] @@ -2193,18 +3159,49 @@ public void testMergeFlowWithDocsMissingAtTheEnd() throws IOException { [null, 5] | [5] [null, 7] | [7] */ + builder.appendDocumentsToStarTree(starTreeDocumentIterator); + assertEquals(10, builder.getStarTreeDocuments().size()); + builder.build(starTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); int count = 0; - while (starTreeDocumentIterator.hasNext()) { + for (StarTreeDocument starTreeDocument : builder.getStarTreeDocuments()) { count++; - StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); - if (starTreeDocument.dimensions[0] == null) { - assertTrue(List.of(5L, 7L).contains(starTreeDocument.dimensions[1])); + if (count <= 10) { + if (starTreeDocument.dimensions[0] == null) { + assertTrue(List.of(5L, 7L).contains(starTreeDocument.dimensions[1])); + } + assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); } - assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); } - assertEquals(10, count); - builder.build(starTreeDocumentIterator); - validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); + + validateStarTree(builder.getRootNode(), 2, 1000, builder.getStarTreeDocuments()); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "sf", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + List.of("field1", "field3"), + compositeField.getMetrics(), + 10, + builder.numStarTreeDocs, + 1000, + Set.of(), + getBuildMode(), + 0, + 363 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + builder.getStarTreeDocuments().size(), + starTreeMetadata, + builder.getStarTreeDocuments() + ); } public void testMergeFlowWithEmptyFieldsInOneSegment() throws IOException { @@ -2216,12 +3213,12 @@ public void testMergeFlowWithEmptyFieldsInOneSegment() throws IOException { List metricsList = List.of(0L, 1L, 2L, 3L, 4L, 5L, 6L); List metricsWithField = List.of(0, 1, 2, 3, 4, 5, 6); - StarTreeField sf = getStarTreeField(MetricStat.VALUE_COUNT); + compositeField = getStarTreeField(MetricStat.VALUE_COUNT); StarTreeValues starTreeValues = getStarTreeValues( getSortedNumericMock(dimList, docsWithField), getSortedNumericMock(dimList2, docsWithField2), getSortedNumericMock(metricsList, metricsWithField), - sf, + compositeField, "6" ); @@ -2229,10 +3226,18 @@ public void testMergeFlowWithEmptyFieldsInOneSegment() throws IOException { DocValues.emptySortedNumeric(), DocValues.emptySortedNumeric(), DocValues.emptySortedNumeric(), - sf, + compositeField, "0" ); - builder = getStarTreeBuilder(sf, getWriteState(0), mapperService); + writeState = getWriteState(0, writeState.segmentInfo.getId()); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] @@ -2243,18 +3248,48 @@ public void testMergeFlowWithEmptyFieldsInOneSegment() throws IOException { [4, 4] | [4] [null, 5] | [5] */ + builder.appendDocumentsToStarTree(starTreeDocumentIterator); + assertEquals(6, builder.getStarTreeDocuments().size()); + builder.build(starTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); int count = 0; - while (starTreeDocumentIterator.hasNext()) { + for (StarTreeDocument starTreeDocument : builder.getStarTreeDocuments()) { count++; - StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); - if (starTreeDocument.dimensions[0] == null) { - assertEquals(5L, (long) starTreeDocument.dimensions[1]); + if (count <= 6) { + if (starTreeDocument.dimensions[0] == null) { + assertEquals(5L, (long) starTreeDocument.dimensions[1]); + } + assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); } - assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); } - assertEquals(6, count); - builder.build(starTreeDocumentIterator); - validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); + validateStarTree(builder.getRootNode(), 2, 1000, builder.getStarTreeDocuments()); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "sf", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + List.of("field1", "field3"), + compositeField.getMetrics(), + 6, + builder.numStarTreeDocs, + 1000, + Set.of(), + getBuildMode(), + 0, + 264 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + builder.getStarTreeDocuments().size(), + starTreeMetadata, + builder.getStarTreeDocuments() + ); } public void testMergeFlowWithDuplicateDimensionValues() throws IOException { @@ -2307,7 +3342,7 @@ public void testMergeFlowWithDuplicateDimensionValues() throws IOException { docCountMetricsWithField.add(i); } - StarTreeField sf = getStarTreeFieldWithDocCount(1, true); + compositeField = getStarTreeFieldWithDocCount(1, true); StarTreeValues starTreeValues = getStarTreeValues( dimList1, docsWithField1, @@ -2321,7 +3356,7 @@ public void testMergeFlowWithDuplicateDimensionValues() throws IOException { metricsWithField, docCountMetricsList, docCountMetricsWithField, - sf + compositeField ); StarTreeValues starTreeValues2 = getStarTreeValues( @@ -2337,10 +3372,17 @@ public void testMergeFlowWithDuplicateDimensionValues() throws IOException { metricsWithField, docCountMetricsList, docCountMetricsWithField, - sf + compositeField + ); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION ); - builder = getStarTreeBuilder(sf, writeState, mapperService); - builder.build(builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2))); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); + builder.build(builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)), new AtomicInteger(), docValuesConsumer); List starTreeDocuments = builder.getStarTreeDocuments(); assertEquals(401, starTreeDocuments.size()); int count = 0; @@ -2371,7 +3413,34 @@ public void testMergeFlowWithDuplicateDimensionValues() throws IOException { count++; } assertEquals(401, count); - validateStarTree(builder.getRootNode(), 4, sf.getStarTreeConfig().maxLeafDocs(), builder.getStarTreeDocuments()); + validateStarTree(builder.getRootNode(), 4, compositeField.getStarTreeConfig().maxLeafDocs(), builder.getStarTreeDocuments()); + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "sf", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + getStarTreeDimensionNames(compositeField.getDimensionsOrder()), + compositeField.getMetrics(), + 100, + builder.numStarTreeDocs, + 1, + Set.of(), + getBuildMode(), + 0, + 13365 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + builder.getStarTreeDocuments().size(), + starTreeMetadata, + builder.getStarTreeDocuments() + ); } public void testMergeFlowWithMaxLeafDocs() throws IOException { @@ -2430,7 +3499,7 @@ public void testMergeFlowWithMaxLeafDocs() throws IOException { metricsWithField1.add(i); } - StarTreeField sf = getStarTreeFieldWithDocCount(3, true); + compositeField = getStarTreeFieldWithDocCount(3, true); StarTreeValues starTreeValues = getStarTreeValues( dimList1, docsWithField1, @@ -2444,7 +3513,7 @@ public void testMergeFlowWithMaxLeafDocs() throws IOException { metricsWithField, metricsList1, metricsWithField1, - sf + compositeField ); StarTreeValues starTreeValues2 = getStarTreeValues( @@ -2460,11 +3529,18 @@ public void testMergeFlowWithMaxLeafDocs() throws IOException { metricsWithField, metricsList1, metricsWithField1, - sf + compositeField ); - builder = getStarTreeBuilder(sf, writeState, mapperService); - builder.build(builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2))); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); + builder.build(builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)), new AtomicInteger(), docValuesConsumer); List starTreeDocuments = builder.getStarTreeDocuments(); /** 635 docs get generated @@ -2520,7 +3596,34 @@ public void testMergeFlowWithMaxLeafDocs() throws IOException { assertEquals(40L, starTreeDocument.metrics[1]); } } - validateStarTree(builder.getRootNode(), 4, sf.getStarTreeConfig().maxLeafDocs(), builder.getStarTreeDocuments()); + validateStarTree(builder.getRootNode(), 4, compositeField.getStarTreeConfig().maxLeafDocs(), builder.getStarTreeDocuments()); + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "sf", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + getStarTreeDimensionNames(compositeField.getDimensionsOrder()), + compositeField.getMetrics(), + 100, + builder.numStarTreeDocs, + 3, + Set.of(), + getBuildMode(), + 0, + 23199 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + builder.getStarTreeDocuments().size(), + starTreeMetadata, + builder.getStarTreeDocuments() + ); } private StarTreeValues getStarTreeValues( @@ -2544,10 +3647,37 @@ private StarTreeValues getStarTreeValues( SortedNumericDocValues d4sndv = getSortedNumericMock(dimList4, docsWithField4); SortedNumericDocValues m1sndv = getSortedNumericMock(metricsList, metricsWithField); SortedNumericDocValues m2sndv = getSortedNumericMock(metricsList1, metricsWithField1); - Map dimDocIdSetIterators = Map.of("field1", d1sndv, "field3", d2sndv, "field5", d3sndv, "field8", d4sndv); - Map metricDocIdSetIterators = new LinkedHashMap<>(); - metricDocIdSetIterators.put("field2", m1sndv); - metricDocIdSetIterators.put("_doc_count", m2sndv); + Map> dimDocIdSetIterators = Map.of( + "field1", + () -> d1sndv, + "field3", + () -> d2sndv, + "field5", + () -> d3sndv, + "field8", + () -> d4sndv + ); + + Map> metricDocIdSetIterators = new LinkedHashMap<>(); + + metricDocIdSetIterators.put( + fullyQualifiedFieldNameForStarTreeMetricsDocValues( + sf.getName(), + "field2", + sf.getMetrics().get(0).getMetrics().get(0).getTypeName() + ), + () -> m1sndv + ); + metricDocIdSetIterators.put( + fullyQualifiedFieldNameForStarTreeMetricsDocValues( + sf.getName(), + "_doc_count", + sf.getMetrics().get(1).getMetrics().get(0).getTypeName() + ), + () -> m2sndv + ); + // metricDocIdSetIterators.put("field2", () -> m1sndv); + // metricDocIdSetIterators.put("_doc_count", () -> m2sndv); StarTreeValues starTreeValues = new StarTreeValues(sf, null, dimDocIdSetIterators, metricDocIdSetIterators, getAttributes(500)); return starTreeValues; } @@ -2606,7 +3736,7 @@ public void testMergeFlowWithDuplicateDimensionValueWithMaxLeafDocs() throws IOE metricsWithField.add(i); } - StarTreeField sf = getStarTreeFieldWithDocCount(3, true); + compositeField = getStarTreeFieldWithDocCount(3, true); StarTreeValues starTreeValues = getStarTreeValues( dimList1, docsWithField1, @@ -2620,7 +3750,7 @@ public void testMergeFlowWithDuplicateDimensionValueWithMaxLeafDocs() throws IOE metricsWithField, docCountMetricsList, docCountMetricsWithField, - sf + compositeField ); StarTreeValues starTreeValues2 = getStarTreeValues( @@ -2636,13 +3766,48 @@ public void testMergeFlowWithDuplicateDimensionValueWithMaxLeafDocs() throws IOE metricsWithField, docCountMetricsList, docCountMetricsWithField, - sf + compositeField + ); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION ); - builder = getStarTreeBuilder(sf, writeState, mapperService); - builder.build(builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2))); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); + builder.build(builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)), new AtomicInteger(), docValuesConsumer); List starTreeDocuments = builder.getStarTreeDocuments(); assertEquals(401, starTreeDocuments.size()); - validateStarTree(builder.getRootNode(), 4, sf.getStarTreeConfig().maxLeafDocs(), builder.getStarTreeDocuments()); + validateStarTree(builder.getRootNode(), 4, compositeField.getStarTreeConfig().maxLeafDocs(), builder.getStarTreeDocuments()); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "sf", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + getStarTreeDimensionNames(compositeField.getDimensionsOrder()), + compositeField.getMetrics(), + 100, + builder.numStarTreeDocs, + compositeField.getStarTreeConfig().maxLeafDocs(), + Set.of(), + getBuildMode(), + 0, + 15345 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + builder.getStarTreeDocuments().size(), + starTreeMetadata, + builder.getStarTreeDocuments() + ); } public static long getLongFromDouble(double value) { @@ -2714,7 +3879,7 @@ public void testMergeFlowWithMaxLeafDocsAndStarTreeNodesAssertion() throws IOExc metricsList.add(1L); metricsWithField.add(i); } - StarTreeField sf = getStarTreeFieldWithDocCount(10, true); + compositeField = getStarTreeFieldWithDocCount(10, true); StarTreeValues starTreeValues = getStarTreeValues( dimList1, docsWithField1, @@ -2728,7 +3893,7 @@ public void testMergeFlowWithMaxLeafDocsAndStarTreeNodesAssertion() throws IOExc metricsWithField, metricsList1, metricsWithField1, - sf + compositeField ); StarTreeValues starTreeValues2 = getStarTreeValues( @@ -2744,10 +3909,17 @@ public void testMergeFlowWithMaxLeafDocsAndStarTreeNodesAssertion() throws IOExc metricsWithField, metricsList1, metricsWithField1, - sf + compositeField ); - builder = getStarTreeBuilder(sf, writeState, mapperService); - builder.build(builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2))); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); + builder.build(builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)), new AtomicInteger(), docValuesConsumer); List starTreeDocuments = builder.getStarTreeDocuments(); Map> dimValueToDocIdMap = new HashMap<>(); traverseStarTree(builder.rootNode, dimValueToDocIdMap, true); @@ -2762,10 +3934,38 @@ public void testMergeFlowWithMaxLeafDocsAndStarTreeNodesAssertion() throws IOExc } } assertEquals(1041, starTreeDocuments.size()); - validateStarTree(builder.getRootNode(), 4, sf.getStarTreeConfig().maxLeafDocs(), builder.getStarTreeDocuments()); + validateStarTree(builder.getRootNode(), 4, compositeField.getStarTreeConfig().maxLeafDocs(), builder.getStarTreeDocuments()); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "sf", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + getStarTreeDimensionNames(compositeField.getDimensionsOrder()), + compositeField.getMetrics(), + 500, + builder.numStarTreeDocs, + compositeField.getStarTreeConfig().maxLeafDocs(), + Set.of(), + getBuildMode(), + 0, + 31779 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + builder.getStarTreeDocuments().size(), + starTreeMetadata, + builder.getStarTreeDocuments() + ); } - private static StarTreeField getStarTreeFieldWithDocCount(int maxLeafDocs, boolean includeDocCountMetric) { + private StarTreeField getStarTreeFieldWithDocCount(int maxLeafDocs, boolean includeDocCountMetric) { Dimension d1 = new NumericDimension("field1"); Dimension d2 = new NumericDimension("field3"); Dimension d3 = new NumericDimension("field5"); @@ -2777,11 +3977,7 @@ private static StarTreeField getStarTreeFieldWithDocCount(int maxLeafDocs, boole m2 = new Metric("_doc_count", List.of(MetricStat.DOC_COUNT)); } List metrics = m2 == null ? List.of(m1) : List.of(m1, m2); - StarTreeFieldConfiguration c = new StarTreeFieldConfiguration( - maxLeafDocs, - new HashSet<>(), - StarTreeFieldConfiguration.StarTreeBuildMode.OFF_HEAP - ); + StarTreeFieldConfiguration c = new StarTreeFieldConfiguration(maxLeafDocs, new HashSet<>(), getBuildMode()); StarTreeField sf = new StarTreeField("sf", dims, metrics, c); return sf; } @@ -2795,23 +3991,24 @@ private void traverseStarTree(InMemoryTreeNode root, Map docIds = new ArrayList<>(); while ((starTreeNode = queue.poll()) != null) { - int dimensionId = starTreeNode.dimensionId; + int dimensionId = starTreeNode.getDimensionId(); if (dimensionId > currentDimensionId) { currentDimensionId = dimensionId; } // store aggregated document of the node - int docId = starTreeNode.aggregatedDocId; + int docId = starTreeNode.getAggregatedDocId(); Map map = dimValueToDocIdMap.getOrDefault(dimensionId, new HashMap<>()); - if (starTreeNode.nodeType == StarTreeNodeType.STAR.getValue()) { + if (starTreeNode.getNodeType() == StarTreeNodeType.STAR.getValue()) { map.put(Long.MAX_VALUE, docId); } else { - map.put(starTreeNode.dimensionValue, docId); + map.put(starTreeNode.getDimensionValue(), docId); } dimValueToDocIdMap.put(dimensionId, map); - if (starTreeNode.children != null && (!traverStarNodes || starTreeNode.nodeType == StarTreeNodeType.STAR.getValue())) { - Iterator childrenIterator = starTreeNode.children.values().iterator(); + if (starTreeNode.getChildren() != null + && (!traverStarNodes || starTreeNode.getNodeType() == StarTreeNodeType.STAR.getValue())) { + Iterator childrenIterator = starTreeNode.getChildren().values().iterator(); while (childrenIterator.hasNext()) { InMemoryTreeNode childNode = childrenIterator.next(); queue.add(childNode); @@ -2872,21 +4069,34 @@ public void testMergeFlow() throws IOException { Metric m2 = new Metric("_doc_count", List.of(MetricStat.DOC_COUNT)); List dims = List.of(d1, d2, d3, d4); List metrics = List.of(m1, m2); - StarTreeFieldConfiguration c = new StarTreeFieldConfiguration( - 1, - new HashSet<>(), - StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP - ); - StarTreeField sf = new StarTreeField("sf", dims, metrics, c); + StarTreeFieldConfiguration c = new StarTreeFieldConfiguration(1, new HashSet<>(), getBuildMode()); + compositeField = new StarTreeField("sf", dims, metrics, c); SortedNumericDocValues d1sndv = getSortedNumericMock(dimList1, docsWithField1); SortedNumericDocValues d2sndv = getSortedNumericMock(dimList2, docsWithField2); SortedNumericDocValues d3sndv = getSortedNumericMock(dimList3, docsWithField3); SortedNumericDocValues d4sndv = getSortedNumericMock(dimList4, docsWithField4); SortedNumericDocValues m1sndv = getSortedNumericMock(metricsList, metricsWithField); SortedNumericDocValues m2sndv = DocValues.emptySortedNumeric(); - Map dimDocIdSetIterators = Map.of("field1", d1sndv, "field3", d2sndv, "field5", d3sndv, "field8", d4sndv); - Map metricDocIdSetIterators = Map.of("field2", m1sndv, "_doc_count", m2sndv); - StarTreeValues starTreeValues = new StarTreeValues(sf, null, dimDocIdSetIterators, metricDocIdSetIterators, getAttributes(1000)); + Map> dimDocIdSetIterators = Map.of( + "field1", + () -> d1sndv, + "field3", + () -> d2sndv, + "field5", + () -> d3sndv, + "field8", + () -> d4sndv + ); + + Map> metricDocIdSetIterators = Map.of("field2", () -> m1sndv, "_doc_count", () -> m2sndv); + + StarTreeValues starTreeValues = new StarTreeValues( + compositeField, + null, + dimDocIdSetIterators, + metricDocIdSetIterators, + getAttributes(1000) + ); SortedNumericDocValues f2d1sndv = getSortedNumericMock(dimList1, docsWithField1); SortedNumericDocValues f2d2sndv = getSortedNumericMock(dimList2, docsWithField2); @@ -2894,26 +4104,34 @@ public void testMergeFlow() throws IOException { SortedNumericDocValues f2d4sndv = getSortedNumericMock(dimList4, docsWithField4); SortedNumericDocValues f2m1sndv = getSortedNumericMock(metricsList, metricsWithField); SortedNumericDocValues f2m2sndv = DocValues.emptySortedNumeric(); - Map f2dimDocIdSetIterators = Map.of( + Map> f2dimDocIdSetIterators = Map.of( "field1", - f2d1sndv, + () -> f2d1sndv, "field3", - f2d2sndv, + () -> f2d2sndv, "field5", - f2d3sndv, + () -> f2d3sndv, "field8", - f2d4sndv + () -> f2d4sndv ); - Map f2metricDocIdSetIterators = Map.of("field2", f2m1sndv, "_doc_count", f2m2sndv); + + Map> f2metricDocIdSetIterators = Map.of("field2", () -> f2m1sndv, "_doc_count", () -> f2m2sndv); StarTreeValues starTreeValues2 = new StarTreeValues( - sf, + compositeField, null, f2dimDocIdSetIterators, f2metricDocIdSetIterators, getAttributes(1000) ); - builder = getStarTreeBuilder(sf, writeState, mapperService); + this.docValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec( + writeState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); + builder = getStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** [0, 0, 0, 0] | [0.0, 2] @@ -2929,10 +4147,38 @@ public void testMergeFlow() throws IOException { assertEquals(starTreeDocument.dimensions[0] * 20.0, starTreeDocument.metrics[0]); assertEquals(2L, starTreeDocument.metrics[1]); } - builder.build(starTreeDocumentIterator); + builder.build(starTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); // Validate the star tree structure validateStarTree(builder.getRootNode(), 4, 1, builder.getStarTreeDocuments()); + + metaOut.close(); + dataOut.close(); + docValuesConsumer.close(); + + StarTreeMetadata starTreeMetadata = new StarTreeMetadata( + "sf", + STAR_TREE, + mock(IndexInput.class), + VERSION_CURRENT, + builder.numStarTreeNodes, + getStarTreeDimensionNames(compositeField.getDimensionsOrder()), + compositeField.getMetrics(), + 1000, + builder.numStarTreeDocs, + compositeField.getStarTreeConfig().maxLeafDocs(), + Set.of(), + getBuildMode(), + 0, + 132165 + ); + + validateStarTreeFileFormats( + builder.getRootNode(), + builder.getStarTreeDocuments().size(), + starTreeMetadata, + builder.getStarTreeDocuments() + ); } private void validateStarTree( @@ -2951,36 +4197,37 @@ private void validateStarTree( assertNotNull(node); // assert dimensions - if (node.dimensionId != StarTreeUtils.ALL) { - assertTrue(node.dimensionId >= 0 && node.dimensionId < totalDimensions); + if (node.getDimensionId() != StarTreeUtils.ALL) { + assertTrue(node.getDimensionId() >= 0 && node.getDimensionId() < totalDimensions); } - if (node.children != null && !node.children.isEmpty()) { - assertEquals(node.dimensionId + 1, node.childDimensionId); - assertTrue(node.childDimensionId < totalDimensions); + + if (node.getChildren() != null && !node.getChildren().isEmpty()) { + assertEquals(node.getDimensionId() + 1, node.getChildDimensionId()); + assertTrue(node.getChildDimensionId() < totalDimensions); InMemoryTreeNode starNode = null; Object[] nonStarNodeCumulativeMetrics = getMetrics(starTreeDocuments); - for (Map.Entry entry : node.children.entrySet()) { + for (Map.Entry entry : node.getChildren().entrySet()) { Long childDimensionValue = entry.getKey(); InMemoryTreeNode child = entry.getValue(); Object[] currMetrics = getMetrics(starTreeDocuments); - if (child.nodeType != StarTreeNodeType.STAR.getValue()) { + if (child.getNodeType() != StarTreeNodeType.STAR.getValue()) { // Validate dimension values in documents - for (int i = child.startDocId; i < child.endDocId; i++) { + for (int i = child.getStartDocId(); i < child.getEndDocId(); i++) { StarTreeDocument doc = starTreeDocuments.get(i); int j = 0; addMetrics(doc, currMetrics, j); - if (child.nodeType != StarTreeNodeType.STAR.getValue()) { - Long dimension = doc.dimensions[child.dimensionId]; + if (child.getNodeType() != StarTreeNodeType.STAR.getValue()) { + Long dimension = doc.dimensions[child.getDimensionId()]; assertEquals(childDimensionValue, dimension); if (dimension != null) { - assertEquals(child.dimensionValue, (long) dimension); + assertEquals(child.getDimensionValue(), (long) dimension); } else { // TODO : fix this ? - assertEquals(child.dimensionValue, StarTreeUtils.ALL); + assertEquals(child.getDimensionValue(), StarTreeUtils.ALL); } } } - Object[] aggregatedMetrics = starTreeDocuments.get(child.aggregatedDocId).metrics; + Object[] aggregatedMetrics = starTreeDocuments.get(child.getAggregatedDocId()).metrics; int j = 0; for (Object metric : currMetrics) { /* @@ -3006,13 +4253,13 @@ private void validateStarTree( // Add star node to queue if (starNode != null) { Object[] starNodeMetrics = getMetrics(starTreeDocuments); - for (int i = starNode.startDocId; i < starNode.endDocId; i++) { + for (int i = starNode.getStartDocId(); i < starNode.getEndDocId(); i++) { StarTreeDocument doc = starTreeDocuments.get(i); int j = 0; addMetrics(doc, starNodeMetrics, j); } int j = 0; - Object[] aggregatedMetrics = starTreeDocuments.get(starNode.aggregatedDocId).metrics; + Object[] aggregatedMetrics = starTreeDocuments.get(starNode.getAggregatedDocId()).metrics; for (Object nonStarNodeCumulativeMetric : nonStarNodeCumulativeMetrics) { assertEquals(nonStarNodeCumulativeMetric, starNodeMetrics[j]); assertEquals(starNodeMetrics[j], aggregatedMetrics[j]); @@ -3032,20 +4279,20 @@ private void validateStarTree( j++; } - assertEquals(-1L, starNode.dimensionValue); + assertEquals(-1L, starNode.getDimensionValue()); queue.offer(new Object[] { starNode, true }); } } else { - assertTrue(node.endDocId - node.startDocId <= maxLeafDocuments); + assertTrue(node.getEndDocId() - node.getStartDocId() <= maxLeafDocuments); } if (currentIsStarNode) { StarTreeDocument prevDoc = null; int docCount = 0; - int docId = node.startDocId; - int dimensionId = node.dimensionId; + int docId = node.getStartDocId(); + int dimensionId = node.getDimensionId(); - while (docId < node.endDocId) { + while (docId < node.getEndDocId()) { StarTreeDocument currentDoc = starTreeDocuments.get(docId); docCount++; @@ -3061,7 +4308,7 @@ private void validateStarTree( } // Verify that the number of generated star documents matches the range in the star node - assertEquals(node.endDocId - node.startDocId, docCount); + assertEquals(node.getEndDocId() - node.getStartDocId(), docCount); } } } @@ -3111,20 +4358,16 @@ private int compareDocuments(StarTreeDocument doc1, StarTreeDocument doc2, int s } Map getAttributes(int numSegmentDocs) { - return Map.of(String.valueOf(NUM_SEGMENT_DOCS), String.valueOf(numSegmentDocs)); + return Map.of(CompositeIndexConstants.SEGMENT_DOCS_COUNT, String.valueOf(numSegmentDocs)); } - private static StarTreeField getStarTreeField(MetricStat count) { + private StarTreeField getStarTreeField(MetricStat count) { Dimension d1 = new NumericDimension("field1"); Dimension d2 = new NumericDimension("field3"); Metric m1 = new Metric("field2", List.of(count)); List dims = List.of(d1, d2); List metrics = List.of(m1); - StarTreeFieldConfiguration c = new StarTreeFieldConfiguration( - 1000, - new HashSet<>(), - StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP - ); + StarTreeFieldConfiguration c = new StarTreeFieldConfiguration(1000, new HashSet<>(), getBuildMode()); return new StarTreeField("sf", dims, metrics, c); } @@ -3179,6 +4422,9 @@ public void tearDown() throws Exception { if (builder != null) { builder.close(); } + docValuesConsumer.close(); + metaOut.close(); + dataOut.close(); directory.close(); } } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilderTests.java index 51ebc02ea8243..22a0fb7875714 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilderTests.java @@ -8,21 +8,24 @@ package org.opensearch.index.compositeindex.datacube.startree.builder; +import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.lucene99.Lucene99Codec; import org.apache.lucene.index.DocValuesType; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.index.VectorEncoding; import org.apache.lucene.index.VectorSimilarityFunction; import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.InfoStream; import org.apache.lucene.util.Version; import org.opensearch.common.settings.Settings; -import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; +import org.opensearch.index.codec.composite.composite99.Composite99DocValuesFormat; import org.opensearch.index.compositeindex.datacube.Dimension; import org.opensearch.index.compositeindex.datacube.Metric; import org.opensearch.index.compositeindex.datacube.MetricStat; @@ -31,6 +34,7 @@ import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricAggregatorInfo; +import org.opensearch.index.compositeindex.datacube.startree.index.StarTreeValues; import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; import org.opensearch.index.fielddata.IndexNumericFieldData; import org.opensearch.index.mapper.ContentPath; @@ -51,6 +55,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -75,9 +80,12 @@ public class BaseStarTreeBuilderTests extends OpenSearchTestCase { private static List metrics; private static Directory directory; private static FieldInfo[] fieldsInfo; - private static SegmentWriteState state; + private static SegmentWriteState writeState; private static StarTreeField starTreeField; + private static IndexOutput dataOut; + private static IndexOutput metaOut; + @BeforeClass public static void setup() throws IOException { @@ -138,7 +146,21 @@ public static void setup() throws IOException { fieldProducerMap.put(fields.get(i), docValuesProducer); } FieldInfos fieldInfos = new FieldInfos(fieldsInfo); - state = new SegmentWriteState(InfoStream.getDefault(), segmentInfo.dir, segmentInfo, fieldInfos, null, newIOContext(random())); + writeState = new SegmentWriteState(InfoStream.getDefault(), segmentInfo.dir, segmentInfo, fieldInfos, null, newIOContext(random())); + + String dataFileName = IndexFileNames.segmentFileName( + writeState.segmentInfo.name, + writeState.segmentSuffix, + Composite99DocValuesFormat.DATA_EXTENSION + ); + dataOut = writeState.directory.createOutput(dataFileName, writeState.context); + + String metaFileName = IndexFileNames.segmentFileName( + writeState.segmentInfo.name, + writeState.segmentSuffix, + Composite99DocValuesFormat.META_EXTENSION + ); + metaOut = writeState.directory.createOutput(metaFileName, writeState.context); mapperService = mock(MapperService.class); DocumentMapper documentMapper = mock(DocumentMapper.class); @@ -157,9 +179,13 @@ public static void setup() throws IOException { ); when(documentMapper.mappers()).thenReturn(fieldMappers); - builder = new BaseStarTreeBuilder(starTreeField, state, mapperService) { + builder = new BaseStarTreeBuilder(metaOut, dataOut, starTreeField, writeState, mapperService) { @Override - public void build(List starTreeValuesSubs) throws IOException {} + public void build( + List starTreeValuesSubs, + AtomicInteger fieldNumberAcrossStarTrees, + DocValuesConsumer starTreeDocValuesConsumer + ) throws IOException {} @Override public void appendStarTreeDocument(StarTreeDocument starTreeDocument) throws IOException {} @@ -224,6 +250,8 @@ public void test_reduceStarTreeDocuments() { @Override public void tearDown() throws Exception { super.tearDown(); + dataOut.close(); + metaOut.close(); directory.close(); } } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilderTests.java index 92382b78f60c6..496558dbc2e83 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilderTests.java @@ -9,7 +9,9 @@ package org.opensearch.index.compositeindex.datacube.startree.builder; import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.IndexOutput; import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; import org.opensearch.index.mapper.MapperService; import java.io.IOException; @@ -17,10 +19,18 @@ public class OffHeapStarTreeBuilderTests extends AbstractStarTreeBuilderTests { @Override public BaseStarTreeBuilder getStarTreeBuilder( + IndexOutput metaOut, + IndexOutput dataOut, StarTreeField starTreeField, SegmentWriteState segmentWriteState, MapperService mapperService ) throws IOException { - return new OffHeapStarTreeBuilder(starTreeField, segmentWriteState, mapperService); + return new OffHeapStarTreeBuilder(metaOut, dataOut, starTreeField, segmentWriteState, mapperService); } + + @Override + StarTreeFieldConfiguration.StarTreeBuildMode getBuildMode() { + return StarTreeFieldConfiguration.StarTreeBuildMode.OFF_HEAP; + } + } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilderTests.java index aed08b7727be7..55cf3bde3cea7 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilderTests.java @@ -9,16 +9,29 @@ package org.opensearch.index.compositeindex.datacube.startree.builder; import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.IndexOutput; import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; import org.opensearch.index.mapper.MapperService; +import java.io.IOException; + public class OnHeapStarTreeBuilderTests extends AbstractStarTreeBuilderTests { + @Override public BaseStarTreeBuilder getStarTreeBuilder( + IndexOutput metaOut, + IndexOutput dataOut, StarTreeField starTreeField, SegmentWriteState segmentWriteState, MapperService mapperService - ) { - return new OnHeapStarTreeBuilder(starTreeField, segmentWriteState, mapperService); + ) throws IOException { + return new OnHeapStarTreeBuilder(metaOut, dataOut, starTreeField, segmentWriteState, mapperService); + } + + @Override + StarTreeFieldConfiguration.StarTreeBuildMode getBuildMode() { + return StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP; } + } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilderTests.java index 828bddfb8aa6e..94fe148eab6b3 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilderTests.java @@ -8,6 +8,7 @@ package org.opensearch.index.compositeindex.datacube.startree.builder; +import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.codecs.lucene99.Lucene99Codec; import org.apache.lucene.index.FieldInfo; @@ -15,6 +16,7 @@ import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.InfoStream; import org.apache.lucene.util.Version; import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; @@ -31,6 +33,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verifyNoInteractions; @@ -45,9 +48,13 @@ public class StarTreesBuilderTests extends OpenSearchTestCase { private StarTreeField starTreeField; private Map fieldProducerMap; private Directory directory; + private IndexOutput dataOut; + private IndexOutput metaOut; public void setUp() throws Exception { super.setUp(); + metaOut = mock(IndexOutput.class); + dataOut = mock(IndexOutput.class); mapperService = mock(MapperService.class); directory = newFSDirectory(createTempDir()); SegmentInfo segmentInfo = new SegmentInfo( @@ -88,19 +95,29 @@ public void setUp() throws Exception { public void test_buildWithNoStarTreeFields() throws IOException { when(mapperService.getCompositeFieldTypes()).thenReturn(new HashSet<>()); - StarTreesBuilder starTreesBuilder = new StarTreesBuilder(segmentWriteState, mapperService); - starTreesBuilder.build(fieldProducerMap); + StarTreesBuilder starTreesBuilder = new StarTreesBuilder(segmentWriteState, mapperService, new AtomicInteger()); + starTreesBuilder.build(metaOut, dataOut, fieldProducerMap, mock(DocValuesConsumer.class)); verifyNoInteractions(docValuesProducer); } public void test_getStarTreeBuilder() throws IOException { when(mapperService.getCompositeFieldTypes()).thenReturn(Set.of(starTreeFieldType)); - StarTreesBuilder starTreesBuilder = new StarTreesBuilder(segmentWriteState, mapperService); - StarTreeBuilder starTreeBuilder = starTreesBuilder.getStarTreeBuilder(starTreeField, segmentWriteState, mapperService); + StarTreesBuilder starTreesBuilder = new StarTreesBuilder(segmentWriteState, mapperService, new AtomicInteger()); + StarTreeBuilder starTreeBuilder = starTreesBuilder.getStarTreeBuilder(metaOut, dataOut, starTreeField, segmentWriteState, mapperService); assertTrue(starTreeBuilder instanceof OnHeapStarTreeBuilder); } + public void test_getStarTreeBuilder_illegalArgument() throws IOException { + when(mapperService.getCompositeFieldTypes()).thenReturn(Set.of(starTreeFieldType)); + StarTreeFieldConfiguration starTreeFieldConfiguration = new StarTreeFieldConfiguration(1, new HashSet<>(), StarTreeFieldConfiguration.StarTreeBuildMode.OFF_HEAP); + StarTreeField starTreeField = new StarTreeField("star_tree", new ArrayList<>(), new ArrayList<>(), starTreeFieldConfiguration); + StarTreesBuilder starTreesBuilder = new StarTreesBuilder(segmentWriteState, mapperService, new AtomicInteger()); + StarTreeBuilder starTreeBuilder = starTreesBuilder.getStarTreeBuilder(metaOut, dataOut, starTreeField, segmentWriteState, mapperService); + assertTrue(starTreeBuilder instanceof OffHeapStarTreeBuilder); + starTreeBuilder.close(); + } + public void test_closeWithNoStarTreeFields() throws IOException { StarTreeFieldConfiguration starTreeFieldConfiguration = new StarTreeFieldConfiguration( 1, @@ -110,7 +127,7 @@ public void test_closeWithNoStarTreeFields() throws IOException { StarTreeField starTreeField = new StarTreeField("star_tree", new ArrayList<>(), new ArrayList<>(), starTreeFieldConfiguration); starTreeFieldType = new StarTreeMapper.StarTreeFieldType("star_tree", starTreeField); when(mapperService.getCompositeFieldTypes()).thenReturn(Set.of(starTreeFieldType)); - StarTreesBuilder starTreesBuilder = new StarTreesBuilder(segmentWriteState, mapperService); + StarTreesBuilder starTreesBuilder = new StarTreesBuilder(segmentWriteState, mapperService, new AtomicInteger()); starTreesBuilder.close(); verifyNoInteractions(docValuesProducer); @@ -119,6 +136,8 @@ public void test_closeWithNoStarTreeFields() throws IOException { @Override public void tearDown() throws Exception { super.tearDown(); + metaOut.close(); + dataOut.close(); directory.close(); } } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/StarTreeFileFormatsTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/StarTreeFileFormatsTests.java index 4653ac8b08198..4d2aa5eaf78cf 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/StarTreeFileFormatsTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/data/StarTreeFileFormatsTests.java @@ -23,7 +23,6 @@ import java.io.IOException; import java.util.ArrayDeque; -import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.Map; @@ -121,8 +120,8 @@ public void test_starTreeSearch() throws IOException { assertNotNull(inMemoryTreeNode); for (int i = 0; i < maxLevels - 1; i++) { - InMemoryTreeNode randomChildNode = randomFrom(inMemoryTreeNode.children.values()); - StarTreeNode randomStarTreeChildNode = starTreeNode.getChildForDimensionValue(randomChildNode.dimensionValue); + InMemoryTreeNode randomChildNode = randomFrom(inMemoryTreeNode.getChildren().values()); + StarTreeNode randomStarTreeChildNode = starTreeNode.getChildForDimensionValue(randomChildNode.getDimensionValue()); assertNotNull(randomStarTreeChildNode); assertStarTreeNode(randomStarTreeChildNode, randomChildNode); @@ -135,18 +134,18 @@ public void test_starTreeSearch() throws IOException { } private void assertStarTreeNode(StarTreeNode starTreeNode, InMemoryTreeNode treeNode) throws IOException { - assertEquals(starTreeNode.getDimensionId(), treeNode.dimensionId); - assertEquals(starTreeNode.getDimensionValue(), treeNode.dimensionValue); - assertEquals(starTreeNode.getStartDocId(), treeNode.startDocId); - assertEquals(starTreeNode.getEndDocId(), treeNode.endDocId); - assertEquals(starTreeNode.getChildDimensionId(), treeNode.childDimensionId); - assertEquals(starTreeNode.getAggregatedDocId(), treeNode.aggregatedDocId); - assertEquals(starTreeNode.getStarTreeNodeType(), treeNode.nodeType); + assertEquals(starTreeNode.getDimensionId(), treeNode.getDimensionId()); + assertEquals(starTreeNode.getDimensionValue(), treeNode.getDimensionValue()); + assertEquals(starTreeNode.getStartDocId(), treeNode.getStartDocId()); + assertEquals(starTreeNode.getEndDocId(), treeNode.getEndDocId()); + assertEquals(starTreeNode.getChildDimensionId(), treeNode.getChildDimensionId()); + assertEquals(starTreeNode.getAggregatedDocId(), treeNode.getAggregatedDocId()); + assertEquals(starTreeNode.getStarTreeNodeType(), treeNode.getNodeType()); if (starTreeNode.getChildDimensionId() != -1) { assertFalse(starTreeNode.isLeaf()); - if (treeNode.children != null) { - assertEquals(starTreeNode.getNumChildren(), treeNode.children.values().size()); + if (treeNode.getChildren() != null) { + assertEquals(starTreeNode.getNumChildren(), treeNode.getChildren().values().size()); } } else { assertTrue(starTreeNode.isLeaf()); @@ -156,16 +155,11 @@ private void assertStarTreeNode(StarTreeNode starTreeNode, InMemoryTreeNode tree public InMemoryTreeNode generateSampleTree(Map inMemoryTreeNodeMap) { // Create the root node - InMemoryTreeNode root = new InMemoryTreeNode(); - root.dimensionId = 0; - root.startDocId = randomInt(); - root.endDocId = randomInt(); - root.childDimensionId = 1; - root.aggregatedDocId = randomInt(); - root.nodeType = (byte) 0; - root.children = new HashMap<>(); + InMemoryTreeNode root = new InMemoryTreeNode(0, randomInt(), randomInt(), (byte) 0, -1); + root.setChildDimensionId(1); + root.setAggregatedDocId(randomInt()); - inMemoryTreeNodeMap.put(root.dimensionValue, root); + inMemoryTreeNodeMap.put(root.getDimensionValue(), root); // Generate the tree recursively generateTreeRecursively(root, 1, inMemoryTreeNodeMap); @@ -181,19 +175,14 @@ private void generateTreeRecursively(InMemoryTreeNode parent, int currentLevel, int numChildren = randomIntBetween(1, 10); for (int i = 0; i < numChildren; i++) { - InMemoryTreeNode child = new InMemoryTreeNode(); dimensionValue++; - child.dimensionId = currentLevel; - child.dimensionValue = dimensionValue; // Assign a unique dimension value for each child - child.startDocId = randomInt(); - child.endDocId = randomInt(); - child.childDimensionId = (currentLevel == this.maxLevels - 1) ? -1 : (currentLevel + 1); - child.aggregatedDocId = randomInt(); - child.nodeType = (byte) 0; - child.children = new HashMap<>(); - - parent.children.put(child.dimensionValue, child); - inMemoryTreeNodeMap.put(child.dimensionValue, child); + InMemoryTreeNode child = new InMemoryTreeNode(currentLevel, randomInt(), randomInt(), (byte) 0, dimensionValue); + + child.setChildDimensionId((currentLevel == this.maxLevels - 1) ? -1 : (currentLevel + 1)); + child.setAggregatedDocId(randomInt()); + + parent.addChildNode(child, child.getDimensionValue()); + inMemoryTreeNodeMap.put(child.getDimensionValue(), child); generateTreeRecursively(child, currentLevel + 1, inMemoryTreeNodeMap); } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadataTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadataTests.java index 62bd74cc0b3fc..6fac5362c3dbb 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadataTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/meta/StarTreeMetadataTests.java @@ -62,6 +62,7 @@ public class StarTreeMetadataTests extends OpenSearchTestCase { private List metrics; private List metricAggregatorInfos = new ArrayList<>(); private int segmentDocumentCount; + private int numStarTreeDocs; private long dataFilePointer; private long dataFileLength; @@ -145,6 +146,7 @@ public void test_starTreeMetadata() throws IOException { dataFileLength = randomNonNegativeLong(); dataFilePointer = randomNonNegativeLong(); segmentDocumentCount = randomInt(Integer.MAX_VALUE); + numStarTreeDocs = randomInt(Integer.MAX_VALUE); metaOut = directory.createOutput("star-tree-metadata", IOContext.DEFAULT); StarTreeWriter starTreeWriter = new StarTreeWriter(); int numberOfNodes = randomInt(Integer.MAX_VALUE); @@ -154,6 +156,7 @@ public void test_starTreeMetadata() throws IOException { metricAggregatorInfos, numberOfNodes, segmentDocumentCount, + numStarTreeDocs, dataFilePointer, dataFileLength ); diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNodeTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNodeTests.java index 6f24728c24f30..9657f867422a0 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNodeTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/fileformats/node/FixedLengthStarTreeNodeTests.java @@ -12,18 +12,18 @@ 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.StarTreeTestUtils; 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.index.compositeindex.datacube.startree.utils.StarTreeUtils; import org.opensearch.test.OpenSearchTestCase; import org.junit.Before; import java.io.IOException; -import java.util.HashMap; import java.util.Iterator; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeUtils.ALL; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -35,6 +35,7 @@ public class FixedLengthStarTreeNodeTests extends OpenSearchTestCase { InMemoryTreeNode node; InMemoryTreeNode starChild; InMemoryTreeNode nullChild; + InMemoryTreeNode childWithMinus1; FixedLengthStarTreeNode starTreeNode; @Before @@ -44,54 +45,42 @@ public void setup() throws IOException { dataOut = directory.createOutput("star-tree-data", IOContext.DEFAULT); StarTreeWriter starTreeWriter = new StarTreeWriter(); - node = new InMemoryTreeNode(); - node.dimensionId = 0; - node.startDocId = randomInt(); - node.endDocId = randomInt(); - node.childDimensionId = 1; - node.aggregatedDocId = randomInt(); - node.nodeType = randomFrom((byte) 0, (byte) -1, (byte) 2); - node.children = new HashMap<>(); - - starChild = new InMemoryTreeNode(); - starChild.dimensionId = node.dimensionId + 1; - starChild.dimensionValue = -1; - starChild.startDocId = randomInt(); - starChild.endDocId = randomInt(); - starChild.childDimensionId = -1; - starChild.aggregatedDocId = randomInt(); - starChild.nodeType = (byte) -2; - starChild.children = new HashMap<>(); - node.children.put(-1L, starChild); - - nullChild = new InMemoryTreeNode(); - nullChild.dimensionId = node.dimensionId + 1; - nullChild.dimensionValue = -1; - nullChild.startDocId = randomInt(); - nullChild.endDocId = randomInt(); - nullChild.childDimensionId = -1; - nullChild.aggregatedDocId = randomInt(); - nullChild.nodeType = (byte) -1; - nullChild.children = new HashMap<>(); - node.children.put(null, nullChild); + node = new InMemoryTreeNode(0, randomInt(), randomInt(), randomFrom((byte) 0, (byte) -1, (byte) 1), -1); + node.setChildDimensionId(1); + node.setAggregatedDocId(randomInt()); + + starChild = new InMemoryTreeNode(node.getDimensionId() + 1, randomInt(), randomInt(), (byte) -1, -1); + starChild.setChildDimensionId(-1); + starChild.setAggregatedDocId(randomInt()); + node.addChildNode(starChild, (long) ALL); + + childWithMinus1 = new InMemoryTreeNode(node.getDimensionId() + 1, randomInt(), randomInt(), (byte) 0, -1); + childWithMinus1.setChildDimensionId(-1); + childWithMinus1.setAggregatedDocId(randomInt()); + node.addChildNode(childWithMinus1, -1L); for (int i = 1; i < randomIntBetween(2, 5); i++) { - InMemoryTreeNode child = new InMemoryTreeNode(); - child.dimensionId = node.dimensionId + 1; - child.dimensionValue = node.dimensionValue + i; // Assign a unique dimension value for each child - child.startDocId = randomInt(); - child.endDocId = randomInt(); - child.childDimensionId = -1; - child.aggregatedDocId = randomInt(); - child.nodeType = (byte) 0; - child.children = new HashMap<>(); - node.children.put(child.dimensionValue, child); + InMemoryTreeNode child = new InMemoryTreeNode( + node.getDimensionId() + 1, + randomInt(), + randomInt(), + (byte) 0, + node.getDimensionValue() + i + ); + child.setChildDimensionId(-1); + child.setAggregatedDocId(randomInt()); + node.addChildNode(child, child.getDimensionValue()); } - long starTreeDataLength = starTreeWriter.writeStarTree(dataOut, node, 1 + node.children.size(), "star-tree"); + nullChild = new InMemoryTreeNode(node.getDimensionId() + 1, randomInt(), randomInt(), (byte) 1, -1); + nullChild.setChildDimensionId(-1); + nullChild.setAggregatedDocId(randomInt()); + node.addChildNode(nullChild, null); + + long starTreeDataLength = starTreeWriter.writeStarTree(dataOut, node, 2 + node.getChildren().size(), "star-tree"); // asserting on the actual length of the star tree data file - assertEquals(starTreeDataLength, 33L * node.children.size() + 33); + assertEquals(starTreeDataLength, 33L * node.getChildren().size() + 2 * 33); dataOut.close(); dataIn = directory.openInput("star-tree-data", IOContext.READONCE); @@ -119,27 +108,27 @@ public void testSerializableDataSize() { } public void testGetDimensionId() throws IOException { - assertEquals(node.dimensionId, starTreeNode.getDimensionId()); + assertEquals(node.getDimensionId(), starTreeNode.getDimensionId()); } public void testGetDimensionValue() throws IOException { - assertEquals(node.dimensionValue, starTreeNode.getDimensionValue()); + assertEquals(node.getDimensionValue(), starTreeNode.getDimensionValue()); } public void testGetStartDocId() throws IOException { - assertEquals(node.startDocId, starTreeNode.getStartDocId()); + assertEquals(node.getStartDocId(), starTreeNode.getStartDocId()); } public void testGetEndDocId() throws IOException { - assertEquals(node.endDocId, starTreeNode.getEndDocId()); + assertEquals(node.getEndDocId(), starTreeNode.getEndDocId()); } public void testGetAggregatedDocId() throws IOException { - assertEquals(node.aggregatedDocId, starTreeNode.getAggregatedDocId()); + assertEquals(node.getAggregatedDocId(), starTreeNode.getAggregatedDocId()); } public void testGetNumChildren() throws IOException { - assertEquals(node.children.size(), starTreeNode.getNumChildren()); + assertEquals(node.getChildren().size(), starTreeNode.getNumChildren() - 1); } public void testIsLeaf() { @@ -151,8 +140,7 @@ public void testGetStarTreeNodeType() throws IOException { } public void testGetChildForDimensionValue() throws IOException { - // TODO: Add a test to verify children with star node, null node and default node with default dimension value -1 - long dimensionValue = randomIntBetween(0, node.children.size() - 3); + long dimensionValue = randomIntBetween(-1, node.getChildren().size() - 3); FixedLengthStarTreeNode childNode = (FixedLengthStarTreeNode) starTreeNode.getChildForDimensionValue(dimensionValue); assertNotNull(childNode); assertEquals(dimensionValue, childNode.getDimensionValue()); @@ -169,11 +157,31 @@ public void testGetChildrenIterator() throws IOException { assertEquals(starTreeNode.getNumChildren(), count); } + public void testChildrenOrder() throws IOException { + Iterator iterator = starTreeNode.getChildrenIterator(); + int count = 0; + while (iterator.hasNext()) { + FixedLengthStarTreeNode child = iterator.next(); + count++; + if (count == 1) { + StarTreeTestUtils.assertStarTreeNode(child, starChild); + } else if (count == 2) { + StarTreeTestUtils.assertStarTreeNode(child, childWithMinus1); + } else if (count == starTreeNode.getNumChildren()) { + StarTreeTestUtils.assertStarTreeNode(child, nullChild); + } else { + StarTreeTestUtils.assertStarTreeNode(child, node.getChildren().get(child.getDimensionValue())); + } + assertNotNull(child); + } + assertEquals(starTreeNode.getNumChildren(), count); + } + public void testGetChildForStarNode() throws IOException { // Assuming the first child is a star node in our test data FixedLengthStarTreeNode starNode = (FixedLengthStarTreeNode) starTreeNode.getChildStarNode(); assertNotNull(starNode); - assertEquals(StarTreeUtils.ALL, starNode.getDimensionValue()); + assertEquals(ALL, starNode.getDimensionValue()); } public void testGetChildForNullNode() throws IOException { @@ -193,14 +201,9 @@ public void testOnlyRootNodePresent() throws IOException { IndexOutput dataOut = directory.createOutput("star-tree-data-1", IOContext.DEFAULT); StarTreeWriter starTreeWriter = new StarTreeWriter(); - InMemoryTreeNode node = new InMemoryTreeNode(); - node.dimensionId = 0; - node.startDocId = randomInt(); - node.endDocId = randomInt(); - node.childDimensionId = 1; - node.aggregatedDocId = randomInt(); - node.nodeType = randomFrom((byte) 0, (byte) -1, (byte) 2); - node.children = new HashMap<>(); + InMemoryTreeNode node = new InMemoryTreeNode(0, randomInt(), randomInt(), randomFrom((byte) 0, (byte) -1, (byte) 2), -1); + node.setChildDimensionId(1); + node.setAggregatedDocId(randomInt()); long starTreeDataLength = starTreeWriter.writeStarTree(dataOut, node, 1, "star-tree"); diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNodeTypeTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNodeTypeTests.java index 81fb620da5af3..f1034e5d1d033 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNodeTypeTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNodeTypeTests.java @@ -14,12 +14,12 @@ public class StarTreeNodeTypeTests extends OpenSearchTestCase { public void testStarNodeType() { assertEquals("star", StarTreeNodeType.STAR.getName()); - assertEquals((byte) -2, StarTreeNodeType.STAR.getValue()); + assertEquals((byte) -1, StarTreeNodeType.STAR.getValue()); } public void testNullNodeType() { assertEquals("null", StarTreeNodeType.NULL.getName()); - assertEquals((byte) -1, StarTreeNodeType.NULL.getValue()); + assertEquals((byte) 1, StarTreeNodeType.NULL.getValue()); } public void testDefaultNodeType() { @@ -28,20 +28,20 @@ public void testDefaultNodeType() { } public void testFromValue() { - assertEquals(StarTreeNodeType.STAR, StarTreeNodeType.fromValue((byte) -2)); - assertEquals(StarTreeNodeType.NULL, StarTreeNodeType.fromValue((byte) -1)); + assertEquals(StarTreeNodeType.STAR, StarTreeNodeType.fromValue((byte) -1)); + assertEquals(StarTreeNodeType.NULL, StarTreeNodeType.fromValue((byte) 1)); assertEquals(StarTreeNodeType.DEFAULT, StarTreeNodeType.fromValue((byte) 0)); } public void testFromValueInvalid() { - IllegalStateException exception = expectThrows(IllegalStateException.class, () -> StarTreeNodeType.fromValue((byte) 1)); - assertEquals("Unrecognized value byte to determine star-tree node type: [1]", exception.getMessage()); + IllegalStateException exception = expectThrows(IllegalStateException.class, () -> StarTreeNodeType.fromValue((byte) 2)); + assertEquals("Unrecognized value byte to determine star-tree node type: [2]", exception.getMessage()); } public void testEnumValues() { StarTreeNodeType[] values = StarTreeNodeType.values(); assertEquals(3, values.length); - assertArrayEquals(new StarTreeNodeType[] { StarTreeNodeType.STAR, StarTreeNodeType.NULL, StarTreeNodeType.DEFAULT }, values); + assertArrayEquals(new StarTreeNodeType[] { StarTreeNodeType.STAR, StarTreeNodeType.DEFAULT, StarTreeNodeType.NULL }, values); } public void testEnumValueOf() { diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeUtilsTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeUtilsTests.java index a1d341615969e..9cca0b04e9ea4 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeUtilsTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeUtilsTests.java @@ -52,7 +52,7 @@ public void testGetFieldInfoList() { public void testGetFieldInfo() { String fieldName = UUID.randomUUID().toString(); int fieldNumber = randomInt(); - assertFieldInfos(StarTreeUtils.getFieldInfo(fieldName, fieldNumber), fieldName, fieldNumber); + assertFieldInfos(StarTreeUtils.getFieldInfo(fieldName, DocValuesType.SORTED_NUMERIC, fieldNumber), fieldName, fieldNumber); } diff --git a/test/framework/src/main/java/org/opensearch/test/OpenSearchTestCase.java b/test/framework/src/main/java/org/opensearch/test/OpenSearchTestCase.java index 6afc7c23d9e66..e07d7ef3c5a7b 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchTestCase.java @@ -815,6 +815,14 @@ public static long randomNonNegativeLong() { return randomLong == Long.MIN_VALUE ? 0 : Math.abs(randomLong); } + /** + * @return a int between 0 and Integer.MAX_VALUE (inclusive) chosen uniformly at random. + */ + public static int randomNonNegativeInt() { + int randomInt = randomInt(); + return randomInt == Integer.MIN_VALUE ? 0 : Math.abs(randomInt); + } + public static float randomFloat() { return random().nextFloat(); }