diff --git a/server/src/main/java/org/apache/lucene/codecs/lucene90/Composite99DocValuesConsumer.java b/server/src/main/java/org/apache/lucene/codecs/lucene90/Composite99DocValuesConsumer.java new file mode 100644 index 0000000000000..04a49d517ff81 --- /dev/null +++ b/server/src/main/java/org/apache/lucene/codecs/lucene90/Composite99DocValuesConsumer.java @@ -0,0 +1,68 @@ +/* + * 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.apache.lucene.codecs.lucene90; + +import org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.SegmentWriteState; + +import java.io.IOException; + +/** + * This class is an abstraction of the {@link DocValuesConsumer} for the Star Tree index structure. + * It is responsible to consume various types of document values (numeric, binary, sorted, sorted numeric, + * and sorted set) for fields in the Star Tree index. + * + * @opensearch.experimental + */ +public class Composite99DocValuesConsumer extends DocValuesConsumer { + + Lucene90DocValuesConsumer lucene90DocValuesConsumer; + + public Composite99DocValuesConsumer( + SegmentWriteState state, + String dataCodec, + String dataExtension, + String metaCodec, + String metaExtension + ) throws IOException { + lucene90DocValuesConsumer = new Lucene90DocValuesConsumer(state, dataCodec, dataExtension, metaCodec, metaExtension); + } + + @Override + public void close() throws IOException { + this.lucene90DocValuesConsumer.close(); + } + + @Override + public void addNumericField(FieldInfo fieldInfo, DocValuesProducer docValuesProducer) throws IOException { + lucene90DocValuesConsumer.addNumericField(fieldInfo, docValuesProducer); + } + + @Override + public void addBinaryField(FieldInfo fieldInfo, DocValuesProducer docValuesProducer) throws IOException { + lucene90DocValuesConsumer.addNumericField(fieldInfo, docValuesProducer); + } + + @Override + public void addSortedField(FieldInfo fieldInfo, DocValuesProducer docValuesProducer) throws IOException { + lucene90DocValuesConsumer.addSortedField(fieldInfo, docValuesProducer); + } + + @Override + public void addSortedNumericField(FieldInfo fieldInfo, DocValuesProducer docValuesProducer) throws IOException { + lucene90DocValuesConsumer.addSortedNumericField(fieldInfo, docValuesProducer); + } + + @Override + public void addSortedSetField(FieldInfo fieldInfo, DocValuesProducer docValuesProducer) throws IOException { + lucene90DocValuesConsumer.addSortedSetField(fieldInfo, docValuesProducer); + } +} diff --git a/server/src/main/java/org/apache/lucene/codecs/lucene90/StarTree99DocValuesProducer.java b/server/src/main/java/org/apache/lucene/codecs/lucene90/StarTree99DocValuesProducer.java new file mode 100644 index 0000000000000..154f4626bc472 --- /dev/null +++ b/server/src/main/java/org/apache/lucene/codecs/lucene90/StarTree99DocValuesProducer.java @@ -0,0 +1,158 @@ +/* + * 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.apache.lucene.codecs.lucene90; + +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexOptions; +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.index.VectorEncoding; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricEntry; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeHelper.fullFieldNameForStarTreeDimensionsDocValues; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeHelper.fullFieldNameForStarTreeMetricsDocValues; + +/** + * This class is a custom abstraction of the {@link DocValuesProducer} for the Star Tree index structure. + * It is responsible for providing access to various types of document values (numeric, binary, sorted, sorted numeric, + * and sorted set) for fields in the Star Tree index. + * + * @opensearch.experimental + */ +public class StarTree99DocValuesProducer extends DocValuesProducer { + + Lucene90DocValuesProducer lucene90DocValuesProducer; + private final List dimensions; + private final List metrics; + private final FieldInfos fieldInfos; + + public StarTree99DocValuesProducer( + SegmentReadState state, + String dataCodec, + String dataExtension, + String metaCodec, + String metaExtension, + List dimensions, + List metricEntries, + String compositeFieldName + ) throws IOException { + this.dimensions = dimensions; + this.metrics = metricEntries; + + // populates the dummy list of field infos to fetch doc id set iterators for respective fields. + this.fieldInfos = new FieldInfos(getFieldInfoList(compositeFieldName)); + SegmentReadState segmentReadState = new SegmentReadState(state.directory, state.segmentInfo, fieldInfos, state.context); + lucene90DocValuesProducer = new Lucene90DocValuesProducer(segmentReadState, dataCodec, dataExtension, metaCodec, metaExtension); + } + + @Override + public NumericDocValues getNumeric(FieldInfo field) throws IOException { + return this.lucene90DocValuesProducer.getNumeric(field); + } + + @Override + public BinaryDocValues getBinary(FieldInfo field) throws IOException { + return this.lucene90DocValuesProducer.getBinary(field); + } + + @Override + public SortedDocValues getSorted(FieldInfo field) throws IOException { + return this.lucene90DocValuesProducer.getSorted(field); + } + + @Override + public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { + return this.lucene90DocValuesProducer.getSortedNumeric(field); + } + + @Override + public SortedSetDocValues getSortedSet(FieldInfo field) throws IOException { + return this.lucene90DocValuesProducer.getSortedSet(field); + } + + @Override + public void checkIntegrity() throws IOException { + this.lucene90DocValuesProducer.checkIntegrity(); + } + + // returns the doc id set iterator based on field name + public SortedNumericDocValues getSortedNumeric(String fieldName) throws IOException { + return this.lucene90DocValuesProducer.getSortedNumeric(fieldInfos.fieldInfo(fieldName)); + } + + @Override + public void close() throws IOException { + this.lucene90DocValuesProducer.close(); + } + + private FieldInfo[] getFieldInfoList(String compositeFieldName) { + FieldInfo[] fieldInfoList = new FieldInfo[this.dimensions.size() + metrics.size()]; + // field number is not really used. We depend on unique field names to get the desired iterator + int fieldNumber = 0; + + for (FieldInfo dimension : this.dimensions) { + fieldInfoList[fieldNumber] = new FieldInfo( + fullFieldNameForStarTreeDimensionsDocValues(compositeFieldName, dimension.getName()), + fieldNumber, + 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 + ); + fieldNumber++; + } + for (MetricEntry metric : metrics) { + fieldInfoList[fieldNumber] = new FieldInfo( + fullFieldNameForStarTreeMetricsDocValues(compositeFieldName, metric.getMetricName(), metric.getMetricStat().getTypeName()), + fieldNumber, + 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 + ); + fieldNumber++; + } + return fieldInfoList; + } + +} diff --git a/server/src/main/java/org/apache/lucene/index/BaseStarTreeBuilder.java b/server/src/main/java/org/apache/lucene/index/BaseStarTreeBuilder.java new file mode 100644 index 0000000000000..a9f55339ca3ad --- /dev/null +++ b/server/src/main/java/org/apache/lucene/index/BaseStarTreeBuilder.java @@ -0,0 +1,818 @@ +/* + * 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.apache.lucene.index; + +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.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; +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.aggregators.MetricAggregatorInfo; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.ValueAggregator; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; +import org.opensearch.index.compositeindex.datacube.startree.builder.StarTreeBuilder; +import org.opensearch.index.compositeindex.datacube.startree.builder.StarTreesBuilder; +import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; +import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeBuilderUtils; +import org.opensearch.index.fielddata.IndexNumericFieldData; +import org.opensearch.index.mapper.Mapper; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.mapper.NumberFieldMapper; + +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.Set; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeHelper.fullFieldNameForStarTreeDimensionsDocValues; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeHelper.fullFieldNameForStarTreeMetricsDocValues; + +/** + * Builder for star tree. Defines the algorithm to construct star-tree + * See {@link StarTreesBuilder} for information around the construction of star-trees based on star-tree fields + * + * @opensearch.experimental + */ +public abstract class BaseStarTreeBuilder implements StarTreeBuilder { + + private static final Logger logger = LogManager.getLogger(BaseStarTreeBuilder.class); + + /** + * Default value for star node + */ + public static final Long STAR_IN_DOC_VALUES_INDEX = null; + + protected final Set skipStarNodeCreationForDimensions; + + protected final List metricAggregatorInfos; + protected final int numMetrics; + protected final int numDimensions; + protected int numStarTreeDocs; + protected int totalSegmentDocs; + protected int numStarTreeNodes; + protected final int maxLeafDocuments; + + protected final StarTreeBuilderUtils.TreeNode rootNode = getNewNode(); + private final StarTreeField starTreeField; + + private final MapperService mapperService; + private final SegmentWriteState writeState; + + private final IndexOutput metaOut; + private final IndexOutput dataOut; + + /** + * Builds star tree based on star tree field configuration consisting of dimensions, metrics and star tree index specific configuration. + * + * @param starTreeField holds the configuration for the star tree + * @param writeState stores the segment write writeState + * @param mapperService helps to find the original type of the field + */ + protected BaseStarTreeBuilder( + IndexOutput metaOut, + IndexOutput dataOut, + StarTreeField starTreeField, + SegmentWriteState writeState, + MapperService mapperService + ) throws IOException { + + logger.debug("Building star tree : {}", starTreeField); + + this.metaOut = metaOut; + this.dataOut = dataOut; + + this.starTreeField = starTreeField; + StarTreeFieldConfiguration starTreeFieldSpec = starTreeField.getStarTreeConfig(); + + List dimensionsSplitOrder = starTreeField.getDimensionsOrder(); + this.numDimensions = dimensionsSplitOrder.size(); + + this.skipStarNodeCreationForDimensions = new HashSet<>(); + this.totalSegmentDocs = writeState.segmentInfo.maxDoc(); + this.mapperService = mapperService; + this.writeState = writeState; + + Set skipStarNodeCreationForDimensions = starTreeFieldSpec.getSkipStarNodeCreationInDims(); + + for (int i = 0; i < numDimensions; i++) { + if (skipStarNodeCreationForDimensions.contains(dimensionsSplitOrder.get(i).getField())) { + this.skipStarNodeCreationForDimensions.add(i); + } + } + + this.metricAggregatorInfos = generateMetricAggregatorInfos(mapperService); + this.numMetrics = metricAggregatorInfos.size(); + this.maxLeafDocuments = starTreeFieldSpec.maxLeafDocs(); + } + + /** + * Generates the configuration required to perform aggregation for all the metrics on a field + * + * @return list of MetricAggregatorInfo + */ + public List generateMetricAggregatorInfos(MapperService mapperService) { + List metricAggregatorInfos = new ArrayList<>(); + for (Metric metric : this.starTreeField.getMetrics()) { + for (MetricStat metricType : metric.getMetrics()) { + IndexNumericFieldData.NumericType numericType; + Mapper fieldMapper = mapperService.documentMapper().mappers().getMapper(metric.getField()); + if (fieldMapper instanceof NumberFieldMapper) { + numericType = ((NumberFieldMapper) fieldMapper).fieldType().numericType(); + } else { + logger.error("unsupported mapper type"); + throw new IllegalStateException("unsupported mapper type"); + } + + MetricAggregatorInfo metricAggregatorInfo = new MetricAggregatorInfo( + metricType, + metric.getField(), + starTreeField.getName(), + numericType + ); + metricAggregatorInfos.add(metricAggregatorInfo); + } + } + 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 metricType : metric.getMetrics()) { + SequentialDocValuesIterator metricReader; + FieldInfo metricFieldInfo = state.fieldInfos.fieldInfo(metric.getField()); + if (metricType != MetricStat.COUNT) { + // Need not initialize the metric reader with relevant doc id set iterator for COUNT metric type + metricReader = new SequentialDocValuesIterator( + fieldProducerMap.get(metricFieldInfo.name).getSortedNumeric(metricFieldInfo) + ); + } else { + metricReader = new SequentialDocValuesIterator(); + } + + 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 + * @param starTreeDocValuesConsumer + * @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); + dimensionReaders[i] = new SequentialDocValuesIterator( + fieldProducerMap.get(dimensionFieldInfo.name).getSortedNumeric(dimensionFieldInfo) + ); + } + Iterator starTreeDocumentIterator = sortAndAggregateSegmentDocuments( + totalSegmentDocs, + 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 + * @param starTreeDocValuesConsumer + * @throws IOException when we are unable to build star-tree + */ + public void build( + Iterator starTreeDocumentIterator, + AtomicInteger fieldNumberAcrossStarTrees, + DocValuesConsumer starTreeDocValuesConsumer + ) 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) { + // serialize the star tree data + serializeStarTree(numSegmentStarTreeDocument); + 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(numSegmentStarTreeDocument); + } + + private void serializeStarTree(int numSegmentStarTreeDocument) throws IOException { + // serialize the star tree data + long dataFilePointer = dataOut.getFilePointer(); + long totalStarTreeDataLength = StarTreeBuilderUtils.serializeStarTree(dataOut, rootNode, numStarTreeNodes); + + // serialize the star tree meta + StarTreeBuilderUtils.serializeStarTreeMetadata( + metaOut, + starTreeField, + writeState, + metricAggregatorInfos, + numSegmentStarTreeDocument, + 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 = new FieldInfo( + fullFieldNameForStarTreeDimensionsDocValues(starTreeField.getName(), starTreeField.getDimensionsOrder().get(i).getField()), + fieldNumberAcrossStarTrees.incrementAndGet(), + 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 + ); + dimensionFieldInfoList[i] = fi; + dimensionWriters.add(new SortedNumericDocValuesWriter(fi, Counter.newCounter())); + } + for (int i = 0; i < metricAggregatorInfos.size(); i++) { + FieldInfo fi = new FieldInfo( + fullFieldNameForStarTreeMetricsDocValues( + starTreeField.getName(), + metricAggregatorInfos.get(i).getField(), + metricAggregatorInfos.get(i).getMetricStat().getTypeName() + ), + fieldNumberAcrossStarTrees.incrementAndGet(), + 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 + ); + metricFieldInfoList[i] = fi; + metricWriters.add(new SortedNumericDocValuesWriter(fi, Counter.newCounter())); + } + + for (int docId = 0; docId < numStarTreeDocs; docId++) { + StarTreeDocument starTreeDocument = getStarTreeDocument(docId); + for (int i = 0; i < starTreeDocument.dimensions.length; i++) { + Long val = starTreeDocument.dimensions[i]; + if (val != null) { + dimensionWriters.get(i).addValue(docId, val); + } + } + + for (int i = 0; i < starTreeDocument.metrics.length; i++) { + try { + switch (metricAggregatorInfos.get(i).getValueAggregators().getAggregatedValueType()) { + case LONG: + metricWriters.get(i).addValue(docId, (Long) starTreeDocument.metrics[i]); + break; + case DOUBLE: + 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.info("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 increment = i; + DocValuesProducer docValuesProducer = new EmptyDocValuesProducer() { + @Override + public SortedNumericDocValues getSortedNumeric(FieldInfo field) { + return docValuesWriters.get(increment).getDocValues(); + } + }; + docValuesConsumer.addSortedNumericField(fieldInfoList[i], docValuesProducer); + } + } + + /** + * Adds a document to the star-tree. + * + * @param starTreeDocument star tree document to be added + * @throws IOException if an I/O error occurs while adding the document + */ + public abstract void appendStarTreeDocument(StarTreeDocument starTreeDocument) throws IOException; + + /** + * Returns the document of the given document id in the star-tree. + * + * @param docId document id + * @return star tree document + * @throws IOException if an I/O error occurs while fetching the star-tree document + */ + public abstract StarTreeDocument getStarTreeDocument(int docId) throws IOException; + + /** + * Retrieves the list of star-tree documents in the star-tree. + * + * @return Star tree documents + */ + public abstract List getStarTreeDocuments(); + + /** + * Returns the value of the dimension for the given dimension id and document in the star-tree. + * + * @param docId document id + * @param dimensionId dimension id + * @return dimension value + */ + public abstract Long getDimensionValue(int docId, int dimensionId) throws IOException; + + /** + * Sorts and aggregates all the documents in the segment as per the configuration, and returns a star-tree document iterator for all the + * aggregated star-tree documents. + * + * @param numDocs number of documents in the given segment + * @param dimensionReaders List of docValues readers to read dimensions from the segment + * @param metricReaders List of docValues readers to read metrics from the segment + * @return Iterator for the aggregated star-tree document + */ + public abstract Iterator sortAndAggregateSegmentDocuments( + int numDocs, + SequentialDocValuesIterator[] dimensionReaders, + List metricReaders + ) throws IOException; + + /** + * Generates aggregated star-tree documents for star-node. This inserts null for the star-node dimension and generates aggregated documents based on the remaining dimensions + * + * @param startDocId start document id (inclusive) in the star-tree + * @param endDocId end document id (exclusive) in the star-tree + * @param dimensionId dimension id of the star-node + * @return Iterator for the aggregated star-tree documents + */ + public abstract Iterator generateStarTreeDocumentsForStarNode(int startDocId, int endDocId, int dimensionId) + throws IOException; + + protected StarTreeDocument getSegmentStarTreeDocument( + int currentDocId, + SequentialDocValuesIterator[] dimensionReaders, + List metricReaders + ) throws IOException { + Long[] dimensions = getStarTreeDimensionsFromSegment(currentDocId, dimensionReaders); + Object[] metrics = getStarTreeMetricsFromSegment(currentDocId, metricReaders); + return new StarTreeDocument(dimensions, metrics); + } + + /** + * Returns the dimension values for the next document from the segment + * + * @return dimension values for each of the star-tree dimension + * @throws IOException when we are unable to iterate to the next doc for the given dimension readers + */ + Long[] getStarTreeDimensionsFromSegment(int currentDocId, SequentialDocValuesIterator[] dimensionReaders) throws IOException { + Long[] dimensions = new Long[numDimensions]; + for (int i = 0; i < numDimensions; i++) { + if (dimensionReaders[i] != null) { + try { + dimensionReaders[i].nextDoc(currentDocId); + } catch (IOException e) { + logger.error("unable to iterate to next doc", e); + throw new RuntimeException("unable to iterate to next doc", e); + } catch (Exception e) { + logger.error("unable to read the dimension values from the segment", e); + throw new IllegalStateException("unable to read the dimension values from the segment", e); + } + dimensions[i] = dimensionReaders[i].value(currentDocId); + } else { + throw new IllegalStateException("dimension readers are empty"); + } + } + return dimensions; + } + + /** + * Returns the metric values for the next document from the segment + * + * @return metric values for each of the star-tree metric + * @throws IOException when we are unable to iterate to the next doc for the given metric readers + */ + private Object[] getStarTreeMetricsFromSegment(int currentDocId, List metricsReaders) throws IOException { + Object[] metrics = new Object[numMetrics]; + for (int i = 0; i < numMetrics; i++) { + SequentialDocValuesIterator metricStatReader = metricsReaders.get(i); + if (metricStatReader != null) { + try { + metricStatReader.nextDoc(currentDocId); + } catch (IOException e) { + logger.error("unable to iterate to next doc", e); + throw new RuntimeException("unable to iterate to next doc", e); + } catch (Exception e) { + logger.error("unable to read the metric values from the segment", e); + throw new IllegalStateException("unable to read the metric values from the segment", e); + } + metrics[i] = metricStatReader.value(currentDocId); + } else { + throw new IllegalStateException("metric readers are empty"); + } + } + return metrics; + } + + /** + * Merges a star-tree document from the segment into an aggregated star-tree document. + * A new aggregated star-tree document is created if the aggregated segment document is null. + * + * @param aggregatedSegmentDocument aggregated star-tree document + * @param segmentDocument segment star-tree document + * @return merged star-tree document + */ + protected StarTreeDocument reduceSegmentStarTreeDocuments( + StarTreeDocument aggregatedSegmentDocument, + StarTreeDocument segmentDocument + ) { + if (aggregatedSegmentDocument == null) { + Long[] dimensions = Arrays.copyOf(segmentDocument.dimensions, numDimensions); + Object[] metrics = new Object[numMetrics]; + for (int i = 0; i < numMetrics; i++) { + try { + ValueAggregator metricValueAggregator = metricAggregatorInfos.get(i).getValueAggregators(); + StarTreeNumericType starTreeNumericType = metricAggregatorInfos.get(i).getAggregatedValueType(); + metrics[i] = metricValueAggregator.getInitialAggregatedValueForSegmentDocValue( + getLong(segmentDocument.metrics[i]), + starTreeNumericType + ); + } catch (Exception e) { + logger.error("Cannot parse initial segment doc value", e); + throw new IllegalStateException("Cannot parse initial segment doc value [" + segmentDocument.metrics[i] + "]"); + } + } + return new StarTreeDocument(dimensions, metrics); + } else { + for (int i = 0; i < numMetrics; i++) { + try { + ValueAggregator metricValueAggregator = metricAggregatorInfos.get(i).getValueAggregators(); + StarTreeNumericType starTreeNumericType = metricAggregatorInfos.get(i).getAggregatedValueType(); + aggregatedSegmentDocument.metrics[i] = metricValueAggregator.mergeAggregatedValueAndSegmentValue( + aggregatedSegmentDocument.metrics[i], + getLong(segmentDocument.metrics[i]), + starTreeNumericType + ); + } catch (Exception e) { + logger.error("Cannot apply segment doc value for aggregation", e); + throw new IllegalStateException("Cannot apply segment doc value for aggregation [" + segmentDocument.metrics[i] + "]"); + } + } + return aggregatedSegmentDocument; + } + } + + /** + * Safely converts the metric value of object type to long. + * + * @param metric value of the metric + * @return converted metric value to long + */ + private static long getLong(Object metric) { + + Long metricValue = null; + try { + if (metric instanceof Long) { + metricValue = (long) metric; + } else if (metric != null) { + metricValue = Long.valueOf(String.valueOf(metric)); + } + } catch (Exception e) { + throw new IllegalStateException("unable to cast segment metric", e); + } + + if (metricValue == null) { + throw new IllegalStateException("unable to cast segment metric"); + } + return metricValue; + } + + /** + * Merges a star-tree document into an aggregated star-tree document. + * A new aggregated star-tree document is created if the aggregated document is null. + * + * @param aggregatedDocument aggregated star-tree document + * @param starTreeDocument segment star-tree document + * @return merged star-tree document + */ + public StarTreeDocument reduceStarTreeDocuments(StarTreeDocument aggregatedDocument, StarTreeDocument starTreeDocument) { + // aggregate the documents + if (aggregatedDocument == null) { + Long[] dimensions = Arrays.copyOf(starTreeDocument.dimensions, numDimensions); + Object[] metrics = new Object[numMetrics]; + for (int i = 0; i < numMetrics; i++) { + try { + metrics[i] = metricAggregatorInfos.get(i).getValueAggregators().getInitialAggregatedValue(starTreeDocument.metrics[i]); + } catch (Exception e) { + logger.error("Cannot get value for aggregation", e); + throw new IllegalStateException("Cannot get value for aggregation[" + starTreeDocument.metrics[i] + "]"); + } + } + return new StarTreeDocument(dimensions, metrics); + } else { + for (int i = 0; i < numMetrics; i++) { + try { + aggregatedDocument.metrics[i] = metricAggregatorInfos.get(i) + .getValueAggregators() + .mergeAggregatedValues(starTreeDocument.metrics[i], aggregatedDocument.metrics[i]); + } catch (Exception e) { + logger.error("Cannot apply value to aggregated document for aggregation", e); + throw new IllegalStateException( + "Cannot apply value to aggregated document for aggregation [" + starTreeDocument.metrics[i] + "]" + ); + } + } + return aggregatedDocument; + } + } + + /** + * Adds a document to star-tree + * + * @param starTreeDocument star-tree document + * @throws IOException throws an exception if we are unable to add the doc + */ + private void appendToStarTree(StarTreeDocument starTreeDocument) throws IOException { + appendStarTreeDocument(starTreeDocument); + numStarTreeDocs++; + } + + /** + * Returns a new star-tree node + * + * @return return new star-tree node + */ + private StarTreeBuilderUtils.TreeNode getNewNode() { + numStarTreeNodes++; + return new StarTreeBuilderUtils.TreeNode(); + } + + /** + * Implements the algorithm to construct a star-tree + * + * @param node star-tree node + * @param startDocId start document id + * @param endDocId end document id + * @throws IOException throws an exception if we are unable to construct the tree + */ + private void constructStarTree(StarTreeBuilderUtils.TreeNode node, int startDocId, int endDocId) throws IOException { + + int childDimensionId = node.dimensionId + 1; + if (childDimensionId == numDimensions) { + return; + } + + // Construct all non-star children nodes + node.childDimensionId = childDimensionId; + Map children = constructNonStarNodes(startDocId, endDocId, childDimensionId); + node.children = children; + + // Construct star-node if required + if (!skipStarNodeCreationForDimensions.contains(childDimensionId) && children.size() > 1) { + children.put((long) StarTreeBuilderUtils.ALL, constructStarNode(startDocId, endDocId, childDimensionId)); + } + + // Further split on child nodes if required + for (StarTreeBuilderUtils.TreeNode child : children.values()) { + if (child.endDocId - child.startDocId > maxLeafDocuments) { + constructStarTree(child, child.startDocId, child.endDocId); + } + } + } + + /** + * Constructs non star tree nodes + * + * @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<>(); + int nodeStartDocId = startDocId; + Long nodeDimensionValue = getDimensionValue(startDocId, dimensionId); + for (int i = startDocId + 1; i < endDocId; i++) { + Long dimensionValue = getDimensionValue(i, dimensionId); + if (!dimensionValue.equals(nodeDimensionValue)) { + StarTreeBuilderUtils.TreeNode child = getNewNode(); + child.dimensionId = dimensionId; + child.dimensionValue = nodeDimensionValue; + child.startDocId = nodeStartDocId; + child.endDocId = i; + nodes.put(nodeDimensionValue, child); + + nodeStartDocId = i; + nodeDimensionValue = dimensionValue; + } + } + StarTreeBuilderUtils.TreeNode lastNode = getNewNode(); + lastNode.dimensionId = dimensionId; + lastNode.dimensionValue = nodeDimensionValue; + lastNode.startDocId = nodeStartDocId; + lastNode.endDocId = endDocId; + nodes.put(nodeDimensionValue, lastNode); + return nodes; + } + + /** + * Constructs star tree nodes + * + * @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 star nodes constructed + * @throws IOException throws an exception if we are unable to construct non-star nodes + */ + private StarTreeBuilderUtils.TreeNode constructStarNode(int startDocId, int endDocId, int dimensionId) throws IOException { + StarTreeBuilderUtils.TreeNode starNode = getNewNode(); + starNode.dimensionId = dimensionId; + starNode.dimensionValue = StarTreeBuilderUtils.ALL; + starNode.isStarNode = true; + starNode.startDocId = numStarTreeDocs; + Iterator starTreeDocumentIterator = generateStarTreeDocumentsForStarNode(startDocId, endDocId, dimensionId); + while (starTreeDocumentIterator.hasNext()) { + appendToStarTree(starTreeDocumentIterator.next()); + } + starNode.endDocId = numStarTreeDocs; + return starNode; + } + + /** + * Returns aggregated star-tree document + * + * @param node star-tree node + * @return aggregated star-tree documents + * @throws IOException throws an exception upon failing to create new aggregated docs based on star tree + */ + private StarTreeDocument createAggregatedDocs(StarTreeBuilderUtils.TreeNode node) throws IOException { + StarTreeDocument aggregatedStarTreeDocument = null; + if (node.children == null) { + // For leaf node + + if (node.startDocId == node.endDocId - 1) { + // If it has only one document, use it as the aggregated document + aggregatedStarTreeDocument = getStarTreeDocument(node.startDocId); + node.aggregatedDocId = node.startDocId; + } else { + // If it has multiple documents, aggregate all of them + for (int i = node.startDocId; i < node.endDocId; 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++) { + aggregatedStarTreeDocument.dimensions[i] = STAR_IN_DOC_VALUES_INDEX; + } + node.aggregatedDocId = numStarTreeDocs; + appendToStarTree(aggregatedStarTreeDocument); + } + } else { + // For non-leaf node + if (node.children.containsKey((long) StarTreeBuilderUtils.ALL)) { + // If it has star child, use the star child aggregated document directly + for (StarTreeBuilderUtils.TreeNode child : node.children.values()) { + if (child.isStarNode) { + aggregatedStarTreeDocument = createAggregatedDocs(child); + node.aggregatedDocId = child.aggregatedDocId; + } else { + createAggregatedDocs(child); + } + } + } else { + // If no star child exists, aggregate all aggregated documents from non-star children + for (StarTreeBuilderUtils.TreeNode child : node.children.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++) { + aggregatedStarTreeDocument.dimensions[i] = STAR_IN_DOC_VALUES_INDEX; + } + node.aggregatedDocId = numStarTreeDocs; + appendToStarTree(aggregatedStarTreeDocument); + } + } + return aggregatedStarTreeDocument; + } + + /** + * Handles the dimension of date time field type + * + * @param fieldName name of the field + * @param val value of the field + * @return returns the converted dimension of the field to a particular granularity + */ + private long handleDateDimension(final String fieldName, final long val) { + // TODO: handle timestamp granularity + return val; + } + + public void close() throws IOException { + + } + +} diff --git a/server/src/main/java/org/opensearch/index/codec/composite/Composite99Codec.java b/server/src/main/java/org/opensearch/index/codec/composite/Composite99Codec.java index de04944e67cd2..2924133970bc5 100644 --- a/server/src/main/java/org/opensearch/index/codec/composite/Composite99Codec.java +++ b/server/src/main/java/org/opensearch/index/codec/composite/Composite99Codec.java @@ -28,7 +28,6 @@ public class Composite99Codec extends FilterCodec { public static final String COMPOSITE_INDEX_CODEC_NAME = "Composite99Codec"; private final MapperService mapperService; - // needed for SPI - this is used in reader path public Composite99Codec() { this(COMPOSITE_INDEX_CODEC_NAME, new Lucene99Codec(), null); } diff --git a/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesFormat.java b/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesFormat.java index 216ed4f68f333..3782cf328c9c1 100644 --- a/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesFormat.java +++ b/server/src/main/java/org/opensearch/index/codec/composite/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 = "sttd"; + + /** Filename extension for the composite index meta */ + public static final String META_EXTENSION = "sttm"; + + /** 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 */ + 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 = "sttddvm"; + + /** Filename extension for the composite index meta doc values */ + public static final String META_DOC_VALUES_EXTENSION = "sttmdvm"; + + /** 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/Composite99DocValuesReader.java b/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesReader.java index 82c844088cfd4..031e74d97a481 100644 --- a/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesReader.java +++ b/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesReader.java @@ -8,19 +8,47 @@ package org.opensearch.index.codec.composite; +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.codecs.lucene90.StarTree99DocValuesProducer; import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.FieldInfo; +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.search.DocIdSetIterator; +import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.lucene.store.IndexInput; import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.index.mapper.CompositeMappedFieldType; +import org.opensearch.common.util.io.IOUtils; +import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; +import org.opensearch.index.compositeindex.CompositeIndexMetadata; +import org.opensearch.index.compositeindex.datacube.Dimension; +import org.opensearch.index.compositeindex.datacube.MergeDimension; +import org.opensearch.index.compositeindex.datacube.Metric; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricEntry; +import org.opensearch.index.compositeindex.datacube.startree.meta.StarTreeMetadata; +import org.opensearch.index.compositeindex.datacube.startree.node.OffHeapStarTree; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTree; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNode; +import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeHelper; import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; /** * Reader for star tree index and star tree doc values from the segments @@ -29,11 +57,113 @@ */ @ExperimentalApi public class Composite99DocValuesReader extends DocValuesProducer implements CompositeIndexReader { - private DocValuesProducer delegate; + private static final Logger logger = LogManager.getLogger(CompositeIndexMetadata.class); - public Composite99DocValuesReader(DocValuesProducer producer, SegmentReadState state) throws IOException { + private final DocValuesProducer delegate; + private final IndexInput dataIn; + private final ChecksumIndexInput metaIn; + private final Map starTreeMap = new LinkedHashMap<>(); + private final Map compositeIndexMetadataMap = new LinkedHashMap<>(); + private final Map compositeDocValuesProducerMap = new LinkedHashMap<>(); + private final List compositeFieldInfos = new ArrayList<>(); + private final SegmentReadState readState; + + public Composite99DocValuesReader(DocValuesProducer producer, SegmentReadState readState) throws IOException { this.delegate = producer; - // TODO : read star tree files + this.readState = readState; + + 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 { + + 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 + ); + + 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) { + long magicMarker = metaIn.readLong(); + + if (magicMarker == -1) { + logger.info("EOF reached for composite index metadata"); + break; + } else if (magicMarker < 0) { + throw new CorruptIndexException("Unknown token encountered: " + magicMarker, metaIn); + } + CompositeIndexMetadata compositeIndexMetadata = new CompositeIndexMetadata(metaIn, magicMarker); + String compositeFieldName = compositeIndexMetadata.getCompositeFieldName(); + compositeFieldInfos.add( + new CompositeIndexFieldInfo(compositeFieldName, compositeIndexMetadata.getCompositeFieldType()) + ); + switch (compositeIndexMetadata.getCompositeFieldType()) { + case STAR_TREE: + StarTreeMetadata starTreeMetadata = compositeIndexMetadata.getStarTreeMetadata(); + StarTree starTree = new OffHeapStarTree(dataIn, starTreeMetadata); + starTreeMap.put(compositeFieldName, starTree); + compositeIndexMetadataMap.put(compositeFieldName, compositeIndexMetadata); + + List dimensionFieldNumbers = starTreeMetadata.getDimensionFieldNumbers(); + List dimensions = new ArrayList<>(); + for (Integer fieldNumber : dimensionFieldNumbers) { + dimensions.add(readState.fieldInfos.fieldInfo(fieldNumber)); + } + + StarTree99DocValuesProducer starTreeDocValuesProducer = new StarTree99DocValuesProducer( + readState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION, + dimensions, + starTreeMetadata.getMetricEntries(), + compositeFieldName + ); + compositeDocValuesProducerMap.put(compositeFieldName, starTreeDocValuesProducer); + + break; + default: + throw new CorruptIndexException("Invalid composite field type found in the file", dataIn); + } + } + } catch (Throwable t) { + priorE = t; + } finally { + CodecUtil.checkFooter(metaIn, priorE); + } + success = true; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); + } + } } @Override @@ -64,26 +194,94 @@ 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(metaIn); + CodecUtil.checksumEntireFile(dataIn); } @Override public void close() throws IOException { delegate.close(); - // Todo: close composite index related files [star tree] files + starTreeMap.clear(); + compositeIndexMetadataMap.clear(); + compositeDocValuesProducerMap.clear(); } @Override - public List getCompositeIndexFields() { - // todo : read from file formats and get the field names. - throw new UnsupportedOperationException(); + public List getCompositeIndexFields() { + return compositeFieldInfos; } @Override - public CompositeIndexValues getCompositeIndexValues(String field, CompositeMappedFieldType.CompositeFieldType fieldType) - throws IOException { - // TODO : read compositeIndexValues [starTreeValues] from star tree files - throw new UnsupportedOperationException(); + public CompositeIndexValues getCompositeIndexValues(CompositeIndexFieldInfo compositeIndexFieldInfo) throws IOException { + + switch (compositeIndexFieldInfo.getType()) { + case STAR_TREE: + CompositeIndexMetadata compositeIndexMetadata = compositeIndexMetadataMap.get(compositeIndexFieldInfo.getField()); + StarTreeMetadata starTreeMetadata = compositeIndexMetadata.getStarTreeMetadata(); + Set skipStarNodeCreationInDimsFieldNumbers = starTreeMetadata.getSkipStarNodeCreationInDims(); + Set skipStarNodeCreationInDims = new HashSet<>(); + for (Integer fieldNumber : skipStarNodeCreationInDimsFieldNumbers) { + skipStarNodeCreationInDims.add(readState.fieldInfos.fieldInfo(fieldNumber).getName()); + } + + List dimensionFieldNumbers = starTreeMetadata.getDimensionFieldNumbers(); + List dimensions = new ArrayList<>(); + List mergeDimensions = new ArrayList<>(); + for (Integer fieldNumber : dimensionFieldNumbers) { + dimensions.add(readState.fieldInfos.fieldInfo(fieldNumber).getName()); + mergeDimensions.add(new MergeDimension(readState.fieldInfos.fieldInfo(fieldNumber).name)); + } + + Map starTreeMetricMap = new ConcurrentHashMap<>(); + for (MetricEntry metricEntry : starTreeMetadata.getMetricEntries()) { + String metricName = metricEntry.getMetricName(); + + Metric metric = starTreeMetricMap.computeIfAbsent(metricName, field -> new Metric(field, new ArrayList<>())); + metric.getMetrics().add(metricEntry.getMetricStat()); + } + List starTreeMetrics = new ArrayList<>(starTreeMetricMap.values()); + + StarTreeField starTreeField = new StarTreeField( + compositeIndexMetadata.getCompositeFieldName(), + mergeDimensions, + starTreeMetrics, + new StarTreeFieldConfiguration( + starTreeMetadata.getMaxLeafDocs(), + skipStarNodeCreationInDims, + starTreeMetadata.getStarTreeBuildMode() + ) + ); + StarTreeNode rootNode = starTreeMap.get(compositeIndexFieldInfo.getField()).getRoot(); + StarTree99DocValuesProducer starTree99DocValuesProducer = (StarTree99DocValuesProducer) compositeDocValuesProducerMap.get( + compositeIndexMetadata.getCompositeFieldName() + ); + Map dimensionsDocIdSetIteratorMap = new LinkedHashMap<>(); + Map metricsDocIdSetIteratorMap = new LinkedHashMap<>(); + + for (String dimension : dimensions) { + dimensionsDocIdSetIteratorMap.put( + dimension, + starTree99DocValuesProducer.getSortedNumeric( + StarTreeHelper.fullFieldNameForStarTreeDimensionsDocValues(starTreeField.getName(), dimension) + ) + ); + } + + for (MetricEntry metricEntry : starTreeMetadata.getMetricEntries()) { + String metricFullName = StarTreeHelper.fullFieldNameForStarTreeMetricsDocValues( + starTreeField.getName(), + metricEntry.getMetricName(), + metricEntry.getMetricStat().getTypeName() + ); + metricsDocIdSetIteratorMap.put(metricFullName, starTree99DocValuesProducer.getSortedNumeric(metricFullName)); + } + + return new StarTreeValues(starTreeField, rootNode, dimensionsDocIdSetIteratorMap, metricsDocIdSetIteratorMap); + + default: + throw new CorruptIndexException("Unsupported composite index field type: ", compositeIndexFieldInfo.getType().getName()); + } + } } diff --git a/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesWriter.java b/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesWriter.java index 75bbf78dbdad2..3c87b37638017 100644 --- a/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesWriter.java +++ b/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesWriter.java @@ -8,22 +8,32 @@ package org.opensearch.index.codec.composite; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.codecs.lucene90.Composite99DocValuesConsumer; import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.MergeState; import org.apache.lucene.index.SegmentWriteState; +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.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.builder.StarTreesBuilder; import org.opensearch.index.mapper.CompositeMappedFieldType; import org.opensearch.index.mapper.MapperService; -import org.opensearch.index.mapper.StarTreeMapper; import java.io.IOException; +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.AtomicReference; /** * This class write the star tree index and star tree doc values @@ -36,13 +46,56 @@ public class Composite99DocValuesWriter extends DocValuesConsumer { private final DocValuesConsumer delegate; private final SegmentWriteState state; private final MapperService mapperService; - AtomicReference mergeState = new AtomicReference<>(); + private MergeState mergeState = null; private final Set compositeMappedFieldTypes; private final Set compositeFieldSet; + private final DocValuesConsumer composite99DocValuesConsumer; + + public IndexOutput dataOut; + public IndexOutput metaOut; private final Map fieldProducerMap = new HashMap<>(); + private static final Logger logger = LogManager.getLogger(Composite99DocValuesWriter.class); + + public Composite99DocValuesWriter(DocValuesConsumer delegate, SegmentWriteState segmentWriteState, MapperService mapperService) + throws IOException { + + boolean success = false; + try { + String dataFileName = IndexFileNames.segmentFileName( + segmentWriteState.segmentInfo.name, + segmentWriteState.segmentSuffix, + Composite99DocValuesFormat.DATA_EXTENSION + ); + dataOut = segmentWriteState.directory.createOutput(dataFileName, segmentWriteState.context); + CodecUtil.writeIndexHeader( + dataOut, + Composite99DocValuesFormat.DATA_CODEC_NAME, + Composite99DocValuesFormat.VERSION_CURRENT, + segmentWriteState.segmentInfo.getId(), + segmentWriteState.segmentSuffix + ); - public Composite99DocValuesWriter(DocValuesConsumer delegate, SegmentWriteState segmentWriteState, MapperService mapperService) { + String metaFileName = IndexFileNames.segmentFileName( + segmentWriteState.segmentInfo.name, + segmentWriteState.segmentSuffix, + Composite99DocValuesFormat.META_EXTENSION + ); + metaOut = segmentWriteState.directory.createOutput(metaFileName, segmentWriteState.context); + CodecUtil.writeIndexHeader( + metaOut, + Composite99DocValuesFormat.META_CODEC_NAME, + Composite99DocValuesFormat.VERSION_CURRENT, + segmentWriteState.segmentInfo.getId(), + segmentWriteState.segmentSuffix + ); + + success = true; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(this); + } + } this.delegate = delegate; this.state = segmentWriteState; @@ -52,6 +105,13 @@ public Composite99DocValuesWriter(DocValuesConsumer delegate, SegmentWriteState for (CompositeMappedFieldType type : compositeMappedFieldTypes) { compositeFieldSet.addAll(type.fields()); } + this.composite99DocValuesConsumer = new Composite99DocValuesConsumer( + segmentWriteState, + Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC, + Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION, + Composite99DocValuesFormat.META_DOC_VALUES_CODEC, + Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION + ); } @Override @@ -73,7 +133,7 @@ public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) th public void addSortedNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { delegate.addSortedNumericField(field, valuesProducer); // Perform this only during flush flow - if (mergeState.get() == null) { + if (mergeState == null) { createCompositeIndicesIfPossible(valuesProducer, field); } } @@ -86,6 +146,24 @@ 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); + } else { + IOUtils.closeWhileHandlingException(dataOut, metaOut); + } + metaOut = dataOut = null; + } } private void createCompositeIndicesIfPossible(DocValuesProducer valuesProducer, FieldInfo field) throws IOException { @@ -97,8 +175,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 instanceof StarTreeMapper.StarTreeFieldType) { - // TODO : Call StarTree builder + if (mappedType.getCompositeIndexType().equals(CompositeMappedFieldType.CompositeFieldType.STAR_TREE)) { + StarTreesBuilder starTreesBuilder = new StarTreesBuilder(state, mapperService); + starTreesBuilder.build(metaOut, dataOut, fieldProducerMap, composite99DocValuesConsumer); } } } @@ -106,9 +185,62 @@ private void createCompositeIndicesIfPossible(DocValuesProducer valuesProducer, @Override public void merge(MergeState mergeState) throws IOException { - this.mergeState.compareAndSet(null, mergeState); + // TODO : check if class variable will cause concurrency issues + this.mergeState = mergeState; super.merge(mergeState); - // TODO : handle merge star tree - // mergeStarTreeFields(mergeState); + mergeCompositeFields(mergeState); + } + + /** + * Merges composite fields from multiple segments + * + * @param mergeState merge state + */ + private void mergeCompositeFields(MergeState mergeState) throws IOException { + mergeStarTreeFields(mergeState); + } + + /** + * Merges star tree data fields from multiple segments + * + * @param mergeState merge state + */ + private void mergeStarTreeFields(MergeState mergeState) throws IOException { + Map> starTreeSubsPerField = new HashMap<>(); + Map starTreeFieldMap = new HashMap<>(); + for (int i = 0; i < mergeState.docValuesProducers.length; i++) { + CompositeIndexReader reader = null; + if (mergeState.docValuesProducers[i] == null) { + continue; + } + if (mergeState.docValuesProducers[i] instanceof CompositeIndexReader) { + reader = (CompositeIndexReader) mergeState.docValuesProducers[i]; + } else { + continue; + } + + List compositeFieldInfo = reader.getCompositeIndexFields(); + for (CompositeIndexFieldInfo fieldInfo : compositeFieldInfo) { + if (fieldInfo.getType().equals(CompositeMappedFieldType.CompositeFieldType.STAR_TREE)) { + CompositeIndexValues compositeIndexValues = reader.getCompositeIndexValues(fieldInfo); + if (compositeIndexValues instanceof StarTreeValues) { + List fieldsList = starTreeSubsPerField.getOrDefault(fieldInfo.getField(), Collections.emptyList()); + if (!starTreeFieldMap.containsKey(fieldInfo.getField())) { + starTreeFieldMap.put(fieldInfo.getField(), ((StarTreeValues) compositeIndexValues).getStarTreeField()); + } + // assert star tree configuration is same across segments + else { + assert starTreeFieldMap.get(fieldInfo.getField()) + .equals(((StarTreeValues) compositeIndexValues).getStarTreeField()); + logger.error("Star tree configuration is not same for segments during merge"); + } + fieldsList.add((StarTreeValues) compositeIndexValues); + starTreeSubsPerField.put(fieldInfo.getField(), fieldsList); + } + } + } + } + final StarTreesBuilder starTreesBuilder = new StarTreesBuilder(state, mapperService); + starTreesBuilder.buildDuringMerge(metaOut, dataOut, starTreeFieldMap, starTreeSubsPerField, composite99DocValuesConsumer); } } diff --git a/server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexFieldInfo.java b/server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexFieldInfo.java new file mode 100644 index 0000000000000..4ac5bcf1b605f --- /dev/null +++ b/server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexFieldInfo.java @@ -0,0 +1,36 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.codec.composite; + +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.mapper.CompositeMappedFieldType; + +/** + * Field info details of composite index fields + * + * @opensearch.experimental + */ +@ExperimentalApi +public class CompositeIndexFieldInfo { + private final String field; + private final CompositeMappedFieldType.CompositeFieldType type; + + public CompositeIndexFieldInfo(String field, CompositeMappedFieldType.CompositeFieldType type) { + this.field = field; + this.type = type; + } + + public String getField() { + return field; + } + + public CompositeMappedFieldType.CompositeFieldType getType() { + return type; + } +} 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 d02438b75377d..a159b0619bcbb 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,7 +9,6 @@ package org.opensearch.index.codec.composite; import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.index.mapper.CompositeMappedFieldType; import java.io.IOException; import java.util.List; @@ -25,10 +24,10 @@ public interface CompositeIndexReader { * Get list of composite index fields from the segment * */ - List getCompositeIndexFields(); + List getCompositeIndexFields(); /** * Get composite index values based on the field name and the field type */ - CompositeIndexValues getCompositeIndexValues(String field, CompositeMappedFieldType.CompositeFieldType fieldType) throws IOException; + CompositeIndexValues getCompositeIndexValues(CompositeIndexFieldInfo fieldInfo) throws IOException; } 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 index 2a5b96ce2620a..baed90273d311 100644 --- 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 @@ -8,10 +8,13 @@ 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.List; +import java.util.Map; /** * Concrete class that holds the star tree associated values from the segment @@ -20,16 +23,41 @@ */ @ExperimentalApi public class StarTreeValues implements CompositeIndexValues { - private final List dimensionsOrder; + private final StarTreeField starTreeField; + private final StarTreeNode root; + private final Map dimensionDocValuesIteratorMap; + private final Map metricDocValuesIteratorMap; - // TODO : come up with full set of vales such as dimensions and metrics doc values + star tree - public StarTreeValues(List dimensionsOrder) { - super(); - this.dimensionsOrder = List.copyOf(dimensionsOrder); + public StarTreeValues( + StarTreeField starTreeField, + StarTreeNode root, + Map dimensionDocValuesIteratorMap, + Map metricDocValuesIteratorMap + ) { + this.starTreeField = starTreeField; + this.root = root; + this.dimensionDocValuesIteratorMap = dimensionDocValuesIteratorMap; + this.metricDocValuesIteratorMap = metricDocValuesIteratorMap; } @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; + } } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexConstants.java b/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexConstants.java new file mode 100644 index 0000000000000..defb4a26de260 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexConstants.java @@ -0,0 +1,26 @@ +/* + * 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; + +/** + * This class contains constants used in the Composite Index implementation. + */ +public class CompositeIndexConstants { + + /** + * The magic marker value used for sanity checks in the Composite Index implementation. + */ + public static final long MAGIC_MARKER = 0xC0950513F1E1DL; // Composite Field + + /** + * The version of the Composite Index implementation. + */ + public static final int VERSION = 1; + +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexMetadata.java b/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexMetadata.java new file mode 100644 index 0000000000000..a3e3bbc687566 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/CompositeIndexMetadata.java @@ -0,0 +1,95 @@ +/* + * 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; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.store.IndexInput; +import org.opensearch.index.compositeindex.datacube.startree.meta.StarTreeMetadata; +import org.opensearch.index.mapper.CompositeMappedFieldType; + +import java.io.IOException; + +import static org.opensearch.index.compositeindex.CompositeIndexConstants.MAGIC_MARKER; +import static org.opensearch.index.compositeindex.CompositeIndexConstants.VERSION; + +/** + * This class represents the metadata of a Composite Index, which includes information about + * the composite field name, type, and the specific metadata for the type of composite field + * (e.g., StarTree metadata). + * + * @opensearch.experimental + */ +public class CompositeIndexMetadata { + + private static final Logger logger = LogManager.getLogger(CompositeIndexMetadata.class); + private final String compositeFieldName; + private final CompositeMappedFieldType.CompositeFieldType compositeFieldType; + private final StarTreeMetadata starTreeMetadata; + + /** + * Constructs a CompositeIndexMetadata object from the provided IndexInput and magic marker. + * + * @param meta the IndexInput containing the metadata + * @param magicMarker the magic marker value + * @throws IOException if an I/O error occurs while reading the metadata + */ + public CompositeIndexMetadata(IndexInput meta, long magicMarker) throws IOException { + if (MAGIC_MARKER != magicMarker) { + logger.error("Invalid composite field magic marker"); + throw new IOException("Invalid composite field magic marker"); + } + int version = meta.readInt(); + if (VERSION != version) { + logger.error("Invalid composite field version"); + throw new IOException("Invalid composite field version"); + } + + compositeFieldName = meta.readString(); + compositeFieldType = CompositeMappedFieldType.CompositeFieldType.fromName(meta.readString()); + + switch (compositeFieldType) { + // support for type of composite fields can be added in the future. + case STAR_TREE: + starTreeMetadata = new StarTreeMetadata(meta, compositeFieldName, compositeFieldType.getName()); + break; + default: + throw new CorruptIndexException("Invalid composite field type present in the file", meta); + } + + } + + /** + * Returns the star-tree metadata. + * + * @return the StarTreeMetadata + */ + public StarTreeMetadata getStarTreeMetadata() { + return starTreeMetadata; + } + + /** + * Returns the name of the composite field. + * + * @return the composite field name + */ + public String getCompositeFieldName() { + return compositeFieldName; + } + + /** + * Returns the type of the composite field. + * + * @return the composite field type + */ + public CompositeMappedFieldType.CompositeFieldType getCompositeFieldType() { + return compositeFieldType; + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/MergeDimension.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/MergeDimension.java new file mode 100644 index 0000000000000..1e15cae2e0029 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/MergeDimension.java @@ -0,0 +1,56 @@ +/* + * 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; + +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.index.mapper.CompositeDataCubeFieldType; + +import java.io.IOException; +import java.util.Objects; + +/** + * Composite index merge dimension class + * + * @opensearch.experimental + */ +public class MergeDimension implements Dimension { + public static final String MERGE = "merge"; + private final String field; + + public MergeDimension(String field) { + this.field = field; + } + + public String getField() { + return field; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject(); + builder.field(CompositeDataCubeFieldType.NAME, field); + builder.field(CompositeDataCubeFieldType.TYPE, MERGE); + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + MergeDimension dimension = (MergeDimension) o; + return Objects.equals(field, dimension.getField()); + } + + @Override + public int hashCode() { + return Objects.hash(field); + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeDocument.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeDocument.java new file mode 100644 index 0000000000000..0ce2b3a5cdac5 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/StarTreeDocument.java @@ -0,0 +1,34 @@ +/* + * 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.opensearch.common.annotation.ExperimentalApi; + +import java.util.Arrays; + +/** + * Star tree document + * + * @opensearch.experimental + */ +@ExperimentalApi +public class StarTreeDocument { + public final Long[] dimensions; + public final Object[] metrics; + + public StarTreeDocument(Long[] dimensions, Object[] metrics) { + this.dimensions = dimensions; + this.metrics = metrics; + } + + @Override + public String toString() { + return Arrays.toString(dimensions) + " | " + Arrays.toString(metrics); + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/CountValueAggregator.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/CountValueAggregator.java new file mode 100644 index 0000000000000..26b99c1f17115 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/CountValueAggregator.java @@ -0,0 +1,65 @@ +/* + * 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.aggregators; + +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; + +/** + * Count value aggregator for star tree + * + * @opensearch.experimental + */ +public class CountValueAggregator implements ValueAggregator { + public static final StarTreeNumericType VALUE_AGGREGATOR_TYPE = StarTreeNumericType.LONG; + + @Override + public MetricStat getAggregationType() { + return MetricStat.COUNT; + } + + @Override + public StarTreeNumericType getAggregatedValueType() { + return VALUE_AGGREGATOR_TYPE; + } + + @Override + public Long getInitialAggregatedValueForSegmentDocValue(Long segmentDocValue, StarTreeNumericType starTreeNumericType) { + return 1L; + } + + @Override + public Long mergeAggregatedValueAndSegmentValue(Long value, Long segmentDocValue, StarTreeNumericType starTreeNumericType) { + return value + 1; + } + + @Override + public Long mergeAggregatedValues(Long value, Long aggregatedValue) { + return value + aggregatedValue; + } + + @Override + public Long getInitialAggregatedValue(Long value) { + return value; + } + + @Override + public int getMaxAggregatedValueByteSize() { + return Long.BYTES; + } + + @Override + public Long toLongValue(Long value) { + return value; + } + + @Override + public Long toStarTreeNumericTypeValue(Long value, StarTreeNumericType type) { + return value; + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MaxValueAggregator.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MaxValueAggregator.java new file mode 100644 index 0000000000000..e80d1c9097f9d --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MaxValueAggregator.java @@ -0,0 +1,75 @@ +/* + * 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.aggregators; + +import org.apache.lucene.util.NumericUtils; +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; + +/** + * Max value aggregator for star tree + * + * @opensearch.experimental + */ +public class MaxValueAggregator implements ValueAggregator { + + public static final StarTreeNumericType VALUE_AGGREGATOR_TYPE = StarTreeNumericType.DOUBLE; + + @Override + public MetricStat getAggregationType() { + return MetricStat.MAX; + } + + @Override + public StarTreeNumericType getAggregatedValueType() { + return VALUE_AGGREGATOR_TYPE; + } + + @Override + public Double getInitialAggregatedValueForSegmentDocValue(Long segmentDocValue, StarTreeNumericType starTreeNumericType) { + return starTreeNumericType.getDoubleValue(segmentDocValue); + } + + @Override + public Double mergeAggregatedValueAndSegmentValue(Double value, Long segmentDocValue, StarTreeNumericType starTreeNumericType) { + return Math.max(value, starTreeNumericType.getDoubleValue(segmentDocValue)); + } + + @Override + public Double mergeAggregatedValues(Double value, Double aggregatedValue) { + return Math.max(value, aggregatedValue); + } + + @Override + public Double getInitialAggregatedValue(Double value) { + return value; + } + + @Override + public int getMaxAggregatedValueByteSize() { + return Double.BYTES; + } + + @Override + public Long toLongValue(Double value) { + try { + return NumericUtils.doubleToSortableLong(value); + } catch (Exception e) { + throw new IllegalStateException("Cannot convert " + value + " to sortable long", e); + } + } + + @Override + public Double toStarTreeNumericTypeValue(Long value, StarTreeNumericType type) { + try { + return type.getDoubleValue(value); + } catch (Exception e) { + throw new IllegalStateException("Cannot convert " + value + " to sortable aggregation type", e); + } + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MetricAggregatorInfo.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MetricAggregatorInfo.java new file mode 100644 index 0000000000000..9b3c633280b08 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MetricAggregatorInfo.java @@ -0,0 +1,119 @@ +/* + * 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.aggregators; + +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; +import org.opensearch.index.fielddata.IndexNumericFieldData; + +import java.util.Comparator; +import java.util.Objects; + +/** + * Builds aggregation function and doc values field pair to support various aggregations + * + * @opensearch.experimental + */ +public class MetricAggregatorInfo implements Comparable { + + public static final String DELIMITER = "_"; + private final String metric; + private final String starFieldName; + private final MetricStat metricStat; + private final String field; + private final ValueAggregator valueAggregators; + private final StarTreeNumericType starTreeNumericType; + + /** + * Constructor for MetricAggregatorInfo + */ + public MetricAggregatorInfo(MetricStat metricStat, String field, String starFieldName, IndexNumericFieldData.NumericType numericType) { + this.metricStat = metricStat; + this.valueAggregators = ValueAggregatorFactory.getValueAggregator(metricStat); + this.starTreeNumericType = StarTreeNumericType.fromNumericType(numericType); + this.field = field; + this.starFieldName = starFieldName; + this.metric = toFieldName(); + } + + /** + * @return metric type + */ + public MetricStat getMetricStat() { + return metricStat; + } + + /** + * @return field Name + */ + public String getField() { + return field; + } + + /** + * @return the metric stat name + */ + public String getMetric() { + return metric; + } + + /** + * @return aggregator for the field value + */ + public ValueAggregator getValueAggregators() { + return valueAggregators; + } + + /** + * @return star tree aggregated value type + */ + public StarTreeNumericType getAggregatedValueType() { + return starTreeNumericType; + } + + /** + * @return field name with metric type and field + */ + public String toFieldName() { + return toFieldName(starFieldName, field, metricStat.getTypeName()); + + } + + public static String toFieldName(String starFieldName, String field, String metricName) { + return starFieldName + DELIMITER + field + DELIMITER + metricName; + } + + @Override + public int hashCode() { + return Objects.hashCode(toFieldName()); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj instanceof MetricAggregatorInfo) { + MetricAggregatorInfo anotherPair = (MetricAggregatorInfo) obj; + return metricStat.equals(anotherPair.metricStat) && field.equals(anotherPair.field); + } + return false; + } + + @Override + public String toString() { + return toFieldName(); + } + + @Override + public int compareTo(MetricAggregatorInfo other) { + return Comparator.comparing((MetricAggregatorInfo o) -> o.field) + .thenComparing((MetricAggregatorInfo o) -> o.metricStat) + .compare(this, other); + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MetricEntry.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MetricEntry.java new file mode 100644 index 0000000000000..683153ecde689 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MetricEntry.java @@ -0,0 +1,35 @@ +/* + * 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.aggregators; + +import org.opensearch.index.compositeindex.datacube.MetricStat; + +/** + * Holds the pair of metric name and it's associated stat + * + * @opensearch.experimental + */ +public class MetricEntry { + + private final String metricName; + private final MetricStat metricStat; + + public MetricEntry(String metricName, MetricStat metricStat) { + this.metricName = metricName; + this.metricStat = metricStat; + } + + public String getMetricName() { + return metricName; + } + + public MetricStat getMetricStat() { + return metricStat; + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MinValueAggregator.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MinValueAggregator.java new file mode 100644 index 0000000000000..26acf44e778a7 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MinValueAggregator.java @@ -0,0 +1,75 @@ +/* + * 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.aggregators; + +import org.apache.lucene.util.NumericUtils; +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; + +/** + * Min value aggregator for star tree + * + * @opensearch.experimental + */ +public class MinValueAggregator implements ValueAggregator { + + public static final StarTreeNumericType VALUE_AGGREGATOR_TYPE = StarTreeNumericType.DOUBLE; + + @Override + public MetricStat getAggregationType() { + return MetricStat.MIN; + } + + @Override + public StarTreeNumericType getAggregatedValueType() { + return VALUE_AGGREGATOR_TYPE; + } + + @Override + public Double getInitialAggregatedValueForSegmentDocValue(Long segmentDocValue, StarTreeNumericType starTreeNumericType) { + return starTreeNumericType.getDoubleValue(segmentDocValue); + } + + @Override + public Double mergeAggregatedValueAndSegmentValue(Double value, Long segmentDocValue, StarTreeNumericType starTreeNumericType) { + return Math.min(value, starTreeNumericType.getDoubleValue(segmentDocValue)); + } + + @Override + public Double mergeAggregatedValues(Double value, Double aggregatedValue) { + return Math.min(value, aggregatedValue); + } + + @Override + public Double getInitialAggregatedValue(Double value) { + return value; + } + + @Override + public int getMaxAggregatedValueByteSize() { + return Double.BYTES; + } + + @Override + public Long toLongValue(Double value) { + try { + return NumericUtils.doubleToSortableLong(value); + } catch (Exception e) { + throw new IllegalStateException("Cannot convert " + value + " to sortable long", e); + } + } + + @Override + public Double toStarTreeNumericTypeValue(Long value, StarTreeNumericType type) { + try { + return type.getDoubleValue(value); + } catch (Exception e) { + throw new IllegalStateException("Cannot convert " + value + " to sortable aggregation type", e); + } + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/SumValueAggregator.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/SumValueAggregator.java new file mode 100644 index 0000000000000..543b0f7f42374 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/SumValueAggregator.java @@ -0,0 +1,97 @@ +/* + * 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.aggregators; + +import org.apache.lucene.util.NumericUtils; +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; +import org.opensearch.search.aggregations.metrics.CompensatedSum; + +/** + * Sum value aggregator for star tree + * + * @opensearch.experimental + */ +public class SumValueAggregator implements ValueAggregator { + + public static final StarTreeNumericType VALUE_AGGREGATOR_TYPE = StarTreeNumericType.DOUBLE; + private double sum = 0; + private double compensation = 0; + private CompensatedSum kahanSummation = new CompensatedSum(0, 0); + + @Override + public MetricStat getAggregationType() { + return MetricStat.SUM; + } + + @Override + public StarTreeNumericType getAggregatedValueType() { + return VALUE_AGGREGATOR_TYPE; + } + + @Override + public Double getInitialAggregatedValueForSegmentDocValue(Long segmentDocValue, StarTreeNumericType starTreeNumericType) { + kahanSummation.reset(0, 0); + kahanSummation.add(starTreeNumericType.getDoubleValue(segmentDocValue)); + compensation = kahanSummation.delta(); + sum = kahanSummation.value(); + return kahanSummation.value(); + } + + @Override + public Double mergeAggregatedValueAndSegmentValue(Double value, Long segmentDocValue, StarTreeNumericType starTreeNumericType) { + assert kahanSummation.value() == value; + kahanSummation.reset(sum, compensation); + kahanSummation.add(starTreeNumericType.getDoubleValue(segmentDocValue)); + compensation = kahanSummation.delta(); + sum = kahanSummation.value(); + return kahanSummation.value(); + } + + @Override + public Double mergeAggregatedValues(Double value, Double aggregatedValue) { + assert kahanSummation.value() == aggregatedValue; + kahanSummation.reset(sum, compensation); + kahanSummation.add(value); + compensation = kahanSummation.delta(); + sum = kahanSummation.value(); + return kahanSummation.value(); + } + + @Override + public Double getInitialAggregatedValue(Double value) { + kahanSummation.reset(0, 0); + kahanSummation.add(value); + compensation = kahanSummation.delta(); + sum = kahanSummation.value(); + return kahanSummation.value(); + } + + @Override + public int getMaxAggregatedValueByteSize() { + return Double.BYTES; + } + + @Override + public Long toLongValue(Double value) { + try { + return NumericUtils.doubleToSortableLong(value); + } catch (Exception e) { + throw new IllegalStateException("Cannot convert " + value + " to sortable long", e); + } + } + + @Override + public Double toStarTreeNumericTypeValue(Long value, StarTreeNumericType type) { + try { + return type.getDoubleValue(value); + } catch (Exception e) { + throw new IllegalStateException("Cannot convert " + value + " to sortable aggregation type", e); + } + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregator.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregator.java new file mode 100644 index 0000000000000..3dd1f85845c17 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregator.java @@ -0,0 +1,64 @@ +/* + * 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.aggregators; + +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; + +/** + * A value aggregator that pre-aggregates on the input values for a specific type of aggregation. + * + * @opensearch.experimental + */ +public interface ValueAggregator { + + /** + * Returns the type of the aggregation. + */ + MetricStat getAggregationType(); + + /** + * Returns the data type of the aggregated value. + */ + StarTreeNumericType getAggregatedValueType(); + + /** + * Returns the initial aggregated value. + */ + A getInitialAggregatedValueForSegmentDocValue(Long segmentDocValue, StarTreeNumericType starTreeNumericType); + + /** + * Applies a segment doc value to the current aggregated value. + */ + A mergeAggregatedValueAndSegmentValue(A value, Long segmentDocValue, StarTreeNumericType starTreeNumericType); + + /** + * Applies an aggregated value to the current aggregated value. + */ + A mergeAggregatedValues(A value, A aggregatedValue); + + /** + * Clones an aggregated value. + */ + A getInitialAggregatedValue(A value); + + /** + * Returns the maximum size in bytes of the aggregated values seen so far. + */ + int getMaxAggregatedValueByteSize(); + + /** + * Converts an aggregated value into a Long type. + */ + Long toLongValue(A value); + + /** + * Converts an aggregated value from a Long type. + */ + A toStarTreeNumericTypeValue(Long rawValue, StarTreeNumericType type); +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregatorFactory.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregatorFactory.java new file mode 100644 index 0000000000000..79103679d5260 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregatorFactory.java @@ -0,0 +1,64 @@ +/* + * 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.aggregators; + +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; + +/** + * Value aggregator factory for a given aggregation type + * + * @opensearch.experimental + */ +public class ValueAggregatorFactory { + private ValueAggregatorFactory() {} + + /** + * Returns a new instance of value aggregator for the given aggregation type. + * + * @param aggregationType Aggregation type + * @return Value aggregator + */ + public static ValueAggregator getValueAggregator(MetricStat aggregationType) { + switch (aggregationType) { + // avg aggregator will be covered in the part of query (using count and sum) + case SUM: + return new SumValueAggregator(); + case COUNT: + return new CountValueAggregator(); + case MIN: + return new MinValueAggregator(); + case MAX: + return new MaxValueAggregator(); + default: + throw new IllegalStateException("Unsupported aggregation type: " + aggregationType); + } + } + + /** + * Returns the data type of the aggregated value for the given aggregation type. + * + * @param aggregationType Aggregation type + * @return Data type of the aggregated value + */ + public static StarTreeNumericType getAggregatedValueType(MetricStat aggregationType) { + switch (aggregationType) { + // other metric types (count, min, max, avg) will be supported in the future + case SUM: + return SumValueAggregator.VALUE_AGGREGATOR_TYPE; + case COUNT: + return CountValueAggregator.VALUE_AGGREGATOR_TYPE; + case MIN: + return MinValueAggregator.VALUE_AGGREGATOR_TYPE; + case MAX: + return MaxValueAggregator.VALUE_AGGREGATOR_TYPE; + default: + throw new IllegalStateException("Unsupported aggregation type: " + aggregationType); + } + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/numerictype/StarTreeNumericType.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/numerictype/StarTreeNumericType.java new file mode 100644 index 0000000000000..57fe573a6a93c --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/numerictype/StarTreeNumericType.java @@ -0,0 +1,66 @@ +/* + * 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.aggregators.numerictype; + +import org.opensearch.index.fielddata.IndexNumericFieldData; + +import java.util.function.Function; + +/** + * Enum to map Star Tree Numeric Types to Lucene's Numeric Type + * + * @opensearch.experimental + */ +public enum StarTreeNumericType { + + // TODO: Handle scaled floats + HALF_FLOAT(IndexNumericFieldData.NumericType.HALF_FLOAT, StarTreeNumericTypeConverters::halfFloatPointToDouble), + FLOAT(IndexNumericFieldData.NumericType.FLOAT, StarTreeNumericTypeConverters::floatPointToDouble), + LONG(IndexNumericFieldData.NumericType.LONG, StarTreeNumericTypeConverters::longToDouble), + DOUBLE(IndexNumericFieldData.NumericType.DOUBLE, StarTreeNumericTypeConverters::sortableLongtoDouble), + INT(IndexNumericFieldData.NumericType.INT, StarTreeNumericTypeConverters::intToDouble), + SHORT(IndexNumericFieldData.NumericType.SHORT, StarTreeNumericTypeConverters::shortToDouble), + BYTE(IndexNumericFieldData.NumericType.BYTE, StarTreeNumericTypeConverters::bytesToDouble), + UNSIGNED_LONG(IndexNumericFieldData.NumericType.UNSIGNED_LONG, StarTreeNumericTypeConverters::unsignedlongToDouble); + + final IndexNumericFieldData.NumericType numericType; + final Function converter; + + StarTreeNumericType(IndexNumericFieldData.NumericType numericType, Function converter) { + this.numericType = numericType; + this.converter = converter; + } + + public double getDoubleValue(long rawValue) { + return this.converter.apply(rawValue); + } + + public static StarTreeNumericType fromNumericType(IndexNumericFieldData.NumericType numericType) { + switch (numericType) { + case HALF_FLOAT: + return StarTreeNumericType.HALF_FLOAT; + case FLOAT: + return StarTreeNumericType.FLOAT; + case LONG: + return StarTreeNumericType.LONG; + case DOUBLE: + return StarTreeNumericType.DOUBLE; + case INT: + return StarTreeNumericType.INT; + case SHORT: + return StarTreeNumericType.SHORT; + case UNSIGNED_LONG: + return StarTreeNumericType.UNSIGNED_LONG; + case BYTE: + return StarTreeNumericType.BYTE; + default: + throw new UnsupportedOperationException("Unknown numeric type [" + numericType + "]"); + } + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/numerictype/StarTreeNumericTypeConverters.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/numerictype/StarTreeNumericTypeConverters.java new file mode 100644 index 0000000000000..eb7647c4f9851 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/numerictype/StarTreeNumericTypeConverters.java @@ -0,0 +1,58 @@ +/* + * 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.aggregators.numerictype; + +import org.apache.lucene.sandbox.document.HalfFloatPoint; +import org.apache.lucene.util.NumericUtils; +import org.opensearch.common.Numbers; +import org.opensearch.common.annotation.ExperimentalApi; + +/** + * Numeric converters used during aggregations of metric values + * + * @opensearch.experimental + */ +@ExperimentalApi +public class StarTreeNumericTypeConverters { + + public static double halfFloatPointToDouble(Long value) { + return HalfFloatPoint.sortableShortToHalfFloat((short) value.longValue()); + } + + public static double floatPointToDouble(Long value) { + return NumericUtils.sortableIntToFloat((int) value.longValue()); + } + + public static double longToDouble(Long value) { + return (double) value; + } + + public static double intToDouble(Long value) { + return (double) value; + } + + public static double shortToDouble(Long value) { + return (double) value; + } + + public static Double sortableLongtoDouble(Long value) { + return NumericUtils.sortableLongToDouble(value); + } + + public static double unsignedlongToDouble(Long value) { + return Numbers.unsignedLongToDouble(value); + } + + public static double bytesToDouble(Long value) { + byte[] bytes = new byte[8]; + NumericUtils.longToSortableBytes(value, bytes, 0); + return NumericUtils.sortableLongToDouble(NumericUtils.sortableBytesToLong(bytes, 0)); + } + +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/numerictype/package-info.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/numerictype/package-info.java new file mode 100644 index 0000000000000..18163056ec3b5 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/numerictype/package-info.java @@ -0,0 +1,14 @@ +/* + * 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. + */ + +/** + * Numeric Types for Composite Index Star Tree + * + * @opensearch.experimental + */ +package org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype; diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/package-info.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/package-info.java new file mode 100644 index 0000000000000..bddd6a46fbbe8 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/package-info.java @@ -0,0 +1,14 @@ +/* + * 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. + */ + +/** + * Aggregators for Composite Index Star Tree + * + * @opensearch.experimental + */ +package org.opensearch.index.compositeindex.datacube.startree.aggregators; 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 new file mode 100644 index 0000000000000..d28607c067a61 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilder.java @@ -0,0 +1,310 @@ +/* + * 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.builder; + +import org.apache.lucene.codecs.DocValuesConsumer; +import org.apache.lucene.index.BaseStarTreeBuilder; +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.startree.StarTreeDocument; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; +import org.opensearch.index.mapper.MapperService; + +import java.io.IOException; +import java.util.ArrayList; +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; + +/** + * On heap based single tree builder + * + * @opensearch.experimental + */ +@ExperimentalApi +public class OnHeapStarTreeBuilder extends BaseStarTreeBuilder { + + private final List starTreeDocuments = new ArrayList<>(); + + /** + * Constructor for OnHeapStarTreeBuilder + * + * @param starTreeField star-tree field + * @param segmentWriteState segment write state + * @param mapperService helps with the numeric type of field + * @throws IOException throws an exception when we are unable to construct a star-tree using on-heap approach + */ + public OnHeapStarTreeBuilder( + IndexOutput metaOut, + IndexOutput dataOut, + StarTreeField starTreeField, + SegmentWriteState segmentWriteState, + MapperService mapperService + ) throws IOException { + super(metaOut, dataOut, starTreeField, segmentWriteState, mapperService); + } + + @Override + public void appendStarTreeDocument(StarTreeDocument starTreeDocument) throws IOException { + starTreeDocuments.add(starTreeDocument); + } + + @Override + public void build( + List starTreeValuesSubs, + AtomicInteger fieldNumberAcrossStarTrees, + DocValuesConsumer starTreeDocValuesConsumer + ) throws IOException { + build(mergeStarTrees(starTreeValuesSubs), fieldNumberAcrossStarTrees, starTreeDocValuesConsumer); + } + + /** + * Sorts and aggregates the star-tree documents from multiple segments and builds star tree based on the newly + * aggregated star-tree documents + * + * @param starTreeValuesSubs StarTreeValues from multiple segments + * @return iterator of star tree documents + */ + Iterator mergeStarTrees(List starTreeValuesSubs) throws IOException { + return sortAndAggregateStarTreeDocuments(mergeStarTreeValues(starTreeValuesSubs)); + } + + /** + * Returns an array of all the starTreeDocuments from all the segments + * + * @param starTreeValuesSubs StarTreeValues from multiple segments + * @return array of star tree documents + */ + StarTreeDocument[] mergeStarTreeValues(List starTreeValuesSubs) throws IOException { + List starTreeDocuments = new ArrayList<>(); + for (StarTreeValues starTreeValues : starTreeValuesSubs) { + List dimensionsSplitOrder = starTreeValues.getStarTreeField().getDimensionsOrder(); + SequentialDocValuesIterator[] dimensionReaders = new SequentialDocValuesIterator[starTreeValues.getStarTreeField() + .getDimensionsOrder() + .size()]; + + for (int i = 0; i < dimensionsSplitOrder.size(); i++) { + String dimension = dimensionsSplitOrder.get(i).getField(); + dimensionReaders[i] = new SequentialDocValuesIterator(starTreeValues.getDimensionDocValuesIteratorMap().get(dimension)); + } + + List metricReaders = new ArrayList<>(); + for (Map.Entry metricDocValuesEntry : starTreeValues.getMetricDocValuesIteratorMap().entrySet()) { + metricReaders.add(new SequentialDocValuesIterator(metricDocValuesEntry.getValue())); + } + + boolean endOfDoc = false; + int currentDocId = 0; + while (!endOfDoc) { + Long[] dims = new Long[starTreeValues.getStarTreeField().getDimensionsOrder().size()]; + int i = 0; + for (SequentialDocValuesIterator dimensionDocValueIterator : dimensionReaders) { + int doc = dimensionDocValueIterator.nextDoc(currentDocId); + Long val = dimensionDocValueIterator.value(currentDocId); + // TODO : figure out how to identify a row with star tree docs here + endOfDoc = (doc == DocIdSetIterator.NO_MORE_DOCS); + if (endOfDoc) { + break; + } + dims[i] = val; + i++; + } + if (endOfDoc) { + break; + } + i = 0; + Object[] metrics = new Object[metricReaders.size()]; + for (SequentialDocValuesIterator metricDocValuesIterator : metricReaders) { + metricDocValuesIterator.nextDoc(currentDocId); + metrics[i] = metricDocValuesIterator.value(currentDocId); + i++; + } + StarTreeDocument starTreeDocument = new StarTreeDocument(dims, metrics); + starTreeDocuments.add(starTreeDocument); + currentDocId++; + } + } + StarTreeDocument[] starTreeDocumentsArr = new StarTreeDocument[starTreeDocuments.size()]; + return starTreeDocuments.toArray(starTreeDocumentsArr); + } + + @Override + public StarTreeDocument getStarTreeDocument(int docId) throws IOException { + return starTreeDocuments.get(docId); + } + + @Override + public List getStarTreeDocuments() { + return starTreeDocuments; + } + + @Override + public Long getDimensionValue(int docId, int dimensionId) throws IOException { + return starTreeDocuments.get(docId).dimensions[dimensionId]; + } + + /** + * Sorts and aggregates all the documents of the segment based on dimension and metrics configuration + * + * @param numDocs number of documents in the given segment + * @param dimensionReaders List of docValues readers to read dimensions from the segment + * @param metricReaders List of docValues readers to read metrics from the segment + * @return Iterator of star-tree documents + * + */ + @Override + public Iterator sortAndAggregateSegmentDocuments( + int numDocs, + SequentialDocValuesIterator[] dimensionReaders, + List metricReaders + ) throws IOException { + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[numDocs]; + for (int currentDocId = 0; currentDocId < numDocs; currentDocId++) { + starTreeDocuments[currentDocId] = getSegmentStarTreeDocument(currentDocId, dimensionReaders, metricReaders); + } + return sortAndAggregateStarTreeDocuments(starTreeDocuments); + } + + /** + * Sorts and aggregates the star-tree documents + * + * @param starTreeDocuments star-tree documents + * @return iterator for star-tree documents + * @throws IOException throws when unable to sort, merge and aggregate star-tree documents + */ + public Iterator sortAndAggregateStarTreeDocuments(StarTreeDocument[] starTreeDocuments) throws IOException { + // sort the documents + Arrays.sort(starTreeDocuments, (o1, o2) -> { + for (int i = 0; i < numDimensions; i++) { + if (o1.dimensions[i] == null && o2.dimensions[i] == null) { + return 0; + } + if (o1.dimensions[i] == null) { + return 1; + } + if (o2.dimensions[i] == null) { + return -1; + } + if (!Objects.equals(o1.dimensions[i], o2.dimensions[i])) { + return Long.compare(o1.dimensions[i], o2.dimensions[i]); + } + } + return 0; + }); + + // merge the documents + return mergeStarTreeDocuments(starTreeDocuments); + } + + /** + * Merges the star-tree documents based on dimensions + * + * @param starTreeDocuments star-tree documents + * @return iterator to aggregate star-tree documents + */ + private Iterator mergeStarTreeDocuments(StarTreeDocument[] starTreeDocuments) { + return new Iterator<>() { + boolean hasNext = true; + StarTreeDocument currentStarTreeDocument = starTreeDocuments[0]; + int docId = 1; + + @Override + public boolean hasNext() { + return hasNext; + } + + @Override + public StarTreeDocument next() { + // aggregate as we move on to the next doc + StarTreeDocument next = reduceSegmentStarTreeDocuments(null, currentStarTreeDocument); + while (docId < starTreeDocuments.length) { + StarTreeDocument starTreeDocument = starTreeDocuments[docId++]; + if (!Arrays.equals(starTreeDocument.dimensions, next.dimensions)) { + currentStarTreeDocument = starTreeDocument; + return next; + } else { + next = reduceSegmentStarTreeDocuments(next, starTreeDocument); + } + } + hasNext = false; + return next; + } + }; + } + + /** + * Generates a star-tree for a given star-node + * + * @param startDocId Start document id in the star-tree + * @param endDocId End document id (exclusive) in the star-tree + * @param dimensionId Dimension id of the star-node + * @return iterator for star-tree documents of star-node + * @throws IOException throws when unable to generate star-tree for star-node + */ + @Override + public Iterator generateStarTreeDocumentsForStarNode(int startDocId, int endDocId, int dimensionId) + throws IOException { + int numDocs = endDocId - startDocId; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[numDocs]; + for (int i = 0; i < numDocs; i++) { + starTreeDocuments[i] = getStarTreeDocument(startDocId + i); + } + Arrays.sort(starTreeDocuments, (o1, o2) -> { + for (int i = dimensionId + 1; i < numDimensions; i++) { + if (!Objects.equals(o1.dimensions[i], o2.dimensions[i])) { + return Long.compare(o1.dimensions[i], o2.dimensions[i]); + } + } + return 0; + }); + return new Iterator() { + boolean hasNext = true; + StarTreeDocument currentStarTreeDocument = starTreeDocuments[0]; + int docId = 1; + + private boolean hasSameDimensions(StarTreeDocument starTreeDocument1, StarTreeDocument starTreeDocument2) { + for (int i = dimensionId + 1; i < numDimensions; i++) { + if (!Objects.equals(starTreeDocument1.dimensions[i], starTreeDocument2.dimensions[i])) { + return false; + } + } + return true; + } + + @Override + public boolean hasNext() { + return hasNext; + } + + @Override + public StarTreeDocument next() { + StarTreeDocument next = reduceStarTreeDocuments(null, currentStarTreeDocument); + next.dimensions[dimensionId] = STAR_IN_DOC_VALUES_INDEX; + while (docId < numDocs) { + StarTreeDocument starTreeDocument = starTreeDocuments[docId++]; + if (!hasSameDimensions(starTreeDocument, currentStarTreeDocument)) { + currentStarTreeDocument = starTreeDocument; + return next; + } else { + next = reduceStarTreeDocuments(next, starTreeDocument); + } + } + hasNext = false; + return next; + } + }; + } +} 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 new file mode 100644 index 0000000000000..4c0e8ad8938e7 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeBuilder.java @@ -0,0 +1,58 @@ +/* + * 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.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 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. + * + * @opensearch.experimental + */ +@ExperimentalApi +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 fieldNumberAcrossStarTrees maintains the unique field number across the fields in the star tree + * @param starTreeDocValuesConsumer + * @throws IOException when we are unable to build star-tree + */ + + void build( + Map fieldProducerMap, + AtomicInteger fieldNumberAcrossStarTrees, + DocValuesConsumer starTreeDocValuesConsumer + ) throws IOException; + + /** + * Builds the star tree using StarTree 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 + * @throws IOException when we are unable to build star-tree + */ + 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 new file mode 100644 index 0000000000000..e4bf365332e7a --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilder.java @@ -0,0 +1,148 @@ +/* + * 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.builder; + +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.mapper.CompositeMappedFieldType; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.mapper.StarTreeMapper; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +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. + * + * @opensearch.experimental + */ +@ExperimentalApi +public class StarTreesBuilder implements Closeable { + + private static final Logger logger = LogManager.getLogger(StarTreesBuilder.class); + + private final List starTreeFields; + private final SegmentWriteState state; + private final MapperService mapperService; + private AtomicInteger fieldNumberAcrossStarTrees; + + public StarTreesBuilder(SegmentWriteState segmentWriteState, MapperService mapperService) { + List starTreeFields = new ArrayList<>(); + for (CompositeMappedFieldType compositeMappedFieldType : mapperService.getCompositeFieldTypes()) { + if (compositeMappedFieldType instanceof StarTreeMapper.StarTreeFieldType) { + StarTreeMapper.StarTreeFieldType starTreeFieldType = (StarTreeMapper.StarTreeFieldType) compositeMappedFieldType; + starTreeFields.add( + new StarTreeField( + starTreeFieldType.name(), + starTreeFieldType.getDimensions(), + starTreeFieldType.getMetrics(), + starTreeFieldType.getStarTreeConfig() + ) + ); + } + } + this.starTreeFields = starTreeFields; + this.state = segmentWriteState; + this.mapperService = mapperService; + this.fieldNumberAcrossStarTrees = new AtomicInteger(); + } + + /** + * Builds the star-trees. + */ + 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; + } + long startTime = System.currentTimeMillis(); + + int numStarTrees = starTreeFields.size(); + logger.debug("Starting building {} star-trees with star-tree fields", numStarTrees); + + // Build all star-trees + for (int i = 0; i < numStarTrees; i++) { + StarTreeField starTreeField = starTreeFields.get(i); + try (StarTreeBuilder starTreeBuilder = getSingleTreeBuilder(metaOut, dataOut, starTreeField, state, mapperService)) { + starTreeBuilder.build(fieldProducerMap, fieldNumberAcrossStarTrees, starTreeDocValuesConsumer); + } + } + logger.debug("Took {} ms to building {} star-trees with star-tree fields", System.currentTimeMillis() - startTime, numStarTrees); + } + + @Override + public void close() throws IOException { + // TODO : close files + } + + /** + * Merges star tree fields from multiple segments + * + * @param metaOut + * @param dataOut + * @param starTreeFieldMap StarTreeField configuration per field + * @param starTreeValuesSubsPerField starTreeValuesSubs per field + * @param starTreeDocValuesConsumer + */ + public void buildDuringMerge( + IndexOutput metaOut, + IndexOutput dataOut, + final Map starTreeFieldMap, + final Map> starTreeValuesSubsPerField, + DocValuesConsumer starTreeDocValuesConsumer + ) throws IOException { + for (Map.Entry> entry : starTreeValuesSubsPerField.entrySet()) { + List starTreeValuesList = entry.getValue(); + StarTreeField starTreeField = starTreeFieldMap.get(entry.getKey()); + StarTreeBuilder builder = getSingleTreeBuilder(metaOut, dataOut, starTreeField, state, mapperService); + builder.build(starTreeValuesList, fieldNumberAcrossStarTrees, starTreeDocValuesConsumer); + } + } + + /** + * Get star-tree builder based on build mode. + */ + private static StarTreeBuilder getSingleTreeBuilder( + IndexOutput metaOut, + IndexOutput dataOut, + StarTreeField starTreeField, + SegmentWriteState state, + MapperService mapperService + ) throws IOException { + switch (starTreeField.getStarTreeConfig().getBuildMode()) { + case ON_HEAP: + return new OnHeapStarTreeBuilder(metaOut, dataOut, starTreeField, state, mapperService); + default: + throw new IllegalArgumentException( + String.format( + Locale.ROOT, + "No star tree implementation is available for [%s] build mode", + starTreeField.getStarTreeConfig().getBuildMode() + ) + ); + } + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/package-info.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/package-info.java new file mode 100644 index 0000000000000..9c97b076371a3 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/package-info.java @@ -0,0 +1,14 @@ +/* + * 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. + */ + +/** + * Builders for Composite Index Star Tree + * + * @opensearch.experimental + */ +package org.opensearch.index.compositeindex.datacube.startree.builder; diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/meta/StarTreeMetadata.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/meta/StarTreeMetadata.java new file mode 100644 index 0000000000000..e328cb061e3f1 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/meta/StarTreeMetadata.java @@ -0,0 +1,228 @@ +/* + * 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.meta; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.store.IndexInput; +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricEntry; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Holds the associated metadata for the building of star-tree + * + * @opensearch.experimental + */ +public class StarTreeMetadata implements TreeMetadata { + private static final Logger logger = LogManager.getLogger(TreeMetadata.class); + private final IndexInput meta; + private final String starTreeFieldName; + private final String starTreeFieldType; + private final List dimensionFieldNumbers; + private final List metricEntries; + private final Integer segmentAggregatedDocCount; + private final Integer maxLeafDocs; + private final Set skipStarNodeCreationInDims; + private final StarTreeFieldConfiguration.StarTreeBuildMode starTreeBuildMode; + private final long dataStartFilePointer; + private final long dataLength; + + public StarTreeMetadata(IndexInput meta, String compositeFieldName, String compositeFieldType) throws IOException { + this.meta = meta; + try { + this.starTreeFieldName = compositeFieldName; + this.starTreeFieldType = compositeFieldType; + this.dimensionFieldNumbers = readStarTreeDimensions(); + this.metricEntries = readMetricEntries(); + this.segmentAggregatedDocCount = readSegmentAggregatedDocCount(); + this.maxLeafDocs = readMaxLeafDocs(); + this.skipStarNodeCreationInDims = readSkipStarNodeCreationInDims(); + this.starTreeBuildMode = readBuildMode(); + this.dataStartFilePointer = readDataStartFilePointer(); + this.dataLength = readDataLength(); + } catch (Exception e) { + logger.error("Unable to read star-tree metadata from the file"); + throw new CorruptIndexException("Unable to read star-tree metadata from the file", meta); + } + } + + @Override + public int readDimensionsCount() throws IOException { + return meta.readInt(); + } + + @Override + public List readStarTreeDimensions() throws IOException { + int dimensionCount = readDimensionsCount(); + List dimensionFieldNumbers = new ArrayList<>(); + + for (int i = 0; i < dimensionCount; i++) { + dimensionFieldNumbers.add(meta.readInt()); + } + + return dimensionFieldNumbers; + } + + @Override + public int readMetricsCount() throws IOException { + return meta.readInt(); + } + + @Override + public List readMetricEntries() throws IOException { + int metricCount = readMetricsCount(); + List metricEntries = new ArrayList<>(); + + for (int i = 0; i < metricCount; i++) { + String metricName = meta.readString(); + String metricStat = meta.readString(); + metricEntries.add(new MetricEntry(metricName, MetricStat.fromTypeName(metricStat))); + } + + return metricEntries; + } + + @Override + public int readSegmentAggregatedDocCount() throws IOException { + return meta.readInt(); + } + + @Override + public int readMaxLeafDocs() throws IOException { + return meta.readInt(); + } + + @Override + public int readSkipStarNodeCreationInDimsCount() throws IOException { + return meta.readInt(); + } + + @Override + public Set readSkipStarNodeCreationInDims() throws IOException { + + int skipStarNodeCreationInDimsCount = readSkipStarNodeCreationInDimsCount(); + Set skipStarNodeCreationInDims = new HashSet<>(); + for (int i = 0; i < skipStarNodeCreationInDimsCount; i++) { + skipStarNodeCreationInDims.add(meta.readInt()); + } + return skipStarNodeCreationInDims; + } + + @Override + public StarTreeFieldConfiguration.StarTreeBuildMode readBuildMode() throws IOException { + return StarTreeFieldConfiguration.StarTreeBuildMode.fromTypeName(meta.readString()); + } + + @Override + public long readDataStartFilePointer() throws IOException { + return meta.readLong(); + } + + @Override + public long readDataLength() throws IOException { + return meta.readLong(); + } + + /** + * Returns the name of the star-tree field. + * + * @return star-tree field name + */ + public String getStarTreeFieldName() { + return starTreeFieldName; + } + + /** + * Returns the type of the star tree field. + * + * @return star-tree field type + */ + public String getStarTreeFieldType() { + return starTreeFieldType; + } + + /** + * Returns the list of dimension field numbers. + * + * @return star-tree dimension field numbers + */ + public List getDimensionFieldNumbers() { + return dimensionFieldNumbers; + } + + /** + * Returns the list of metric entries. + * + * @return star-tree metric entries + */ + public List getMetricEntries() { + return metricEntries; + } + + /** + * Returns the aggregated document count for the star-tree. + * + * @return the aggregated document count for the star-tree. + */ + public Integer getSegmentAggregatedDocCount() { + return segmentAggregatedDocCount; + } + + /** + * Returns the max leaf docs for the star-tree. + * + * @return the max leaf docs. + */ + public Integer getMaxLeafDocs() { + return maxLeafDocs; + } + + /** + * Returns the set of dimensions for which star node will not be created in the star-tree. + * + * @return the set of dimensions. + */ + public Set getSkipStarNodeCreationInDims() { + return skipStarNodeCreationInDims; + } + + /** + * Returns the build mode for the star-tree. + * + * @return the star-tree build mode. + */ + public StarTreeFieldConfiguration.StarTreeBuildMode getStarTreeBuildMode() { + return starTreeBuildMode; + } + + /** + * Returns the file pointer to the start of the star-tree data. + * + * @return start file pointer for star-tree data + */ + public long getDataStartFilePointer() { + return dataStartFilePointer; + } + + /** + * Returns the length of star-tree data + * + * @return star-tree length + */ + public long getDataLength() { + return dataLength; + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/meta/TreeMetadata.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/meta/TreeMetadata.java new file mode 100644 index 0000000000000..9859afad95a74 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/meta/TreeMetadata.java @@ -0,0 +1,112 @@ +/* + * 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.meta; + +import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricEntry; + +import java.io.IOException; +import java.util.List; +import java.util.Set; + +/** + * An interface for metadata of the star-tree + * + * @opensearch.experimental + */ +public interface TreeMetadata { + + /** + * Reads the count of dimensions in the star-tree. + * + * @return the count of dimensions + * @throws IOException if an I/O error occurs while reading the dimensions count + */ + int readDimensionsCount() throws IOException; + + /** + * Reads the list of dimension ordinals in the star-tree. + * + * @return the list of dimension ordinals + * @throws IOException if an I/O error occurs while reading the dimension ordinals + */ + List readStarTreeDimensions() throws IOException; + + /** + * Reads the count of metrics in the star-tree. + * + * @return the count of metrics + * @throws IOException if an I/O error occurs while reading the metrics count + */ + int readMetricsCount() throws IOException; + + /** + * Reads the list of metric entries in the star-tree. + * + * @return the list of metric entries + * @throws IOException if an I/O error occurs while reading the metric entries + */ + List readMetricEntries() throws IOException; + + /** + * Reads the aggregated document count for the segment in the star-tree. + * + * @return the aggregated document count for the segment + * @throws IOException if an I/O error occurs while reading the aggregated document count + */ + int readSegmentAggregatedDocCount() throws IOException; + + /** + * Reads the max leaf docs for the star-tree. + * + * @return the max leaf docs for the star-tree + * @throws IOException if an I/O error occurs while reading the max leaf docs + */ + int readMaxLeafDocs() throws IOException; + + /** + * Reads the count of dimensions where star node will not be created in the star-tree. + * + * @return the count of dimensions + * @throws IOException if an I/O error occurs while reading the skip star node dimensions count + */ + int readSkipStarNodeCreationInDimsCount() throws IOException; + + /** + * Reads the list of dimensions field numbers to be skipped for star node creation in the star-tree. + * + * @return the set of dimensions field numbers to be skipped for star node creation. + * @throws IOException if an I/O error occurs while reading the dimensions + */ + Set readSkipStarNodeCreationInDims() throws IOException; + + /** + * Reads the build mode for the star-tree. + * + * @return the star-tree build mode + * @throws IOException if an I/O error occurs while reading the build mode + */ + StarTreeFieldConfiguration.StarTreeBuildMode readBuildMode() throws IOException; + + /** + * Reads the file pointer to the start of the star-tree data. + * + * @return the file pointer to the start of the star-tree data + * @throws IOException if an I/O error occurs while reading the star-tree data start file pointer + */ + long readDataStartFilePointer() throws IOException; + + /** + * Reads the length of the data of the star-tree. + * + * @return the length of the data + * @throws IOException if an I/O error occurs while reading the data length + */ + long readDataLength() throws IOException; +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/meta/package-info.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/meta/package-info.java new file mode 100644 index 0000000000000..568cacea4b59a --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/meta/package-info.java @@ -0,0 +1,14 @@ +/* + * 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. + */ + +/** + * Meta package for star tree + * + * @opensearch.experimental + */ +package org.opensearch.index.compositeindex.datacube.startree.meta; diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/OffHeapStarTree.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/OffHeapStarTree.java new file mode 100644 index 0000000000000..1f16bd924dac6 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/OffHeapStarTree.java @@ -0,0 +1,65 @@ +/* + * 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.node; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.RandomAccessInput; +import org.opensearch.index.compositeindex.datacube.startree.meta.StarTreeMetadata; + +import java.io.IOException; + +import static org.opensearch.index.compositeindex.CompositeIndexConstants.MAGIC_MARKER; +import static org.opensearch.index.compositeindex.CompositeIndexConstants.VERSION; + +/** + * Off heap implementation of the star-tree. + * + * @opensearch.experimental + */ +public class OffHeapStarTree implements StarTree { + private static final Logger logger = LogManager.getLogger(OffHeapStarTree.class); + private final OffHeapStarTreeNode root; + private final Integer numNodes; + + public OffHeapStarTree(IndexInput data, StarTreeMetadata starTreeMetadata) throws IOException { + long magicMarker = data.readLong(); + if (MAGIC_MARKER != magicMarker) { + logger.error("Invalid magic marker"); + throw new IOException("Invalid magic marker"); + } + int version = data.readInt(); + if (VERSION != version) { + logger.error("Invalid star tree version"); + throw new IOException("Invalid version"); + } + numNodes = data.readInt(); // num nodes + + RandomAccessInput in = data.randomAccessSlice( + starTreeMetadata.getDataStartFilePointer(), + starTreeMetadata.getDataStartFilePointer() + starTreeMetadata.getDataLength() + ); + root = new OffHeapStarTreeNode(in, 0); + } + + @Override + public StarTreeNode getRoot() { + return root; + } + + /** + * Returns the number of nodes in star-tree + * + * @return number of nodes in te star-tree + */ + public Integer getNumNodes() { + return numNodes; + } + +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/OffHeapStarTreeNode.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/OffHeapStarTreeNode.java new file mode 100644 index 0000000000000..91ad56ef70eb3 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/OffHeapStarTreeNode.java @@ -0,0 +1,183 @@ +/* + * 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.node; + +import org.apache.lucene.store.RandomAccessInput; + +import java.io.IOException; +import java.util.Iterator; + +/** + * Off heap implementation of {@link StarTreeNode} + * + * @opensearch.experimental + */ +public class OffHeapStarTreeNode implements StarTreeNode { + public static final int NUM_INT_SERIALIZABLE_FIELDS = 7; + public static final int NUM_LONG_SERIALIZABLE_FIELDS = 1; + public static final long SERIALIZABLE_DATA_SIZE_IN_BYTES = (Integer.BYTES * NUM_INT_SERIALIZABLE_FIELDS) + (Long.BYTES + * NUM_LONG_SERIALIZABLE_FIELDS); + private static final int DIMENSION_ID_OFFSET = 0; + private static final int DIMENSION_VALUE_OFFSET = DIMENSION_ID_OFFSET + Integer.BYTES; + private static final int START_DOC_ID_OFFSET = DIMENSION_VALUE_OFFSET + Long.BYTES; + private static final int END_DOC_ID_OFFSET = START_DOC_ID_OFFSET + Integer.BYTES; + private static final int AGGREGATE_DOC_ID_OFFSET = END_DOC_ID_OFFSET + Integer.BYTES; + private static final int IS_STAR_NODE_OFFSET = AGGREGATE_DOC_ID_OFFSET + Integer.BYTES; + private static final int FIRST_CHILD_ID_OFFSET = IS_STAR_NODE_OFFSET + Integer.BYTES; + private static final int LAST_CHILD_ID_OFFSET = FIRST_CHILD_ID_OFFSET + Integer.BYTES; + + public static final int INVALID_ID = -1; + + private final int nodeId; + private final int firstChildId; + + RandomAccessInput in; + + public OffHeapStarTreeNode(RandomAccessInput in, int nodeId) throws IOException { + this.in = in; + this.nodeId = nodeId; + firstChildId = getInt(FIRST_CHILD_ID_OFFSET); + } + + private int getInt(int fieldOffset) throws IOException { + return in.readInt(nodeId * SERIALIZABLE_DATA_SIZE_IN_BYTES + fieldOffset); + } + + private long getLong(int fieldOffset) throws IOException { + return in.readLong(nodeId * SERIALIZABLE_DATA_SIZE_IN_BYTES + fieldOffset); + } + + private byte getByte(int fieldOffset) throws IOException { + return in.readByte(nodeId * SERIALIZABLE_DATA_SIZE_IN_BYTES + fieldOffset); + } + + @Override + public int getDimensionId() throws IOException { + return getInt(DIMENSION_ID_OFFSET); + } + + @Override + public long getDimensionValue() throws IOException { + return getLong(DIMENSION_VALUE_OFFSET); + } + + @Override + public int getChildDimensionId() throws IOException { + if (firstChildId == INVALID_ID) { + return INVALID_ID; + } else { + return in.readInt(firstChildId * SERIALIZABLE_DATA_SIZE_IN_BYTES); + } + } + + @Override + public int getStartDocId() throws IOException { + return getInt(START_DOC_ID_OFFSET); + } + + @Override + public int getEndDocId() throws IOException { + return getInt(END_DOC_ID_OFFSET); + } + + @Override + public int getAggregatedDocId() throws IOException { + return getInt(AGGREGATE_DOC_ID_OFFSET); + } + + @Override + public int getNumChildren() throws IOException { + if (firstChildId == INVALID_ID) { + return 0; + } else { + return getInt(LAST_CHILD_ID_OFFSET) - firstChildId + 1; + } + } + + @Override + public boolean isLeaf() { + return firstChildId == INVALID_ID; + } + + @Override + public boolean isStarNode() throws IOException { + return getByte(IS_STAR_NODE_OFFSET) != 0; + } + + @Override + public StarTreeNode getChildForDimensionValue(long dimensionValue) throws IOException { + // there will be no children for leaf nodes + if (isLeaf()) { + return null; + } + + // Specialize star node for performance + if (dimensionValue == ALL) { + return handleStarNode(); + } + + return binarySearchChild(dimensionValue); + } + + private OffHeapStarTreeNode handleStarNode() throws IOException { + OffHeapStarTreeNode firstNode = new OffHeapStarTreeNode(in, firstChildId); + if (firstNode.getDimensionValue() == ALL) { + return firstNode; + } else { + return null; + } + } + + private OffHeapStarTreeNode binarySearchChild(long dimensionValue) throws IOException { + // Binary search to find child node + int low = firstChildId; + int high = getInt(LAST_CHILD_ID_OFFSET); + + while (low <= high) { + int mid = low + (high - low) / 2; + OffHeapStarTreeNode midNode = new OffHeapStarTreeNode(in, mid); + long midNodeDimensionValue = midNode.getDimensionValue(); + + if (midNodeDimensionValue == dimensionValue) { + return midNode; + } else if (midNodeDimensionValue < dimensionValue) { + low = mid + 1; + } else { + high = mid - 1; + } + } + return null; + } + + @Override + public Iterator getChildrenIterator() throws IOException { + return new Iterator<>() { + private int currentChildId = firstChildId; + private final int lastChildId = getInt(LAST_CHILD_ID_OFFSET); + + @Override + public boolean hasNext() { + return currentChildId <= lastChildId; + } + + @Override + public OffHeapStarTreeNode next() { + try { + return new OffHeapStarTreeNode(in, currentChildId++); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTree.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTree.java new file mode 100644 index 0000000000000..e21dc225a4c8f --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTree.java @@ -0,0 +1,23 @@ +/* + * 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.node; + +/** + * Interface for star-tree. + * + * @opensearch.experimental + */ +public interface StarTree { + + /** + * Fetches the root node of the star-tree. + * @return the root of the star-tree + */ + StarTreeNode getRoot(); + +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNode.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNode.java new file mode 100644 index 0000000000000..59522ffa4be89 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNode.java @@ -0,0 +1,112 @@ +/* + * 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.node; + +import org.opensearch.common.annotation.ExperimentalApi; + +import java.io.IOException; +import java.util.Iterator; + +/** + * Interface that represents star tree node + * + * @opensearch.experimental + */ +@ExperimentalApi +public interface StarTreeNode { + long ALL = -1l; + + /** + * Returns the dimension ID of the current star-tree node. + * + * @return the dimension ID + * @throws IOException if an I/O error occurs while reading the dimension ID + */ + int getDimensionId() throws IOException; + + /** + * Returns the dimension value of the current star-tree node. + * + * @return the dimension value + * @throws IOException if an I/O error occurs while reading the dimension value + */ + long getDimensionValue() throws IOException; + + /** + * Returns the dimension ID of the child star-tree node. + * + * @return the child dimension ID + * @throws IOException if an I/O error occurs while reading the child dimension ID + */ + int getChildDimensionId() throws IOException; + + /** + * Returns the start document ID of the current star-tree node. + * + * @return the start document ID + * @throws IOException if an I/O error occurs while reading the start document ID + */ + int getStartDocId() throws IOException; + + /** + * Returns the end document ID of the current star-tree node. + * + * @return the end document ID + * @throws IOException if an I/O error occurs while reading the end document ID + */ + int getEndDocId() throws IOException; + + /** + * Returns the aggregated document ID of the current star-tree node. + * + * @return the aggregated document ID + * @throws IOException if an I/O error occurs while reading the aggregated document ID + */ + int getAggregatedDocId() throws IOException; + + /** + * Returns the number of children of the current star-tree node. + * + * @return the number of children + * @throws IOException if an I/O error occurs while reading the number of children + */ + int getNumChildren() throws IOException; + + /** + * Checks if the current node is a leaf star-tree node. + * + * @return true if the node is a leaf node, false otherwise + */ + boolean isLeaf(); + + /** + * Checks if the current node is a star node. + * + * @return true if the node is a star node, false otherwise + * @throws IOException if an I/O error occurs while reading the star node status + */ + boolean isStarNode() throws IOException; + + /** + * Returns the child star-tree node for the given dimension value. + * + * @param dimensionValue the dimension value + * @return the child node for the given dimension value or null if child is not present + * @throws IOException if an I/O error occurs while retrieving the child node + */ + StarTreeNode getChildForDimensionValue(long dimensionValue) throws IOException; + + /** + * Returns an iterator over the children of the current star-tree node. + * + * @return an iterator over the children + * @throws IOException if an I/O error occurs while retrieving the children iterator + */ + Iterator getChildrenIterator() throws IOException; +} 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 new file mode 100644 index 0000000000000..19d12bc6318d7 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/package-info.java @@ -0,0 +1,14 @@ +/* + * 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. + */ + +/** + * 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/package-info.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/package-info.java index 4f4e670478e2f..6d6cb420f4a9e 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/package-info.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/package-info.java @@ -7,5 +7,7 @@ */ /** * Core classes for handling star tree index. + * + * @opensearch.experimental */ package org.opensearch.index.compositeindex.datacube.startree; 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 new file mode 100644 index 0000000000000..0a9625b151183 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/SequentialDocValuesIterator.java @@ -0,0 +1,160 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.compositeindex.datacube.startree.utils; + +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.search.DocIdSetIterator; +import org.opensearch.common.annotation.ExperimentalApi; + +import java.io.IOException; + +/** + * Coordinates the reading of documents across multiple DocIdSetIterators. + * It encapsulates a single DocIdSetIterator and maintains the latest document ID and its associated value. + * + * @opensearch.experimental + */ +@ExperimentalApi +public class SequentialDocValuesIterator { + + /** + * The doc id set iterator associated for each field. + */ + private final DocIdSetIterator docIdSetIterator; + + /** + * The value associated with the latest document. + */ + private Long docValue; + + /** + * The id of the latest document. + */ + private int docId = -1; + + /** + * Constructs a new SequentialDocValuesIterator instance with the given DocIdSetIterator. + * + * @param docIdSetIterator the DocIdSetIterator to be associated with this instance + */ + public SequentialDocValuesIterator(DocIdSetIterator docIdSetIterator) { + this.docIdSetIterator = docIdSetIterator; + } + + /** + * Creates a SequentialDocValuesIterator with an empty DocIdSetIterator. + * + */ + public SequentialDocValuesIterator() { + this.docIdSetIterator = new DocIdSetIterator() { + @Override + public int docID() { + return NO_MORE_DOCS; + } + + @Override + public int nextDoc() { + return NO_MORE_DOCS; + } + + @Override + public int advance(int target) { + return NO_MORE_DOCS; + } + + @Override + public long cost() { + return 0; + } + }; + } + + /** + * Returns the value associated with the latest document. + * + * @return the value associated with the latest document + */ + public Long getDocValue() { + return docValue; + } + + /** + * Sets the value associated with the latest document. + * + * @param docValue the value to be associated with the latest document + */ + public void setDocValue(Long docValue) { + this.docValue = docValue; + } + + /** + * Returns the id of the latest document. + * + * @return the id of the latest document + */ + public int getDocId() { + return docId; + } + + /** + * Sets the id of the latest document. + * + * @param docId the ID of the latest document + */ + public void setDocId(int docId) { + this.docId = docId; + } + + /** + * Returns the DocIdSetIterator associated with this instance. + * + * @return the DocIdSetIterator associated with this instance + */ + public DocIdSetIterator getDocIdSetIterator() { + return docIdSetIterator; + } + + public int nextDoc(int currentDocId) throws IOException { + // if doc id stored is less than or equal to the requested doc id , return the stored doc id + if (docId >= currentDocId) { + return docId; + } + setDocId(this.docIdSetIterator.nextDoc()); + return docId; + } + + public Long value(int currentDocId) throws IOException { + if (this.getDocIdSetIterator() instanceof SortedNumericDocValues) { + SortedNumericDocValues sortedNumericDocValues = (SortedNumericDocValues) this.getDocIdSetIterator(); + if (currentDocId < 0) { + throw new IllegalStateException("invalid doc id to fetch the next value"); + } + if (currentDocId == DocIdSetIterator.NO_MORE_DOCS) { + throw new IllegalStateException("DocValuesIterator is already exhausted"); + } + + if (docId == DocIdSetIterator.NO_MORE_DOCS) { + return null; + } + + if (docValue == null) { + setDocValue(sortedNumericDocValues.nextValue()); + } + if (docId == currentDocId) { + Long nextValue = docValue; + docValue = null; + return nextValue; + } else { + return null; + } + } else { + throw new IllegalStateException("Unsupported Iterator requested for SequentialDocValuesIterator"); + } + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeBuilderUtils.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeBuilderUtils.java new file mode 100644 index 0000000000000..4e12f8e60d449 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeBuilderUtils.java @@ -0,0 +1,105 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ +package org.opensearch.index.compositeindex.datacube.startree.utils; + +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.IndexOutput; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricAggregatorInfo; +import org.opensearch.index.mapper.CompositeMappedFieldType; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +/** + * Util class for building star tree + * + * @opensearch.experimental + */ +public class StarTreeBuilderUtils { + + private StarTreeBuilderUtils() {} + + public static final int ALL = -1; + + /** + * 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. + */ + @ExperimentalApi + public static class TreeNode { + + /** + * The dimension id for the dimension (field) associated with this star-tree node. + */ + public int dimensionId = ALL; + + /** + * The starting document id (inclusive) associated with this star-tree node. + */ + public int startDocId = ALL; + + /** + * The ending document id (exclusive) associated with this star-tree node. + */ + public int endDocId = ALL; + + /** + * The aggregated document id associated with this star-tree node. + */ + public int aggregatedDocId = ALL; + + /** + * The child dimension identifier associated with this star-tree node. + */ + public int childDimensionId = ALL; + + /** + * The value of the dimension associated with this star-tree node. + */ + public long dimensionValue = ALL; + + /** + * A flag indicating whether this node is a star node (a node that represents an aggregation of all dimensions). + */ + public boolean isStarNode = false; + + /** + * A map containing the child nodes of this star-tree node, keyed by their dimension id. + */ + public Map children; + } + + public static long serializeStarTree(IndexOutput dataOut, TreeNode rootNode, int numNodes) throws IOException { + return StarTreeDataSerializer.serializeStarTree(dataOut, rootNode, numNodes); + } + + public static void serializeStarTreeMetadata( + IndexOutput metaOut, + StarTreeField starTreeField, + SegmentWriteState writeState, + List metricAggregatorInfos, + Integer segmentAggregatedCount, + long dataFilePointer, + long dataFileLength + ) throws IOException { + StarTreeMetaSerializer.serializeStarTreeMetadata( + metaOut, + CompositeMappedFieldType.CompositeFieldType.STAR_TREE, + starTreeField, + writeState, + metricAggregatorInfos, + segmentAggregatedCount, + dataFilePointer, + dataFileLength + ); + } + +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDataSerializer.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDataSerializer.java new file mode 100644 index 0000000000000..54059458a6d0d --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDataSerializer.java @@ -0,0 +1,138 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.compositeindex.datacube.startree.utils; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.store.IndexOutput; + +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.CompositeIndexConstants.MAGIC_MARKER; +import static org.opensearch.index.compositeindex.CompositeIndexConstants.VERSION; +import static org.opensearch.index.compositeindex.datacube.startree.node.OffHeapStarTreeNode.SERIALIZABLE_DATA_SIZE_IN_BYTES; +import static org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeBuilderUtils.ALL; + +/** + * Utility class for serializing a star-tree data structure. + * + * @opensearch.experimental + */ +public class StarTreeDataSerializer { + + private static final Logger logger = LogManager.getLogger(StarTreeDataSerializer.class); + + /** + * Serializes the star-tree data structure. + * + * @param indexOutput the IndexOutput to write the star-tree data + * @param rootNode the root node of the star-tree + * @param numNodes the total number of nodes in the star-tree + * @return the total size in bytes of the serialized star-tree data + * @throws IOException if an I/O error occurs while writing the star-tree data + */ + public static long serializeStarTree(IndexOutput indexOutput, StarTreeBuilderUtils.TreeNode rootNode, int numNodes) throws IOException { + int headerSizeInBytes = computeStarTreeDataHeaderByteSize(); + long totalSizeInBytes = headerSizeInBytes + (long) numNodes * SERIALIZABLE_DATA_SIZE_IN_BYTES; + + logger.info("Star tree size in bytes : {}", totalSizeInBytes); + + writeStarTreeHeader(indexOutput, numNodes); + writeStarTreeNodes(indexOutput, rootNode); + return totalSizeInBytes; + } + + /** + * Computes the byte size of the star-tree data header. + * + * @return the byte size of the star-tree data header + */ + private static int computeStarTreeDataHeaderByteSize() { + // Magic marker (8), version (4) + int headerSizeInBytes = 12; + + // For number of nodes. + headerSizeInBytes += Integer.BYTES; + return headerSizeInBytes; + } + + /** + * Writes the star-tree data header. + * + * @param output the IndexOutput to write the header + * @param numNodes the total number of nodes in the star-tree + * @throws IOException if an I/O error occurs while writing the header + */ + private static void writeStarTreeHeader(IndexOutput output, int numNodes) throws IOException { + output.writeLong(MAGIC_MARKER); + output.writeInt(VERSION); + output.writeInt(numNodes); + } + + /** + * Writes the star-tree nodes in a breadth-first order. + * + * @param output the IndexOutput to write the nodes + * @param rootNode the root node of the star-tree + * @throws IOException if an I/O error occurs while writing the nodes + */ + private static void writeStarTreeNodes(IndexOutput output, StarTreeBuilderUtils.TreeNode rootNode) throws IOException { + Queue queue = new LinkedList<>(); + queue.add(rootNode); + + int currentNodeId = 0; + while (!queue.isEmpty()) { + StarTreeBuilderUtils.TreeNode node = queue.remove(); + + if (node.children == null) { + writeStarTreeNode(output, node, ALL, ALL); + } else { + + // Sort all children nodes based on dimension value + List sortedChildren = new ArrayList<>(node.children.values()); + sortedChildren.sort(Comparator.comparingLong(o -> o.dimensionValue)); + + int firstChildId = currentNodeId + queue.size() + 1; + int lastChildId = firstChildId + sortedChildren.size() - 1; + writeStarTreeNode(output, node, firstChildId, lastChildId); + + queue.addAll(sortedChildren); + } + + currentNodeId++; + } + } + + /** + * Writes a single star-tree node + * + * @param output the IndexOutput to write the node + * @param node the star tree node to write + * @param firstChildId the ID of the first child node + * @param lastChildId the ID of the last child node + * @throws IOException if an I/O error occurs while writing the node + */ + private static void writeStarTreeNode(IndexOutput output, StarTreeBuilderUtils.TreeNode 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.isStarNode == false ? (byte) 0 : (byte) 1); + output.writeInt(firstChildId); + output.writeInt(lastChildId); + } + +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeHelper.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeHelper.java new file mode 100644 index 0000000000000..4192d6deff99e --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeHelper.java @@ -0,0 +1,38 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.compositeindex.datacube.startree.utils; + +import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricAggregatorInfo; + +/** + * This class contains helper methods used throughout the Star Tree index implementation. + * + * @opensearch.experimental + */ +public class StarTreeHelper { + + /** + * The suffix appended to dimension field names in the Star Tree index. + */ + public static final String DIMENSION_SUFFIX = "_dim"; + + /** + * The suffix appended to metric field names in the Star Tree index. + */ + public static final String METRIC_SUFFIX = "_metric"; + + public static String fullFieldNameForStarTreeDimensionsDocValues(String starTreeFieldName, String dimensionName) { + return starTreeFieldName + "_" + dimensionName + "_" + DIMENSION_SUFFIX; + } + + public static String fullFieldNameForStarTreeMetricsDocValues(String name, String fieldName, String metricName) { + return MetricAggregatorInfo.toFieldName(name, fieldName, metricName) + "_" + METRIC_SUFFIX; + } + +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeMetaSerializer.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeMetaSerializer.java new file mode 100644 index 0000000000000..6f4b15611a03a --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeMetaSerializer.java @@ -0,0 +1,223 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.compositeindex.datacube.startree.utils; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.IndexOutput; +import org.opensearch.index.compositeindex.datacube.Dimension; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricAggregatorInfo; +import org.opensearch.index.mapper.CompositeMappedFieldType; + +import java.io.IOException; +import java.util.List; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.opensearch.index.compositeindex.CompositeIndexConstants.MAGIC_MARKER; +import static org.opensearch.index.compositeindex.CompositeIndexConstants.VERSION; + +/** + * The utility class for serializing the metadata of a star-tree data structure. + * The metadata includes information about the dimensions, metrics, and other relevant details + * related to the star tree. + * + * @opensearch.experimental + */ +public class StarTreeMetaSerializer { + + private static final Logger logger = LogManager.getLogger(StarTreeMetaSerializer.class); + + /** + * Serializes the star-tree metadata. + * + * @param metaOut the IndexOutput to write the metadata + * @param compositeFieldType the composite field type of the star-tree field + * @param starTreeField the star-tree field + * @param writeState the segment write state + * @param metricAggregatorInfos the list of metric aggregator information + * @param segmentAggregatedCount the aggregated document count 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 serializing the metadata + */ + public static void serializeStarTreeMetadata( + IndexOutput metaOut, + CompositeMappedFieldType.CompositeFieldType compositeFieldType, + StarTreeField starTreeField, + SegmentWriteState writeState, + List metricAggregatorInfos, + Integer segmentAggregatedCount, + long dataFilePointer, + long dataFileLength + ) throws IOException { + long totalSizeInBytes = 0; + + // header size + totalSizeInBytes += computeHeaderByteSize(compositeFieldType, starTreeField.getName()); + // number of dimensions + totalSizeInBytes += Integer.BYTES; + // dimension field numbers + totalSizeInBytes += (long) starTreeField.getDimensionsOrder().size() * Integer.BYTES; + // metric count + totalSizeInBytes += Integer.BYTES; + // metric - metric stat pair + totalSizeInBytes += computeMetricEntriesSizeInBytes(metricAggregatorInfos); + // segment aggregated document count + totalSizeInBytes += Integer.BYTES; + // max leaf docs + totalSizeInBytes += Integer.BYTES; + // skip star node creation dimensions count + totalSizeInBytes += Integer.BYTES; + // skip star node creation dimensions field numbers + totalSizeInBytes += (long) starTreeField.getStarTreeConfig().getSkipStarNodeCreationInDims().size() * Integer.BYTES; + // data start file pointer + totalSizeInBytes += Long.BYTES; + // data length + totalSizeInBytes += Long.BYTES; + + logger.info("Star tree size in bytes : {}", totalSizeInBytes); + + writeMetaHeader(metaOut, compositeFieldType, starTreeField.getName()); + writeMeta(metaOut, writeState, metricAggregatorInfos, starTreeField, segmentAggregatedCount, dataFilePointer, dataFileLength); + } + + /** + * Computes the byte size required to store the star-tree metric entry. + * + * @param metricAggregatorInfos the list of metric aggregator information + * @return the byte size required to store the metric-metric stat pairs + */ + private static long computeMetricEntriesSizeInBytes(List metricAggregatorInfos) { + + long totalMetricEntriesSize = 0; + + for (MetricAggregatorInfo metricAggregatorInfo : metricAggregatorInfos) { + totalMetricEntriesSize += metricAggregatorInfo.getField().getBytes(UTF_8).length; + totalMetricEntriesSize += metricAggregatorInfo.getMetricStat().getTypeName().getBytes(UTF_8).length; + } + + return totalMetricEntriesSize; + } + + /** + * Computes the byte size of the star-tree metadata header. + * + * @param compositeFieldType the composite field type of the star-tree field + * @param starTreeFieldName the name of the star-tree field + * @return the byte size of the star-tree metadata header + */ + private static int computeHeaderByteSize(CompositeMappedFieldType.CompositeFieldType compositeFieldType, String starTreeFieldName) { + // Magic marker (8), version (4), size of header (4) + int headerSizeInBytes = 16; + + // For star-tree field name + headerSizeInBytes += starTreeFieldName.getBytes(UTF_8).length; + + // For star tree field type + headerSizeInBytes += compositeFieldType.getName().getBytes(UTF_8).length; + + return headerSizeInBytes; + } + + /** + * Writes the star-tree metadata header. + * + * @param metaOut the IndexOutput to write the header + * @param compositeFieldType the composite field type of the star-tree field + * @param starTreeFieldName the name of the star-tree field + * @throws IOException if an I/O error occurs while writing the header + */ + private static void writeMetaHeader( + IndexOutput metaOut, + CompositeMappedFieldType.CompositeFieldType compositeFieldType, + String starTreeFieldName + ) throws IOException { + // magic marker for sanity + metaOut.writeLong(MAGIC_MARKER); + + // version + metaOut.writeInt(VERSION); + + // star tree field name + metaOut.writeString(starTreeFieldName); + + // star tree field type + metaOut.writeString(compositeFieldType.getName()); + } + + /** + * Writes the star-tree metadata. + * + * @param metaOut the IndexOutput to write the metadata + * @param writeState the segment write state + * @param metricAggregatorInfos the list of metric aggregator information + * @param starTreeField the star tree field + * @param segmentAggregatedDocCount the aggregated document count 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 + */ + private static void writeMeta( + IndexOutput metaOut, + SegmentWriteState writeState, + List metricAggregatorInfos, + StarTreeField starTreeField, + Integer segmentAggregatedDocCount, + long dataFilePointer, + long dataFileLength + ) throws IOException { + + // number of dimensions + metaOut.writeInt(starTreeField.getDimensionsOrder().size()); + + // dimensions + for (Dimension dimension : starTreeField.getDimensionsOrder()) { + int dimensionFieldNumber = writeState.fieldInfos.fieldInfo(dimension.getField()).getFieldNumber(); + metaOut.writeInt(dimensionFieldNumber); + } + + // number of metrics + metaOut.writeInt(metricAggregatorInfos.size()); + + // metric - metric stat pair + for (MetricAggregatorInfo metricAggregatorInfo : metricAggregatorInfos) { + String metricName = metricAggregatorInfo.getField(); + String metricStatName = metricAggregatorInfo.getMetricStat().getTypeName(); + metaOut.writeString(metricName); + metaOut.writeString(metricStatName); + } + + // segment aggregated document count + metaOut.writeInt(segmentAggregatedDocCount); + + // max leaf docs + metaOut.writeInt(starTreeField.getStarTreeConfig().maxLeafDocs()); + + // number of skip star node creation dimensions + metaOut.writeInt(starTreeField.getStarTreeConfig().maxLeafDocs()); + + // skip star node creations + for (String dimension : starTreeField.getStarTreeConfig().getSkipStarNodeCreationInDims()) { + int dimensionFieldNumber = writeState.fieldInfos.fieldInfo(dimension).getFieldNumber(); + metaOut.writeInt(dimensionFieldNumber); + } + + // star tree build-mode + metaOut.writeString(starTreeField.getStarTreeConfig().getBuildMode().getTypeName()); + + // star-tree data file pointer + metaOut.writeLong(dataFilePointer); + + // star-tree data file length + metaOut.writeLong(dataFileLength); + + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/package-info.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/package-info.java new file mode 100644 index 0000000000000..c7e8b04d42178 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/package-info.java @@ -0,0 +1,14 @@ +/* + * 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. + */ + +/** + * Utility to support Composite Index Star Tree + * + * @opensearch.experimental + */ +package org.opensearch.index.compositeindex.datacube.startree.utils; diff --git a/server/src/main/java/org/opensearch/index/compositeindex/package-info.java b/server/src/main/java/org/opensearch/index/compositeindex/package-info.java index 59f18efec26b1..9a88f88d9850a 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/package-info.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/package-info.java @@ -8,6 +8,7 @@ /** * Core classes for handling composite indices. - * @opensearch.experimental + * + * @opensearch.experimental */ package org.opensearch.index.compositeindex; diff --git a/server/src/main/java/org/opensearch/index/mapper/CompositeMappedFieldType.java b/server/src/main/java/org/opensearch/index/mapper/CompositeMappedFieldType.java index e067e70621304..f4bdb19abd8cc 100644 --- a/server/src/main/java/org/opensearch/index/mapper/CompositeMappedFieldType.java +++ b/server/src/main/java/org/opensearch/index/mapper/CompositeMappedFieldType.java @@ -68,10 +68,14 @@ public static CompositeFieldType fromName(String name) { return metric; } } - throw new IllegalArgumentException("Invalid metric stat: " + name); + throw new IllegalArgumentException("Invalid composite field type: " + name); } } + public CompositeFieldType getCompositeIndexType() { + return type; + } + public List fields() { return fields; } diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java index 3f2edd1a6c5a5..fe9ed57fa77b8 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java @@ -8,7 +8,9 @@ package org.opensearch.gateway.remote; +import org.opensearch.Version; import org.opensearch.action.LatchedActionListener; +import org.opensearch.cluster.AbstractNamedDiffable; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.ClusterState.Custom; @@ -21,8 +23,11 @@ import org.opensearch.common.util.TestCapturingListener; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.compress.Compressor; import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.gateway.remote.model.RemoteClusterBlocks; import org.opensearch.gateway.remote.model.RemoteClusterStateCustoms; import org.opensearch.gateway.remote.model.RemoteDiscoveryNodes; @@ -46,10 +51,6 @@ import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.CLUSTER_STATE_ATTRIBUTE; import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.CLUSTER_STATE_ATTRIBUTES_CURRENT_CODEC_VERSION; import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.DISCOVERY_NODES; -import static org.opensearch.gateway.remote.RemoteClusterStateTestUtils.TestClusterStateCustom1; -import static org.opensearch.gateway.remote.RemoteClusterStateTestUtils.TestClusterStateCustom2; -import static org.opensearch.gateway.remote.RemoteClusterStateTestUtils.TestClusterStateCustom3; -import static org.opensearch.gateway.remote.RemoteClusterStateTestUtils.TestClusterStateCustom4; import static org.opensearch.gateway.remote.RemoteClusterStateUtils.CLUSTER_STATE_EPHEMERAL_PATH_TOKEN; import static org.opensearch.gateway.remote.RemoteClusterStateUtils.CLUSTER_STATE_PATH_TOKEN; import static org.opensearch.gateway.remote.RemoteClusterStateUtils.CUSTOM_DELIMITER; @@ -337,22 +338,22 @@ public void testGetAsyncMetadataReadAction_Exception() throws IOException, Inter public void testGetUpdatedCustoms() { Map previousCustoms = Map.of( - TestClusterStateCustom1.TYPE, - new TestClusterStateCustom1("data1"), - TestClusterStateCustom2.TYPE, - new TestClusterStateCustom2("data2"), - TestClusterStateCustom3.TYPE, - new TestClusterStateCustom3("data3") + TestCustom1.TYPE, + new TestCustom1("data1"), + TestCustom2.TYPE, + new TestCustom2("data2"), + TestCustom3.TYPE, + new TestCustom3("data3") ); ClusterState previousState = ClusterState.builder(new ClusterName("test-cluster")).customs(previousCustoms).build(); Map currentCustoms = Map.of( - TestClusterStateCustom2.TYPE, - new TestClusterStateCustom2("data2"), - TestClusterStateCustom3.TYPE, - new TestClusterStateCustom3("data3-changed"), - TestClusterStateCustom4.TYPE, - new TestClusterStateCustom4("data4") + TestCustom2.TYPE, + new TestCustom2("data2"), + TestCustom3.TYPE, + new TestCustom3("data3-changed"), + TestCustom4.TYPE, + new TestCustom4("data4") ); ClusterState currentState = ClusterState.builder(new ClusterName("test-cluster")).customs(currentCustoms).build(); @@ -367,14 +368,136 @@ public void testGetUpdatedCustoms() { assertThat(customsDiff.getDeletes(), is(Collections.emptyList())); Map expectedCustoms = Map.of( - TestClusterStateCustom3.TYPE, - new TestClusterStateCustom3("data3-changed"), - TestClusterStateCustom4.TYPE, - new TestClusterStateCustom4("data4") + TestCustom3.TYPE, + new TestCustom3("data3-changed"), + TestCustom4.TYPE, + new TestCustom4("data4") ); customsDiff = remoteClusterStateAttributesManager.getUpdatedCustoms(currentState, previousState, true, false); assertThat(customsDiff.getUpserts(), is(expectedCustoms)); - assertThat(customsDiff.getDeletes(), is(List.of(TestClusterStateCustom1.TYPE))); + assertThat(customsDiff.getDeletes(), is(List.of(TestCustom1.TYPE))); + } + + private static abstract class AbstractTestCustom extends AbstractNamedDiffable implements ClusterState.Custom { + + private final String value; + + AbstractTestCustom(String value) { + this.value = value; + } + + AbstractTestCustom(StreamInput in) throws IOException { + this.value = in.readString(); + } + + @Override + public Version getMinimalSupportedVersion() { + return Version.CURRENT; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(value); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder; + } + + @Override + public boolean isPrivate() { + return true; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + AbstractTestCustom that = (AbstractTestCustom) o; + + if (!value.equals(that.value)) return false; + + return true; + } + + @Override + public int hashCode() { + return value.hashCode(); + } + } + + private static class TestCustom1 extends AbstractTestCustom { + + private static final String TYPE = "custom_1"; + + TestCustom1(String value) { + super(value); + } + + TestCustom1(StreamInput in) throws IOException { + super(in); + } + + @Override + public String getWriteableName() { + return TYPE; + } + } + + private static class TestCustom2 extends AbstractTestCustom { + + private static final String TYPE = "custom_2"; + + TestCustom2(String value) { + super(value); + } + + TestCustom2(StreamInput in) throws IOException { + super(in); + } + + @Override + public String getWriteableName() { + return TYPE; + } + } + + private static class TestCustom3 extends AbstractTestCustom { + + private static final String TYPE = "custom_3"; + + TestCustom3(String value) { + super(value); + } + + TestCustom3(StreamInput in) throws IOException { + super(in); + } + + @Override + public String getWriteableName() { + return TYPE; + } + } + + private static class TestCustom4 extends AbstractTestCustom { + + private static final String TYPE = "custom_4"; + + TestCustom4(String value) { + super(value); + } + + TestCustom4(StreamInput in) throws IOException { + super(in); + } + + @Override + public String getWriteableName() { + return TYPE; + } } } diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java index 6cd9cbbf13848..d983a4d8c4027 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java @@ -9,14 +9,12 @@ package org.opensearch.gateway.remote; import org.opensearch.Version; -import org.opensearch.action.LatchedActionListener; import org.opensearch.cluster.ClusterModule; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.RepositoryCleanupInProgress; -import org.opensearch.cluster.block.ClusterBlocks; +import org.opensearch.cluster.RepositoryCleanupInProgress.Entry; import org.opensearch.cluster.coordination.CoordinationMetadata; -import org.opensearch.cluster.metadata.DiffableStringMap; import org.opensearch.cluster.metadata.IndexGraveyard; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.IndexTemplateMetadata; @@ -24,12 +22,10 @@ import org.opensearch.cluster.metadata.TemplatesMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.routing.remote.InternalRemoteRoutingTableService; import org.opensearch.cluster.routing.remote.NoopRemoteRoutingTableService; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.CheckedRunnable; import org.opensearch.common.blobstore.AsyncMultiStreamBlobContainer; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobMetadata; @@ -42,7 +38,6 @@ import org.opensearch.common.compress.DeflateCompressor; import org.opensearch.common.lucene.store.ByteArrayIndexInput; import org.opensearch.common.network.NetworkModule; -import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; @@ -50,17 +45,13 @@ import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; -import org.opensearch.core.common.io.stream.NamedWriteableRegistry; -import org.opensearch.core.compress.Compressor; import org.opensearch.core.index.Index; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedIndexMetadata; import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; import org.opensearch.gateway.remote.model.RemoteClusterMetadataManifest; import org.opensearch.gateway.remote.model.RemoteClusterStateManifestInfo; -import org.opensearch.gateway.remote.model.RemotePersistentSettingsMetadata; -import org.opensearch.gateway.remote.model.RemoteReadResult; -import org.opensearch.gateway.remote.model.RemoteTransientSettingsMetadata; +import org.opensearch.gateway.remote.model.RemoteIndexMetadata; import org.opensearch.index.remote.RemoteIndexPathUploader; import org.opensearch.indices.IndicesModule; import org.opensearch.repositories.FilterRepository; @@ -70,6 +61,7 @@ import org.opensearch.repositories.blobstore.ChecksumWritableBlobStoreFormat; import org.opensearch.repositories.fs.FsRepository; import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.TestCustomMetadata; import org.opensearch.test.VersionUtils; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; @@ -83,6 +75,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.EnumSet; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -99,51 +92,23 @@ import java.util.stream.Stream; import org.mockito.ArgumentCaptor; -import org.mockito.ArgumentMatcher; import org.mockito.ArgumentMatchers; import org.mockito.Mockito; -import static java.util.Collections.emptyList; -import static java.util.Collections.emptyMap; import static java.util.stream.Collectors.toList; import static org.opensearch.common.util.FeatureFlags.REMOTE_PUBLICATION_EXPERIMENTAL; import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V1; -import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V2; -import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.CLUSTER_BLOCKS; -import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.CLUSTER_STATE_ATTRIBUTE; -import static org.opensearch.gateway.remote.RemoteClusterStateTestUtils.CustomMetadata1; -import static org.opensearch.gateway.remote.RemoteClusterStateTestUtils.CustomMetadata2; -import static org.opensearch.gateway.remote.RemoteClusterStateTestUtils.CustomMetadata3; -import static org.opensearch.gateway.remote.RemoteClusterStateTestUtils.TestClusterStateCustom1; -import static org.opensearch.gateway.remote.RemoteClusterStateTestUtils.TestClusterStateCustom2; -import static org.opensearch.gateway.remote.RemoteClusterStateTestUtils.TestClusterStateCustom3; import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; import static org.opensearch.gateway.remote.RemoteClusterStateUtils.FORMAT_PARAMS; import static org.opensearch.gateway.remote.RemoteClusterStateUtils.getFormattedIndexFileName; -import static org.opensearch.gateway.remote.model.RemoteClusterBlocks.CLUSTER_BLOCKS_FORMAT; -import static org.opensearch.gateway.remote.model.RemoteClusterBlocksTests.randomClusterBlocks; import static org.opensearch.gateway.remote.model.RemoteClusterMetadataManifest.MANIFEST_CURRENT_CODEC_VERSION; -import static org.opensearch.gateway.remote.model.RemoteClusterStateCustoms.CLUSTER_STATE_CUSTOM; import static org.opensearch.gateway.remote.model.RemoteCoordinationMetadata.COORDINATION_METADATA; import static org.opensearch.gateway.remote.model.RemoteCoordinationMetadata.COORDINATION_METADATA_FORMAT; -import static org.opensearch.gateway.remote.model.RemoteCustomMetadata.CUSTOM_DELIMITER; -import static org.opensearch.gateway.remote.model.RemoteCustomMetadata.CUSTOM_METADATA; -import static org.opensearch.gateway.remote.model.RemoteCustomMetadata.readFrom; -import static org.opensearch.gateway.remote.model.RemoteDiscoveryNodes.DISCOVERY_NODES; -import static org.opensearch.gateway.remote.model.RemoteDiscoveryNodes.DISCOVERY_NODES_FORMAT; -import static org.opensearch.gateway.remote.model.RemoteDiscoveryNodesTests.getDiscoveryNodes; import static org.opensearch.gateway.remote.model.RemoteGlobalMetadata.GLOBAL_METADATA_FORMAT; -import static org.opensearch.gateway.remote.model.RemoteHashesOfConsistentSettings.HASHES_OF_CONSISTENT_SETTINGS; -import static org.opensearch.gateway.remote.model.RemoteHashesOfConsistentSettings.HASHES_OF_CONSISTENT_SETTINGS_FORMAT; -import static org.opensearch.gateway.remote.model.RemoteHashesOfConsistentSettingsTests.getHashesOfConsistentSettings; -import static org.opensearch.gateway.remote.model.RemoteIndexMetadata.INDEX; -import static org.opensearch.gateway.remote.model.RemoteIndexMetadata.INDEX_METADATA_FORMAT; import static org.opensearch.gateway.remote.model.RemotePersistentSettingsMetadata.SETTINGS_METADATA_FORMAT; import static org.opensearch.gateway.remote.model.RemotePersistentSettingsMetadata.SETTING_METADATA; import static org.opensearch.gateway.remote.model.RemoteTemplatesMetadata.TEMPLATES_METADATA; import static org.opensearch.gateway.remote.model.RemoteTemplatesMetadata.TEMPLATES_METADATA_FORMAT; -import static org.opensearch.gateway.remote.model.RemoteTemplatesMetadataTests.getTemplatesMetadata; -import static org.opensearch.gateway.remote.model.RemoteTransientSettingsMetadata.TRANSIENT_SETTING_METADATA; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT; @@ -155,19 +120,11 @@ import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.ArgumentMatchers.anyList; -import static org.mockito.ArgumentMatchers.anyMap; import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class RemoteClusterStateServiceTests extends OpenSearchTestCase { @@ -178,22 +135,11 @@ public class RemoteClusterStateServiceTests extends OpenSearchTestCase { private Supplier repositoriesServiceSupplier; private RepositoriesService repositoriesService; private BlobStoreRepository blobStoreRepository; - private Compressor compressor; private BlobStore blobStore; private Settings settings; private boolean publicationEnabled; - private NamedWriteableRegistry namedWriteableRegistry; private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); - private static final String NODE_ID = "test-node"; - private static final String COORDINATION_METADATA_FILENAME = "coordination-metadata-file__1"; - private static final String PERSISTENT_SETTINGS_FILENAME = "persistent-settings-file__1"; - private static final String TRANSIENT_SETTINGS_FILENAME = "transient-settings-file__1"; - private static final String TEMPLATES_METADATA_FILENAME = "templates-metadata-file__1"; - private static final String DISCOVERY_NODES_FILENAME = "discovery-nodes-file__1"; - private static final String CLUSTER_BLOCKS_FILENAME = "cluster-blocks-file__1"; - private static final String HASHES_OF_CONSISTENT_SETTINGS_FILENAME = "consistent-settings-hashes-file__1"; - @Before public void setup() { repositoriesServiceSupplier = mock(Supplier.class); @@ -218,11 +164,6 @@ public void setup() { .put(RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true) .put("node.attr." + REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY, "routing_repository") .build(); - List writeableEntries = ClusterModule.getNamedWriteables(); - writeableEntries.add(new NamedWriteableRegistry.Entry(Metadata.Custom.class, CustomMetadata1.TYPE, CustomMetadata1::new)); - writeableEntries.add(new NamedWriteableRegistry.Entry(Metadata.Custom.class, CustomMetadata2.TYPE, CustomMetadata2::new)); - writeableEntries.add(new NamedWriteableRegistry.Entry(Metadata.Custom.class, CustomMetadata3.TYPE, CustomMetadata3::new)); - namedWriteableRegistry = new NamedWriteableRegistry(writeableEntries); clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); clusterService = mock(ClusterService.class); @@ -235,7 +176,6 @@ public void setup() { ).flatMap(Function.identity()).collect(toList()) ); - compressor = new DeflateCompressor(); blobStoreRepository = mock(BlobStoreRepository.class); blobStore = mock(BlobStore.class); when(blobStoreRepository.blobStore()).thenReturn(blobStore); @@ -251,7 +191,7 @@ public void setup() { () -> 0L, threadPool, List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)), - namedWriteableRegistry + writableRegistry() ); } @@ -335,7 +275,6 @@ public void testWriteFullMetadataSuccess() throws IOException { assertThat(manifest.getSettingsMetadata(), notNullValue()); assertThat(manifest.getTemplatesMetadata(), notNullValue()); assertFalse(manifest.getCustomMetadataMap().isEmpty()); - assertThat(manifest.getCustomMetadataMap().containsKey(CustomMetadata1.TYPE), is(true)); assertThat(manifest.getClusterBlocksMetadata(), nullValue()); assertThat(manifest.getDiscoveryNodesMetadata(), nullValue()); assertThat(manifest.getTransientSettingsMetadata(), nullValue()); @@ -359,12 +298,7 @@ public void testWriteFullMetadataSuccessPublicationEnabled() throws IOException writableRegistry() ); final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()) - .customs( - Map.of( - RepositoryCleanupInProgress.TYPE, - new RepositoryCleanupInProgress(List.of(new RepositoryCleanupInProgress.Entry("test-repo", 10L))) - ) - ) + .customs(Map.of(RepositoryCleanupInProgress.TYPE, new RepositoryCleanupInProgress(List.of(new Entry("test-repo", 10L))))) .build(); mockBlobStoreObjects(); remoteClusterStateService.start(); @@ -396,7 +330,6 @@ public void testWriteFullMetadataSuccessPublicationEnabled() throws IOException assertThat(manifest.getSettingsMetadata(), notNullValue()); assertThat(manifest.getTemplatesMetadata(), notNullValue()); assertFalse(manifest.getCustomMetadataMap().isEmpty()); - assertThat(manifest.getCustomMetadataMap().containsKey(CustomMetadata1.TYPE), is(true)); assertThat(manifest.getClusterStateCustomMap().size(), is(1)); assertThat(manifest.getClusterStateCustomMap().containsKey(RepositoryCleanupInProgress.TYPE), is(true)); } @@ -455,7 +388,7 @@ public void testWriteFullMetadataInParallelSuccess() throws IOException { .provideStream(0) .getInputStream() .readAllBytes(); - IndexMetadata writtenIndexMetadata = INDEX_METADATA_FORMAT.deserialize( + IndexMetadata writtenIndexMetadata = RemoteIndexMetadata.INDEX_METADATA_FORMAT.deserialize( capturedWriteContext.get("metadata").getFileName(), blobStoreRepository.getNamedXContentRegistry(), new BytesArray(writtenBytes) @@ -512,35 +445,24 @@ public void testTimeoutWhileWritingManifestFile() throws IOException { ArgumentCaptor> actionListenerArgumentCaptor = ArgumentCaptor.forClass(ActionListener.class); - doAnswer((i) -> { + doAnswer((i) -> { // For Global Metadata + actionListenerArgumentCaptor.getValue().onResponse(null); + return null; + }).doAnswer((i) -> { // For Index Metadata + actionListenerArgumentCaptor.getValue().onResponse(null); + return null; + }).doAnswer((i) -> { // For Manifest file perform No Op, so latch in code will timeout return null; }).when(container).asyncBlobUpload(any(WriteContext.class), actionListenerArgumentCaptor.capture()); remoteClusterStateService.start(); - RemoteClusterStateService spiedService = spy(remoteClusterStateService); - when( - spiedService.writeMetadataInParallel( - any(), - anyList(), - anyMap(), - anyMap(), - anyBoolean(), - anyBoolean(), - anyBoolean(), - anyBoolean(), - anyBoolean(), - anyBoolean(), - anyMap(), - anyBoolean(), - anyList() - ) - ).thenReturn(new RemoteClusterStateUtils.UploadedMetadataResults()); - RemoteStateTransferException ex = expectThrows( - RemoteStateTransferException.class, - () -> spiedService.writeFullMetadata(clusterState, randomAlphaOfLength(10)) - ); - assertTrue(ex.getMessage().contains("Timed out waiting for transfer of manifest file to complete")); + try { + remoteClusterStateService.writeFullMetadata(clusterState, randomAlphaOfLength(10)); + } catch (Exception e) { + assertTrue(e instanceof RemoteStateTransferException); + assertTrue(e.getMessage().contains("Timed out waiting for transfer of following metadata to complete")); + } } public void testWriteFullMetadataInParallelFailureForIndexMetadata() throws IOException { @@ -736,991 +658,6 @@ public void testWriteIncrementalMetadataSuccessWhenPublicationEnabled() throws I assertThat(manifest.getIndicesRouting().size(), is(1)); } - public void testTimeoutWhileWritingMetadata() throws IOException { - AsyncMultiStreamBlobContainer container = (AsyncMultiStreamBlobContainer) mockBlobStoreObjects(AsyncMultiStreamBlobContainer.class); - doNothing().when(container).asyncBlobUpload(any(), any()); - int writeTimeout = 2; - Settings newSettings = Settings.builder() - .put("cluster.remote_store.state.global_metadata.upload_timeout", writeTimeout + "s") - .build(); - clusterSettings.applySettings(newSettings); - remoteClusterStateService.start(); - RemoteStateTransferException exception = assertThrows( - RemoteStateTransferException.class, - () -> remoteClusterStateService.writeMetadataInParallel( - ClusterState.EMPTY_STATE, - emptyList(), - emptyMap(), - emptyMap(), - true, - true, - true, - true, - true, - true, - emptyMap(), - true, - emptyList() - ) - ); - assertTrue(exception.getMessage().startsWith("Timed out waiting for transfer of following metadata to complete")); - } - - public void testGetClusterStateForManifest_IncludeEphemeral() throws IOException { - ClusterMetadataManifest manifest = generateClusterMetadataManifestWithAllAttributes().build(); - mockBlobStoreObjects(); - remoteClusterStateService.start(); - RemoteReadResult mockedResult = mock(RemoteReadResult.class); - RemoteIndexMetadataManager mockedIndexManager = mock(RemoteIndexMetadataManager.class); - RemoteGlobalMetadataManager mockedGlobalMetadataManager = mock(RemoteGlobalMetadataManager.class); - RemoteClusterStateAttributesManager mockedClusterStateAttributeManager = mock(RemoteClusterStateAttributesManager.class); - remoteClusterStateService.setRemoteIndexMetadataManager(mockedIndexManager); - remoteClusterStateService.setRemoteGlobalMetadataManager(mockedGlobalMetadataManager); - remoteClusterStateService.setRemoteClusterStateAttributesManager(mockedClusterStateAttributeManager); - ArgumentCaptor> listenerArgumentCaptor = ArgumentCaptor.forClass( - LatchedActionListener.class - ); - when(mockedIndexManager.getAsyncIndexMetadataReadAction(any(), anyString(), listenerArgumentCaptor.capture())).thenReturn( - () -> listenerArgumentCaptor.getValue().onResponse(mockedResult) - ); - when(mockedGlobalMetadataManager.getAsyncMetadataReadAction(any(), anyString(), listenerArgumentCaptor.capture())).thenReturn( - () -> listenerArgumentCaptor.getValue().onResponse(mockedResult) - ); - when(mockedClusterStateAttributeManager.getAsyncMetadataReadAction(anyString(), any(), listenerArgumentCaptor.capture())) - .thenReturn(() -> listenerArgumentCaptor.getValue().onResponse(mockedResult)); - when(mockedResult.getComponent()).thenReturn(COORDINATION_METADATA); - RemoteClusterStateService mockService = spy(remoteClusterStateService); - mockService.getClusterStateForManifest(ClusterName.DEFAULT.value(), manifest, NODE_ID, true); - verify(mockService, times(1)).readClusterStateInParallel( - any(), - eq(manifest), - eq(manifest.getClusterUUID()), - eq(NODE_ID), - eq(manifest.getIndices()), - eq(manifest.getCustomMetadataMap()), - eq(true), - eq(true), - eq(true), - eq(true), - eq(true), - eq(true), - eq(manifest.getIndicesRouting()), - eq(true), - eq(manifest.getClusterStateCustomMap()), - eq(true) - ); - } - - public void testGetClusterStateForManifest_ExcludeEphemeral() throws IOException { - ClusterMetadataManifest manifest = generateClusterMetadataManifestWithAllAttributes().build(); - mockBlobStoreObjects(); - remoteClusterStateService.start(); - RemoteReadResult mockedResult = mock(RemoteReadResult.class); - RemoteIndexMetadataManager mockedIndexManager = mock(RemoteIndexMetadataManager.class); - RemoteGlobalMetadataManager mockedGlobalMetadataManager = mock(RemoteGlobalMetadataManager.class); - RemoteClusterStateAttributesManager mockedClusterStateAttributeManager = mock(RemoteClusterStateAttributesManager.class); - ArgumentCaptor> listenerArgumentCaptor = ArgumentCaptor.forClass( - LatchedActionListener.class - ); - when(mockedIndexManager.getAsyncIndexMetadataReadAction(any(), anyString(), listenerArgumentCaptor.capture())).thenReturn( - () -> listenerArgumentCaptor.getValue().onResponse(mockedResult) - ); - when(mockedGlobalMetadataManager.getAsyncMetadataReadAction(any(), anyString(), listenerArgumentCaptor.capture())).thenReturn( - () -> listenerArgumentCaptor.getValue().onResponse(mockedResult) - ); - when(mockedClusterStateAttributeManager.getAsyncMetadataReadAction(anyString(), any(), listenerArgumentCaptor.capture())) - .thenReturn(() -> listenerArgumentCaptor.getValue().onResponse(mockedResult)); - when(mockedResult.getComponent()).thenReturn(COORDINATION_METADATA); - remoteClusterStateService.setRemoteIndexMetadataManager(mockedIndexManager); - remoteClusterStateService.setRemoteGlobalMetadataManager(mockedGlobalMetadataManager); - remoteClusterStateService.setRemoteClusterStateAttributesManager(mockedClusterStateAttributeManager); - RemoteClusterStateService spiedService = spy(remoteClusterStateService); - spiedService.getClusterStateForManifest(ClusterName.DEFAULT.value(), manifest, NODE_ID, false); - verify(spiedService, times(1)).readClusterStateInParallel( - any(), - eq(manifest), - eq(manifest.getClusterUUID()), - eq(NODE_ID), - eq(manifest.getIndices()), - eq(manifest.getCustomMetadataMap()), - eq(true), - eq(true), - eq(false), - eq(true), - eq(false), - eq(false), - eq(emptyList()), - eq(false), - eq(emptyMap()), - eq(false) - ); - } - - public void testGetClusterStateFromManifest_CodecV1() throws IOException { - ClusterMetadataManifest manifest = generateClusterMetadataManifestWithAllAttributes().codecVersion(CODEC_V1).build(); - mockBlobStoreObjects(); - remoteClusterStateService.start(); - final Index index = new Index("test-index", "index-uuid"); - final Settings idxSettings = Settings.builder() - .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetadata.SETTING_INDEX_UUID, index.getUUID()) - .build(); - final IndexMetadata indexMetadata = new IndexMetadata.Builder(index.getName()).settings(idxSettings) - .numberOfShards(1) - .numberOfReplicas(0) - .build(); - RemoteIndexMetadataManager mockedIndexManager = mock(RemoteIndexMetadataManager.class); - RemoteGlobalMetadataManager mockedGlobalMetadataManager = mock(RemoteGlobalMetadataManager.class); - remoteClusterStateService.setRemoteIndexMetadataManager(mockedIndexManager); - remoteClusterStateService.setRemoteGlobalMetadataManager(mockedGlobalMetadataManager); - ArgumentCaptor> listenerArgumentCaptor = ArgumentCaptor.forClass( - LatchedActionListener.class - ); - when(mockedIndexManager.getAsyncIndexMetadataReadAction(any(), anyString(), listenerArgumentCaptor.capture())).thenReturn( - () -> listenerArgumentCaptor.getValue().onResponse(new RemoteReadResult(indexMetadata, INDEX, INDEX)) - ); - when(mockedGlobalMetadataManager.getGlobalMetadata(anyString(), eq(manifest))).thenReturn(Metadata.EMPTY_METADATA); - RemoteClusterStateService spiedService = spy(remoteClusterStateService); - spiedService.getClusterStateForManifest(ClusterName.DEFAULT.value(), manifest, NODE_ID, true); - verify(spiedService, times(1)).readClusterStateInParallel( - any(), - eq(manifest), - eq(manifest.getClusterUUID()), - eq(NODE_ID), - eq(manifest.getIndices()), - eq(emptyMap()), - eq(false), - eq(false), - eq(false), - eq(false), - eq(false), - eq(false), - eq(emptyList()), - eq(false), - eq(emptyMap()), - eq(false) - ); - verify(mockedGlobalMetadataManager, times(1)).getGlobalMetadata(eq(manifest.getClusterUUID()), eq(manifest)); - } - - public void testGetClusterStateUsingDiffFailWhenDiffManifestAbsent() { - ClusterMetadataManifest manifest = ClusterMetadataManifest.builder().build(); - ClusterState previousState = ClusterState.EMPTY_STATE; - AssertionError error = assertThrows( - AssertionError.class, - () -> remoteClusterStateService.getClusterStateUsingDiff(manifest, previousState, "test-node") - ); - assertEquals("Diff manifest null which is required for downloading cluster state", error.getMessage()); - } - - public void testGetClusterStateUsingDiff_NoDiff() throws IOException { - ClusterStateDiffManifest diffManifest = ClusterStateDiffManifest.builder().build(); - ClusterState clusterState = generateClusterStateWithAllAttributes().build(); - ClusterMetadataManifest manifest = ClusterMetadataManifest.builder() - .diffManifest(diffManifest) - .stateUUID(clusterState.stateUUID()) - .stateVersion(clusterState.version()) - .metadataVersion(clusterState.metadata().version()) - .clusterUUID(clusterState.getMetadata().clusterUUID()) - .routingTableVersion(clusterState.routingTable().version()) - .build(); - ClusterState updatedClusterState = remoteClusterStateService.getClusterStateUsingDiff(manifest, clusterState, "test-node"); - assertEquals(clusterState.getClusterName(), updatedClusterState.getClusterName()); - assertEquals(clusterState.metadata().clusterUUID(), updatedClusterState.metadata().clusterUUID()); - assertEquals(clusterState.metadata().version(), updatedClusterState.metadata().version()); - assertEquals(clusterState.metadata().coordinationMetadata(), updatedClusterState.metadata().coordinationMetadata()); - assertEquals(clusterState.metadata().getIndices(), updatedClusterState.metadata().getIndices()); - assertEquals(clusterState.metadata().templates(), updatedClusterState.metadata().templates()); - assertEquals(clusterState.metadata().persistentSettings(), updatedClusterState.metadata().persistentSettings()); - assertEquals(clusterState.metadata().transientSettings(), updatedClusterState.metadata().transientSettings()); - assertEquals(clusterState.metadata().getCustoms(), updatedClusterState.metadata().getCustoms()); - assertEquals(clusterState.metadata().hashesOfConsistentSettings(), updatedClusterState.metadata().hashesOfConsistentSettings()); - assertEquals(clusterState.getCustoms(), updatedClusterState.getCustoms()); - assertEquals(clusterState.stateUUID(), updatedClusterState.stateUUID()); - assertEquals(clusterState.version(), updatedClusterState.version()); - assertEquals(clusterState.getRoutingTable().version(), updatedClusterState.getRoutingTable().version()); - assertEquals(clusterState.getRoutingTable().getIndicesRouting(), updatedClusterState.getRoutingTable().getIndicesRouting()); - assertEquals(clusterState.getNodes(), updatedClusterState.getNodes()); - assertEquals(clusterState.getBlocks(), updatedClusterState.getBlocks()); - } - - public void testGetClusterStateUsingDiff() throws IOException { - ClusterState clusterState = generateClusterStateWithAllAttributes().build(); - ClusterState.Builder expectedClusterStateBuilder = ClusterState.builder(clusterState); - Metadata.Builder mb = Metadata.builder(clusterState.metadata()); - ClusterStateDiffManifest.Builder diffManifestBuilder = ClusterStateDiffManifest.builder(); - ClusterMetadataManifest.Builder manifestBuilder = ClusterMetadataManifest.builder(); - BlobContainer blobContainer = mockBlobStoreObjects(); - if (randomBoolean()) { - // updated coordination metadata - CoordinationMetadata coordinationMetadata = CoordinationMetadata.builder() - .term(clusterState.metadata().coordinationMetadata().term() + 1) - .build(); - mb.coordinationMetadata(coordinationMetadata); - diffManifestBuilder.coordinationMetadataUpdated(true); - manifestBuilder.coordinationMetadata(new UploadedMetadataAttribute(COORDINATION_METADATA, COORDINATION_METADATA_FILENAME)); - when(blobContainer.readBlob(COORDINATION_METADATA_FILENAME)).thenAnswer(i -> { - BytesReference bytes = COORDINATION_METADATA_FORMAT.serialize( - coordinationMetadata, - COORDINATION_METADATA_FILENAME, - compressor, - FORMAT_PARAMS - ); - return new ByteArrayInputStream(bytes.streamInput().readAllBytes()); - }); - } - if (randomBoolean()) { - // updated templates - TemplatesMetadata templatesMetadata = TemplatesMetadata.builder() - .put( - IndexTemplateMetadata.builder("template" + randomAlphaOfLength(3)) - .patterns(Arrays.asList("bar-*", "foo-*")) - .settings(Settings.builder().put("random_index_setting_" + randomAlphaOfLength(3), randomAlphaOfLength(5)).build()) - .build() - ) - .build(); - mb.templates(templatesMetadata); - diffManifestBuilder.templatesMetadataUpdated(true); - manifestBuilder.templatesMetadata(new UploadedMetadataAttribute(TEMPLATES_METADATA, TEMPLATES_METADATA_FILENAME)); - when(blobContainer.readBlob(TEMPLATES_METADATA_FILENAME)).thenAnswer(i -> { - BytesReference bytes = TEMPLATES_METADATA_FORMAT.serialize( - templatesMetadata, - TEMPLATES_METADATA_FILENAME, - compressor, - FORMAT_PARAMS - ); - return new ByteArrayInputStream(bytes.streamInput().readAllBytes()); - }); - } - if (randomBoolean()) { - // updated persistent settings - Settings persistentSettings = Settings.builder() - .put("random_persistent_setting_" + randomAlphaOfLength(3), randomAlphaOfLength(5)) - .build(); - mb.persistentSettings(persistentSettings); - diffManifestBuilder.settingsMetadataUpdated(true); - manifestBuilder.settingMetadata(new UploadedMetadataAttribute(SETTING_METADATA, PERSISTENT_SETTINGS_FILENAME)); - when(blobContainer.readBlob(PERSISTENT_SETTINGS_FILENAME)).thenAnswer(i -> { - BytesReference bytes = RemotePersistentSettingsMetadata.SETTINGS_METADATA_FORMAT.serialize( - persistentSettings, - PERSISTENT_SETTINGS_FILENAME, - compressor, - FORMAT_PARAMS - ); - return new ByteArrayInputStream(bytes.streamInput().readAllBytes()); - }); - } - if (randomBoolean()) { - // updated transient settings - Settings transientSettings = Settings.builder() - .put("random_transient_setting_" + randomAlphaOfLength(3), randomAlphaOfLength(5)) - .build(); - mb.transientSettings(transientSettings); - diffManifestBuilder.transientSettingsMetadataUpdate(true); - manifestBuilder.transientSettingsMetadata( - new UploadedMetadataAttribute(TRANSIENT_SETTING_METADATA, TRANSIENT_SETTINGS_FILENAME) - ); - when(blobContainer.readBlob(TRANSIENT_SETTINGS_FILENAME)).thenAnswer(i -> { - BytesReference bytes = RemoteTransientSettingsMetadata.SETTINGS_METADATA_FORMAT.serialize( - transientSettings, - TRANSIENT_SETTINGS_FILENAME, - compressor, - FORMAT_PARAMS - ); - return new ByteArrayInputStream(bytes.streamInput().readAllBytes()); - }); - } - if (randomBoolean()) { - // updated customs - CustomMetadata2 addedCustom = new CustomMetadata2(randomAlphaOfLength(10)); - mb.putCustom(addedCustom.getWriteableName(), addedCustom); - diffManifestBuilder.customMetadataUpdated(Collections.singletonList(addedCustom.getWriteableName())); - manifestBuilder.customMetadataMap( - Map.of(addedCustom.getWriteableName(), new UploadedMetadataAttribute(addedCustom.getWriteableName(), "custom-md2-file__1")) - ); - when(blobContainer.readBlob("custom-md2-file__1")).thenAnswer(i -> { - ChecksumWritableBlobStoreFormat customMetadataFormat = new ChecksumWritableBlobStoreFormat<>( - "custom", - is -> readFrom(is, namedWriteableRegistry, addedCustom.getWriteableName()) - ); - BytesReference bytes = customMetadataFormat.serialize(addedCustom, "custom-md2-file__1", compressor); - return new ByteArrayInputStream(bytes.streamInput().readAllBytes()); - }); - } - if (randomBoolean()) { - Set customsToRemove = clusterState.metadata().customs().keySet(); - customsToRemove.forEach(mb::removeCustom); - diffManifestBuilder.customMetadataDeleted(new ArrayList<>(customsToRemove)); - } - if (randomBoolean()) { - // updated hashes of consistent settings - DiffableStringMap hashesOfConsistentSettings = new DiffableStringMap(Map.of("secure_setting_key", "secure_setting_value")); - mb.hashesOfConsistentSettings(hashesOfConsistentSettings); - diffManifestBuilder.hashesOfConsistentSettingsUpdated(true); - manifestBuilder.hashesOfConsistentSettings( - new UploadedMetadataAttribute(HASHES_OF_CONSISTENT_SETTINGS, HASHES_OF_CONSISTENT_SETTINGS_FILENAME) - ); - when(blobContainer.readBlob(HASHES_OF_CONSISTENT_SETTINGS_FILENAME)).thenAnswer(i -> { - BytesReference bytes = HASHES_OF_CONSISTENT_SETTINGS_FORMAT.serialize( - hashesOfConsistentSettings, - HASHES_OF_CONSISTENT_SETTINGS_FILENAME, - compressor - ); - return new ByteArrayInputStream(bytes.streamInput().readAllBytes()); - }); - } - if (randomBoolean()) { - // updated index metadata - IndexMetadata indexMetadata = new IndexMetadata.Builder("add-test-index").settings( - Settings.builder() - .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetadata.SETTING_INDEX_UUID, "add-test-index-uuid") - .build() - ).numberOfShards(1).numberOfReplicas(0).build(); - mb.put(indexMetadata, true); - diffManifestBuilder.indicesUpdated(Collections.singletonList(indexMetadata.getIndex().getName())); - manifestBuilder.indices( - List.of( - new UploadedIndexMetadata(indexMetadata.getIndex().getName(), indexMetadata.getIndexUUID(), "add-test-index-file__2") - ) - ); - when(blobContainer.readBlob("add-test-index-file__2")).thenAnswer(i -> { - BytesReference bytes = INDEX_METADATA_FORMAT.serialize(indexMetadata, "add-test-index-file__2", compressor, FORMAT_PARAMS); - return new ByteArrayInputStream(bytes.streamInput().readAllBytes()); - }); - } - if (randomBoolean()) { - // remove index metadata - Set indicesToDelete = clusterState.metadata().getIndices().keySet(); - indicesToDelete.forEach(mb::remove); - diffManifestBuilder.indicesDeleted(new ArrayList<>(indicesToDelete)); - } - if (randomBoolean()) { - // update nodes - DiscoveryNode node = new DiscoveryNode("node_id", buildNewFakeTransportAddress(), Version.CURRENT); - DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(clusterState.nodes()).add(node); - expectedClusterStateBuilder.nodes(nodesBuilder.build()); - diffManifestBuilder.discoveryNodesUpdated(true); - manifestBuilder.discoveryNodesMetadata(new UploadedMetadataAttribute(DISCOVERY_NODES, DISCOVERY_NODES_FILENAME)); - when(blobContainer.readBlob(DISCOVERY_NODES_FILENAME)).thenAnswer(invocationOnMock -> { - BytesReference bytes = DISCOVERY_NODES_FORMAT.serialize(nodesBuilder.build(), DISCOVERY_NODES_FILENAME, compressor); - return new ByteArrayInputStream(bytes.streamInput().readAllBytes()); - }); - } - if (randomBoolean()) { - // update blocks - ClusterBlocks newClusterBlock = randomClusterBlocks(); - expectedClusterStateBuilder.blocks(newClusterBlock); - diffManifestBuilder.clusterBlocksUpdated(true); - manifestBuilder.clusterBlocksMetadata(new UploadedMetadataAttribute(CLUSTER_BLOCKS, CLUSTER_BLOCKS_FILENAME)); - when(blobContainer.readBlob(CLUSTER_BLOCKS_FILENAME)).thenAnswer(invocationOnMock -> { - BytesReference bytes = CLUSTER_BLOCKS_FORMAT.serialize(newClusterBlock, CLUSTER_BLOCKS_FILENAME, compressor); - return new ByteArrayInputStream(bytes.streamInput().readAllBytes()); - }); - - } - ClusterState expectedClusterState = expectedClusterStateBuilder.metadata(mb).build(); - ClusterStateDiffManifest diffManifest = diffManifestBuilder.build(); - manifestBuilder.diffManifest(diffManifest) - .stateUUID(clusterState.stateUUID()) - .stateVersion(clusterState.version()) - .metadataVersion(clusterState.metadata().version()) - .clusterUUID(clusterState.getMetadata().clusterUUID()) - .routingTableVersion(clusterState.getRoutingTable().version()); - - remoteClusterStateService.start(); - ClusterState updatedClusterState = remoteClusterStateService.getClusterStateUsingDiff( - manifestBuilder.build(), - clusterState, - NODE_ID - ); - - assertEquals(expectedClusterState.getClusterName(), updatedClusterState.getClusterName()); - assertEquals(expectedClusterState.stateUUID(), updatedClusterState.stateUUID()); - assertEquals(expectedClusterState.version(), updatedClusterState.version()); - assertEquals(expectedClusterState.metadata().clusterUUID(), updatedClusterState.metadata().clusterUUID()); - assertEquals(expectedClusterState.getRoutingTable().version(), updatedClusterState.getRoutingTable().version()); - assertNotEquals(diffManifest.isClusterBlocksUpdated(), updatedClusterState.getBlocks().equals(clusterState.getBlocks())); - assertNotEquals(diffManifest.isDiscoveryNodesUpdated(), updatedClusterState.getNodes().equals(clusterState.getNodes())); - assertNotEquals( - diffManifest.isCoordinationMetadataUpdated(), - updatedClusterState.getMetadata().coordinationMetadata().equals(clusterState.getMetadata().coordinationMetadata()) - ); - assertNotEquals( - diffManifest.isTemplatesMetadataUpdated(), - updatedClusterState.getMetadata().templates().equals(clusterState.getMetadata().getTemplates()) - ); - assertNotEquals( - diffManifest.isSettingsMetadataUpdated(), - updatedClusterState.getMetadata().persistentSettings().equals(clusterState.getMetadata().persistentSettings()) - ); - assertNotEquals( - diffManifest.isTransientSettingsMetadataUpdated(), - updatedClusterState.getMetadata().transientSettings().equals(clusterState.getMetadata().transientSettings()) - ); - diffManifest.getIndicesUpdated().forEach(indexName -> { - IndexMetadata updatedIndexMetadata = updatedClusterState.metadata().index(indexName); - IndexMetadata originalIndexMetadata = clusterState.metadata().index(indexName); - assertNotEquals(originalIndexMetadata, updatedIndexMetadata); - }); - diffManifest.getCustomMetadataUpdated().forEach(customMetadataName -> { - Metadata.Custom updatedCustomMetadata = updatedClusterState.metadata().custom(customMetadataName); - Metadata.Custom originalCustomMetadata = clusterState.metadata().custom(customMetadataName); - assertNotEquals(originalCustomMetadata, updatedCustomMetadata); - }); - diffManifest.getClusterStateCustomUpdated().forEach(clusterStateCustomName -> { - ClusterState.Custom updateClusterStateCustom = updatedClusterState.customs().get(clusterStateCustomName); - ClusterState.Custom originalClusterStateCustom = clusterState.customs().get(clusterStateCustomName); - assertNotEquals(originalClusterStateCustom, updateClusterStateCustom); - }); - diffManifest.getIndicesRoutingUpdated().forEach(indexName -> { - IndexRoutingTable updatedIndexRoutingTable = updatedClusterState.getRoutingTable().getIndicesRouting().get(indexName); - IndexRoutingTable originalIndexingRoutingTable = clusterState.getRoutingTable().getIndicesRouting().get(indexName); - assertNotEquals(originalIndexingRoutingTable, updatedIndexRoutingTable); - }); - diffManifest.getIndicesDeleted() - .forEach(indexName -> { assertFalse(updatedClusterState.metadata().getIndices().containsKey(indexName)); }); - diffManifest.getCustomMetadataDeleted().forEach(customMetadataName -> { - assertFalse(updatedClusterState.metadata().customs().containsKey(customMetadataName)); - }); - diffManifest.getClusterStateCustomDeleted().forEach(clusterStateCustomName -> { - assertFalse(updatedClusterState.customs().containsKey(clusterStateCustomName)); - }); - diffManifest.getIndicesRoutingDeleted().forEach(indexName -> { - assertFalse(updatedClusterState.getRoutingTable().getIndicesRouting().containsKey(indexName)); - }); - } - - public void testReadClusterStateInParallel_TimedOut() throws IOException { - ClusterState previousClusterState = generateClusterStateWithAllAttributes().build(); - ClusterMetadataManifest manifest = generateClusterMetadataManifestWithAllAttributes().build(); - BlobContainer container = mockBlobStoreObjects(); - int readTimeOut = 2; - Settings newSettings = Settings.builder().put("cluster.remote_store.state.read_timeout", readTimeOut + "s").build(); - clusterSettings.applySettings(newSettings); - when(container.readBlob(anyString())).thenAnswer(invocationOnMock -> { - Thread.sleep(readTimeOut * 1000 + 100); - return null; - }); - remoteClusterStateService.start(); - RemoteStateTransferException exception = expectThrows( - RemoteStateTransferException.class, - () -> remoteClusterStateService.readClusterStateInParallel( - previousClusterState, - manifest, - manifest.getClusterUUID(), - NODE_ID, - emptyList(), - emptyMap(), - true, - true, - true, - true, - true, - true, - emptyList(), - true, - emptyMap(), - true - ) - ); - assertEquals("Timed out waiting to read cluster state from remote within timeout " + readTimeOut + "s", exception.getMessage()); - } - - public void testReadClusterStateInParallel_ExceptionDuringRead() throws IOException { - ClusterState previousClusterState = generateClusterStateWithAllAttributes().build(); - ClusterMetadataManifest manifest = generateClusterMetadataManifestWithAllAttributes().build(); - BlobContainer container = mockBlobStoreObjects(); - Exception mockException = new IOException("mock exception"); - when(container.readBlob(anyString())).thenThrow(mockException); - remoteClusterStateService.start(); - RemoteStateTransferException exception = expectThrows( - RemoteStateTransferException.class, - () -> remoteClusterStateService.readClusterStateInParallel( - previousClusterState, - manifest, - manifest.getClusterUUID(), - NODE_ID, - emptyList(), - emptyMap(), - true, - true, - true, - true, - true, - true, - emptyList(), - true, - emptyMap(), - true - ) - ); - assertEquals("Exception during reading cluster state from remote", exception.getMessage()); - assertTrue(exception.getSuppressed().length > 0); - assertEquals(mockException, exception.getSuppressed()[0]); - } - - public void testReadClusterStateInParallel_UnexpectedResult() throws IOException { - ClusterState previousClusterState = generateClusterStateWithAllAttributes().build(); - // index already present in previous state - List uploadedIndexMetadataList = new ArrayList<>( - List.of(new UploadedIndexMetadata("test-index", "test-index-uuid", "test-index-file__2")) - ); - // new index to be added - List newIndicesToRead = List.of( - new UploadedIndexMetadata("test-index-1", "test-index-1-uuid", "test-index-1-file__2") - ); - uploadedIndexMetadataList.addAll(newIndicesToRead); - // existing custom metadata - Map uploadedCustomMetadataMap = new HashMap<>( - Map.of( - "custom_md_1", - new UploadedMetadataAttribute("custom_md_1", "test-custom1-file__1"), - "custom_md_2", - new UploadedMetadataAttribute("custom_md_2", "test-custom2-file__1") - ) - ); - // new custom metadata to be added - Map newCustomMetadataMap = Map.of( - "custom_md_3", - new UploadedMetadataAttribute("custom_md_3", "test-custom3-file__1") - ); - uploadedCustomMetadataMap.putAll(newCustomMetadataMap); - // already existing cluster state customs - Map uploadedClusterStateCustomMap = new HashMap<>( - Map.of( - "custom_1", - new UploadedMetadataAttribute("custom_1", "test-cluster-state-custom1-file__1"), - "custom_2", - new UploadedMetadataAttribute("custom_2", "test-cluster-state-custom2-file__1") - ) - ); - // new customs uploaded - Map newClusterStateCustoms = Map.of( - "custom_3", - new UploadedMetadataAttribute("custom_3", "test-cluster-state-custom3-file__1") - ); - uploadedClusterStateCustomMap.putAll(newClusterStateCustoms); - ClusterMetadataManifest manifest = ClusterMetadataManifest.builder() - .clusterUUID(previousClusterState.getMetadata().clusterUUID()) - .indices(uploadedIndexMetadataList) - .coordinationMetadata(new UploadedMetadataAttribute(COORDINATION_METADATA, COORDINATION_METADATA_FILENAME)) - .settingMetadata(new UploadedMetadataAttribute(SETTING_METADATA, PERSISTENT_SETTINGS_FILENAME)) - .transientSettingsMetadata(new UploadedMetadataAttribute(TRANSIENT_SETTING_METADATA, TRANSIENT_SETTINGS_FILENAME)) - .templatesMetadata(new UploadedMetadataAttribute(TEMPLATES_METADATA, TEMPLATES_METADATA_FILENAME)) - .hashesOfConsistentSettings( - new UploadedMetadataAttribute(HASHES_OF_CONSISTENT_SETTINGS, HASHES_OF_CONSISTENT_SETTINGS_FILENAME) - ) - .customMetadataMap(uploadedCustomMetadataMap) - .discoveryNodesMetadata(new UploadedMetadataAttribute(DISCOVERY_NODES, DISCOVERY_NODES_FILENAME)) - .clusterBlocksMetadata(new UploadedMetadataAttribute(CLUSTER_BLOCKS, CLUSTER_BLOCKS_FILENAME)) - .clusterStateCustomMetadataMap(uploadedClusterStateCustomMap) - .build(); - - RemoteReadResult mockResult = mock(RemoteReadResult.class); - RemoteIndexMetadataManager mockIndexMetadataManager = mock(RemoteIndexMetadataManager.class); - CheckedRunnable mockRunnable = mock(CheckedRunnable.class); - ArgumentCaptor> latchCapture = ArgumentCaptor.forClass(LatchedActionListener.class); - when(mockIndexMetadataManager.getAsyncIndexMetadataReadAction(anyString(), anyString(), latchCapture.capture())).thenReturn( - mockRunnable - ); - RemoteGlobalMetadataManager mockGlobalMetadataManager = mock(RemoteGlobalMetadataManager.class); - when(mockGlobalMetadataManager.getAsyncMetadataReadAction(any(), anyString(), latchCapture.capture())).thenReturn(mockRunnable); - RemoteClusterStateAttributesManager mockClusterStateAttributeManager = mock(RemoteClusterStateAttributesManager.class); - when(mockClusterStateAttributeManager.getAsyncMetadataReadAction(anyString(), any(), latchCapture.capture())).thenReturn( - mockRunnable - ); - doAnswer(invocationOnMock -> { - latchCapture.getValue().onResponse(mockResult); - return null; - }).when(mockRunnable).run(); - when(mockResult.getComponent()).thenReturn("mock-result"); - remoteClusterStateService.start(); - remoteClusterStateService.setRemoteIndexMetadataManager(mockIndexMetadataManager); - remoteClusterStateService.setRemoteGlobalMetadataManager(mockGlobalMetadataManager); - remoteClusterStateService.setRemoteClusterStateAttributesManager(mockClusterStateAttributeManager); - IllegalStateException exception = expectThrows( - IllegalStateException.class, - () -> remoteClusterStateService.readClusterStateInParallel( - previousClusterState, - manifest, - manifest.getClusterUUID(), - NODE_ID, - newIndicesToRead, - newCustomMetadataMap, - true, - true, - true, - true, - true, - true, - emptyList(), - true, - newClusterStateCustoms, - true - ) - ); - assertEquals("Unknown component: mock-result", exception.getMessage()); - newIndicesToRead.forEach( - uploadedIndexMetadata -> verify(mockIndexMetadataManager, times(1)).getAsyncIndexMetadataReadAction( - eq(previousClusterState.getMetadata().clusterUUID()), - eq(uploadedIndexMetadata.getUploadedFilename()), - any() - ) - ); - verify(mockGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(COORDINATION_METADATA_FILENAME)), - eq(COORDINATION_METADATA), - any() - ); - verify(mockGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(PERSISTENT_SETTINGS_FILENAME)), - eq(SETTING_METADATA), - any() - ); - verify(mockGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(TRANSIENT_SETTINGS_FILENAME)), - eq(TRANSIENT_SETTING_METADATA), - any() - ); - verify(mockGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(TEMPLATES_METADATA_FILENAME)), - eq(TEMPLATES_METADATA), - any() - ); - verify(mockGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(HASHES_OF_CONSISTENT_SETTINGS_FILENAME)), - eq(HASHES_OF_CONSISTENT_SETTINGS), - any() - ); - newCustomMetadataMap.keySet().forEach(uploadedCustomMetadataKey -> { - verify(mockGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(newCustomMetadataMap.get(uploadedCustomMetadataKey).getUploadedFilename())), - eq(uploadedCustomMetadataKey), - any() - ); - }); - verify(mockClusterStateAttributeManager, times(1)).getAsyncMetadataReadAction( - eq(DISCOVERY_NODES), - argThat(new BlobNameMatcher(DISCOVERY_NODES_FILENAME)), - any() - ); - verify(mockClusterStateAttributeManager, times(1)).getAsyncMetadataReadAction( - eq(CLUSTER_BLOCKS), - argThat(new BlobNameMatcher(CLUSTER_BLOCKS_FILENAME)), - any() - ); - newClusterStateCustoms.keySet().forEach(uploadedClusterStateCustomMetadataKey -> { - verify(mockClusterStateAttributeManager, times(1)).getAsyncMetadataReadAction( - eq(String.join(CUSTOM_DELIMITER, CLUSTER_STATE_CUSTOM, uploadedClusterStateCustomMetadataKey)), - argThat(new BlobNameMatcher(newClusterStateCustoms.get(uploadedClusterStateCustomMetadataKey).getUploadedFilename())), - any() - ); - }); - } - - public void testReadClusterStateInParallel_Success() throws IOException { - ClusterState previousClusterState = generateClusterStateWithAllAttributes().build(); - String indexFilename = "test-index-1-file__2"; - String customMetadataFilename = "test-custom3-file__1"; - String clusterStateCustomFilename = "test-cluster-state-custom3-file__1"; - // index already present in previous state - List uploadedIndexMetadataList = new ArrayList<>( - List.of(new UploadedIndexMetadata("test-index", "test-index-uuid", "test-index-file__2")) - ); - // new index to be added - List newIndicesToRead = List.of( - new UploadedIndexMetadata("test-index-1", "test-index-1-uuid", indexFilename) - ); - uploadedIndexMetadataList.addAll(newIndicesToRead); - // existing custom metadata - Map uploadedCustomMetadataMap = new HashMap<>( - Map.of( - "custom_md_1", - new UploadedMetadataAttribute("custom_md_1", "test-custom1-file__1"), - "custom_md_2", - new UploadedMetadataAttribute("custom_md_2", "test-custom2-file__1") - ) - ); - // new custom metadata to be added - Map newCustomMetadataMap = Map.of( - "custom_md_3", - new UploadedMetadataAttribute("custom_md_3", customMetadataFilename) - ); - uploadedCustomMetadataMap.putAll(newCustomMetadataMap); - // already existing cluster state customs - Map uploadedClusterStateCustomMap = new HashMap<>( - Map.of( - "custom_1", - new UploadedMetadataAttribute("custom_1", "test-cluster-state-custom1-file__1"), - "custom_2", - new UploadedMetadataAttribute("custom_2", "test-cluster-state-custom2-file__1") - ) - ); - // new customs uploaded - Map newClusterStateCustoms = Map.of( - "custom_3", - new UploadedMetadataAttribute("custom_3", clusterStateCustomFilename) - ); - uploadedClusterStateCustomMap.putAll(newClusterStateCustoms); - - ClusterMetadataManifest manifest = generateClusterMetadataManifestWithAllAttributes().indices(uploadedIndexMetadataList) - .customMetadataMap(uploadedCustomMetadataMap) - .clusterStateCustomMetadataMap(uploadedClusterStateCustomMap) - .build(); - - IndexMetadata newIndexMetadata = new IndexMetadata.Builder("test-index-1").state(IndexMetadata.State.OPEN) - .settings(Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT).build()) - .numberOfShards(1) - .numberOfReplicas(1) - .build(); - CustomMetadata3 customMetadata3 = new CustomMetadata3("custom_md_3"); - CoordinationMetadata updatedCoordinationMetadata = CoordinationMetadata.builder() - .term(previousClusterState.metadata().coordinationMetadata().term() + 1) - .build(); - Settings updatedPersistentSettings = Settings.builder() - .put("random_persistent_setting_" + randomAlphaOfLength(3), randomAlphaOfLength(5)) - .build(); - Settings updatedTransientSettings = Settings.builder() - .put("random_transient_setting_" + randomAlphaOfLength(3), randomAlphaOfLength(5)) - .build(); - TemplatesMetadata updatedTemplateMetadata = getTemplatesMetadata(); - DiffableStringMap updatedHashesOfConsistentSettings = getHashesOfConsistentSettings(); - DiscoveryNodes updatedDiscoveryNodes = getDiscoveryNodes(); - ClusterBlocks updatedClusterBlocks = randomClusterBlocks(); - TestClusterStateCustom3 updatedClusterStateCustom3 = new TestClusterStateCustom3("custom_3"); - - RemoteIndexMetadataManager mockedIndexManager = mock(RemoteIndexMetadataManager.class); - RemoteGlobalMetadataManager mockedGlobalMetadataManager = mock(RemoteGlobalMetadataManager.class); - RemoteClusterStateAttributesManager mockedClusterStateAttributeManager = mock(RemoteClusterStateAttributesManager.class); - - when( - mockedIndexManager.getAsyncIndexMetadataReadAction( - eq(manifest.getClusterUUID()), - eq(indexFilename), - any(LatchedActionListener.class) - ) - ).thenAnswer(invocationOnMock -> { - LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( - new RemoteReadResult(newIndexMetadata, INDEX, "test-index-1") - ); - }); - when( - mockedGlobalMetadataManager.getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(customMetadataFilename)), - eq("custom_md_3"), - any() - ) - ).thenAnswer(invocationOnMock -> { - LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( - new RemoteReadResult(customMetadata3, CUSTOM_METADATA, "custom_md_3") - ); - }); - when( - mockedGlobalMetadataManager.getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(COORDINATION_METADATA_FILENAME)), - eq(COORDINATION_METADATA), - any() - ) - ).thenAnswer(invocationOnMock -> { - LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( - new RemoteReadResult(updatedCoordinationMetadata, COORDINATION_METADATA, COORDINATION_METADATA) - ); - }); - when( - mockedGlobalMetadataManager.getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(PERSISTENT_SETTINGS_FILENAME)), - eq(SETTING_METADATA), - any() - ) - ).thenAnswer(invocationOnMock -> { - LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - - return (CheckedRunnable) () -> latchedActionListener.onResponse( - new RemoteReadResult(updatedPersistentSettings, SETTING_METADATA, SETTING_METADATA) - ); - }); - when( - mockedGlobalMetadataManager.getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(TRANSIENT_SETTINGS_FILENAME)), - eq(TRANSIENT_SETTING_METADATA), - any() - ) - ).thenAnswer(invocationOnMock -> { - LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( - new RemoteReadResult(updatedTransientSettings, TRANSIENT_SETTING_METADATA, TRANSIENT_SETTING_METADATA) - ); - }); - when( - mockedGlobalMetadataManager.getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(TEMPLATES_METADATA_FILENAME)), - eq(TEMPLATES_METADATA), - any() - ) - ).thenAnswer(invocationOnMock -> { - LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( - new RemoteReadResult(updatedTemplateMetadata, TEMPLATES_METADATA, TEMPLATES_METADATA) - ); - }); - when( - mockedGlobalMetadataManager.getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(HASHES_OF_CONSISTENT_SETTINGS_FILENAME)), - eq(HASHES_OF_CONSISTENT_SETTINGS), - any() - ) - ).thenAnswer(invocationOnMock -> { - LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( - new RemoteReadResult(updatedHashesOfConsistentSettings, HASHES_OF_CONSISTENT_SETTINGS, HASHES_OF_CONSISTENT_SETTINGS) - ); - }); - when( - mockedClusterStateAttributeManager.getAsyncMetadataReadAction( - eq(DISCOVERY_NODES), - argThat(new BlobNameMatcher(DISCOVERY_NODES_FILENAME)), - any() - ) - ).thenAnswer(invocationOnMock -> { - LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( - new RemoteReadResult(updatedDiscoveryNodes, CLUSTER_STATE_ATTRIBUTE, DISCOVERY_NODES) - ); - }); - when( - mockedClusterStateAttributeManager.getAsyncMetadataReadAction( - eq(CLUSTER_BLOCKS), - argThat(new BlobNameMatcher(CLUSTER_BLOCKS_FILENAME)), - any() - ) - ).thenAnswer(invocationOnMock -> { - LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( - new RemoteReadResult(updatedClusterBlocks, CLUSTER_STATE_ATTRIBUTE, CLUSTER_BLOCKS) - ); - }); - when( - mockedClusterStateAttributeManager.getAsyncMetadataReadAction( - eq(String.join(CUSTOM_DELIMITER, CLUSTER_STATE_CUSTOM, updatedClusterStateCustom3.getWriteableName())), - argThat(new BlobNameMatcher(clusterStateCustomFilename)), - any() - ) - ).thenAnswer(invocationOnMock -> { - LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( - new RemoteReadResult( - updatedClusterStateCustom3, - CLUSTER_STATE_ATTRIBUTE, - String.join(CUSTOM_DELIMITER, CLUSTER_STATE_CUSTOM, updatedClusterStateCustom3.getWriteableName()) - ) - ); - }); - - remoteClusterStateService.start(); - remoteClusterStateService.setRemoteIndexMetadataManager(mockedIndexManager); - remoteClusterStateService.setRemoteGlobalMetadataManager(mockedGlobalMetadataManager); - remoteClusterStateService.setRemoteClusterStateAttributesManager(mockedClusterStateAttributeManager); - - ClusterState updatedClusterState = remoteClusterStateService.readClusterStateInParallel( - previousClusterState, - manifest, - manifest.getClusterUUID(), - NODE_ID, - newIndicesToRead, - newCustomMetadataMap, - true, - true, - true, - true, - true, - true, - emptyList(), - true, - newClusterStateCustoms, - true - ); - - assertEquals(uploadedIndexMetadataList.size(), updatedClusterState.metadata().indices().size()); - assertTrue(updatedClusterState.metadata().indices().containsKey("test-index-1")); - assertEquals(newIndexMetadata, updatedClusterState.metadata().index(newIndexMetadata.getIndex())); - uploadedCustomMetadataMap.keySet().forEach(key -> assertTrue(updatedClusterState.metadata().customs().containsKey(key))); - assertEquals(customMetadata3, updatedClusterState.metadata().custom(customMetadata3.getWriteableName())); - assertEquals( - previousClusterState.metadata().coordinationMetadata().term() + 1, - updatedClusterState.metadata().coordinationMetadata().term() - ); - assertEquals(updatedPersistentSettings, updatedClusterState.metadata().persistentSettings()); - assertEquals(updatedTransientSettings, updatedClusterState.metadata().transientSettings()); - assertEquals(updatedTemplateMetadata.getTemplates(), updatedClusterState.metadata().templates()); - assertEquals(updatedHashesOfConsistentSettings, updatedClusterState.metadata().hashesOfConsistentSettings()); - assertEquals(updatedDiscoveryNodes.getSize(), updatedClusterState.getNodes().getSize()); - updatedDiscoveryNodes.getNodes().forEach((nodeId, node) -> assertEquals(updatedClusterState.getNodes().get(nodeId), node)); - assertEquals(updatedDiscoveryNodes.getClusterManagerNodeId(), updatedClusterState.getNodes().getClusterManagerNodeId()); - assertEquals(updatedClusterBlocks, updatedClusterState.blocks()); - uploadedClusterStateCustomMap.keySet().forEach(key -> assertTrue(updatedClusterState.customs().containsKey(key))); - assertEquals(updatedClusterStateCustom3, updatedClusterState.custom("custom_3")); - newIndicesToRead.forEach( - uploadedIndexMetadata -> verify(mockedIndexManager, times(1)).getAsyncIndexMetadataReadAction( - eq(previousClusterState.getMetadata().clusterUUID()), - eq(uploadedIndexMetadata.getUploadedFilename()), - any() - ) - ); - verify(mockedGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(COORDINATION_METADATA_FILENAME)), - eq(COORDINATION_METADATA), - any() - ); - verify(mockedGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(PERSISTENT_SETTINGS_FILENAME)), - eq(SETTING_METADATA), - any() - ); - verify(mockedGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(TRANSIENT_SETTINGS_FILENAME)), - eq(TRANSIENT_SETTING_METADATA), - any() - ); - verify(mockedGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(TEMPLATES_METADATA_FILENAME)), - eq(TEMPLATES_METADATA), - any() - ); - verify(mockedGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(HASHES_OF_CONSISTENT_SETTINGS_FILENAME)), - eq(HASHES_OF_CONSISTENT_SETTINGS), - any() - ); - newCustomMetadataMap.keySet().forEach(uploadedCustomMetadataKey -> { - verify(mockedGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(newCustomMetadataMap.get(uploadedCustomMetadataKey).getUploadedFilename())), - eq(uploadedCustomMetadataKey), - any() - ); - }); - verify(mockedClusterStateAttributeManager, times(1)).getAsyncMetadataReadAction( - eq(DISCOVERY_NODES), - argThat(new BlobNameMatcher(DISCOVERY_NODES_FILENAME)), - any() - ); - verify(mockedClusterStateAttributeManager, times(1)).getAsyncMetadataReadAction( - eq(CLUSTER_BLOCKS), - argThat(new BlobNameMatcher(CLUSTER_BLOCKS_FILENAME)), - any() - ); - newClusterStateCustoms.keySet().forEach(uploadedClusterStateCustomMetadataKey -> { - verify(mockedClusterStateAttributeManager, times(1)).getAsyncMetadataReadAction( - eq(String.join(CUSTOM_DELIMITER, CLUSTER_STATE_CUSTOM, uploadedClusterStateCustomMetadataKey)), - argThat(new BlobNameMatcher(newClusterStateCustoms.get(uploadedClusterStateCustomMetadataKey).getUploadedFilename())), - any() - ); - }); - } - /* * Here we will verify the migration of manifest file from codec V0. * @@ -2254,14 +1191,13 @@ public void testReadLatestMetadataManifestSuccessButIndexMetadataFetchIOExceptio .stateVersion(1L) .stateUUID("state-uuid") .clusterUUID("cluster-uuid") - .codecVersion(CODEC_V2) .nodeId("nodeA") .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) .previousClusterUUID("prev-cluster-uuid") .build(); BlobContainer blobContainer = mockBlobStoreObjects(); - mockBlobContainer(blobContainer, expectedManifest, Map.of(), CODEC_V2); + mockBlobContainer(blobContainer, expectedManifest, Map.of()); when(blobContainer.readBlob(uploadedIndexMetadata.getUploadedFilename())).thenThrow(FileNotFoundException.class); remoteClusterStateService.start(); @@ -2289,11 +1225,11 @@ public void testReadLatestMetadataManifestSuccess() throws IOException { .clusterUUID("cluster-uuid") .nodeId("nodeA") .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) - .codecVersion(CODEC_V2) + .codecVersion(ClusterMetadataManifest.CODEC_V0) .previousClusterUUID("prev-cluster-uuid") .build(); - mockBlobContainer(mockBlobStoreObjects(), expectedManifest, new HashMap<>(), CODEC_V2); + mockBlobContainer(mockBlobStoreObjects(), expectedManifest, new HashMap<>()); remoteClusterStateService.start(); final ClusterMetadataManifest manifest = remoteClusterStateService.getLatestClusterMetadataManifest( clusterState.getClusterName().value(), @@ -2417,10 +1353,10 @@ public void testReadLatestIndexMetadataSuccess() throws IOException { .nodeId("nodeA") .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) .previousClusterUUID("prev-cluster-uuid") - .codecVersion(CODEC_V2) + .codecVersion(ClusterMetadataManifest.CODEC_V0) .build(); - mockBlobContainer(mockBlobStoreObjects(), expectedManifest, Map.of(index.getUUID(), indexMetadata), CODEC_V2); + mockBlobContainer(mockBlobStoreObjects(), expectedManifest, Map.of(index.getUUID(), indexMetadata)); Map indexMetadataMap = remoteClusterStateService.getLatestClusterState( clusterState.getClusterName().value(), @@ -2665,7 +1601,6 @@ public void testWriteFullMetadataInParallelSuccessWithRoutingTable() throws IOEx .clusterUUID("cluster-uuid") .previousClusterUUID("prev-cluster-uuid") .routingTableVersion(1) - .codecVersion(CODEC_V2) .indicesRouting(List.of(uploadedIndiceRoutingMetadata)) .build(); @@ -2922,7 +1857,7 @@ private void mockObjectsForGettingPreviousClusterUUID( BlobContainer[] mockBlobContainerOrderedArray = new BlobContainer[mockBlobContainerOrderedList.size()]; mockBlobContainerOrderedList.toArray(mockBlobContainerOrderedArray); when(blobStore.blobContainer(ArgumentMatchers.any())).thenReturn(uuidBlobContainer, mockBlobContainerOrderedArray); - when(blobStoreRepository.getCompressor()).thenReturn(compressor); + when(blobStoreRepository.getCompressor()).thenReturn(new DeflateCompressor()); } private ClusterMetadataManifest generateV1ClusterMetadataManifest( @@ -3051,7 +1986,7 @@ private void mockBlobContainer( } String fileName = uploadedIndexMetadata.getUploadedFilename(); when(blobContainer.readBlob(getFormattedIndexFileName(fileName))).thenAnswer((invocationOnMock) -> { - BytesReference bytesIndexMetadata = INDEX_METADATA_FORMAT.serialize( + BytesReference bytesIndexMetadata = RemoteIndexMetadata.INDEX_METADATA_FORMAT.serialize( indexMetadata, fileName, blobStoreRepository.getCompressor(), @@ -3083,7 +2018,7 @@ private void mockBlobContainerForGlobalMetadata( FORMAT_PARAMS ); when(blobContainer.readBlob(mockManifestFileName)).thenReturn(new ByteArrayInputStream(bytes.streamInput().readAllBytes())); - if (codecVersion >= CODEC_V2) { + if (codecVersion >= ClusterMetadataManifest.CODEC_V2) { String coordinationFileName = getFileNameFromPath(clusterMetadataManifest.getCoordinationMetadata().getUploadedFilename()); when(blobContainer.readBlob(COORDINATION_METADATA_FORMAT.blobName(coordinationFileName))).thenAnswer((invocationOnMock) -> { BytesReference bytesReference = COORDINATION_METADATA_FORMAT.serialize( @@ -3122,6 +2057,12 @@ private void mockBlobContainerForGlobalMetadata( .stream() .collect(Collectors.toMap(Map.Entry::getKey, entry -> getFileNameFromPath(entry.getValue().getUploadedFilename()))); + // ChecksumBlobStoreFormat customMetadataFormat = new ChecksumBlobStoreFormat<>( + // "custom", + // METADATA_NAME_PLAIN_FORMAT, + // null + // ); + ChecksumWritableBlobStoreFormat customMetadataFormat = new ChecksumWritableBlobStoreFormat<>("custom", null); for (Map.Entry entry : customFileMap.entrySet()) { String custom = entry.getKey(); @@ -3206,105 +2147,32 @@ static ClusterState.Builder generateClusterStateWithOneIndex() { .routingTable(RoutingTable.builder().addAsNew(indexMetadata).version(1L).build()); } - static ClusterState.Builder generateClusterStateWithAllAttributes() { - final Index index = new Index("test-index", "index-uuid"); - final Settings idxSettings = Settings.builder() - .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetadata.SETTING_INDEX_UUID, index.getUUID()) - .build(); - final IndexMetadata indexMetadata = new IndexMetadata.Builder(index.getName()).settings(idxSettings) - .numberOfShards(1) - .numberOfReplicas(0) - .build(); - final CoordinationMetadata coordinationMetadata = CoordinationMetadata.builder().term(1L).build(); - final Settings settings = Settings.builder().put("mock-settings", true).build(); - final Settings transientSettings = Settings.builder().put("mock-transient-settings", true).build(); - final DiffableStringMap hashesOfConsistentSettings = new DiffableStringMap(emptyMap()); - final TemplatesMetadata templatesMetadata = TemplatesMetadata.builder() - .put(IndexTemplateMetadata.builder("template-1").patterns(List.of("test-index* ")).build()) - .build(); - final CustomMetadata1 customMetadata1 = new CustomMetadata1("custom-metadata-1"); - final CustomMetadata2 customMetadata2 = new CustomMetadata2("custom-metadata-2"); - final DiscoveryNodes nodes = nodesWithLocalNodeClusterManager(); - final ClusterBlocks clusterBlocks = randomClusterBlocks(); - final TestClusterStateCustom1 custom1 = new RemoteClusterStateTestUtils.TestClusterStateCustom1("custom-1"); - final TestClusterStateCustom2 custom2 = new RemoteClusterStateTestUtils.TestClusterStateCustom2("custom-2"); - return ClusterState.builder(ClusterName.DEFAULT) - .version(1L) - .stateUUID("state-uuid") - .metadata( - Metadata.builder() - .version(randomNonNegativeLong()) - .put(indexMetadata, true) - .clusterUUID("cluster-uuid") - .coordinationMetadata(coordinationMetadata) - .persistentSettings(settings) - .transientSettings(transientSettings) - .hashesOfConsistentSettings(hashesOfConsistentSettings) - .templates(templatesMetadata) - .putCustom(customMetadata1.getWriteableName(), customMetadata1) - .putCustom(customMetadata2.getWriteableName(), customMetadata2) - .build() - ) - .routingTable(RoutingTable.builder().addAsNew(indexMetadata).version(1L).build()) - .nodes(nodes) - .blocks(clusterBlocks) - .putCustom(custom1.getWriteableName(), custom1) - .putCustom(custom2.getWriteableName(), custom2); - } - - static ClusterMetadataManifest.Builder generateClusterMetadataManifestWithAllAttributes() { - return ClusterMetadataManifest.builder() - .codecVersion(CODEC_V2) - .clusterUUID("cluster-uuid") - .indices(List.of(new UploadedIndexMetadata("test-index", "test-index-uuid", "test-index-file__2"))) - .customMetadataMap( - Map.of( - "custom_md_1", - new UploadedMetadataAttribute("custom_md_1", "test-custom1-file__1"), - "custom_md_2", - new UploadedMetadataAttribute("custom_md_2", "test-custom2-file__1") - ) - ) - .coordinationMetadata(new UploadedMetadataAttribute(COORDINATION_METADATA, COORDINATION_METADATA_FILENAME)) - .settingMetadata(new UploadedMetadataAttribute(SETTING_METADATA, PERSISTENT_SETTINGS_FILENAME)) - .transientSettingsMetadata(new UploadedMetadataAttribute(TRANSIENT_SETTING_METADATA, TRANSIENT_SETTINGS_FILENAME)) - .templatesMetadata(new UploadedMetadataAttribute(TEMPLATES_METADATA, TEMPLATES_METADATA_FILENAME)) - .hashesOfConsistentSettings( - new UploadedMetadataAttribute(HASHES_OF_CONSISTENT_SETTINGS, HASHES_OF_CONSISTENT_SETTINGS_FILENAME) - ) - .discoveryNodesMetadata(new UploadedMetadataAttribute(DISCOVERY_NODES, DISCOVERY_NODES_FILENAME)) - .clusterBlocksMetadata(new UploadedMetadataAttribute(CLUSTER_BLOCKS, CLUSTER_BLOCKS_FILENAME)) - .clusterStateCustomMetadataMap( - Map.of( - "custom_1", - new UploadedMetadataAttribute("custom_1", "test-cluster-state-custom1-file__1"), - "custom_2", - new UploadedMetadataAttribute("custom_2", "test-cluster-state-custom2-file__1") - ) - ); - } - static DiscoveryNodes nodesWithLocalNodeClusterManager() { final DiscoveryNode localNode = new DiscoveryNode("cluster-manager-id", buildNewFakeTransportAddress(), Version.CURRENT); return DiscoveryNodes.builder().clusterManagerNodeId("cluster-manager-id").localNodeId("cluster-manager-id").add(localNode).build(); } - private class BlobNameMatcher implements ArgumentMatcher { - private final String expectedBlobName; + private static class CustomMetadata1 extends TestCustomMetadata { + public static final String TYPE = "custom_md_1"; - BlobNameMatcher(String expectedBlobName) { - this.expectedBlobName = expectedBlobName; + CustomMetadata1(String data) { + super(data); } @Override - public boolean matches(AbstractRemoteWritableBlobEntity argument) { - return argument != null && expectedBlobName.equals(argument.getFullBlobName()); + public String getWriteableName() { + return TYPE; } @Override - public String toString() { - return "BlobNameMatcher[Expected blobName: " + expectedBlobName + "]"; + public Version getMinimalSupportedVersion() { + return Version.CURRENT; + } + + @Override + public EnumSet context() { + return EnumSet.of(Metadata.XContentContext.GATEWAY); } } + } 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/composite/datacube/startree/StarTreeDocValuesFormatTests.java index 6c6d26656e4de..967ed0ca49faa 100644 --- a/server/src/test/java/org/opensearch/index/codec/composite/datacube/startree/StarTreeDocValuesFormatTests.java +++ b/server/src/test/java/org/opensearch/index/codec/composite/datacube/startree/StarTreeDocValuesFormatTests.java @@ -8,6 +8,8 @@ package org.opensearch.index.codec.composite.datacube.startree; +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope; + import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.codecs.Codec; @@ -19,7 +21,17 @@ import org.apache.lucene.tests.index.BaseDocValuesFormatTestCase; import org.apache.lucene.tests.index.RandomIndexWriter; import org.apache.lucene.tests.util.LuceneTestCase; +import org.opensearch.Version; +import org.opensearch.cluster.ClusterModule; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.CheckedConsumer; import org.opensearch.common.Rounding; +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.Composite99Codec; import org.opensearch.index.compositeindex.datacube.DateDimension; import org.opensearch.index.compositeindex.datacube.Dimension; @@ -30,26 +42,44 @@ import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.mapper.StarTreeMapper; +import org.junit.AfterClass; +import org.junit.BeforeClass; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Set; -import org.mockito.Mockito; +import static org.opensearch.common.util.FeatureFlags.STAR_TREE_INDEX; +import static org.opensearch.index.engine.EngineTestCase.createMapperService; /** * Star tree doc values Lucene tests */ @LuceneTestCase.SuppressSysoutChecks(bugUrl = "we log a lot on purpose") +@ThreadLeakScope(ThreadLeakScope.Scope.NONE) public class StarTreeDocValuesFormatTests extends BaseDocValuesFormatTestCase { + @BeforeClass + public static void createMapper() throws Exception { + FeatureFlags.initializeFeatureFlags(Settings.builder().put(STAR_TREE_INDEX, "true").build()); + } + + @AfterClass + public static void clearMapper() { + FeatureFlags.initializeFeatureFlags(Settings.EMPTY); + } + @Override protected Codec getCodec() { - MapperService service = Mockito.mock(MapperService.class); - Mockito.when(service.getCompositeFieldTypes()).thenReturn(Set.of(getStarTreeFieldType())); final Logger testLogger = LogManager.getLogger(StarTreeDocValuesFormatTests.class); - return new Composite99Codec(Lucene99Codec.Mode.BEST_SPEED, service, testLogger); + MapperService mapperService = null; + try { + mapperService = createMapperService(getExpandedMapping("status", "size")); + } catch (IOException e) { + throw new RuntimeException(e); + } + Codec codec = new Composite99Codec(Lucene99Codec.Mode.BEST_SPEED, mapperService, testLogger); + return codec; } private StarTreeMapper.StarTreeFieldType getStarTreeFieldType() { @@ -72,7 +102,7 @@ private static StarTreeField getStarTreeField(List d1Cale StarTreeFieldConfiguration config = new StarTreeFieldConfiguration( 100, Collections.emptySet(), - StarTreeFieldConfiguration.StarTreeBuildMode.OFF_HEAP + StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP // TODO : change it ); return new StarTreeField("starTree", dims, metrics, config); @@ -107,4 +137,71 @@ public void testStarTreeDocValues() throws IOException { // TODO : validate star tree structures that got created directory.close(); } + + private XContentBuilder getExpandedMapping(String dim, String metric) 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.startArray("ordered_dimensions"); + b.startObject(); + b.field("name", "sndv"); + b.endObject(); + b.startObject(); + b.field("name", "dv"); + b.endObject(); + b.endArray(); + b.startArray("metrics"); + b.startObject(); + b.field("name", "field"); + b.startArray("stats"); + b.value("sum"); + b.value("count"); // TODO : THIS TEST FAILS. + b.endArray(); + b.endObject(); + b.endArray(); + b.endObject(); + b.endObject(); + b.endObject(); + b.startObject("properties"); + b.startObject("sndv"); + b.field("type", "integer"); + b.endObject(); + b.startObject("dv"); + b.field("type", "integer"); + b.endObject(); + b.startObject("field"); + b.field("type", "integer"); + b.endObject(); + b.endObject(); + }); + } + + private XContentBuilder topMapping(CheckedConsumer buildFields) throws IOException { + XContentBuilder builder = XContentFactory.jsonBuilder().startObject().startObject("_doc"); + buildFields.accept(builder); + return builder.endObject().endObject(); + } + + private MapperService createMapperService(XContentBuilder builder) throws IOException { + IndexMetadata indexMetadata = IndexMetadata.builder("test") + .settings( + Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + ) + .putMapping(builder.toString()) + .build(); + MapperService mapperService = MapperTestUtils.newMapperService( + new NamedXContentRegistry(ClusterModule.getNamedXWriteables()), + createTempDir(), + Settings.EMPTY, + "test" + ); + mapperService.merge(indexMetadata, MapperService.MergeReason.INDEX_TEMPLATE); + return mapperService; + } } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/CountValueAggregatorTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/CountValueAggregatorTests.java new file mode 100644 index 0000000000000..e30e203406a6c --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/CountValueAggregatorTests.java @@ -0,0 +1,53 @@ +/* + * 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.aggregators; + +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; +import org.opensearch.test.OpenSearchTestCase; + +public class CountValueAggregatorTests extends OpenSearchTestCase { + private final CountValueAggregator aggregator = new CountValueAggregator(); + + public void testGetAggregationType() { + assertEquals(MetricStat.COUNT.getTypeName(), aggregator.getAggregationType().getTypeName()); + } + + public void testGetAggregatedValueType() { + assertEquals(CountValueAggregator.VALUE_AGGREGATOR_TYPE, aggregator.getAggregatedValueType()); + } + + public void testGetInitialAggregatedValueForSegmentDocValue() { + assertEquals(1L, aggregator.getInitialAggregatedValueForSegmentDocValue(randomLong(), StarTreeNumericType.LONG), 0.0); + } + + public void testMergeAggregatedValueAndSegmentValue() { + assertEquals(3L, aggregator.mergeAggregatedValueAndSegmentValue(2L, 3L, StarTreeNumericType.LONG), 0.0); + } + + public void testMergeAggregatedValues() { + assertEquals(5L, aggregator.mergeAggregatedValues(2L, 3L), 0.0); + } + + public void testGetInitialAggregatedValue() { + assertEquals(3L, aggregator.getInitialAggregatedValue(3L), 0.0); + } + + public void testGetMaxAggregatedValueByteSize() { + assertEquals(Long.BYTES, aggregator.getMaxAggregatedValueByteSize()); + } + + public void testToLongValue() { + assertEquals(3L, aggregator.toLongValue(3L), 0.0); + } + + public void testToStarTreeNumericTypeValue() { + assertEquals(3L, aggregator.toStarTreeNumericTypeValue(3L, StarTreeNumericType.LONG), 0.0); + } +} diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MaxValueAggregatorTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MaxValueAggregatorTests.java new file mode 100644 index 0000000000000..194d667cffbd7 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MaxValueAggregatorTests.java @@ -0,0 +1,58 @@ +/* + * 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.aggregators; + +import org.apache.lucene.util.NumericUtils; +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; +import org.opensearch.test.OpenSearchTestCase; + +public class MaxValueAggregatorTests extends OpenSearchTestCase { + private final MaxValueAggregator aggregator = new MaxValueAggregator(); + + public void testGetAggregationType() { + assertEquals(MetricStat.MAX.getTypeName(), aggregator.getAggregationType().getTypeName()); + } + + public void testGetAggregatedValueType() { + assertEquals(MaxValueAggregator.VALUE_AGGREGATOR_TYPE, aggregator.getAggregatedValueType()); + } + + public void testGetInitialAggregatedValueForSegmentDocValue() { + assertEquals(1.0, aggregator.getInitialAggregatedValueForSegmentDocValue(1L, StarTreeNumericType.LONG), 0.0); + assertThrows( + NullPointerException.class, + () -> aggregator.getInitialAggregatedValueForSegmentDocValue(null, StarTreeNumericType.DOUBLE) + ); + } + + public void testMergeAggregatedValueAndSegmentValue() { + assertEquals(3.0, aggregator.mergeAggregatedValueAndSegmentValue(2.0, 3L, StarTreeNumericType.LONG), 0.0); + } + + public void testMergeAggregatedValues() { + assertEquals(3.0, aggregator.mergeAggregatedValues(2.0, 3.0), 0.0); + } + + public void testGetInitialAggregatedValue() { + assertEquals(3.0, aggregator.getInitialAggregatedValue(3.0), 0.0); + } + + public void testGetMaxAggregatedValueByteSize() { + assertEquals(Double.BYTES, aggregator.getMaxAggregatedValueByteSize()); + } + + public void testToLongValue() { + assertEquals(NumericUtils.doubleToSortableLong(3.0), aggregator.toLongValue(3.0), 0.0); + } + + public void testToStarTreeNumericTypeValue() { + assertEquals(NumericUtils.sortableLongToDouble(3L), aggregator.toStarTreeNumericTypeValue(3L, StarTreeNumericType.DOUBLE), 0.0); + } +} diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MetricAggregatorInfoTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MetricAggregatorInfoTests.java new file mode 100644 index 0000000000000..73e6aeb44cfd7 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MetricAggregatorInfoTests.java @@ -0,0 +1,113 @@ +/* + * 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.aggregators; + +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.fielddata.IndexNumericFieldData; +import org.opensearch.test.OpenSearchTestCase; + +public class MetricAggregatorInfoTests extends OpenSearchTestCase { + + public void testConstructor() { + MetricAggregatorInfo pair = new MetricAggregatorInfo( + MetricStat.SUM, + "column1", + "star_tree_field", + IndexNumericFieldData.NumericType.DOUBLE + ); + assertEquals(MetricStat.SUM, pair.getMetricStat()); + assertEquals("column1", pair.getField()); + } + + public void testCountStarConstructor() { + MetricAggregatorInfo pair = new MetricAggregatorInfo( + MetricStat.COUNT, + "anything", + "star_tree_field", + IndexNumericFieldData.NumericType.DOUBLE + ); + assertEquals(MetricStat.COUNT, pair.getMetricStat()); + assertEquals("anything", pair.getField()); + } + + public void testToFieldName() { + MetricAggregatorInfo pair = new MetricAggregatorInfo( + MetricStat.SUM, + "column2", + "star_tree_field", + IndexNumericFieldData.NumericType.DOUBLE + ); + assertEquals("star_tree_field_column2_sum", pair.toFieldName()); + } + + public void testEquals() { + MetricAggregatorInfo pair1 = new MetricAggregatorInfo( + MetricStat.SUM, + "column1", + "star_tree_field", + IndexNumericFieldData.NumericType.DOUBLE + ); + MetricAggregatorInfo pair2 = new MetricAggregatorInfo( + MetricStat.SUM, + "column1", + "star_tree_field", + IndexNumericFieldData.NumericType.DOUBLE + ); + assertEquals(pair1, pair2); + assertNotEquals( + pair1, + new MetricAggregatorInfo(MetricStat.COUNT, "column1", "star_tree_field", IndexNumericFieldData.NumericType.DOUBLE) + ); + assertNotEquals( + pair1, + new MetricAggregatorInfo(MetricStat.SUM, "column2", "star_tree_field", IndexNumericFieldData.NumericType.DOUBLE) + ); + } + + public void testHashCode() { + MetricAggregatorInfo pair1 = new MetricAggregatorInfo( + MetricStat.SUM, + "column1", + "star_tree_field", + IndexNumericFieldData.NumericType.DOUBLE + ); + MetricAggregatorInfo pair2 = new MetricAggregatorInfo( + MetricStat.SUM, + "column1", + "star_tree_field", + IndexNumericFieldData.NumericType.DOUBLE + ); + assertEquals(pair1.hashCode(), pair2.hashCode()); + } + + public void testCompareTo() { + MetricAggregatorInfo pair1 = new MetricAggregatorInfo( + MetricStat.SUM, + "column1", + "star_tree_field", + IndexNumericFieldData.NumericType.DOUBLE + ); + MetricAggregatorInfo pair2 = new MetricAggregatorInfo( + MetricStat.SUM, + "column2", + "star_tree_field", + IndexNumericFieldData.NumericType.DOUBLE + ); + MetricAggregatorInfo pair3 = new MetricAggregatorInfo( + MetricStat.COUNT, + "column1", + "star_tree_field", + IndexNumericFieldData.NumericType.DOUBLE + ); + assertTrue(pair1.compareTo(pair2) < 0); + assertTrue(pair2.compareTo(pair1) > 0); + assertTrue(pair1.compareTo(pair3) > 0); + assertTrue(pair3.compareTo(pair1) < 0); + } +} diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MinValueAggregatorTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MinValueAggregatorTests.java new file mode 100644 index 0000000000000..8f8ce259bc62c --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MinValueAggregatorTests.java @@ -0,0 +1,58 @@ +/* + * 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.aggregators; + +import org.apache.lucene.util.NumericUtils; +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; +import org.opensearch.test.OpenSearchTestCase; + +public class MinValueAggregatorTests extends OpenSearchTestCase { + private final MinValueAggregator aggregator = new MinValueAggregator(); + + public void testGetAggregationType() { + assertEquals(MetricStat.MIN.getTypeName(), aggregator.getAggregationType().getTypeName()); + } + + public void testGetAggregatedValueType() { + assertEquals(MinValueAggregator.VALUE_AGGREGATOR_TYPE, aggregator.getAggregatedValueType()); + } + + public void testGetInitialAggregatedValueForSegmentDocValue() { + assertEquals(1.0, aggregator.getInitialAggregatedValueForSegmentDocValue(1L, StarTreeNumericType.LONG), 0.0); + assertThrows( + NullPointerException.class, + () -> aggregator.getInitialAggregatedValueForSegmentDocValue(null, StarTreeNumericType.DOUBLE) + ); + } + + public void testMergeAggregatedValueAndSegmentValue() { + assertEquals(2.0, aggregator.mergeAggregatedValueAndSegmentValue(2.0, 3L, StarTreeNumericType.LONG), 0.0); + } + + public void testMergeAggregatedValues() { + assertEquals(2.0, aggregator.mergeAggregatedValues(2.0, 3.0), 0.0); + } + + public void testGetInitialAggregatedValue() { + assertEquals(3.0, aggregator.getInitialAggregatedValue(3.0), 0.0); + } + + public void testGetMaxAggregatedValueByteSize() { + assertEquals(Double.BYTES, aggregator.getMaxAggregatedValueByteSize()); + } + + public void testToLongValue() { + assertEquals(NumericUtils.doubleToSortableLong(3.0), aggregator.toLongValue(3.0), 0.0); + } + + public void testToStarTreeNumericTypeValue() { + assertEquals(NumericUtils.sortableLongToDouble(3L), aggregator.toStarTreeNumericTypeValue(3L, StarTreeNumericType.DOUBLE), 0.0); + } +} diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/SumValueAggregatorTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/SumValueAggregatorTests.java new file mode 100644 index 0000000000000..3fb627e7cd434 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/SumValueAggregatorTests.java @@ -0,0 +1,72 @@ +/* + * 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.aggregators; + +import org.apache.lucene.util.NumericUtils; +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +public class SumValueAggregatorTests extends OpenSearchTestCase { + + private SumValueAggregator aggregator; + + @Before + public void setup() { + aggregator = new SumValueAggregator(); + } + + public void testGetAggregationType() { + assertEquals(MetricStat.SUM.getTypeName(), aggregator.getAggregationType().getTypeName()); + } + + public void testGetAggregatedValueType() { + assertEquals(SumValueAggregator.VALUE_AGGREGATOR_TYPE, aggregator.getAggregatedValueType()); + } + + public void testGetInitialAggregatedValueForSegmentDocValue() { + assertEquals(1.0, aggregator.getInitialAggregatedValueForSegmentDocValue(1L, StarTreeNumericType.LONG), 0.0); + assertThrows( + NullPointerException.class, + () -> aggregator.getInitialAggregatedValueForSegmentDocValue(null, StarTreeNumericType.DOUBLE) + ); + } + + public void testMergeAggregatedValueAndSegmentValue() { + aggregator.getInitialAggregatedValue(2.0); + assertEquals(5.0, aggregator.mergeAggregatedValueAndSegmentValue(2.0, 3L, StarTreeNumericType.LONG), 0.0); + } + + public void testMergeAggregatedValueAndSegmentValue_nullSegmentDocValue() { + aggregator.getInitialAggregatedValue(2.0); + assertThrows(NullPointerException.class, () -> aggregator.mergeAggregatedValueAndSegmentValue(2.0, null, StarTreeNumericType.LONG)); + } + + public void testMergeAggregatedValues() { + aggregator.getInitialAggregatedValue(3.0); + assertEquals(5.0, aggregator.mergeAggregatedValues(2.0, 3.0), 0.0); + } + + public void testGetInitialAggregatedValue() { + assertEquals(3.14, aggregator.getInitialAggregatedValue(3.14), 0.0); + } + + public void testGetMaxAggregatedValueByteSize() { + assertEquals(Double.BYTES, aggregator.getMaxAggregatedValueByteSize()); + } + + public void testToLongValue() { + assertEquals(NumericUtils.doubleToSortableLong(3.14), aggregator.toLongValue(3.14), 0.0); + } + + public void testToStarTreeNumericTypeValue() { + assertEquals(NumericUtils.sortableLongToDouble(3L), aggregator.toStarTreeNumericTypeValue(3L, StarTreeNumericType.DOUBLE), 0.0); + } +} diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregatorFactoryTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregatorFactoryTests.java new file mode 100644 index 0000000000000..ce61ab839cc61 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregatorFactoryTests.java @@ -0,0 +1,27 @@ +/* + * 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.aggregators; + +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; +import org.opensearch.test.OpenSearchTestCase; + +public class ValueAggregatorFactoryTests extends OpenSearchTestCase { + + public void testGetValueAggregatorForSumType() { + ValueAggregator aggregator = ValueAggregatorFactory.getValueAggregator(MetricStat.SUM); + assertNotNull(aggregator); + assertEquals(SumValueAggregator.class, aggregator.getClass()); + } + + public void testGetAggregatedValueTypeForSumType() { + StarTreeNumericType starTreeNumericType = ValueAggregatorFactory.getAggregatedValueType(MetricStat.SUM); + assertEquals(SumValueAggregator.VALUE_AGGREGATOR_TYPE, starTreeNumericType); + } +} 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 new file mode 100644 index 0000000000000..91049b160379f --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilderTests.java @@ -0,0 +1,254 @@ +/* + * 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.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.BaseStarTreeBuilder; +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.Composite99DocValuesFormat; +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.NumericDimension; +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.aggregators.MetricAggregatorInfo; +import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; +import org.opensearch.index.fielddata.IndexNumericFieldData; +import org.opensearch.index.mapper.ContentPath; +import org.opensearch.index.mapper.DocumentMapper; +import org.opensearch.index.mapper.Mapper; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.mapper.MappingLookup; +import org.opensearch.index.mapper.NumberFieldMapper; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.BeforeClass; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +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; + +public class BaseStarTreeBuilderTests extends OpenSearchTestCase { + + private static BaseStarTreeBuilder builder; + private static MapperService mapperService; + private static List dimensionsOrder; + private static List fields = List.of( + "field1", + "field2", + "field3", + "field4", + "field5", + "field6", + "field7", + "field8", + "field9", + "field10" + ); + private static List metrics; + private static Directory directory; + private static FieldInfo[] fieldsInfo; + private static SegmentWriteState writeState; + private static StarTreeField starTreeField; + + private static IndexOutput dataOut; + private static IndexOutput metaOut; + + @BeforeClass + public static void setup() throws IOException { + + dimensionsOrder = List.of( + new NumericDimension("field1"), + new NumericDimension("field3"), + new NumericDimension("field5"), + new NumericDimension("field8") + ); + metrics = List.of(new Metric("field2", List.of(MetricStat.SUM)), new Metric("field4", List.of(MetricStat.SUM))); + + starTreeField = new StarTreeField( + "test", + dimensionsOrder, + metrics, + new StarTreeFieldConfiguration(1, Set.of("field8"), StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP) + ); + + DocValuesProducer docValuesProducer = mock(DocValuesProducer.class); + directory = newFSDirectory(createTempDir()); + SegmentInfo segmentInfo = new SegmentInfo( + directory, + Version.LATEST, + Version.LUCENE_9_11_0, + "test_segment", + 5, + false, + false, + new Lucene99Codec(), + new HashMap<>(), + UUID.randomUUID().toString().substring(0, 16).getBytes(StandardCharsets.UTF_8), + new HashMap<>(), + null + ); + + fieldsInfo = new FieldInfo[fields.size()]; + Map 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())); + + 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); + when(mapperService.documentMapper()).thenReturn(documentMapper); + Settings settings = Settings.builder().put(settings(org.opensearch.Version.CURRENT).build()).build(); + NumberFieldMapper numberFieldMapper1 = new NumberFieldMapper.Builder("field2", NumberFieldMapper.NumberType.DOUBLE, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper2 = new NumberFieldMapper.Builder("field4", NumberFieldMapper.NumberType.DOUBLE, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + MappingLookup fieldMappers = new MappingLookup( + Set.of(numberFieldMapper1, numberFieldMapper2), + Collections.emptyList(), + Collections.emptyList(), + 0, + null + ); + when(documentMapper.mappers()).thenReturn(fieldMappers); + + builder = new BaseStarTreeBuilder(metaOut, dataOut, starTreeField, writeState, mapperService) { + @Override + public void build( + List starTreeValuesSubs, + AtomicInteger fieldNumberAcrossStarTrees, + DocValuesConsumer starTreeDocValuesConsumer + ) throws IOException {} + + @Override + public void appendStarTreeDocument(StarTreeDocument starTreeDocument) throws IOException {} + + @Override + public StarTreeDocument getStarTreeDocument(int docId) throws IOException { + return null; + } + + @Override + public List getStarTreeDocuments() { + return List.of(); + } + + @Override + public Long getDimensionValue(int docId, int dimensionId) throws IOException { + return 0L; + } + + @Override + public Iterator sortAndAggregateSegmentDocuments( + int numDocs, + SequentialDocValuesIterator[] dimensionReaders, + List metricReaders + ) throws IOException { + return null; + } + + @Override + public Iterator generateStarTreeDocumentsForStarNode(int startDocId, int endDocId, int dimensionId) + throws IOException { + return null; + } + }; + } + + public void test_generateMetricAggregatorInfos() { + List metricAggregatorInfos = builder.generateMetricAggregatorInfos(mapperService); + List expectedMetricAggregatorInfos = List.of( + new MetricAggregatorInfo(MetricStat.SUM, "field2", starTreeField.getName(), IndexNumericFieldData.NumericType.DOUBLE), + new MetricAggregatorInfo(MetricStat.SUM, "field4", starTreeField.getName(), IndexNumericFieldData.NumericType.DOUBLE) + ); + assertEquals(metricAggregatorInfos, expectedMetricAggregatorInfos); + } + + public void test_reduceStarTreeDocuments() { + StarTreeDocument starTreeDocument1 = new StarTreeDocument(new Long[] { 1L, 3L, 5L, 8L }, new Double[] { 4.0, 8.0 }); + StarTreeDocument starTreeDocument2 = new StarTreeDocument(new Long[] { 1L, 3L, 5L, 8L }, new Double[] { 10.0, 6.0 }); + + StarTreeDocument expectedeMergedStarTreeDocument = new StarTreeDocument(new Long[] { 1L, 3L, 5L, 8L }, new Double[] { 14.0, 14.0 }); + StarTreeDocument mergedStarTreeDocument = builder.reduceStarTreeDocuments(null, starTreeDocument1); + StarTreeDocument resultStarTreeDocument = builder.reduceStarTreeDocuments(mergedStarTreeDocument, starTreeDocument2); + + assertEquals(resultStarTreeDocument.metrics[0], expectedeMergedStarTreeDocument.metrics[0]); + assertEquals(resultStarTreeDocument.metrics[1], expectedeMergedStarTreeDocument.metrics[1]); + } + + @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/OnHeapStarTreeBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilderTests.java new file mode 100644 index 0000000000000..1e6e1b3197ede --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilderTests.java @@ -0,0 +1,806 @@ +/* + * 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.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.SortedNumericDocValues; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.sandbox.document.HalfFloatPoint; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.Directory; +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.Composite99DocValuesFormat; +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.NumericDimension; +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.mapper.ContentPath; +import org.opensearch.index.mapper.DocumentMapper; +import org.opensearch.index.mapper.Mapper; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.mapper.MappingLookup; +import org.opensearch.index.mapper.NumberFieldMapper; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +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.Set; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class OnHeapStarTreeBuilderTests extends OpenSearchTestCase { + + private OnHeapStarTreeBuilder builder; + private MapperService mapperService; + private List dimensionsOrder; + private List fields = List.of( + "field1", + "field2", + "field3", + "field4", + "field5", + "field6", + "field7", + "field8", + "field9", + "field10" + ); + private List metrics; + private Directory directory; + private FieldInfo[] fieldsInfo; + private StarTreeField compositeField; + private Map fieldProducerMap; + private SegmentWriteState writeState; + private IndexOutput dataOut; + private IndexOutput metaOut; + private DocValuesConsumer docValuesConsumer; + + @Before + public void setup() throws IOException { + dimensionsOrder = List.of( + new NumericDimension("field1"), + new NumericDimension("field3"), + new NumericDimension("field5"), + new NumericDimension("field8") + ); + metrics = List.of( + new Metric("field2", List.of(MetricStat.SUM)), + new Metric("field4", List.of(MetricStat.SUM)), + new Metric("field6", List.of(MetricStat.COUNT)), + new Metric("field9", List.of(MetricStat.MIN)), + new Metric("field10", List.of(MetricStat.MAX)) + ); + + DocValuesProducer docValuesProducer = mock(DocValuesProducer.class); + docValuesConsumer = mock(DocValuesConsumer.class); + + compositeField = new StarTreeField( + "test", + dimensionsOrder, + metrics, + new StarTreeFieldConfiguration(1, Set.of("field8"), StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP) + ); + directory = newFSDirectory(createTempDir()); + SegmentInfo segmentInfo = new SegmentInfo( + directory, + Version.LATEST, + Version.LUCENE_9_11_0, + "test_segment", + 5, + 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())); + + 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); + when(mapperService.documentMapper()).thenReturn(documentMapper); + Settings settings = Settings.builder().put(settings(org.opensearch.Version.CURRENT).build()).build(); + NumberFieldMapper numberFieldMapper1 = new NumberFieldMapper.Builder("field2", NumberFieldMapper.NumberType.DOUBLE, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper2 = new NumberFieldMapper.Builder("field4", NumberFieldMapper.NumberType.DOUBLE, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper3 = new NumberFieldMapper.Builder("field6", NumberFieldMapper.NumberType.DOUBLE, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper4 = new NumberFieldMapper.Builder("field9", NumberFieldMapper.NumberType.DOUBLE, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper5 = new NumberFieldMapper.Builder("field10", NumberFieldMapper.NumberType.DOUBLE, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + MappingLookup fieldMappers = new MappingLookup( + Set.of(numberFieldMapper1, numberFieldMapper2, numberFieldMapper3, numberFieldMapper4, numberFieldMapper5), + Collections.emptyList(), + Collections.emptyList(), + 0, + null + ); + when(documentMapper.mappers()).thenReturn(fieldMappers); + builder = new OnHeapStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); + } + + public void test_sortAndAggregateStarTreeDocuments() 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 }); + + List inorderStarTreeDocuments = List.of( + new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Object[] { 21.0, 14.0, 2L, 8.0, 20.0 }), + new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Object[] { 35.0, 34.0, 3L, 6.0, 24.0 }) + ); + Iterator expectedStarTreeDocumentIterator = inorderStarTreeDocuments.iterator(); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + Long[] metrics = new Long[starTreeDocuments[0].metrics.length]; + for (int j = 0; j < metrics.length; j++) { + metrics[j] = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[j]); + } + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, metrics); + } + + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateStarTreeDocuments(segmentStarTreeDocuments); + int numOfAggregatedDocuments = 0; + while (segmentStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { + StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); + StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); + for (int dim = 0; dim < 4; dim++) { + assertEquals(expectedStarTreeDocument.dimensions[dim], resultStarTreeDocument.dimensions[dim]); + } + + for (int met = 0; met < 5; met++) { + assertEquals(expectedStarTreeDocument.metrics[met], resultStarTreeDocument.metrics[met]); + } + numOfAggregatedDocuments++; + } + assertEquals(inorderStarTreeDocuments.size(), numOfAggregatedDocuments); + + } + + public void test_sortAndAggregateStarTreeDocuments_nullMetric() 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(), 8.0, 20.0 }); + starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 14.0, 12.0, randomDouble(), 8.0, 20.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, null, randomDouble(), 8.0, 20.0 }); + StarTreeDocument expectedStarTreeDocument = new StarTreeDocument( + new Long[] { 2L, 4L, 3L, 4L }, + new Object[] { 21.0, 14.0, 2L, 8.0, 20.0 } + ); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + Long[] metrics = new Long[starTreeDocuments[0].metrics.length]; + for (int j = 0; j < metrics.length; j++) { + metrics[j] = starTreeDocuments[i].metrics[j] != null + ? NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[j]) + : null; + } + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, metrics); + } + + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateStarTreeDocuments(segmentStarTreeDocuments); + + StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); + for (int dim = 0; dim < 4; dim++) { + assertEquals(expectedStarTreeDocument.dimensions[dim], resultStarTreeDocument.dimensions[dim]); + } + + for (int met = 0; met < 5; met++) { + assertEquals(expectedStarTreeDocument.metrics[met], resultStarTreeDocument.metrics[met]); + } + + assertThrows( + "Null metric should have resulted in IllegalStateException", + IllegalStateException.class, + segmentStarTreeDocumentIterator::next + ); + + } + + public void test_sortAndAggregateStarTreeDocument_longMaxAndLongMinDimensions() throws IOException { + + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + + starTreeDocuments[0] = new StarTreeDocument( + new Long[] { Long.MIN_VALUE, 4L, 3L, 4L }, + new Double[] { 12.0, 10.0, randomDouble(), 8.0, 20.0 } + ); + starTreeDocuments[1] = new StarTreeDocument( + new Long[] { 3L, 4L, 2L, Long.MAX_VALUE }, + new Double[] { 10.0, 6.0, randomDouble(), 12.0, 10.0 } + ); + starTreeDocuments[2] = new StarTreeDocument( + new Long[] { 3L, 4L, 2L, Long.MAX_VALUE }, + new Double[] { 14.0, 12.0, randomDouble(), 6.0, 24.0 } + ); + starTreeDocuments[3] = new StarTreeDocument( + new Long[] { Long.MIN_VALUE, 4L, 3L, 4L }, + new Double[] { 9.0, 4.0, randomDouble(), 9.0, 12.0 } + ); + starTreeDocuments[4] = new StarTreeDocument( + new Long[] { 3L, 4L, 2L, Long.MAX_VALUE }, + new Double[] { 11.0, 16.0, randomDouble(), 8.0, 13.0 } + ); + + List inorderStarTreeDocuments = List.of( + new StarTreeDocument(new Long[] { Long.MIN_VALUE, 4L, 3L, 4L }, new Object[] { 21.0, 14.0, 2L, 8.0, 20.0 }), + new StarTreeDocument(new Long[] { 3L, 4L, 2L, Long.MAX_VALUE }, new Object[] { 35.0, 34.0, 3L, 6.0, 24.0 }) + ); + Iterator expectedStarTreeDocumentIterator = inorderStarTreeDocuments.iterator(); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + Long[] metrics = new Long[starTreeDocuments[0].metrics.length]; + for (int j = 0; j < metrics.length; j++) { + metrics[j] = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[j]); + } + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, metrics); + } + + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateStarTreeDocuments(segmentStarTreeDocuments); + int numOfAggregatedDocuments = 0; + while (segmentStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { + StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); + StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); + + for (int dim = 0; dim < 4; dim++) { + assertEquals(expectedStarTreeDocument.dimensions[dim], resultStarTreeDocument.dimensions[dim]); + } + + for (int met = 0; met < 5; met++) { + assertEquals(expectedStarTreeDocument.metrics[met], resultStarTreeDocument.metrics[met]); + } + + numOfAggregatedDocuments++; + } + + assertEquals(inorderStarTreeDocuments.size(), numOfAggregatedDocuments); + + } + + public void test_build_DoubleMaxAndDoubleMinMetrics() throws IOException { + + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + + starTreeDocuments[0] = new StarTreeDocument( + new Long[] { 2L, 4L, 3L, 4L }, + new Double[] { Double.MAX_VALUE, 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, Double.MIN_VALUE, 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 }); + + List inorderStarTreeDocuments = List.of( + new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Object[] { Double.MAX_VALUE + 9, 14.0, 2L, 8.0, 20.0 }), + new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Object[] { 35.0, Double.MIN_VALUE + 22, 3L, 6.0, 24.0 }) + ); + Iterator expectedStarTreeDocumentIterator = inorderStarTreeDocuments.iterator(); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + Long[] metrics = new Long[starTreeDocuments[0].metrics.length]; + for (int j = 0; j < metrics.length; j++) { + metrics[j] = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[j]); + } + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, metrics); + } + + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateStarTreeDocuments(segmentStarTreeDocuments); + int numOfAggregatedDocuments = 0; + while (segmentStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { + StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); + StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); + + for (int dim = 0; dim < 4; dim++) { + assertEquals(expectedStarTreeDocument.dimensions[dim], resultStarTreeDocument.dimensions[dim]); + } + + for (int met = 0; met < 5; met++) { + assertEquals(expectedStarTreeDocument.metrics[met], resultStarTreeDocument.metrics[met]); + } + + numOfAggregatedDocuments++; + } + + assertEquals(inorderStarTreeDocuments.size(), numOfAggregatedDocuments); + + } + + public void test_build_halfFloatMetrics() throws IOException { + + 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("field2", NumberFieldMapper.NumberType.HALF_FLOAT, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper2 = new NumberFieldMapper.Builder("field4", NumberFieldMapper.NumberType.HALF_FLOAT, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper3 = new NumberFieldMapper.Builder("field6", NumberFieldMapper.NumberType.HALF_FLOAT, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper4 = new NumberFieldMapper.Builder("field9", NumberFieldMapper.NumberType.HALF_FLOAT, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper5 = new NumberFieldMapper.Builder( + "field10", + NumberFieldMapper.NumberType.HALF_FLOAT, + false, + true + ).build(new Mapper.BuilderContext(settings, new ContentPath())); + MappingLookup fieldMappers = new MappingLookup( + Set.of(numberFieldMapper1, numberFieldMapper2, numberFieldMapper3, numberFieldMapper4, numberFieldMapper5), + Collections.emptyList(), + Collections.emptyList(), + 0, + null + ); + when(documentMapper.mappers()).thenReturn(fieldMappers); + builder = new OnHeapStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); + + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + + starTreeDocuments[0] = new StarTreeDocument( + new Long[] { 2L, 4L, 3L, 4L }, + new HalfFloatPoint[] { + new HalfFloatPoint("hf1", 12), + new HalfFloatPoint("hf6", 10), + new HalfFloatPoint("field6", 10), + new HalfFloatPoint("field9", 8), + new HalfFloatPoint("field10", 20) } + ); + starTreeDocuments[1] = new StarTreeDocument( + new Long[] { 3L, 4L, 2L, 1L }, + new HalfFloatPoint[] { + new HalfFloatPoint("hf2", 10), + new HalfFloatPoint("hf7", 6), + new HalfFloatPoint("field6", 10), + new HalfFloatPoint("field9", 12), + new HalfFloatPoint("field10", 10) } + ); + starTreeDocuments[2] = new StarTreeDocument( + new Long[] { 3L, 4L, 2L, 1L }, + new HalfFloatPoint[] { + new HalfFloatPoint("hf3", 14), + new HalfFloatPoint("hf8", 12), + new HalfFloatPoint("field6", 10), + new HalfFloatPoint("field9", 6), + new HalfFloatPoint("field10", 24) } + ); + starTreeDocuments[3] = new StarTreeDocument( + new Long[] { 2L, 4L, 3L, 4L }, + new HalfFloatPoint[] { + new HalfFloatPoint("hf4", 9), + new HalfFloatPoint("hf9", 4), + new HalfFloatPoint("field6", 10), + new HalfFloatPoint("field9", 9), + new HalfFloatPoint("field10", 12) } + ); + starTreeDocuments[4] = new StarTreeDocument( + new Long[] { 3L, 4L, 2L, 1L }, + new HalfFloatPoint[] { + new HalfFloatPoint("hf5", 11), + new HalfFloatPoint("hf10", 16), + new HalfFloatPoint("field6", 10), + new HalfFloatPoint("field9", 8), + new HalfFloatPoint("field10", 13) } + ); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + Long[] metrics = new Long[starTreeDocuments[0].metrics.length]; + for (int j = 0; j < metrics.length; j++) { + metrics[j] = (long) HalfFloatPoint.halfFloatToSortableShort( + ((HalfFloatPoint) starTreeDocuments[i].metrics[j]).numericValue().floatValue() + ); + } + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, metrics); + } + + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateStarTreeDocuments(segmentStarTreeDocuments); + builder.build(segmentStarTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); + List resultStarTreeDocuments = builder.getStarTreeDocuments(); + assertEquals(8, resultStarTreeDocuments.size()); + + Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator(); + assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); + } + + public void test_build_floatMetrics() throws IOException { + + 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("field2", NumberFieldMapper.NumberType.FLOAT, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper2 = new NumberFieldMapper.Builder("field4", NumberFieldMapper.NumberType.FLOAT, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper3 = new NumberFieldMapper.Builder("field6", NumberFieldMapper.NumberType.FLOAT, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper4 = new NumberFieldMapper.Builder("field9", NumberFieldMapper.NumberType.FLOAT, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper5 = new NumberFieldMapper.Builder("field10", NumberFieldMapper.NumberType.FLOAT, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + MappingLookup fieldMappers = new MappingLookup( + Set.of(numberFieldMapper1, numberFieldMapper2, numberFieldMapper3, numberFieldMapper4, numberFieldMapper5), + Collections.emptyList(), + Collections.emptyList(), + 0, + null + ); + when(documentMapper.mappers()).thenReturn(fieldMappers); + builder = new OnHeapStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); + + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + + starTreeDocuments[0] = new StarTreeDocument( + new Long[] { 2L, 4L, 3L, 4L }, + new Float[] { 12.0F, 10.0F, randomFloat(), 8.0F, 20.0F } + ); + starTreeDocuments[1] = new StarTreeDocument( + new Long[] { 3L, 4L, 2L, 1L }, + new Float[] { 10.0F, 6.0F, randomFloat(), 12.0F, 10.0F } + ); + starTreeDocuments[2] = new StarTreeDocument( + new Long[] { 3L, 4L, 2L, 1L }, + new Float[] { 14.0F, 12.0F, randomFloat(), 6.0F, 24.0F } + ); + starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Float[] { 9.0F, 4.0F, randomFloat(), 9.0F, 12.0F }); + starTreeDocuments[4] = new StarTreeDocument( + new Long[] { 3L, 4L, 2L, 1L }, + new Float[] { 11.0F, 16.0F, randomFloat(), 8.0F, 13.0F } + ); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + Long[] metrics = new Long[starTreeDocuments[0].metrics.length]; + for (int j = 0; j < metrics.length; j++) { + metrics[j] = (long) NumericUtils.floatToSortableInt((Float) starTreeDocuments[i].metrics[j]); + } + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, metrics); + } + + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateStarTreeDocuments(segmentStarTreeDocuments); + builder.build(segmentStarTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); + + List resultStarTreeDocuments = builder.getStarTreeDocuments(); + assertEquals(8, resultStarTreeDocuments.size()); + + Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator(); + assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); + } + + public void test_build_longMetrics() throws IOException { + + 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("field2", NumberFieldMapper.NumberType.LONG, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper2 = new NumberFieldMapper.Builder("field4", NumberFieldMapper.NumberType.LONG, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper3 = new NumberFieldMapper.Builder("field6", NumberFieldMapper.NumberType.LONG, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper4 = new NumberFieldMapper.Builder("field9", NumberFieldMapper.NumberType.LONG, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper5 = new NumberFieldMapper.Builder("field10", NumberFieldMapper.NumberType.LONG, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + MappingLookup fieldMappers = new MappingLookup( + Set.of(numberFieldMapper1, numberFieldMapper2, numberFieldMapper3, numberFieldMapper4, numberFieldMapper5), + Collections.emptyList(), + Collections.emptyList(), + 0, + null + ); + when(documentMapper.mappers()).thenReturn(fieldMappers); + builder = new OnHeapStarTreeBuilder(metaOut, dataOut, compositeField, writeState, mapperService); + + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + + starTreeDocuments[0] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Long[] { 12L, 10L, randomLong(), 8L, 20L }); + starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Long[] { 10L, 6L, randomLong(), 12L, 10L }); + starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Long[] { 14L, 12L, randomLong(), 6L, 24L }); + starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Long[] { 9L, 4L, randomLong(), 9L, 12L }); + starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Long[] { 11L, 16L, randomLong(), 8L, 13L }); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + Long[] metrics = new Long[starTreeDocuments[0].metrics.length]; + for (int j = 0; j < metrics.length; j++) { + metrics[j] = (Long) starTreeDocuments[i].metrics[j]; + } + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, metrics); + } + + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateStarTreeDocuments(segmentStarTreeDocuments); + builder.build(segmentStarTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); + + List resultStarTreeDocuments = builder.getStarTreeDocuments(); + assertEquals(8, resultStarTreeDocuments.size()); + + Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator(); + assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); + } + + private static Iterator getExpectedStarTreeDocumentIterator() { + List expectedStarTreeDocuments = List.of( + new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Object[] { 21.0, 14.0, 2L }), + new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Object[] { 35.0, 34.0, 3L }), + new StarTreeDocument(new Long[] { null, 4L, 2L, 1L }, new Object[] { 35.0, 34.0, 3L }), + new StarTreeDocument(new Long[] { null, 4L, 3L, 4L }, new Object[] { 21.0, 14.0, 2L }), + new StarTreeDocument(new Long[] { null, 4L, null, 1L }, new Object[] { 35.0, 34.0, 3L }), + new StarTreeDocument(new Long[] { null, 4L, null, 4L }, new Object[] { 21.0, 14.0, 2L }), + new StarTreeDocument(new Long[] { null, 4L, null, null }, new Object[] { 56.0, 48.0, 5L }), + new StarTreeDocument(new Long[] { null, null, null, null }, new Object[] { 56.0, 48.0, 5L }) + ); + return expectedStarTreeDocuments.iterator(); + } + + public void test_build() 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[] metrics = new Long[starTreeDocuments[0].metrics.length]; + for (int j = 0; j < metrics.length; j++) { + metrics[j] = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[j]); + } + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, metrics); + } + + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateStarTreeDocuments(segmentStarTreeDocuments); + builder.build(segmentStarTreeDocumentIterator, new AtomicInteger(), docValuesConsumer); + + List resultStarTreeDocuments = builder.getStarTreeDocuments(); + assertEquals(8, resultStarTreeDocuments.size()); + + Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator(); + assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); + } + + private static void assertStarTreeDocuments( + List resultStarTreeDocuments, + Iterator expectedStarTreeDocumentIterator + ) { + Iterator resultStarTreeDocumentIterator = resultStarTreeDocuments.iterator(); + while (resultStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { + StarTreeDocument resultStarTreeDocument = resultStarTreeDocumentIterator.next(); + StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); + + assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); + assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); + assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); + assertEquals(expectedStarTreeDocument.dimensions[3], resultStarTreeDocument.dimensions[3]); + assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); + assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); + assertEquals(expectedStarTreeDocument.metrics[2], resultStarTreeDocument.metrics[2]); + } + } + + public void testMergeFlow() throws IOException { + List dimList = List.of(0L, 1L, 3L, 4L, 5L); + List docsWithField = List.of(0, 1, 3, 4, 5); + List dimList2 = List.of(0L, 1L, 2L, 3L, 4L, 5L); + List docsWithField2 = List.of(0, 1, 2, 3, 4, 5); + + List metricsList = List.of( + getLongFromDouble(0.0), + getLongFromDouble(10.0), + getLongFromDouble(20.0), + getLongFromDouble(30.0), + getLongFromDouble(40.0), + getLongFromDouble(50.0) + ); + List metricsWithField = List.of(0, 1, 2, 3, 4, 5); + + Dimension d1 = new NumericDimension("field1"); + Dimension d2 = new NumericDimension("field3"); + Metric m1 = new Metric("field2", List.of(MetricStat.SUM)); + List dims = List.of(d1, d2); + List metrics = List.of(m1); + StarTreeFieldConfiguration c = new StarTreeFieldConfiguration( + 1000, + new HashSet<>(), + StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP + ); + StarTreeField sf = new StarTreeField("sf", dims, metrics, c); + SortedNumericDocValues d1sndv = getSortedNumericMock(dimList, docsWithField); + SortedNumericDocValues d2sndv = getSortedNumericMock(dimList2, docsWithField2); + SortedNumericDocValues m1sndv = getSortedNumericMock(metricsList, metricsWithField); + Map dimDocIdSetIterators = Map.of("field1", d1sndv, "field3", d2sndv); + Map metricDocIdSetIterators = Map.of("field2", m1sndv); + StarTreeValues starTreeValues = new StarTreeValues(sf, null, dimDocIdSetIterators, metricDocIdSetIterators); + + SortedNumericDocValues f2d1sndv = getSortedNumericMock(dimList, docsWithField); + SortedNumericDocValues f2d2sndv = getSortedNumericMock(dimList2, docsWithField2); + SortedNumericDocValues f2m1sndv = getSortedNumericMock(metricsList, metricsWithField); + Map f2dimDocIdSetIterators = Map.of("field1", f2d1sndv, "field3", f2d2sndv); + Map f2metricDocIdSetIterators = Map.of("field2", f2m1sndv); + StarTreeValues starTreeValues2 = new StarTreeValues(sf, null, f2dimDocIdSetIterators, f2metricDocIdSetIterators); + OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(metaOut, dataOut, sf, writeState, mapperService); + Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); + + /** + * Asserting following dim / metrics [ dim1, dim2 / Sum [ metric] ] + * [0, 0] | [0.0] + * [1, 1] | [20.0] + * [3, 3] | [60.0] + * [4, 4] | [80.0] + * [5, 5] | [100.0] + * [null, 2] | [40.0] + */ + + while (starTreeDocumentIterator.hasNext()) { + StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); + assertEquals( + starTreeDocument.dimensions[0] != null ? starTreeDocument.dimensions[0] * 2 * 10.0 : 40.0, + starTreeDocument.metrics[0] + ); + } + } + + private Long getLongFromDouble(Double num) { + if (num == null) { + return null; + } + return NumericUtils.doubleToSortableLong(num); + } + + private SortedNumericDocValues getSortedNumericMock(List dimList, List docsWithField) { + return new SortedNumericDocValues() { + int index = -1; + + @Override + public long nextValue() throws IOException { + return dimList.get(index); + } + + @Override + public int docValueCount() { + return 0; + } + + @Override + public boolean advanceExact(int target) throws IOException { + return false; + } + + @Override + public int docID() { + return index; + } + + @Override + public int nextDoc() throws IOException { + if (index == docsWithField.size() - 1) { + return NO_MORE_DOCS; + } + index++; + return docsWithField.get(index); + } + + @Override + public int advance(int target) throws IOException { + return 0; + } + + @Override + public long cost() { + return 0; + } + }; + } + + @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/SequentialIteratorTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/SequentialIteratorTests.java new file mode 100644 index 0000000000000..8f9943a4e73ef --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/SequentialIteratorTests.java @@ -0,0 +1,133 @@ +/* + * 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.builder; + +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.util.BytesRef; +import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.BeforeClass; + +import java.io.IOException; +import java.util.Collections; + +import org.mockito.Mockito; + +import static org.mockito.Mockito.when; + +public class SequentialIteratorTests extends OpenSearchTestCase { + + private static FieldInfo mockFieldInfo; + + @BeforeClass + public static void setup() { + mockFieldInfo = new FieldInfo( + "field", + 1, + false, + false, + true, + IndexOptions.NONE, + DocValuesType.NONE, + -1, + Collections.emptyMap(), + 0, + 0, + 0, + 0, + VectorEncoding.FLOAT32, + VectorSimilarityFunction.EUCLIDEAN, + false, + false + ); + } + + public void testCreateIterator_SortedNumeric() throws IOException { + DocValuesProducer producer = Mockito.mock(DocValuesProducer.class); + SortedNumericDocValues iterator = Mockito.mock(SortedNumericDocValues.class); + when(producer.getSortedNumeric(mockFieldInfo)).thenReturn(iterator); + SequentialDocValuesIterator result = new SequentialDocValuesIterator(producer.getSortedNumeric(mockFieldInfo)); + assertEquals(iterator.getClass(), result.getDocIdSetIterator().getClass()); + } + + public void testCreateIterator_UnsupportedType() throws IOException { + DocValuesProducer producer = Mockito.mock(DocValuesProducer.class); + BinaryDocValues iterator = Mockito.mock(BinaryDocValues.class); + when(producer.getBinary(mockFieldInfo)).thenReturn(iterator); + SequentialDocValuesIterator result = new SequentialDocValuesIterator(producer.getBinary(mockFieldInfo)); + assertEquals(iterator.getClass(), result.getDocIdSetIterator().getClass()); + when(iterator.nextDoc()).thenReturn(0); + when(iterator.binaryValue()).thenReturn(new BytesRef("123")); + + IllegalStateException exception = expectThrows(IllegalStateException.class, () -> { + result.nextDoc(0); + result.value(0); + }); + assertEquals("Unsupported Iterator requested for SequentialDocValuesIterator", exception.getMessage()); + } + + public void testGetNextValue_SortedNumeric() throws IOException { + SortedNumericDocValues iterator = Mockito.mock(SortedNumericDocValues.class); + when(iterator.nextDoc()).thenReturn(0); + when(iterator.nextValue()).thenReturn(123L); + SequentialDocValuesIterator sequentialDocValuesIterator = new SequentialDocValuesIterator(iterator); + sequentialDocValuesIterator.nextDoc(0); + long result = sequentialDocValuesIterator.value(0); + assertEquals(123L, result); + } + + public void testGetNextValue_UnsupportedIterator() { + DocIdSetIterator iterator = Mockito.mock(DocIdSetIterator.class); + SequentialDocValuesIterator sequentialDocValuesIterator = new SequentialDocValuesIterator(iterator); + + IllegalStateException exception = expectThrows(IllegalStateException.class, () -> { sequentialDocValuesIterator.value(0); }); + assertEquals("Unsupported Iterator requested for SequentialDocValuesIterator", exception.getMessage()); + } + + public void testNextDoc() throws IOException { + SortedNumericDocValues iterator = Mockito.mock(SortedNumericDocValues.class); + SequentialDocValuesIterator sequentialDocValuesIterator = new SequentialDocValuesIterator(iterator); + when(iterator.nextDoc()).thenReturn(5); + + int result = sequentialDocValuesIterator.nextDoc(5); + assertEquals(5, result); + } + + public void test_multipleCoordinatedDocumentReader() throws IOException { + SortedNumericDocValues iterator1 = Mockito.mock(SortedNumericDocValues.class); + SortedNumericDocValues iterator2 = Mockito.mock(SortedNumericDocValues.class); + + SequentialDocValuesIterator sequentialDocValuesIterator1 = new SequentialDocValuesIterator(iterator1); + SequentialDocValuesIterator sequentialDocValuesIterator2 = new SequentialDocValuesIterator(iterator2); + + when(iterator1.nextDoc()).thenReturn(0); + when(iterator2.nextDoc()).thenReturn(1); + + when(iterator1.nextValue()).thenReturn(9L); + when(iterator2.nextValue()).thenReturn(9L); + + sequentialDocValuesIterator1.nextDoc(0); + sequentialDocValuesIterator2.nextDoc(0); + assertEquals(0, sequentialDocValuesIterator1.getDocId()); + assertEquals(9L, (long) sequentialDocValuesIterator1.value(0)); + assertNotEquals(0, sequentialDocValuesIterator2.getDocId()); + assertEquals(1, sequentialDocValuesIterator2.getDocId()); + assertEquals(9L, (long) sequentialDocValuesIterator2.value(1)); + + } + +}