From b9d72e0f76af209e3ca7f93c1ec576a5c211d479 Mon Sep 17 00:00:00 2001 From: Bharathwaj G Date: Wed, 24 Jul 2024 09:09:39 +0530 Subject: [PATCH 1/7] Off heap changes for star tree Signed-off-by: Bharathwaj G --- .../index/mapper/StarTreeMapperIT.java | 4 +- .../composite/Composite99DocValuesWriter.java | 8 +- .../startree/builder/BaseStarTreeBuilder.java | 35 +- .../builder/OffHeapStarTreeBuilder.java | 896 ++++++++++++++++++ .../builder/OnHeapStarTreeBuilder.java | 24 +- .../startree/builder/StarTreesBuilder.java | 9 +- .../index/mapper/StarTreeMapper.java | 3 +- .../builder/AbstractStarTreeBuilderTests.java | 100 +- .../builder/OffHeapStarTreeBuilderTests.java | 75 ++ .../builder/StarTreesBuilderTests.java | 8 - .../index/mapper/StarTreeMapperTests.java | 4 +- 11 files changed, 1099 insertions(+), 67 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java create mode 100644 server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilderTests.java diff --git a/server/src/internalClusterTest/java/org/opensearch/index/mapper/StarTreeMapperIT.java b/server/src/internalClusterTest/java/org/opensearch/index/mapper/StarTreeMapperIT.java index 8e5193b650868..1cabb8b617ce3 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/mapper/StarTreeMapperIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/mapper/StarTreeMapperIT.java @@ -275,7 +275,7 @@ public void testValidCompositeIndex() { assertEquals(expectedMetrics, starTreeFieldType.getMetrics().get(0).getMetrics()); assertEquals(10000, starTreeFieldType.getStarTreeConfig().maxLeafDocs()); assertEquals( - StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP, + StarTreeFieldConfiguration.StarTreeBuildMode.OFF_HEAP, starTreeFieldType.getStarTreeConfig().getBuildMode() ); assertEquals(Collections.emptySet(), starTreeFieldType.getStarTreeConfig().getSkipStarNodeCreationInDims()); @@ -359,7 +359,7 @@ public void testUpdateIndexWhenMappingIsSame() { assertEquals(expectedMetrics, starTreeFieldType.getMetrics().get(0).getMetrics()); assertEquals(10000, starTreeFieldType.getStarTreeConfig().maxLeafDocs()); assertEquals( - StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP, + StarTreeFieldConfiguration.StarTreeBuildMode.OFF_HEAP, starTreeFieldType.getStarTreeConfig().getBuildMode() ); assertEquals(Collections.emptySet(), starTreeFieldType.getStarTreeConfig().getSkipStarNodeCreationInDims()); 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 3859d3c998573..b44ea2ae6cb52 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,8 +8,6 @@ package org.opensearch.index.codec.composite; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.index.DocValues; @@ -50,9 +48,9 @@ public class Composite99DocValuesWriter extends DocValuesConsumer { private final Set compositeMappedFieldTypes; private final Set compositeFieldSet; private final Set segmentFieldSet; + private final boolean segmentHasCompositeFields; private final Map fieldProducerMap = new HashMap<>(); - private static final Logger logger = LogManager.getLogger(Composite99DocValuesWriter.class); public Composite99DocValuesWriter(DocValuesConsumer delegate, SegmentWriteState segmentWriteState, MapperService mapperService) { @@ -70,6 +68,8 @@ public Composite99DocValuesWriter(DocValuesConsumer delegate, SegmentWriteState for (CompositeMappedFieldType type : compositeMappedFieldTypes) { compositeFieldSet.addAll(type.fields()); } + // check if there are any composite fields which are part of the segment + segmentHasCompositeFields = !Collections.disjoint(segmentFieldSet, compositeFieldSet); } @Override @@ -91,7 +91,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.get() == null && segmentHasCompositeFields) { createCompositeIndicesIfPossible(valuesProducer, field); } } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java index 7187fade882ea..ffdabfe65ea3a 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java @@ -72,7 +72,7 @@ public abstract class BaseStarTreeBuilder implements StarTreeBuilder { protected final TreeNode rootNode = getNewNode(); - private final StarTreeField starTreeField; + protected final StarTreeField starTreeField; private final MapperService mapperService; private final SegmentWriteState state; static String NUM_SEGMENT_DOCS = "numSegmentDocs"; @@ -141,6 +141,37 @@ public List generateMetricAggregatorInfos(MapperService ma return metricAggregatorInfos; } + /** + * Get star tree document from the segment for the current docId with the dimensionReaders and metricReaders + */ + protected StarTreeDocument getStarTreeDocument( + int currentDocId, + SequentialDocValuesIterator[] dimensionReaders, + List metricReaders + ) throws IOException { + Long[] dims = new Long[numDimensions]; + int i = 0; + for (SequentialDocValuesIterator dimensionDocValueIterator : dimensionReaders) { + dimensionDocValueIterator.nextDoc(currentDocId); + Long val = dimensionDocValueIterator.value(currentDocId); + dims[i] = val; + i++; + } + i = 0; + Object[] metrics = new Object[metricReaders.size()]; + for (SequentialDocValuesIterator metricDocValuesIterator : metricReaders) { + metricDocValuesIterator.nextDoc(currentDocId); + // As part of merge, we traverse the star tree doc values + // The type of data stored in metric fields is different from the + // actual indexing field they're based on + metrics[i] = metricAggregatorInfos.get(i) + .getValueAggregators() + .toStarTreeNumericTypeValue(metricDocValuesIterator.value(currentDocId)); + i++; + } + return new StarTreeDocument(dims, metrics); + } + /** * Adds a document to the star-tree. * @@ -163,7 +194,7 @@ public List generateMetricAggregatorInfos(MapperService ma * * @return Star tree documents */ - public abstract List getStarTreeDocuments(); + public abstract List getStarTreeDocuments() throws IOException; /** * Returns the value of the dimension for the given dimension id and document in the star-tree. diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java new file mode 100644 index 0000000000000..2a679afbfda21 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java @@ -0,0 +1,896 @@ +/* + * 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.index.SegmentWriteState; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.RandomAccessInput; +import org.apache.lucene.store.TrackingDirectoryWrapper; +import org.apache.lucene.util.IntroSorter; +import org.apache.lucene.util.NumericUtils; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.util.io.IOUtils; +import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; +import org.opensearch.index.compositeindex.datacube.Dimension; +import org.opensearch.index.compositeindex.datacube.Metric; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeDocument; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericTypeConverters; +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.Collections; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Off-heap implementation of the star tree builder. + * + *

+ * Segment documents are stored in a single file named 'segment.documents' for sorting and aggregation. A document ID array is created, + * and the document IDs in the array are swapped during sorting based on the actual segment document values in the file. + *

+ * Star tree documents are stored in multiple 'star-tree.documents' files. The algorithm works as follows: + *

    + *
  1. Initially, aggregated documents are created based on the segment documents.
  2. + *
  3. Further, star tree documents are generated (e.g., in the {@code generateStarTreeDocumentsForStarNode} method) by reading the current + * aggregated documents and creating new aggregated star tree documents, which are appended to the 'star-tree.documents' files.
  4. + *
  5. This process is repeated until all combinations of star tree documents are generated.
  6. + *
+ *

In cases where previously written star tree documents need to be read from the 'star-tree.documents' files, the current + * 'star-tree.documents' file is closed, and the values are read. Then, the derived values gets appended to a new 'star-tree.documents' file. + * This is necessary because Lucene maintains immutability of data, and an {@code IndexOutput} cannot be kept open while creating an + * {@code IndexInput} on the same file, as all file contents may not be visible in the reader. Therefore, the {@code IndexOutput} must be + * closed to ensure all data can be read before creating an {@code IndexInput}. Additionally, an {@code IndexOutput} cannot be reopened, + * so a new file is created for the new star tree documents. + *

The set of 'star-tree.documents' files is maintained, and a tracker array is used to keep track of the start document ID for each file. + * Once the number of files reaches a set threshold, the files are merged. + + @opensearch.experimental + **/ +@ExperimentalApi +public class OffHeapStarTreeBuilder extends BaseStarTreeBuilder { + private static final Logger logger = LogManager.getLogger(OffHeapStarTreeBuilder.class); + private static final String SEGMENT_DOC_FILE_NAME = "segment.documents"; + private static final String STAR_TREE_DOC_FILE_NAME = "star-tree.documents"; + // TODO : Should this be via settings ? + private static final int DEFAULT_FILE_COUNT_MERGE_THRESHOLD = 5; + private final int fileCountMergeThreshold; + private final List starTreeDocumentOffsets; + private int numReadableStarTreeDocuments; + final IndexOutput segmentDocsFileOutput; + private IndexOutput starTreeDocsFileOutput; + private IndexInput starTreeDocsFileInput; + private IndexInput segmentDocsFileInput; + private RandomAccessInput segmentRandomInput; + private RandomAccessInput starTreeDocsFileRandomInput; + private final SegmentWriteState state; + private final Map fileToEndDocIdMap; + private int starTreeFileCount = -1; + private int prevStartDocId = Integer.MAX_VALUE; + private int currBytes = 0; + private int docSizeInBytes = -1; + private final TrackingDirectoryWrapper tmpDirectory; + + /** + * 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 state stores the segment write state + * @param mapperService helps to find the original type of the field + */ + protected OffHeapStarTreeBuilder(StarTreeField starTreeField, SegmentWriteState state, MapperService mapperService) throws IOException { + this(starTreeField, state, mapperService, DEFAULT_FILE_COUNT_MERGE_THRESHOLD); + } + + /** + * 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 state stores the segment write state + * @param mapperService helps to find the original type of the field + * @param fileThreshold threshold for number of files after which we merge the files + */ + protected OffHeapStarTreeBuilder(StarTreeField starTreeField, SegmentWriteState state, MapperService mapperService, int fileThreshold) + throws IOException { + super(starTreeField, state, mapperService); + this.fileCountMergeThreshold = fileThreshold; + this.state = state; + this.tmpDirectory = new TrackingDirectoryWrapper(state.directory); + fileToEndDocIdMap = new LinkedHashMap<>(); // maintain order + try { + starTreeDocsFileOutput = createStarTreeDocumentsFileOutput(); + segmentDocsFileOutput = tmpDirectory.createTempOutput(SEGMENT_DOC_FILE_NAME, state.segmentSuffix, state.context); + } catch (IOException e) { + IOUtils.closeWhileHandlingException(starTreeDocsFileOutput); + IOUtils.close(this); + throw e; + } + starTreeDocumentOffsets = new ArrayList<>(); + } + + /** + * Creates a new star tree document temporary file to store star tree documents. + */ + IndexOutput createStarTreeDocumentsFileOutput() throws IOException { + starTreeFileCount++; + return tmpDirectory.createTempOutput(STAR_TREE_DOC_FILE_NAME + starTreeFileCount, state.segmentSuffix, state.context); + } + + @Override + public void appendStarTreeDocument(StarTreeDocument starTreeDocument) throws IOException { + int bytes = writeStarTreeDocument(starTreeDocument, starTreeDocsFileOutput, true); + if (docSizeInBytes == -1) { + docSizeInBytes = bytes; + } + assert docSizeInBytes == bytes; + starTreeDocumentOffsets.add(currBytes); + currBytes += bytes; + } + + /** + * Builds star tree based on the star tree values from multiple segments + * + * @param starTreeValuesSubs contains the star tree values from multiple segments + */ + @Override + public void build(List starTreeValuesSubs) throws IOException { + try { + build(mergeStarTrees(starTreeValuesSubs)); + } finally { + try { + for (String file : tmpDirectory.getCreatedFiles()) { + tmpDirectory.deleteFile(file); + } + } catch (final IOException ignored) {} + } + } + + /** + * 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 { + int docBytesLength = 0; + int numDocs = 0; + int[] sortedDocIds; + try { + 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())); + } + int currentDocId = 0; + int numSegmentDocs = Integer.parseInt( + starTreeValues.getAttributes().getOrDefault(NUM_SEGMENT_DOCS, String.valueOf(DocIdSetIterator.NO_MORE_DOCS)) + ); + while (currentDocId < numSegmentDocs) { + StarTreeDocument starTreeDocument = getStarTreeDocument(currentDocId, dimensionReaders, metricReaders); + int bytes = writeStarTreeDocument(starTreeDocument, segmentDocsFileOutput, true); + numDocs++; + docBytesLength = bytes; + currentDocId++; + } + } + sortedDocIds = new int[numDocs]; + for (int i = 0; i < numDocs; i++) { + sortedDocIds[i] = i; + } + } finally { + segmentDocsFileOutput.close(); + } + + if (numDocs == 0) { + return Collections.emptyIterator(); + } + + return sortAndReduceDocuments(sortedDocIds, numDocs, docBytesLength, true); + } + + /** + * Sorts and reduces the star tree documents based on the dimensions during flush flow + */ + private Iterator sortAndReduceDocuments(int[] sortedDocIds, int numDocs, int docBytesLength) throws IOException { + return sortAndReduceDocuments(sortedDocIds, numDocs, docBytesLength, false); + } + + /** + * Sorts and reduces the star tree documents based on the dimensions + */ + private Iterator sortAndReduceDocuments(int[] sortedDocIds, int numDocs, int docBytesLength, boolean isMerge) + throws IOException { + try { + segmentDocsFileInput = tmpDirectory.openInput(segmentDocsFileOutput.getName(), state.context); + final long documentBytes = docBytesLength; + segmentRandomInput = segmentDocsFileInput.randomAccessSlice(0, segmentDocsFileInput.length()); + if (sortedDocIds == null || sortedDocIds.length == 0) { + logger.debug("Sorted doc ids array is null"); + return Collections.emptyIterator(); + } + new IntroSorter() { + private long[] dimensions; + + @Override + protected void swap(int i, int j) { + int temp = sortedDocIds[i]; + sortedDocIds[i] = sortedDocIds[j]; + sortedDocIds[j] = temp; + } + + @Override + protected void setPivot(int i) { + long offset = (long) sortedDocIds[i] * documentBytes; + dimensions = new long[starTreeField.getDimensionsOrder().size()]; + try { + for (int j = 0; j < dimensions.length; j++) { + dimensions[j] = segmentRandomInput.readLong(offset + (long) j * Long.BYTES); + } + } catch (IOException e) { + throw new RuntimeException("Sort documents failed ", e); + } + } + + @Override + protected int comparePivot(int j) { + long offset = (long) sortedDocIds[j] * documentBytes; + try { + for (int i = 0; i < dimensions.length; i++) { + long dimension = segmentRandomInput.readLong(offset + (long) i * Long.BYTES); + if (dimensions[i] != dimension) { + return Long.compare(dimensions[i], dimension); + } + } + } catch (IOException e) { + throw new RuntimeException("Sort documents failed ", e); + } + return 0; + } + }.sort(0, numDocs); + + // Create an iterator for aggregated documents + IndexInput finalSegmentDocsFileInput = segmentDocsFileInput; + return new Iterator() { + boolean _hasNext = true; + StarTreeDocument currentDocument; + + { + currentDocument = getSegmentStarTreeDocument(sortedDocIds[0], documentBytes, isMerge); + } + + int _docId = 1; + + @Override + public boolean hasNext() { + return _hasNext; + } + + @Override + public StarTreeDocument next() { + StarTreeDocument next = reduceSegmentStarTreeDocuments(null, currentDocument, isMerge); + while (_docId < numDocs) { + StarTreeDocument doc; + try { + doc = getSegmentStarTreeDocument(sortedDocIds[_docId++], documentBytes, isMerge); + } catch (IOException e) { + throw new RuntimeException("Reducing documents failed ", e); + } + if (!Arrays.equals(doc.dimensions, next.dimensions)) { + currentDocument = doc; + return next; + } else { + next = reduceSegmentStarTreeDocuments(next, doc, isMerge); + } + } + _hasNext = false; + IOUtils.closeWhileHandlingException(finalSegmentDocsFileInput); + try { + tmpDirectory.deleteFile(segmentDocsFileOutput.getName()); + } catch (final IOException ignored) {} + return next; + } + }; + } catch (IOException ex) { + IOUtils.closeWhileHandlingException(segmentDocsFileInput); + throw ex; + } + } + + /** + * Get segment star tree document from the segment.documents file + */ + public StarTreeDocument getSegmentStarTreeDocument(int docID, long documentBytes, boolean isMerge) throws IOException { + return readStarTreeDocument(segmentRandomInput, docID * documentBytes, isMerge); + } + + /** + * Get star tree document for the given docId from the star-tree documents file + */ + @Override + public StarTreeDocument getStarTreeDocument(int docId) throws IOException { + ensureDocumentReadable(docId); + return readStarTreeDocument(starTreeDocsFileRandomInput, starTreeDocumentOffsets.get(docId), true); + } + + // This should be only used for testing + @Override + public List getStarTreeDocuments() throws IOException { + List starTreeDocuments = new ArrayList<>(); + for (int i = 0; i < numStarTreeDocs; i++) { + starTreeDocuments.add(getStarTreeDocument(i)); + } + return starTreeDocuments; + } + + @Override + public Long getDimensionValue(int docId, int dimensionId) throws IOException { + ensureDocumentReadable(docId); + return starTreeDocsFileRandomInput.readLong((starTreeDocumentOffsets.get(docId) + ((long) dimensionId * Long.BYTES))); + } + + /** + * Sorts and aggregates all the documents of the segment based on dimension and metrics configuration + * + * @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( + SequentialDocValuesIterator[] dimensionReaders, + List metricReaders + ) throws IOException { + // Write all dimensions for segment documents into the buffer, and sort all documents using an int + // array + int documentBytesLength = 0; + int[] sortedDocIds = new int[totalSegmentDocs]; + for (int i = 0; i < totalSegmentDocs; i++) { + sortedDocIds[i] = i; + } + + try { + for (int i = 0; i < totalSegmentDocs; i++) { + StarTreeDocument document = getSegmentStarTreeDocument(i, dimensionReaders, metricReaders); + documentBytesLength = writeStarTreeDocument(document, segmentDocsFileOutput, false); + } + } finally { + segmentDocsFileOutput.close(); + } + + // Create an iterator for aggregated documents + return sortAndReduceDocuments(sortedDocIds, totalSegmentDocs, documentBytesLength); + } + + /** + * 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 { + // End doc id is not inclusive but start doc is inclusive + // Hence we need to check if buffer is readable till endDocId - 1 + ensureDocumentReadable(endDocId - 1); + + // Sort all documents using an int array + int numDocs = endDocId - startDocId; + int[] sortedDocIds = new int[numDocs]; + for (int i = 0; i < numDocs; i++) { + sortedDocIds[i] = startDocId + i; + } + new IntroSorter() { + private long[] dimensions; + + @Override + protected void swap(int i, int j) { + int temp = sortedDocIds[i]; + sortedDocIds[i] = sortedDocIds[j]; + sortedDocIds[j] = temp; + } + + @Override + protected void setPivot(int i) { + long offset = starTreeDocumentOffsets.get(sortedDocIds[i]); + dimensions = new long[starTreeField.getDimensionsOrder().size()]; + try { + for (int j = dimensionId + 1; j < dimensions.length; j++) { + dimensions[j] = starTreeDocsFileRandomInput.readLong(offset + (long) j * Long.BYTES); + } + } catch (IOException e) { + throw new RuntimeException("Sort documents failed ", e); + } + } + + @Override + protected int comparePivot(int j) { + long offset = starTreeDocumentOffsets.get(sortedDocIds[j]); + try { + for (int i = dimensionId + 1; i < dimensions.length; i++) { + long dimension = starTreeDocsFileRandomInput.readLong(offset + (long) i * Long.BYTES); + if (dimensions[i] != dimension) { + return Long.compare(dimensions[i], dimension); + } + } + } catch (IOException e) { + throw new RuntimeException("Sort documents failed ", e); + } + return 0; + } + }.sort(0, numDocs); + + // Create an iterator for aggregated documents + return new Iterator() { + boolean _hasNext = true; + StarTreeDocument _currentdocument = getStarTreeDocument(sortedDocIds[0]); + int _docId = 1; + + private boolean hasSameDimensions(StarTreeDocument document1, StarTreeDocument document2) { + for (int i = dimensionId + 1; i < starTreeField.getDimensionsOrder().size(); i++) { + if (!Objects.equals(document1.dimensions[i], document2.dimensions[i])) { + return false; + } + } + return true; + } + + @Override + public boolean hasNext() { + return _hasNext; + } + + @Override + public StarTreeDocument next() { + StarTreeDocument next = reduceStarTreeDocuments(null, _currentdocument); + next.dimensions[dimensionId] = STAR_IN_DOC_VALUES_INDEX; + while (_docId < numDocs) { + StarTreeDocument document; + try { + document = getStarTreeDocument(sortedDocIds[_docId++]); + } catch (IOException e) { + throw new RuntimeException(e); + } + if (!hasSameDimensions(document, _currentdocument)) { + _currentdocument = document; + return next; + } else { + next = reduceStarTreeDocuments(next, document); + } + } + _hasNext = false; + return next; + } + }; + } + + /** + * Write the star tree document to file associated with dimensions and metrics + */ + int writeStarTreeDocument(StarTreeDocument starTreeDocument, IndexOutput output, boolean isAggregatedDoc) throws IOException { + int numBytes = writeDimensions(starTreeDocument, output); + numBytes += writeMetrics(starTreeDocument, output, isAggregatedDoc); + return numBytes; + } + + /** + * Write dimensions to file + */ + int writeDimensions(StarTreeDocument starTreeDocument, IndexOutput output) throws IOException { + int numBytes = 0; + // Initialize to set bits for dimensions which are null + byte dimensionNullBitSet = 0; + List dimensionsByteBitSets = new ArrayList<>(); + for (int i = 0; i < starTreeDocument.dimensions.length; i++) { + if (i % 8 == 0 && i > 0) { + dimensionsByteBitSets.add(dimensionNullBitSet); + dimensionNullBitSet = 0; + } + if (starTreeDocument.dimensions[i] == null) { + // Set the corresponding bit in dimensionNullBitSet to 1 (present) + dimensionNullBitSet |= (byte) (1 << (i % 8)); + starTreeDocument.dimensions[i] = 0L; + } + output.writeLong(starTreeDocument.dimensions[i]); + numBytes += Long.BYTES; + } + dimensionsByteBitSets.add(dimensionNullBitSet); + for (Byte dimBitSet : dimensionsByteBitSets) { + output.writeByte(dimBitSet); + numBytes += Byte.BYTES; + } + return numBytes; + } + + /** + * Write star tree document metrics to file + */ + private int writeMetrics(StarTreeDocument starTreeDocument, IndexOutput output, boolean isAggregatedDoc) throws IOException { + int numBytes = 0; + List byteBitSets = new ArrayList<>(); + byte metricsNullBitSet = 0; + for (int i = 0; i < starTreeDocument.metrics.length; i++) { + if (i % 8 == 0 && i > 0) { + output.writeByte(metricsNullBitSet); + byteBitSets.add(metricsNullBitSet); + metricsNullBitSet = 0; + } + switch (metricAggregatorInfos.get(i).getValueAggregators().getAggregatedValueType()) { + case LONG: + if (starTreeDocument.metrics[i] == null) { + starTreeDocument.metrics[i] = 0L; + metricsNullBitSet |= (byte) (1 << (i % 8)); + } + output.writeLong((Long) starTreeDocument.metrics[i]); + numBytes += Long.BYTES; + break; + case DOUBLE: + if (isAggregatedDoc) { + if (starTreeDocument.metrics[i] == null) { + starTreeDocument.metrics[i] = 0.0; + metricsNullBitSet |= (byte) (1 << (i % 8)); + } + long val = NumericUtils.doubleToSortableLong((Double) starTreeDocument.metrics[i]); + output.writeLong(val); + numBytes += Long.BYTES; + } else { + if (starTreeDocument.metrics[i] == null) { + starTreeDocument.metrics[i] = 0L; + metricsNullBitSet |= (byte) (1 << (i % 8)); + } + output.writeLong((Long) starTreeDocument.metrics[i]); + numBytes += Long.BYTES; + } + break; + case INT: + case FLOAT: + default: + throw new IllegalStateException("Unsupported metric type"); + } + } + byteBitSets.add(metricsNullBitSet); + for (Byte bitSet : byteBitSets) { + output.writeByte(bitSet); + numBytes += Byte.BYTES; + } + return numBytes; + } + + /** + * Reads the star tree document from file with given offset + * + * @param input RandomAccessInput + * @param offset Offset in the file + * @param shouldReadAggregatedDocs boolean to indicate if aggregated star tree docs should be read + * @return StarTreeDocument + * @throws IOException IOException in case of I/O errors + */ + private StarTreeDocument readStarTreeDocument(RandomAccessInput input, long offset, boolean shouldReadAggregatedDocs) + throws IOException { + int dimSize = starTreeField.getDimensionsOrder().size(); + Long[] dimensions = new Long[dimSize]; + offset = readDimensions(dimensions, input, offset); + + int numMetrics = getNumMetrics(); + Object[] metrics = new Object[numMetrics]; + offset = readMetrics(input, offset, numMetrics, metrics, shouldReadAggregatedDocs); + + return new StarTreeDocument(dimensions, metrics); + } + + /** + * Read dimensions from file + */ + long readDimensions(Long[] dimensions, RandomAccessInput input, long offset) throws IOException { + for (int i = 0; i < dimensions.length; i++) { + try { + dimensions[i] = input.readLong(offset); + } catch (Exception e) { + logger.error("Error reading dimension value at offset {} for dimension {}", offset, i); + throw e; + } + offset += Long.BYTES; + } + offset += setNullValuesForDimensions(input, offset, dimensions); + return offset; + } + + /** + * Read star tree metrics from file + */ + private long readMetrics(RandomAccessInput input, long offset, int numMetrics, Object[] metrics, boolean shouldReadAggregatedDocs) + throws IOException { + for (int i = 0; i < numMetrics; i++) { + switch (metricAggregatorInfos.get(i).getValueAggregators().getAggregatedValueType()) { + case LONG: + metrics[i] = input.readLong(offset); + offset += Long.BYTES; + break; + case DOUBLE: + long val = input.readLong(offset); + if (shouldReadAggregatedDocs) { + metrics[i] = StarTreeNumericTypeConverters.sortableLongtoDouble(val); + } else { + metrics[i] = val; + } + offset += Long.BYTES; + break; + default: + throw new IllegalStateException("Unsupported metric type"); + } + } + offset += setNullForMetrics(input, offset, numMetrics, metrics); + return offset; + } + + private int getNumMetrics() { + int numMetrics = 0; + for (Metric metric : starTreeField.getMetrics()) { + numMetrics += metric.getMetrics().size(); + } + return numMetrics; + } + + /** + * Sets null/identity equivalent for the metrics when applicable based on the bitset + */ + int setNullForMetrics(RandomAccessInput input, long offset, int numMetrics, Object[] metrics) throws IOException { + int numBytes = 0; + byte nullMetricsBitSet = input.readByte(offset + numBytes); + numBytes += Byte.BYTES; + for (int i = 0; i < numMetrics; i++) { + if (i > 0 && i % 8 == 0) { + nullMetricsBitSet = input.readByte(offset + numBytes); + numBytes += Byte.BYTES; + } + boolean isMetricNull; + switch (metricAggregatorInfos.get(i).getValueAggregators().getAggregatedValueType()) { + case LONG: + isMetricNull = (nullMetricsBitSet & (1 << (i % 8))) != 0; + if (isMetricNull) { + // TODO : get the identity value of metrics here + metrics[i] = 0L; + } + break; + case DOUBLE: + isMetricNull = (nullMetricsBitSet & (1 << (i % 8))) != 0; + if (isMetricNull) { + // TODO : get the identity value of metrics here + metrics[i] = 0; + } + break; + + case FLOAT: + case INT: + default: + throw new IllegalStateException(); + } + } + return numBytes; + } + + /** + * Set null values for dimension based on the bitset + */ + private int setNullValuesForDimensions(RandomAccessInput input, long offset, Long[] dimensions) throws IOException { + int numBytes = 0; + byte nullDimensionsBitSet = input.readByte(offset + numBytes); + numBytes += Byte.BYTES; + for (int i = 0; i < dimensions.length; i++) { + if (i > 0 && i % 8 == 0) { + nullDimensionsBitSet = input.readByte(offset + numBytes); + numBytes += Byte.BYTES; + } + boolean isDimensionNull = (nullDimensionsBitSet & (1L << (i % 8))) != 0; + if (isDimensionNull) { + dimensions[i] = null; + } + } + return numBytes; + } + + /** + * Load the correct StarTreeDocuments file based on the docId + */ + private void ensureDocumentReadable(int docId) throws IOException { + ensureDocumentReadable(docId, true); + } + + /** + * Load the correct StarTreeDocuments file based on the docId + * + * @param docId requested doc id + * @param shouldCreateFileOutput this flag is used to indicate whether to create a new file output which is not needed during file format write operation + */ + private void ensureDocumentReadable(int docId, boolean shouldCreateFileOutput) throws IOException { + if (docId >= prevStartDocId && docId < numReadableStarTreeDocuments) { + return; + } + IOUtils.closeWhileHandlingException(starTreeDocsFileInput); + starTreeDocsFileInput = null; + /* + * If docId is less then the _numDocs , then we need to find a previous file associated with doc id + * The fileToByteSizeMap is in the following format + * file1 -> 521 + * file2 -> 780 + * which represents that file1 contains all docs till "520". + * + * "prevStartDocId" tracks the "start doc id" of the range in the present 'star-tree.documents' file + * "numReadableStarTreeDocuments" tracks the "end doc id + 1" of the range of docs in the present file + * + * IMPORTANT : This is case where the requested file is not the file which is being currently written to + */ + try { + if (docId < numStarTreeDocs) { + int prevStartDocId = 0; + for (Map.Entry entry : fileToEndDocIdMap.entrySet()) { + if (docId < entry.getValue()) { + starTreeDocsFileInput = tmpDirectory.openInput(entry.getKey(), state.context); + starTreeDocsFileRandomInput = starTreeDocsFileInput.randomAccessSlice( + starTreeDocsFileInput.getFilePointer(), + starTreeDocsFileInput.length() - starTreeDocsFileInput.getFilePointer() + ); + numReadableStarTreeDocuments = entry.getValue(); + break; + } + prevStartDocId = entry.getValue(); + } + this.prevStartDocId = prevStartDocId; + } + + if (starTreeDocsFileInput != null) { + return; + } + } catch (IOException ex) { + if (starTreeDocsFileOutput != null) { + IOUtils.closeWhileHandlingException(starTreeDocsFileOutput); + } + } + /* + * This is the case where the requested document id is in the currently open star.documents file + * So we close the current file, create/open a new file and read the current file. + */ + if (starTreeDocsFileOutput != null) { + IOUtils.closeWhileHandlingException(starTreeDocsFileOutput); + } + currBytes = 0; + if (starTreeDocsFileOutput != null) { + fileToEndDocIdMap.put(starTreeDocsFileOutput.getName(), numStarTreeDocs); + } + + if (shouldCreateFileOutput) { + starTreeDocsFileOutput = createStarTreeDocumentsFileOutput(); + } + + // Check if we need to merge files + if (fileToEndDocIdMap.size() >= fileCountMergeThreshold) { + mergeFiles(); + } + + if (starTreeDocsFileRandomInput != null) { + starTreeDocsFileRandomInput = null; + } + try { + int prevStartDocId = 0; + for (Map.Entry fileToEndDocId : fileToEndDocIdMap.entrySet()) { + if (docId <= fileToEndDocId.getValue() - 1) { + starTreeDocsFileInput = tmpDirectory.openInput(fileToEndDocId.getKey(), state.context); + starTreeDocsFileRandomInput = starTreeDocsFileInput.randomAccessSlice( + starTreeDocsFileInput.getFilePointer(), + starTreeDocsFileInput.length() - starTreeDocsFileInput.getFilePointer() + ); + numReadableStarTreeDocuments = fileToEndDocId.getValue(); + break; + } + prevStartDocId = fileToEndDocId.getValue(); + } + this.prevStartDocId = prevStartDocId; + } catch (IOException e) { + IOUtils.close(this); + throw e; + } + } + + /** + * Merge temporary star tree files once the number of files reach threshold + */ + private void mergeFiles() throws IOException { + try (IndexOutput mergedOutput = createStarTreeDocumentsFileOutput()) { + long st = System.currentTimeMillis(); + + long mergeBytes = 0L; + for (Map.Entry entry : fileToEndDocIdMap.entrySet()) { + IndexInput input = tmpDirectory.openInput(entry.getKey(), state.context); + mergedOutput.copyBytes(input, input.length()); + mergeBytes += input.length(); + input.close(); + } + logger.debug( + "Created merge file : {} in : {} ms with size of : {} KB", + starTreeDocsFileOutput.getName(), + System.currentTimeMillis() - st, + mergeBytes / 1024 + ); + // Delete the old files + for (String fileName : fileToEndDocIdMap.keySet()) { + tmpDirectory.deleteFile(fileName); + } + // Clear the fileToByteSizeMap and add the merged file + fileToEndDocIdMap.clear(); + fileToEndDocIdMap.put(mergedOutput.getName(), numStarTreeDocs); + } + + int curr = 0; + for (int i = 0; i < starTreeDocumentOffsets.size(); i++) { + starTreeDocumentOffsets.set(i, curr); + curr += docSizeInBytes; + } + + } + + /** + * Close the open segment files, star tree document files and associated data in/outputs. + * Delete all the temporary segment files and star tree document files + * + * @throws IOException IOException in case of I/O errors + */ + @Override + public void close() throws IOException { + try { + if (starTreeDocsFileOutput != null) { + IOUtils.closeWhileHandlingException(starTreeDocsFileOutput); + try { + tmpDirectory.deleteFile(starTreeDocsFileOutput.getName()); + } catch (IOException ignored) {} + } + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + IOUtils.closeWhileHandlingException(starTreeDocsFileInput, segmentDocsFileInput, starTreeDocsFileOutput, segmentDocsFileOutput); + } + try { + if (this.segmentDocsFileOutput != null) { + // Delete all temporary segment document files + tmpDirectory.deleteFile(segmentDocsFileOutput.getName()); + } + } catch (IOException ignored) {} + // Delete all temporary star tree document files + for (String file : fileToEndDocIdMap.keySet()) { + try { + tmpDirectory.deleteFile(file); + } catch (IOException ignored) {} + } + super.close(); + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilder.java index 1599be2e76a56..8ff111d3b41d9 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilder.java @@ -127,34 +127,12 @@ StarTreeDocument[] getSegmentsStarTreeDocuments(List starTreeVal metricReaders.add(new SequentialDocValuesIterator(metricDocValuesEntry.getValue())); } - boolean endOfDoc = false; int currentDocId = 0; int numSegmentDocs = Integer.parseInt( starTreeValues.getAttributes().getOrDefault(NUM_SEGMENT_DOCS, String.valueOf(DocIdSetIterator.NO_MORE_DOCS)) ); while (currentDocId < numSegmentDocs) { - Long[] dims = new Long[dimensionsSplitOrder.size()]; - int i = 0; - for (SequentialDocValuesIterator dimensionDocValueIterator : dimensionReaders) { - dimensionDocValueIterator.nextDoc(currentDocId); - Long val = dimensionDocValueIterator.value(currentDocId); - dims[i] = val; - i++; - } - i = 0; - Object[] metrics = new Object[metricReaders.size()]; - for (SequentialDocValuesIterator metricDocValuesIterator : metricReaders) { - metricDocValuesIterator.nextDoc(currentDocId); - // As part of merge, we traverse the star tree doc values - // The type of data stored in metric fields is different from the - // actual indexing field they're based on - metrics[i] = metricAggregatorInfos.get(i) - .getValueAggregators() - .toStarTreeNumericTypeValue(metricDocValuesIterator.value(currentDocId)); - i++; - } - StarTreeDocument starTreeDocument = new StarTreeDocument(dims, metrics); - starTreeDocuments.add(starTreeDocument); + starTreeDocuments.add(getStarTreeDocument(currentDocId, dimensionReaders, metricReaders)); currentDocId++; } } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilder.java index 6c3d476aa3a55..a744443417105 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilder.java @@ -102,9 +102,9 @@ public void buildDuringMerge(final Map> starTreeVal continue; } StarTreeField starTreeField = starTreeValuesList.get(0).getStarTreeField(); - StarTreeBuilder builder = getSingleTreeBuilder(starTreeField, state, mapperService); - builder.build(starTreeValuesList); - builder.close(); + try (StarTreeBuilder builder = getSingleTreeBuilder(starTreeField, state, mapperService)) { + builder.build(starTreeValuesList); + } } logger.debug( "Took {} ms to merge {} star-trees with star-tree fields", @@ -122,8 +122,7 @@ StarTreeBuilder getSingleTreeBuilder(StarTreeField starTreeField, SegmentWriteSt case ON_HEAP: return new OnHeapStarTreeBuilder(starTreeField, state, mapperService); case OFF_HEAP: - // TODO - // return new OffHeapStarTreeBuilder(starTreeField, state, mapperService); + return new OffHeapStarTreeBuilder(starTreeField, state, mapperService); default: throw new IllegalArgumentException( String.format( diff --git a/server/src/main/java/org/opensearch/index/mapper/StarTreeMapper.java b/server/src/main/java/org/opensearch/index/mapper/StarTreeMapper.java index d2debe762e9be..d9539f9dc0c82 100644 --- a/server/src/main/java/org/opensearch/index/mapper/StarTreeMapper.java +++ b/server/src/main/java/org/opensearch/index/mapper/StarTreeMapper.java @@ -84,8 +84,7 @@ public static class Builder extends ParametrizedFieldMapper.Builder { List.of(XContentMapValues.nodeStringArrayValue(paramMap.getOrDefault(SKIP_STAR_NODE_IN_DIMS, new ArrayList()))) ); paramMap.remove(SKIP_STAR_NODE_IN_DIMS); - // TODO : change this to off heap once off heap gets implemented - StarTreeFieldConfiguration.StarTreeBuildMode buildMode = StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP; + StarTreeFieldConfiguration.StarTreeBuildMode buildMode = StarTreeFieldConfiguration.StarTreeBuildMode.OFF_HEAP; List dimensions = buildDimensions(name, paramMap, context); paramMap.remove(ORDERED_DIMENSIONS); diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java index 76a7875919a8b..bde688fc6afea 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java @@ -411,7 +411,9 @@ public void test_sortAndAggregateStarTreeDocuments_nullDimensionsAndNullMetrics( starTreeDocuments[3] = new StarTreeDocument(new Long[] { null, null, null, null }, new Double[] { null, null, null }); starTreeDocuments[4] = new StarTreeDocument(new Long[] { null, null, null, null }, new Double[] { null, null, null }); - List inorderStarTreeDocuments = List.of(); + List inorderStarTreeDocuments = List.of( + new StarTreeDocument(new Long[] { null, null, null, null }, new Object[] { 0.0, 0.0, 5L }) + ); Iterator expectedStarTreeDocumentIterator = inorderStarTreeDocuments.iterator(); StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; @@ -1055,7 +1057,7 @@ public void testFlushFlow() throws IOException { SortedNumericDocValues m1sndv = getSortedNumericMock(metricsList, metricsWithField); SortedNumericDocValues m2sndv = getSortedNumericMock(metricsList, metricsWithField); - OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, getWriteState(6), mapperService); + builder = getStarTreeBuilder(sf, getWriteState(6), mapperService); SequentialDocValuesIterator[] dimDvs = { new SequentialDocValuesIterator(d1sndv), new SequentialDocValuesIterator(d2sndv) }; Iterator starTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( dimDvs, @@ -1120,7 +1122,7 @@ public void testFlushFlowBuild() throws IOException { SortedNumericDocValues d2sndv = getSortedNumericMock(dimList2, docsWithField2); SortedNumericDocValues m1sndv = getSortedNumericMock(metricsList, metricsWithField); - BaseStarTreeBuilder builder = getStarTreeBuilder(sf, getWriteState(100), mapperService); + builder = getStarTreeBuilder(sf, getWriteState(100), mapperService); DocValuesProducer d1vp = getDocValuesProducer(d1sndv); DocValuesProducer d2vp = getDocValuesProducer(d2sndv); @@ -1209,7 +1211,7 @@ public void testMergeFlowWithSum() throws IOException { sf, "6" ); - OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, getWriteState(6), mapperService); + builder = getStarTreeBuilder(sf, getWriteState(6), mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** * Asserting following dim / metrics [ dim1, dim2 / Sum [ metric] ] @@ -1259,7 +1261,7 @@ public void testMergeFlowWithCount() throws IOException { sf, "6" ); - OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, getWriteState(6), mapperService); + builder = getStarTreeBuilder(sf, getWriteState(6), mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] @@ -1336,7 +1338,7 @@ public void testMergeFlowWithDifferentDocsFromSegments() throws IOException { sf, "4" ); - OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, getWriteState(4), mapperService); + builder = getStarTreeBuilder(sf, getWriteState(4), mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] @@ -1363,6 +1365,66 @@ public void testMergeFlowWithDifferentDocsFromSegments() throws IOException { assertEquals(9, count); } + public void testMergeFlowNumSegmentsDocs() throws IOException { + List dimList = List.of(0L, 1L, 2L, 3L, 4L, 5L, 6L, -1L, -1L, -1L); + List docsWithField = List.of(0, 1, 2, 3, 4, 5, 6, 7, 8, 9); + List dimList2 = List.of(0L, 1L, 2L, 3L, 4L, 5L, 6L, -1L, -1L, -1L); + List docsWithField2 = List.of(0, 1, 2, 3, 4, 5, 6, 7, 8, 9); + + List metricsList = List.of(0L, 1L, 2L, 3L, 4L, 5L, 6L, -1L, -1L, -1L); + List metricsWithField = List.of(0, 1, 2, 3, 4, 5, 6, 7, 8, 9); + + List dimList3 = List.of(5L, 6L, 7L, 8L, -1L); + List docsWithField3 = List.of(0, 1, 2, 3, 4); + List dimList4 = List.of(5L, 6L, 7L, 8L, -1L); + List docsWithField4 = List.of(0, 1, 2, 3, 4); + + List metricsList2 = List.of(5L, 6L, 7L, 8L, 9L); + List metricsWithField2 = List.of(0, 1, 2, 3, 4); + + StarTreeField sf = getStarTreeField(MetricStat.COUNT); + StarTreeValues starTreeValues = getStarTreeValues( + getSortedNumericMock(dimList, docsWithField), + getSortedNumericMock(dimList2, docsWithField2), + getSortedNumericMock(metricsList, metricsWithField), + sf, + "6" + ); + + StarTreeValues starTreeValues2 = getStarTreeValues( + getSortedNumericMock(dimList3, docsWithField3), + getSortedNumericMock(dimList4, docsWithField4), + getSortedNumericMock(metricsList2, metricsWithField2), + sf, + "4" + ); + builder = getStarTreeBuilder(sf, getWriteState(4), mapperService); + Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); + /** + * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] + [0, 0] | [0] + [1, 1] | [1] + [2, 2] | [2] + [3, 3] | [3] + [4, 4] | [4] + [5, 5] | [10] + [6, 6] | [6] + [7, 7] | [7] + [8, 8] | [8] + */ + int count = 0; + while (starTreeDocumentIterator.hasNext()) { + count++; + StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); + if (Objects.equals(starTreeDocument.dimensions[0], 5L)) { + assertEquals(starTreeDocument.dimensions[0] * 2, starTreeDocument.metrics[0]); + } else { + assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); + } + } + assertEquals(9, count); + } + public void testMergeFlowWithMissingDocs() throws IOException { List dimList = List.of(0L, 1L, 2L, 3L, 4L, 6L); List docsWithField = List.of(0, 1, 2, 3, 4, 6); @@ -1396,7 +1458,7 @@ public void testMergeFlowWithMissingDocs() throws IOException { sf, "4" ); - OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, getWriteState(4), mapperService); + builder = getStarTreeBuilder(sf, getWriteState(4), mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] @@ -1456,7 +1518,7 @@ public void testMergeFlowWithMissingDocsInSecondDim() throws IOException { sf, "4" ); - OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, getWriteState(4), mapperService); + builder = getStarTreeBuilder(sf, getWriteState(4), mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] @@ -1517,7 +1579,7 @@ public void testMergeFlowWithDocsMissingAtTheEnd() throws IOException { sf, "4" ); - OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, writeState, mapperService); + builder = getStarTreeBuilder(sf, writeState, mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] @@ -1569,7 +1631,7 @@ public void testMergeFlowWithEmptyFieldsInOneSegment() throws IOException { sf, "0" ); - OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, getWriteState(0), mapperService); + builder = getStarTreeBuilder(sf, getWriteState(0), mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] @@ -1664,8 +1726,8 @@ public void testMergeFlowWithDuplicateDimensionValues() throws IOException { metricsWithField, sf ); - OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, writeState, mapperService); - builder.build(List.of(starTreeValues, starTreeValues2)); + builder = getStarTreeBuilder(sf, writeState, mapperService); + builder.build(builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2))); List starTreeDocuments = builder.getStarTreeDocuments(); assertEquals(401, starTreeDocuments.size()); int count = 0; @@ -1774,8 +1836,8 @@ public void testMergeFlowWithMaxLeafDocs() throws IOException { sf ); - OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, writeState, mapperService); - builder.build(List.of(starTreeValues, starTreeValues2)); + builder = getStarTreeBuilder(sf, writeState, mapperService); + builder.build(builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2))); List starTreeDocuments = builder.getStarTreeDocuments(); /** 635 docs get generated @@ -1892,8 +1954,8 @@ public void testMergeFlowWithDuplicateDimensionValueWithMaxLeafDocs() throws IOE metricsWithField, sf ); - OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, writeState, mapperService); - builder.build(List.of(starTreeValues, starTreeValues2)); + builder = getStarTreeBuilder(sf, writeState, mapperService); + builder.build(builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2))); List starTreeDocuments = builder.getStarTreeDocuments(); assertEquals(401, starTreeDocuments.size()); builder.close(); @@ -1991,8 +2053,8 @@ public void testMergeFlowWithMaxLeafDocsAndStarTreeNodesAssertion() throws IOExc metricsWithField, sf ); - OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, writeState, mapperService); - builder.build(List.of(starTreeValues, starTreeValues2)); + builder = getStarTreeBuilder(sf, writeState, mapperService); + builder.build(builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2))); List starTreeDocuments = builder.getStarTreeDocuments(); Map> dimValueToDocIdMap = new HashMap<>(); traverseStarTree(builder.rootNode, dimValueToDocIdMap, true); @@ -2151,7 +2213,7 @@ public void testMergeFlow() throws IOException { getAttributes(1000) ); - BaseStarTreeBuilder builder = getStarTreeBuilder(sf, writeState, mapperService); + builder = getStarTreeBuilder(sf, writeState, mapperService); Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); /** [0, 0, 0, 0] | [0.0] diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilderTests.java new file mode 100644 index 0000000000000..e42b6b96706e7 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilderTests.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.builder; + +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.FSDirectory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.RandomAccessInput; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeDocument; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.mapper.MapperService; + +import java.io.IOException; +import java.nio.file.Path; + +public class OffHeapStarTreeBuilderTests extends AbstractStarTreeBuilderTests { + @Override + public BaseStarTreeBuilder getStarTreeBuilder( + StarTreeField starTreeField, + SegmentWriteState segmentWriteState, + MapperService mapperService + ) throws IOException { + return new OffHeapStarTreeBuilder(starTreeField, segmentWriteState, mapperService, randomIntBetween(2, 6)); + } + + public void testDimensions() throws IOException { + for (int k = 0; k < 10; k++) { + int randomDimensionSize = randomIntBetween(2, 63); + Long[] dims = new Long[randomDimensionSize]; + for (int i = 0; i < randomDimensionSize; i++) { + dims[i] = randomLong(); + } + assertNullAndValuesInDims(dims); + } + } + + private void assertNullAndValuesInDims(Long[] dims) throws IOException { + int randomNullIndex1 = randomIntBetween(0, dims.length - 1); + int randomNullIndex2 = randomIntBetween(0, dims.length - 1); + dims[randomNullIndex1] = null; + dims[randomNullIndex2] = null; + Object[] metrics = new Object[64]; + StarTreeDocument doc = new StarTreeDocument(dims, metrics); + + Path basePath = createTempDir("OffHeapTests"); + FSDirectory fsDirectory = FSDirectory.open(basePath); + String TEST_FILE = "test_file"; + IndexOutput indexOutput = fsDirectory.createOutput(TEST_FILE, IOContext.DEFAULT); + OffHeapStarTreeBuilder builder = (OffHeapStarTreeBuilder) getStarTreeBuilder(compositeField, writeState, mapperService); + builder.writeDimensions(doc, indexOutput); + indexOutput.close(); + Long[] dims1 = new Long[dims.length]; + IndexInput in = fsDirectory.openInput(TEST_FILE, IOContext.DEFAULT); + RandomAccessInput randomAccessInput = in.randomAccessSlice(0, in.length()); + builder.readDimensions(dims1, randomAccessInput, 0); + for (int i = 0; i < dims.length; i++) { + if (i == randomNullIndex1 || i == randomNullIndex2) { + assertNull(dims1[i]); + } else { + assertEquals(dims[i], dims1[i]); + } + } + in.close(); + builder.close(); + fsDirectory.close(); + } +} diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilderTests.java index 564ab110fa7a5..9e275e3898222 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilderTests.java @@ -101,14 +101,6 @@ public void test_getStarTreeBuilder() throws IOException { assertTrue(starTreeBuilder instanceof OnHeapStarTreeBuilder); } - public void test_getStarTreeBuilder_illegalArgument() { - when(mapperService.getCompositeFieldTypes()).thenReturn(Set.of(starTreeFieldType)); - StarTreeFieldConfiguration starTreeFieldConfiguration = new StarTreeFieldConfiguration(1, new HashSet<>(), StarTreeFieldConfiguration.StarTreeBuildMode.OFF_HEAP); - StarTreeField starTreeField = new StarTreeField("star_tree", new ArrayList<>(), new ArrayList<>(), starTreeFieldConfiguration); - StarTreesBuilder starTreesBuilder = new StarTreesBuilder(segmentWriteState, mapperService); - assertThrows(IllegalArgumentException.class, () -> starTreesBuilder.getSingleTreeBuilder(starTreeField, segmentWriteState, mapperService)); - } - public void test_closeWithNoStarTreeFields() throws IOException { StarTreeFieldConfiguration starTreeFieldConfiguration = new StarTreeFieldConfiguration( 1, diff --git a/server/src/test/java/org/opensearch/index/mapper/StarTreeMapperTests.java b/server/src/test/java/org/opensearch/index/mapper/StarTreeMapperTests.java index 3144b1b007924..132d2ff5a566a 100644 --- a/server/src/test/java/org/opensearch/index/mapper/StarTreeMapperTests.java +++ b/server/src/test/java/org/opensearch/index/mapper/StarTreeMapperTests.java @@ -69,7 +69,7 @@ public void testValidStarTree() throws IOException { List expectedMetrics = Arrays.asList(MetricStat.SUM, MetricStat.AVG); assertEquals(expectedMetrics, starTreeFieldType.getMetrics().get(0).getMetrics()); assertEquals(100, starTreeFieldType.getStarTreeConfig().maxLeafDocs()); - assertEquals(StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP, starTreeFieldType.getStarTreeConfig().getBuildMode()); + assertEquals(StarTreeFieldConfiguration.StarTreeBuildMode.OFF_HEAP, starTreeFieldType.getStarTreeConfig().getBuildMode()); assertEquals( new HashSet<>(Arrays.asList("@timestamp", "status")), starTreeFieldType.getStarTreeConfig().getSkipStarNodeCreationInDims() @@ -101,7 +101,7 @@ public void testValidStarTreeDefaults() throws IOException { ); assertEquals(expectedMetrics, starTreeFieldType.getMetrics().get(0).getMetrics()); assertEquals(10000, starTreeFieldType.getStarTreeConfig().maxLeafDocs()); - assertEquals(StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP, starTreeFieldType.getStarTreeConfig().getBuildMode()); + assertEquals(StarTreeFieldConfiguration.StarTreeBuildMode.OFF_HEAP, starTreeFieldType.getStarTreeConfig().getBuildMode()); assertEquals(Collections.emptySet(), starTreeFieldType.getStarTreeConfig().getSkipStarNodeCreationInDims()); } } From 4ec529f1233e8bb09b71710bd83b135fd091ae33 Mon Sep 17 00:00:00 2001 From: Bharathwaj G Date: Tue, 30 Jul 2024 18:39:25 +0530 Subject: [PATCH 2/7] addressing review comments Signed-off-by: Bharathwaj G --- .../aggregators/CountValueAggregator.java | 5 + .../aggregators/SumValueAggregator.java | 5 + .../startree/aggregators/ValueAggregator.java | 5 + .../startree/builder/BaseStarTreeBuilder.java | 7 +- .../builder/OffHeapStarTreeBuilder.java | 517 ++++++++---------- .../utils/StarTreeDocumentBitSetUtil.java | 97 ++++ .../builder/AbstractStarTreeBuilderTests.java | 62 +++ 7 files changed, 395 insertions(+), 303 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtil.java 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 index 5390b6728b9b6..ed159ee2efb7b 100644 --- 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 @@ -68,4 +68,9 @@ public Long toLongValue(Long value) { public Long toStarTreeNumericTypeValue(Long value) { return value; } + + @Override + public Long getIdentityMetricValue() { + return 0L; + } } 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 index 385549216e4d6..a471f0e2bd960 100644 --- 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 @@ -103,4 +103,9 @@ public Double toStarTreeNumericTypeValue(Long value) { throw new IllegalStateException("Cannot convert " + value + " to sortable aggregation type", e); } } + + @Override + public Double getIdentityMetricValue() { + return 0D; + } } 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 index 93230ed012b13..048582cc530e5 100644 --- 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 @@ -61,4 +61,9 @@ public interface ValueAggregator { * Converts an aggregated value from a Long type. */ A toStarTreeNumericTypeValue(Long rawValue); + + /** + * Fetches a value that does not alter the result of aggregations + */ + A getIdentityMetricValue(); } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java index ffdabfe65ea3a..2a7d67bdc0022 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java @@ -361,8 +361,13 @@ protected StarTreeDocument reduceSegmentStarTreeDocuments( * @return converted metric value to long */ private static long getLong(Object metric) { - Long metricValue = null; + // TODO : remove this after we merge identity changes + if (metric instanceof Double) { + if (0D == (double) metric) { + return 0L; + } + } try { if (metric instanceof Long) { metricValue = (long) metric; diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java index 2a679afbfda21..2c279f4a57e77 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java @@ -22,11 +22,11 @@ import org.opensearch.common.util.io.IOUtils; import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; import org.opensearch.index.compositeindex.datacube.Dimension; -import org.opensearch.index.compositeindex.datacube.Metric; import org.opensearch.index.compositeindex.datacube.startree.StarTreeDocument; import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericTypeConverters; import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; +import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeDocumentBitSetUtil; import org.opensearch.index.mapper.MapperService; import java.io.IOException; @@ -38,6 +38,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.function.Function; /** * Off-heap implementation of the star tree builder. @@ -81,7 +82,7 @@ public class OffHeapStarTreeBuilder extends BaseStarTreeBuilder { private RandomAccessInput segmentRandomInput; private RandomAccessInput starTreeDocsFileRandomInput; private final SegmentWriteState state; - private final Map fileToEndDocIdMap; + private final LinkedHashMap fileToEndDocIdMap;// maintain order private int starTreeFileCount = -1; private int prevStartDocId = Integer.MAX_VALUE; private int currBytes = 0; @@ -115,7 +116,7 @@ protected OffHeapStarTreeBuilder(StarTreeField starTreeField, SegmentWriteState this.fileCountMergeThreshold = fileThreshold; this.state = state; this.tmpDirectory = new TrackingDirectoryWrapper(state.directory); - fileToEndDocIdMap = new LinkedHashMap<>(); // maintain order + fileToEndDocIdMap = new LinkedHashMap<>(); try { starTreeDocsFileOutput = createStarTreeDocumentsFileOutput(); segmentDocsFileOutput = tmpDirectory.createTempOutput(SEGMENT_DOC_FILE_NAME, state.segmentSuffix, state.context); @@ -156,11 +157,12 @@ public void build(List starTreeValuesSubs) throws IOException { try { build(mergeStarTrees(starTreeValuesSubs)); } finally { - try { - for (String file : tmpDirectory.getCreatedFiles()) { + for (String file : tmpDirectory.getCreatedFiles()) { + try { tmpDirectory.deleteFile(file); - } - } catch (final IOException ignored) {} + } catch (final IOException ignored) {} + } + } } @@ -236,70 +238,28 @@ private Iterator sortAndReduceDocuments(int[] sortedDocIds, in logger.debug("Sorted doc ids array is null"); return Collections.emptyIterator(); } - new IntroSorter() { - private long[] dimensions; - - @Override - protected void swap(int i, int j) { - int temp = sortedDocIds[i]; - sortedDocIds[i] = sortedDocIds[j]; - sortedDocIds[j] = temp; - } - - @Override - protected void setPivot(int i) { - long offset = (long) sortedDocIds[i] * documentBytes; - dimensions = new long[starTreeField.getDimensionsOrder().size()]; - try { - for (int j = 0; j < dimensions.length; j++) { - dimensions[j] = segmentRandomInput.readLong(offset + (long) j * Long.BYTES); - } - } catch (IOException e) { - throw new RuntimeException("Sort documents failed ", e); - } - } - - @Override - protected int comparePivot(int j) { - long offset = (long) sortedDocIds[j] * documentBytes; - try { - for (int i = 0; i < dimensions.length; i++) { - long dimension = segmentRandomInput.readLong(offset + (long) i * Long.BYTES); - if (dimensions[i] != dimension) { - return Long.compare(dimensions[i], dimension); - } - } - } catch (IOException e) { - throw new RuntimeException("Sort documents failed ", e); - } - return 0; - } - }.sort(0, numDocs); + sortDocuments(sortedDocIds, (index) -> (sortedDocIds[index] * documentBytes), -1, numDocs, segmentRandomInput); // Create an iterator for aggregated documents IndexInput finalSegmentDocsFileInput = segmentDocsFileInput; return new Iterator() { - boolean _hasNext = true; - StarTreeDocument currentDocument; - - { - currentDocument = getSegmentStarTreeDocument(sortedDocIds[0], documentBytes, isMerge); - } + boolean hasNext = true; + StarTreeDocument currentDocument = getSegmentStarTreeDocument(sortedDocIds[0], documentBytes, isMerge); - int _docId = 1; + int docId = 1; @Override public boolean hasNext() { - return _hasNext; + return hasNext; } @Override public StarTreeDocument next() { StarTreeDocument next = reduceSegmentStarTreeDocuments(null, currentDocument, isMerge); - while (_docId < numDocs) { + while (docId < numDocs) { StarTreeDocument doc; try { - doc = getSegmentStarTreeDocument(sortedDocIds[_docId++], documentBytes, isMerge); + doc = getSegmentStarTreeDocument(sortedDocIds[docId++], documentBytes, isMerge); } catch (IOException e) { throw new RuntimeException("Reducing documents failed ", e); } @@ -310,7 +270,7 @@ public StarTreeDocument next() { next = reduceSegmentStarTreeDocuments(next, doc, isMerge); } } - _hasNext = false; + hasNext = false; IOUtils.closeWhileHandlingException(finalSegmentDocsFileInput); try { tmpDirectory.deleteFile(segmentDocsFileOutput.getName()); @@ -411,51 +371,20 @@ public Iterator generateStarTreeDocumentsForStarNode(int start for (int i = 0; i < numDocs; i++) { sortedDocIds[i] = startDocId + i; } - new IntroSorter() { - private long[] dimensions; - @Override - protected void swap(int i, int j) { - int temp = sortedDocIds[i]; - sortedDocIds[i] = sortedDocIds[j]; - sortedDocIds[j] = temp; - } - - @Override - protected void setPivot(int i) { - long offset = starTreeDocumentOffsets.get(sortedDocIds[i]); - dimensions = new long[starTreeField.getDimensionsOrder().size()]; - try { - for (int j = dimensionId + 1; j < dimensions.length; j++) { - dimensions[j] = starTreeDocsFileRandomInput.readLong(offset + (long) j * Long.BYTES); - } - } catch (IOException e) { - throw new RuntimeException("Sort documents failed ", e); - } - } - - @Override - protected int comparePivot(int j) { - long offset = starTreeDocumentOffsets.get(sortedDocIds[j]); - try { - for (int i = dimensionId + 1; i < dimensions.length; i++) { - long dimension = starTreeDocsFileRandomInput.readLong(offset + (long) i * Long.BYTES); - if (dimensions[i] != dimension) { - return Long.compare(dimensions[i], dimension); - } - } - } catch (IOException e) { - throw new RuntimeException("Sort documents failed ", e); - } - return 0; - } - }.sort(0, numDocs); + sortDocuments( + sortedDocIds, + (index) -> Long.valueOf(starTreeDocumentOffsets.get(sortedDocIds[index])), + dimensionId, + numDocs, + starTreeDocsFileRandomInput + ); // Create an iterator for aggregated documents return new Iterator() { - boolean _hasNext = true; - StarTreeDocument _currentdocument = getStarTreeDocument(sortedDocIds[0]); - int _docId = 1; + boolean hasNext = true; + StarTreeDocument currentDocument = getStarTreeDocument(sortedDocIds[0]); + int docId = 1; private boolean hasSameDimensions(StarTreeDocument document1, StarTreeDocument document2) { for (int i = dimensionId + 1; i < starTreeField.getDimensionsOrder().size(); i++) { @@ -468,33 +397,91 @@ private boolean hasSameDimensions(StarTreeDocument document1, StarTreeDocument d @Override public boolean hasNext() { - return _hasNext; + return hasNext; } @Override public StarTreeDocument next() { - StarTreeDocument next = reduceStarTreeDocuments(null, _currentdocument); + StarTreeDocument next = reduceStarTreeDocuments(null, currentDocument); next.dimensions[dimensionId] = STAR_IN_DOC_VALUES_INDEX; - while (_docId < numDocs) { + while (docId < numDocs) { StarTreeDocument document; try { - document = getStarTreeDocument(sortedDocIds[_docId++]); + document = getStarTreeDocument(sortedDocIds[docId++]); } catch (IOException e) { throw new RuntimeException(e); } - if (!hasSameDimensions(document, _currentdocument)) { - _currentdocument = document; + if (!hasSameDimensions(document, currentDocument)) { + currentDocument = document; return next; } else { next = reduceStarTreeDocuments(next, document); } } - _hasNext = false; + hasNext = false; return next; } }; } + private void sortDocuments( + int[] sortedDocIds, + Function offsetSupplier, + int dimensionId, + int numDocs, + RandomAccessInput randomAccessInput + ) { + new IntroSorter() { + private Long[] dimensions; + + @Override + protected void swap(int i, int j) { + int temp = sortedDocIds[i]; + sortedDocIds[i] = sortedDocIds[j]; + sortedDocIds[j] = temp; + } + + @Override + protected void setPivot(int i) { + + long offset = offsetSupplier.apply(i); + dimensions = new Long[starTreeField.getDimensionsOrder().size()]; + try { + readDimensions(dimensions, randomAccessInput, offset); + } catch (IOException e) { + throw new RuntimeException("Sort documents failed ", e); + } + } + + @Override + protected int comparePivot(int j) { + long offset = offsetSupplier.apply(j); + Long[] dimensionsFromOutput = new Long[starTreeField.getDimensionsOrder().size()]; + try { + readDimensions(dimensionsFromOutput, randomAccessInput, offset); + } catch (IOException e) { + throw new RuntimeException("Sort documents failed ", e); + } + for (int i = dimensionId + 1; i < dimensions.length; i++) { + Long dimension = dimensionsFromOutput[i]; + if (dimensions[i] == null && dimension == null) { + return 0; + } + if (dimension == null) { + return -1; + } + if (dimensions[i] == null) { + return 1; + } + if (!Objects.equals(dimensions[i], dimension)) { + return Long.compare(dimensions[i], dimension); + } + } + return 0; + } + }.sort(0, numDocs); + } + /** * Write the star tree document to file associated with dimensions and metrics */ @@ -509,27 +496,11 @@ int writeStarTreeDocument(StarTreeDocument starTreeDocument, IndexOutput output, */ int writeDimensions(StarTreeDocument starTreeDocument, IndexOutput output) throws IOException { int numBytes = 0; - // Initialize to set bits for dimensions which are null - byte dimensionNullBitSet = 0; - List dimensionsByteBitSets = new ArrayList<>(); for (int i = 0; i < starTreeDocument.dimensions.length; i++) { - if (i % 8 == 0 && i > 0) { - dimensionsByteBitSets.add(dimensionNullBitSet); - dimensionNullBitSet = 0; - } - if (starTreeDocument.dimensions[i] == null) { - // Set the corresponding bit in dimensionNullBitSet to 1 (present) - dimensionNullBitSet |= (byte) (1 << (i % 8)); - starTreeDocument.dimensions[i] = 0L; - } - output.writeLong(starTreeDocument.dimensions[i]); + output.writeLong(starTreeDocument.dimensions[i] == null ? 0L : starTreeDocument.dimensions[i]); numBytes += Long.BYTES; } - dimensionsByteBitSets.add(dimensionNullBitSet); - for (Byte dimBitSet : dimensionsByteBitSets) { - output.writeByte(dimBitSet); - numBytes += Byte.BYTES; - } + numBytes += StarTreeDocumentBitSetUtil.writeBitSet(starTreeDocument.dimensions, output); return numBytes; } @@ -538,52 +509,28 @@ int writeDimensions(StarTreeDocument starTreeDocument, IndexOutput output) throw */ private int writeMetrics(StarTreeDocument starTreeDocument, IndexOutput output, boolean isAggregatedDoc) throws IOException { int numBytes = 0; - List byteBitSets = new ArrayList<>(); - byte metricsNullBitSet = 0; for (int i = 0; i < starTreeDocument.metrics.length; i++) { - if (i % 8 == 0 && i > 0) { - output.writeByte(metricsNullBitSet); - byteBitSets.add(metricsNullBitSet); - metricsNullBitSet = 0; - } switch (metricAggregatorInfos.get(i).getValueAggregators().getAggregatedValueType()) { case LONG: - if (starTreeDocument.metrics[i] == null) { - starTreeDocument.metrics[i] = 0L; - metricsNullBitSet |= (byte) (1 << (i % 8)); - } - output.writeLong((Long) starTreeDocument.metrics[i]); + output.writeLong(starTreeDocument.metrics[i] == null ? 0L : (Long) starTreeDocument.metrics[i]); numBytes += Long.BYTES; break; case DOUBLE: if (isAggregatedDoc) { - if (starTreeDocument.metrics[i] == null) { - starTreeDocument.metrics[i] = 0.0; - metricsNullBitSet |= (byte) (1 << (i % 8)); - } - long val = NumericUtils.doubleToSortableLong((Double) starTreeDocument.metrics[i]); + long val = NumericUtils.doubleToSortableLong( + starTreeDocument.metrics[i] == null ? 0.0 : (Double) starTreeDocument.metrics[i] + ); output.writeLong(val); - numBytes += Long.BYTES; } else { - if (starTreeDocument.metrics[i] == null) { - starTreeDocument.metrics[i] = 0L; - metricsNullBitSet |= (byte) (1 << (i % 8)); - } - output.writeLong((Long) starTreeDocument.metrics[i]); - numBytes += Long.BYTES; + output.writeLong(starTreeDocument.metrics[i] == null ? 0L : (Long) starTreeDocument.metrics[i]); } + numBytes += Long.BYTES; break; - case INT: - case FLOAT: default: throw new IllegalStateException("Unsupported metric type"); } } - byteBitSets.add(metricsNullBitSet); - for (Byte bitSet : byteBitSets) { - output.writeByte(bitSet); - numBytes += Byte.BYTES; - } + numBytes += StarTreeDocumentBitSetUtil.writeBitSet(starTreeDocument.metrics, output); return numBytes; } @@ -602,7 +549,6 @@ private StarTreeDocument readStarTreeDocument(RandomAccessInput input, long offs Long[] dimensions = new Long[dimSize]; offset = readDimensions(dimensions, input, offset); - int numMetrics = getNumMetrics(); Object[] metrics = new Object[numMetrics]; offset = readMetrics(input, offset, numMetrics, metrics, shouldReadAggregatedDocs); @@ -622,7 +568,7 @@ long readDimensions(Long[] dimensions, RandomAccessInput input, long offset) thr } offset += Long.BYTES; } - offset += setNullValuesForDimensions(input, offset, dimensions); + offset += StarTreeDocumentBitSetUtil.readAndSetNullBasedOnBitSet(input, offset, dimensions); return offset; } @@ -650,76 +596,15 @@ private long readMetrics(RandomAccessInput input, long offset, int numMetrics, O throw new IllegalStateException("Unsupported metric type"); } } - offset += setNullForMetrics(input, offset, numMetrics, metrics); + offset += StarTreeDocumentBitSetUtil.readAndSetIdentityValueBasedOnBitSet( + input, + offset, + metrics, + index -> metricAggregatorInfos.get(index).getValueAggregators().getIdentityMetricValue() + ); return offset; } - private int getNumMetrics() { - int numMetrics = 0; - for (Metric metric : starTreeField.getMetrics()) { - numMetrics += metric.getMetrics().size(); - } - return numMetrics; - } - - /** - * Sets null/identity equivalent for the metrics when applicable based on the bitset - */ - int setNullForMetrics(RandomAccessInput input, long offset, int numMetrics, Object[] metrics) throws IOException { - int numBytes = 0; - byte nullMetricsBitSet = input.readByte(offset + numBytes); - numBytes += Byte.BYTES; - for (int i = 0; i < numMetrics; i++) { - if (i > 0 && i % 8 == 0) { - nullMetricsBitSet = input.readByte(offset + numBytes); - numBytes += Byte.BYTES; - } - boolean isMetricNull; - switch (metricAggregatorInfos.get(i).getValueAggregators().getAggregatedValueType()) { - case LONG: - isMetricNull = (nullMetricsBitSet & (1 << (i % 8))) != 0; - if (isMetricNull) { - // TODO : get the identity value of metrics here - metrics[i] = 0L; - } - break; - case DOUBLE: - isMetricNull = (nullMetricsBitSet & (1 << (i % 8))) != 0; - if (isMetricNull) { - // TODO : get the identity value of metrics here - metrics[i] = 0; - } - break; - - case FLOAT: - case INT: - default: - throw new IllegalStateException(); - } - } - return numBytes; - } - - /** - * Set null values for dimension based on the bitset - */ - private int setNullValuesForDimensions(RandomAccessInput input, long offset, Long[] dimensions) throws IOException { - int numBytes = 0; - byte nullDimensionsBitSet = input.readByte(offset + numBytes); - numBytes += Byte.BYTES; - for (int i = 0; i < dimensions.length; i++) { - if (i > 0 && i % 8 == 0) { - nullDimensionsBitSet = input.readByte(offset + numBytes); - numBytes += Byte.BYTES; - } - boolean isDimensionNull = (nullDimensionsBitSet & (1L << (i % 8))) != 0; - if (isDimensionNull) { - dimensions[i] = null; - } - } - return numBytes; - } - /** * Load the correct StarTreeDocuments file based on the docId */ @@ -734,129 +619,157 @@ private void ensureDocumentReadable(int docId) throws IOException { * @param shouldCreateFileOutput this flag is used to indicate whether to create a new file output which is not needed during file format write operation */ private void ensureDocumentReadable(int docId, boolean shouldCreateFileOutput) throws IOException { - if (docId >= prevStartDocId && docId < numReadableStarTreeDocuments) { - return; - } - IOUtils.closeWhileHandlingException(starTreeDocsFileInput); - starTreeDocsFileInput = null; - /* - * If docId is less then the _numDocs , then we need to find a previous file associated with doc id - * The fileToByteSizeMap is in the following format - * file1 -> 521 - * file2 -> 780 - * which represents that file1 contains all docs till "520". - * - * "prevStartDocId" tracks the "start doc id" of the range in the present 'star-tree.documents' file - * "numReadableStarTreeDocuments" tracks the "end doc id + 1" of the range of docs in the present file - * - * IMPORTANT : This is case where the requested file is not the file which is being currently written to - */ try { - if (docId < numStarTreeDocs) { - int prevStartDocId = 0; - for (Map.Entry entry : fileToEndDocIdMap.entrySet()) { - if (docId < entry.getValue()) { - starTreeDocsFileInput = tmpDirectory.openInput(entry.getKey(), state.context); - starTreeDocsFileRandomInput = starTreeDocsFileInput.randomAccessSlice( - starTreeDocsFileInput.getFilePointer(), - starTreeDocsFileInput.length() - starTreeDocsFileInput.getFilePointer() - ); - numReadableStarTreeDocuments = entry.getValue(); - break; - } - prevStartDocId = entry.getValue(); - } - this.prevStartDocId = prevStartDocId; + if (docId >= prevStartDocId && docId < numReadableStarTreeDocuments) { + return; } + IOUtils.closeWhileHandlingException(starTreeDocsFileInput); + starTreeDocsFileInput = null; + + if (docId < numStarTreeDocs) { + loadPreviousStarTreeDocumentFile(docId); + } if (starTreeDocsFileInput != null) { return; } + closeAndMaybeCreateNewFile(shouldCreateFileOutput); + + int prevStartDocId = 0; + for (Map.Entry fileToEndDocId : fileToEndDocIdMap.entrySet()) { + if (docId <= fileToEndDocId.getValue() - 1) { + loadStarTreeDocumentFile(fileToEndDocId.getKey(), fileToEndDocId.getValue()); + break; + } + prevStartDocId = fileToEndDocId.getValue(); + } + this.prevStartDocId = prevStartDocId; } catch (IOException ex) { - if (starTreeDocsFileOutput != null) { - IOUtils.closeWhileHandlingException(starTreeDocsFileOutput); + IOUtils.close(this); + throw ex; + } + } + + /** + * If docId is less then the numDocs , then we need to find a previous file associated with doc id + * The fileToByteSizeMap is in the following format + * file1 -> 521 + * file2 -> 780 + * which represents that file1 contains all docs till "520". + *

+ * "prevStartDocId" tracks the "start doc id" of the range in the present 'star-tree.documents' file + * "numReadableStarTreeDocuments" tracks the "end doc id + 1" of the range of docs in the present file + *

+ * IMPORTANT : This is case where the requested file is not the file which is being currently written to + */ + private void loadPreviousStarTreeDocumentFile(int docId) throws IOException { + int prevStartDocId = 0; + for (Map.Entry entry : fileToEndDocIdMap.entrySet()) { + if (docId < entry.getValue()) { + loadStarTreeDocumentFile(entry.getKey(), entry.getValue()); + break; } + prevStartDocId = entry.getValue(); } - /* - * This is the case where the requested document id is in the currently open star.documents file - * So we close the current file, create/open a new file and read the current file. - */ + this.prevStartDocId = prevStartDocId; + } + + /** + * Load the requested star-tree.documents file + */ + private void loadStarTreeDocumentFile(String fileName, int endDocId) throws IOException { + starTreeDocsFileInput = tmpDirectory.openInput(fileName, state.context); + starTreeDocsFileRandomInput = starTreeDocsFileInput.randomAccessSlice( + starTreeDocsFileInput.getFilePointer(), + starTreeDocsFileInput.length() - starTreeDocsFileInput.getFilePointer() + ); + numReadableStarTreeDocuments = endDocId; + } + + /** + * This case handles when the requested document ID is beyond the range of the currently open 'star-tree.documents' file. + * In this scenario, the following steps are taken: + * + * 1. Close the current 'star-tree.documents' file. + * 2. Create a new 'star-tree.documents' file if the operation involves appending new documents. + * If the operation is only for reading existing documents, a new file is not created. + */ + private void closeAndMaybeCreateNewFile(boolean shouldCreateFileForAppend) throws IOException { if (starTreeDocsFileOutput != null) { - IOUtils.closeWhileHandlingException(starTreeDocsFileOutput); + IOUtils.close(starTreeDocsFileOutput); } currBytes = 0; if (starTreeDocsFileOutput != null) { fileToEndDocIdMap.put(starTreeDocsFileOutput.getName(), numStarTreeDocs); } - if (shouldCreateFileOutput) { + if (shouldCreateFileForAppend) { starTreeDocsFileOutput = createStarTreeDocumentsFileOutput(); - } - - // Check if we need to merge files - if (fileToEndDocIdMap.size() >= fileCountMergeThreshold) { - mergeFiles(); + if (fileToEndDocIdMap.size() >= fileCountMergeThreshold) { + mergeFiles(); + } } if (starTreeDocsFileRandomInput != null) { starTreeDocsFileRandomInput = null; } - try { - int prevStartDocId = 0; - for (Map.Entry fileToEndDocId : fileToEndDocIdMap.entrySet()) { - if (docId <= fileToEndDocId.getValue() - 1) { - starTreeDocsFileInput = tmpDirectory.openInput(fileToEndDocId.getKey(), state.context); - starTreeDocsFileRandomInput = starTreeDocsFileInput.randomAccessSlice( - starTreeDocsFileInput.getFilePointer(), - starTreeDocsFileInput.length() - starTreeDocsFileInput.getFilePointer() - ); - numReadableStarTreeDocuments = fileToEndDocId.getValue(); - break; - } - prevStartDocId = fileToEndDocId.getValue(); - } - this.prevStartDocId = prevStartDocId; - } catch (IOException e) { - IOUtils.close(this); - throw e; - } } /** * Merge temporary star tree files once the number of files reach threshold */ private void mergeFiles() throws IOException { + long st = System.currentTimeMillis(); try (IndexOutput mergedOutput = createStarTreeDocumentsFileOutput()) { - long st = System.currentTimeMillis(); - - long mergeBytes = 0L; - for (Map.Entry entry : fileToEndDocIdMap.entrySet()) { - IndexInput input = tmpDirectory.openInput(entry.getKey(), state.context); - mergedOutput.copyBytes(input, input.length()); - mergeBytes += input.length(); - input.close(); - } + long mergeBytes = mergeFilesToOutput(mergedOutput); logger.debug( "Created merge file : {} in : {} ms with size of : {} KB", starTreeDocsFileOutput.getName(), System.currentTimeMillis() - st, mergeBytes / 1024 ); - // Delete the old files - for (String fileName : fileToEndDocIdMap.keySet()) { - tmpDirectory.deleteFile(fileName); - } - // Clear the fileToByteSizeMap and add the merged file + + deleteOldFiles(); fileToEndDocIdMap.clear(); fileToEndDocIdMap.put(mergedOutput.getName(), numStarTreeDocs); + resetStarTreeDocumentOffsets(); } + } + /** + * Merge all files to single IndexOutput + */ + private long mergeFilesToOutput(IndexOutput mergedOutput) throws IOException { + long mergeBytes = 0L; + for (Map.Entry entry : fileToEndDocIdMap.entrySet()) { + IndexInput input = tmpDirectory.openInput(entry.getKey(), state.context); + mergedOutput.copyBytes(input, input.length()); + mergeBytes += input.length(); + input.close(); + } + return mergeBytes; + } + + /** + * Delete the old startree.documents files + */ + private void deleteOldFiles() throws IOException { + for (String fileName : fileToEndDocIdMap.keySet()) { + try { + tmpDirectory.deleteFile(fileName); + } catch (IOException ignored) {} + } + } + + /** + * Reset the star tree document offsets based on the merged file + */ + private void resetStarTreeDocumentOffsets() { int curr = 0; for (int i = 0; i < starTreeDocumentOffsets.size(); i++) { starTreeDocumentOffsets.set(i, curr); curr += docSizeInBytes; } - } /** diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtil.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtil.java new file mode 100644 index 0000000000000..929a2cd9cea28 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtil.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.utils; + +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.RandomAccessInput; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; + +/** + * Helper class to read/write bitset for null values and identity values. + */ +public class StarTreeDocumentBitSetUtil { + /** + * Write bitset for null values. + * + * @param array array of objects + * @param output output stream + * @return number of bytes written + * @throws IOException if an I/O error occurs while writing to the output stream + */ + public static int writeBitSet(Object[] array, IndexOutput output) throws IOException { + int numBytes = 0; + List nullBitSetList = new ArrayList<>(); + byte nullBitSet = 0; + for (int i = 0; i < array.length; i++) { + if (i % 8 == 0 && i > 0) { + nullBitSetList.add(nullBitSet); + nullBitSet = 0; + } + if (array[i] == null) { + // Set the corresponding bit in dimensionNullBitSet to 1 (present) + nullBitSet |= (byte) (1 << (i % 8)); + } + } + nullBitSetList.add(nullBitSet); + for (Byte bitSet : nullBitSetList) { + output.writeByte(bitSet); + numBytes += Byte.BYTES; + } + return numBytes; + } + + /** + * Set null values based on bitset. + */ + public static int readAndSetNullBasedOnBitSet(RandomAccessInput input, long offset, Object[] array) throws IOException { + int numBytes = 0; + byte nullDimensionsBitSet = input.readByte(offset + numBytes); + numBytes += Byte.BYTES; + for (int i = 0; i < array.length; i++) { + if (i > 0 && i % 8 == 0) { + nullDimensionsBitSet = input.readByte(offset + numBytes); + numBytes += Byte.BYTES; + } + boolean isElementNull = (nullDimensionsBitSet & (1L << (i % 8))) != 0; + if (isElementNull) { + array[i] = null; + } + } + return numBytes; + } + + /** + * Set identity values based on bitset. + */ + public static int readAndSetIdentityValueBasedOnBitSet( + RandomAccessInput input, + long offset, + Object[] array, + Function identityValueSupplier + ) throws IOException { + int numBytes = 0; + byte nullDimensionsBitSet = input.readByte(offset + numBytes); + numBytes += Byte.BYTES; + for (int i = 0; i < array.length; i++) { + if (i > 0 && i % 8 == 0) { + nullDimensionsBitSet = input.readByte(offset + numBytes); + numBytes += Byte.BYTES; + } + boolean isElementNull = (nullDimensionsBitSet & (1L << (i % 8))) != 0; + if (isElementNull) { + array[i] = identityValueSupplier.apply(i); + } + } + return numBytes; + } +} diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java index bde688fc6afea..b0d1f3692c4bd 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java @@ -1485,6 +1485,68 @@ public void testMergeFlowWithMissingDocs() throws IOException { assertEquals(10, count); } + public void testMergeFlowWithMissingDocsWithZero() throws IOException { + List dimList = List.of(0L, 0L, 0L, 0L); + List docsWithField = List.of(0, 1, 2, 6); + List dimList2 = List.of(0L, 0L, 0L, 0L); + List docsWithField2 = List.of(0, 1, 2, 6); + + List metricsList = List.of(0L, 1L, 2L, 3L, 4L, 5L, 6L); + List metricsWithField = List.of(0, 1, 2, 3, 4, 5, 6); + + List dimList3 = List.of(5L, 6L, 8L, -1L); + List docsWithField3 = List.of(0, 1, 3, 4); + List dimList4 = List.of(5L, 6L, 7L, 8L, -1L); + List docsWithField4 = List.of(0, 1, 2, 3, 4); + + List metricsList2 = List.of(5L, 6L, 7L, 8L, 9L); + List metricsWithField2 = List.of(0, 1, 2, 3, 4); + + StarTreeField sf = getStarTreeField(MetricStat.COUNT); + StarTreeValues starTreeValues = getStarTreeValues( + getSortedNumericMock(dimList, docsWithField), + getSortedNumericMock(dimList2, docsWithField2), + getSortedNumericMock(metricsList, metricsWithField), + sf, + "7" + ); + + StarTreeValues starTreeValues2 = getStarTreeValues( + getSortedNumericMock(dimList3, docsWithField3), + getSortedNumericMock(dimList4, docsWithField4), + getSortedNumericMock(metricsList2, metricsWithField2), + sf, + "4" + ); + builder = getStarTreeBuilder(sf, getWriteState(4), mapperService); + Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); + /** + * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] + [0, 0] | [9] + [5, 5] | [5] + [6, 6] | [6] + [8, 8] | [8] + [null, null] | [12] + [null, 7] | [7] + */ + int count = 0; + while (starTreeDocumentIterator.hasNext()) { + count++; + StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); + System.out.println(starTreeDocument); + if (starTreeDocument.dimensions[0] == null && starTreeDocument.dimensions[1] == null) { + assertEquals(12L, (long) starTreeDocument.metrics[0]); + } else if (starTreeDocument.dimensions[0] == null) { + assertEquals(7L, starTreeDocument.metrics[0]); + } else if (starTreeDocument.dimensions[0] == 0) { + assertEquals(9L, starTreeDocument.metrics[0]); + } else { + assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); + } + } + assertEquals(6, count); + } + public void testMergeFlowWithMissingDocsInSecondDim() throws IOException { List dimList2 = List.of(0L, 1L, 2L, 3L, 4L, 6L); List docsWithField2 = List.of(0, 1, 2, 3, 4, 6); From 91be29b970acdc1f9aa534a286f9632bdff3648b Mon Sep 17 00:00:00 2001 From: Bharathwaj G Date: Wed, 31 Jul 2024 01:22:48 +0530 Subject: [PATCH 3/7] Addressing comments Signed-off-by: Bharathwaj G --- .../builder/OffHeapStarTreeBuilder.java | 151 +++++++++--------- .../startree/builder/StarTreesBuilder.java | 6 +- .../builder/AbstractStarTreeBuilderTests.java | 68 +++++++- .../builder/StarTreesBuilderTests.java | 2 +- .../SequentialDocValuesIteratorTests.java | 2 - .../StarTreeDocumentBitSetUtilTests.java | 71 ++++++++ 6 files changed, 219 insertions(+), 81 deletions(-) create mode 100644 server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtilTests.java diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java index 2c279f4a57e77..2e9d6b7d449ed 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java @@ -84,7 +84,7 @@ public class OffHeapStarTreeBuilder extends BaseStarTreeBuilder { private final SegmentWriteState state; private final LinkedHashMap fileToEndDocIdMap;// maintain order private int starTreeFileCount = -1; - private int prevStartDocId = Integer.MAX_VALUE; + private int currentDocStartId = Integer.MAX_VALUE; private int currBytes = 0; private int docSizeInBytes = -1; private final TrackingDirectoryWrapper tmpDirectory; @@ -139,10 +139,6 @@ IndexOutput createStarTreeDocumentsFileOutput() throws IOException { @Override public void appendStarTreeDocument(StarTreeDocument starTreeDocument) throws IOException { int bytes = writeStarTreeDocument(starTreeDocument, starTreeDocsFileOutput, true); - if (docSizeInBytes == -1) { - docSizeInBytes = bytes; - } - assert docSizeInBytes == bytes; starTreeDocumentOffsets.add(currBytes); currBytes += bytes; } @@ -160,10 +156,42 @@ public void build(List starTreeValuesSubs) throws IOException { for (String file : tmpDirectory.getCreatedFiles()) { try { tmpDirectory.deleteFile(file); - } catch (final IOException ignored) {} + } catch (final IOException ignored) { + logDeleteFileError(file); + } } + } + } + /** + * Sorts and aggregates all the documents of the segment based on dimension and metrics configuration + * + * @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( + SequentialDocValuesIterator[] dimensionReaders, + List metricReaders + ) throws IOException { + // Write all dimensions for segment documents into the buffer, + // and sort all documents using an int array + int documentBytesLength = 0; + int[] sortedDocIds = new int[totalSegmentDocs]; + for (int i = 0; i < totalSegmentDocs; i++) { + sortedDocIds[i] = i; } + try { + for (int i = 0; i < totalSegmentDocs; i++) { + StarTreeDocument document = getSegmentStarTreeDocument(i, dimensionReaders, metricReaders); + documentBytesLength = writeStarTreeDocument(document, segmentDocsFileOutput, false); + } + } finally { + segmentDocsFileOutput.close(); + } + // Create an iterator for aggregated documents + return sortAndReduceDocuments(sortedDocIds, totalSegmentDocs, documentBytesLength); } /** @@ -176,7 +204,7 @@ public void build(List starTreeValuesSubs) throws IOException { Iterator mergeStarTrees(List starTreeValuesSubs) throws IOException { int docBytesLength = 0; int numDocs = 0; - int[] sortedDocIds; + int[] docIds; try { for (StarTreeValues starTreeValues : starTreeValuesSubs) { List dimensionsSplitOrder = starTreeValues.getStarTreeField().getDimensionsOrder(); @@ -203,9 +231,9 @@ Iterator mergeStarTrees(List starTreeValuesSub currentDocId++; } } - sortedDocIds = new int[numDocs]; + docIds = new int[numDocs]; for (int i = 0; i < numDocs; i++) { - sortedDocIds[i] = i; + docIds[i] = i; } } finally { segmentDocsFileOutput.close(); @@ -215,14 +243,14 @@ Iterator mergeStarTrees(List starTreeValuesSub return Collections.emptyIterator(); } - return sortAndReduceDocuments(sortedDocIds, numDocs, docBytesLength, true); + return sortAndReduceDocuments(docIds, numDocs, docBytesLength, true); } /** * Sorts and reduces the star tree documents based on the dimensions during flush flow */ - private Iterator sortAndReduceDocuments(int[] sortedDocIds, int numDocs, int docBytesLength) throws IOException { - return sortAndReduceDocuments(sortedDocIds, numDocs, docBytesLength, false); + private Iterator sortAndReduceDocuments(int[] docIds, int numDocs, int docBytesLength) throws IOException { + return sortAndReduceDocuments(docIds, numDocs, docBytesLength, false); } /** @@ -238,7 +266,7 @@ private Iterator sortAndReduceDocuments(int[] sortedDocIds, in logger.debug("Sorted doc ids array is null"); return Collections.emptyIterator(); } - sortDocuments(sortedDocIds, (index) -> (sortedDocIds[index] * documentBytes), -1, numDocs, segmentRandomInput); + sortDocumentsOffHeap(sortedDocIds, (index) -> (sortedDocIds[index] * documentBytes), -1, numDocs, segmentRandomInput); // Create an iterator for aggregated documents IndexInput finalSegmentDocsFileInput = segmentDocsFileInput; @@ -274,7 +302,9 @@ public StarTreeDocument next() { IOUtils.closeWhileHandlingException(finalSegmentDocsFileInput); try { tmpDirectory.deleteFile(segmentDocsFileOutput.getName()); - } catch (final IOException ignored) {} + } catch (final IOException ignored) { + logDeleteFileError(segmentDocsFileOutput.getName()); + } return next; } }; @@ -292,7 +322,7 @@ public StarTreeDocument getSegmentStarTreeDocument(int docID, long documentBytes } /** - * Get star tree document for the given docId from the star-tree documents file + * Get star tree document for the given docId from the star-tree.documents file */ @Override public StarTreeDocument getStarTreeDocument(int docId) throws IOException { @@ -316,39 +346,6 @@ public Long getDimensionValue(int docId, int dimensionId) throws IOException { return starTreeDocsFileRandomInput.readLong((starTreeDocumentOffsets.get(docId) + ((long) dimensionId * Long.BYTES))); } - /** - * Sorts and aggregates all the documents of the segment based on dimension and metrics configuration - * - * @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( - SequentialDocValuesIterator[] dimensionReaders, - List metricReaders - ) throws IOException { - // Write all dimensions for segment documents into the buffer, and sort all documents using an int - // array - int documentBytesLength = 0; - int[] sortedDocIds = new int[totalSegmentDocs]; - for (int i = 0; i < totalSegmentDocs; i++) { - sortedDocIds[i] = i; - } - - try { - for (int i = 0; i < totalSegmentDocs; i++) { - StarTreeDocument document = getSegmentStarTreeDocument(i, dimensionReaders, metricReaders); - documentBytesLength = writeStarTreeDocument(document, segmentDocsFileOutput, false); - } - } finally { - segmentDocsFileOutput.close(); - } - - // Create an iterator for aggregated documents - return sortAndReduceDocuments(sortedDocIds, totalSegmentDocs, documentBytesLength); - } - /** * Generates a star-tree for a given star-node * @@ -372,7 +369,7 @@ public Iterator generateStarTreeDocumentsForStarNode(int start sortedDocIds[i] = startDocId + i; } - sortDocuments( + sortDocumentsOffHeap( sortedDocIds, (index) -> Long.valueOf(starTreeDocumentOffsets.get(sortedDocIds[index])), dimensionId, @@ -424,7 +421,10 @@ public StarTreeDocument next() { }; } - private void sortDocuments( + /** + * Sort documents based on the dimension values off heap using intro sorter. + */ + private void sortDocumentsOffHeap( int[] sortedDocIds, Function offsetSupplier, int dimensionId, @@ -443,7 +443,6 @@ protected void swap(int i, int j) { @Override protected void setPivot(int i) { - long offset = offsetSupplier.apply(i); dimensions = new Long[starTreeField.getDimensionsOrder().size()]; try { @@ -464,16 +463,16 @@ protected int comparePivot(int j) { } for (int i = dimensionId + 1; i < dimensions.length; i++) { Long dimension = dimensionsFromOutput[i]; - if (dimensions[i] == null && dimension == null) { - return 0; - } - if (dimension == null) { - return -1; - } - if (dimensions[i] == null) { - return 1; - } if (!Objects.equals(dimensions[i], dimension)) { + if (dimensions[i] == null && dimension == null) { + return 0; + } + if (dimension == null) { + return -1; + } + if (dimensions[i] == null) { + return 1; + } return Long.compare(dimensions[i], dimension); } } @@ -488,6 +487,10 @@ protected int comparePivot(int j) { int writeStarTreeDocument(StarTreeDocument starTreeDocument, IndexOutput output, boolean isAggregatedDoc) throws IOException { int numBytes = writeDimensions(starTreeDocument, output); numBytes += writeMetrics(starTreeDocument, output, isAggregatedDoc); + if (docSizeInBytes == -1) { + docSizeInBytes = numBytes; + } + assert docSizeInBytes == numBytes; return numBytes; } @@ -547,11 +550,12 @@ private StarTreeDocument readStarTreeDocument(RandomAccessInput input, long offs throws IOException { int dimSize = starTreeField.getDimensionsOrder().size(); Long[] dimensions = new Long[dimSize]; + long initialOffset = offset; offset = readDimensions(dimensions, input, offset); Object[] metrics = new Object[numMetrics]; offset = readMetrics(input, offset, numMetrics, metrics, shouldReadAggregatedDocs); - + assert (offset - initialOffset) == docSizeInBytes; return new StarTreeDocument(dimensions, metrics); } @@ -620,13 +624,11 @@ private void ensureDocumentReadable(int docId) throws IOException { */ private void ensureDocumentReadable(int docId, boolean shouldCreateFileOutput) throws IOException { try { - if (docId >= prevStartDocId && docId < numReadableStarTreeDocuments) { + if (docId >= currentDocStartId && docId < numReadableStarTreeDocuments) { return; } - IOUtils.closeWhileHandlingException(starTreeDocsFileInput); starTreeDocsFileInput = null; - if (docId < numStarTreeDocs) { loadPreviousStarTreeDocumentFile(docId); } @@ -634,7 +636,6 @@ private void ensureDocumentReadable(int docId, boolean shouldCreateFileOutput) t return; } closeAndMaybeCreateNewFile(shouldCreateFileOutput); - int prevStartDocId = 0; for (Map.Entry fileToEndDocId : fileToEndDocIdMap.entrySet()) { if (docId <= fileToEndDocId.getValue() - 1) { @@ -643,7 +644,7 @@ private void ensureDocumentReadable(int docId, boolean shouldCreateFileOutput) t } prevStartDocId = fileToEndDocId.getValue(); } - this.prevStartDocId = prevStartDocId; + this.currentDocStartId = prevStartDocId; } catch (IOException ex) { IOUtils.close(this); throw ex; @@ -653,8 +654,8 @@ private void ensureDocumentReadable(int docId, boolean shouldCreateFileOutput) t /** * If docId is less then the numDocs , then we need to find a previous file associated with doc id * The fileToByteSizeMap is in the following format - * file1 -> 521 - * file2 -> 780 + * file1 == 521 + * file2 == 780 * which represents that file1 contains all docs till "520". *

* "prevStartDocId" tracks the "start doc id" of the range in the present 'star-tree.documents' file @@ -671,7 +672,7 @@ private void loadPreviousStarTreeDocumentFile(int docId) throws IOException { } prevStartDocId = entry.getValue(); } - this.prevStartDocId = prevStartDocId; + this.currentDocStartId = prevStartDocId; } /** @@ -702,14 +703,12 @@ private void closeAndMaybeCreateNewFile(boolean shouldCreateFileForAppend) throw if (starTreeDocsFileOutput != null) { fileToEndDocIdMap.put(starTreeDocsFileOutput.getName(), numStarTreeDocs); } - if (shouldCreateFileForAppend) { starTreeDocsFileOutput = createStarTreeDocumentsFileOutput(); if (fileToEndDocIdMap.size() >= fileCountMergeThreshold) { mergeFiles(); } } - if (starTreeDocsFileRandomInput != null) { starTreeDocsFileRandomInput = null; } @@ -753,11 +752,13 @@ private long mergeFilesToOutput(IndexOutput mergedOutput) throws IOException { /** * Delete the old startree.documents files */ - private void deleteOldFiles() throws IOException { + private void deleteOldFiles() { for (String fileName : fileToEndDocIdMap.keySet()) { try { tmpDirectory.deleteFile(fileName); - } catch (IOException ignored) {} + } catch (IOException ignored) { + logDeleteFileError(fileName); + } } } @@ -806,4 +807,8 @@ public void close() throws IOException { } super.close(); } + + private void logDeleteFileError(String file) { + logger.error("Error deleting file {}", file); + } } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilder.java index a744443417105..3b376d7c34351 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilder.java @@ -75,7 +75,7 @@ public void build(Map fieldProducerMap) throws IOExce // Build all star-trees for (StarTreeField starTreeField : starTreeFields) { - try (StarTreeBuilder starTreeBuilder = getSingleTreeBuilder(starTreeField, state, mapperService)) { + try (StarTreeBuilder starTreeBuilder = getStarTreeBuilder(starTreeField, state, mapperService)) { starTreeBuilder.build(fieldProducerMap); } } @@ -102,7 +102,7 @@ public void buildDuringMerge(final Map> starTreeVal continue; } StarTreeField starTreeField = starTreeValuesList.get(0).getStarTreeField(); - try (StarTreeBuilder builder = getSingleTreeBuilder(starTreeField, state, mapperService)) { + try (StarTreeBuilder builder = getStarTreeBuilder(starTreeField, state, mapperService)) { builder.build(starTreeValuesList); } } @@ -116,7 +116,7 @@ public void buildDuringMerge(final Map> starTreeVal /** * Get star-tree builder based on build mode. */ - StarTreeBuilder getSingleTreeBuilder(StarTreeField starTreeField, SegmentWriteState state, MapperService mapperService) + StarTreeBuilder getStarTreeBuilder(StarTreeField starTreeField, SegmentWriteState state, MapperService mapperService) throws IOException { switch (starTreeField.getStarTreeConfig().getBuildMode()) { case ON_HEAP: diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java index b0d1f3692c4bd..616a6e10509c8 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java @@ -1526,14 +1526,13 @@ public void testMergeFlowWithMissingDocsWithZero() throws IOException { [5, 5] | [5] [6, 6] | [6] [8, 8] | [8] - [null, null] | [12] [null, 7] | [7] + [null, null] | [12] */ int count = 0; while (starTreeDocumentIterator.hasNext()) { count++; StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); - System.out.println(starTreeDocument); if (starTreeDocument.dimensions[0] == null && starTreeDocument.dimensions[1] == null) { assertEquals(12L, (long) starTreeDocument.metrics[0]); } else if (starTreeDocument.dimensions[0] == null) { @@ -1547,6 +1546,71 @@ public void testMergeFlowWithMissingDocsWithZero() throws IOException { assertEquals(6, count); } + public void testMergeFlowWithMissingDocsWithZeroComplexCase() throws IOException { + List dimList = List.of(0L, 0L, 0L, 0L, 0L); + List docsWithField = List.of(0, 1, 2, 6, 8); + List dimList2 = List.of(0L, 0L, 0L, 0L); + List docsWithField2 = List.of(0, 1, 2, 6); + + List metricsList = List.of(0L, 1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L); + List metricsWithField = List.of(0, 1, 2, 3, 4, 5, 6, 7, 8, 9); + + List dimList3 = List.of(5L, 6L, 8L, -1L); + List docsWithField3 = List.of(0, 1, 3, 4); + List dimList4 = List.of(5L, 6L, 7L, 8L, -1L); + List docsWithField4 = List.of(0, 1, 2, 3, 4); + + List metricsList2 = List.of(5L, 6L, 7L, 8L, 9L); + List metricsWithField2 = List.of(0, 1, 2, 3, 4); + + StarTreeField sf = getStarTreeField(MetricStat.COUNT); + StarTreeValues starTreeValues = getStarTreeValues( + getSortedNumericMock(dimList, docsWithField), + getSortedNumericMock(dimList2, docsWithField2), + getSortedNumericMock(metricsList, metricsWithField), + sf, + "9" + ); + + StarTreeValues starTreeValues2 = getStarTreeValues( + getSortedNumericMock(dimList3, docsWithField3), + getSortedNumericMock(dimList4, docsWithField4), + getSortedNumericMock(metricsList2, metricsWithField2), + sf, + "4" + ); + builder = getStarTreeBuilder(sf, getWriteState(4), mapperService); + Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); + /** + * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] + [0, 0] | [9] + [0, null] | [8] + [5, 5] | [5] + [6, 6] | [6] + [8, 8] | [8] + [null, 7] | [7] + [null, null] | [19] + */ + int count = 0; + while (starTreeDocumentIterator.hasNext()) { + count++; + StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); + if (starTreeDocument.dimensions[0] == null && starTreeDocument.dimensions[1] == null) { + assertEquals(19L, (long) starTreeDocument.metrics[0]); + assertEquals(7, count); + } else if (starTreeDocument.dimensions[0] == null) { + assertEquals(7L, starTreeDocument.metrics[0]); + } else if (starTreeDocument.dimensions[1] == null) { + assertEquals(8L, starTreeDocument.metrics[0]); + } else if (starTreeDocument.dimensions[0] == 0) { + assertEquals(9L, starTreeDocument.metrics[0]); + } else { + assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); + } + } + assertEquals(7, count); + } + public void testMergeFlowWithMissingDocsInSecondDim() throws IOException { List dimList2 = List.of(0L, 1L, 2L, 3L, 4L, 6L); List docsWithField2 = List.of(0, 1, 2, 3, 4, 6); diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilderTests.java index 9e275e3898222..828bddfb8aa6e 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilderTests.java @@ -97,7 +97,7 @@ public void test_buildWithNoStarTreeFields() throws IOException { public void test_getStarTreeBuilder() throws IOException { when(mapperService.getCompositeFieldTypes()).thenReturn(Set.of(starTreeFieldType)); StarTreesBuilder starTreesBuilder = new StarTreesBuilder(segmentWriteState, mapperService); - StarTreeBuilder starTreeBuilder = starTreesBuilder.getSingleTreeBuilder(starTreeField, segmentWriteState, mapperService); + StarTreeBuilder starTreeBuilder = starTreesBuilder.getStarTreeBuilder(starTreeField, segmentWriteState, mapperService); assertTrue(starTreeBuilder instanceof OnHeapStarTreeBuilder); } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/SequentialDocValuesIteratorTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/SequentialDocValuesIteratorTests.java index dfc83125b2806..f56f7d9906ae1 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/SequentialDocValuesIteratorTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/SequentialDocValuesIteratorTests.java @@ -127,7 +127,5 @@ public void test_multipleCoordinatedDocumentReader() throws IOException { assertNotEquals(0, sequentialDocValuesIterator2.getDocId()); assertEquals(1, sequentialDocValuesIterator2.getDocId()); assertEquals(9L, (long) sequentialDocValuesIterator2.value(1)); - } - } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtilTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtilTests.java new file mode 100644 index 0000000000000..3c882fa964bd9 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtilTests.java @@ -0,0 +1,71 @@ +/* + * 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.store.FSDirectory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.RandomAccessInput; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.function.Function; + +/** + * Unit tests for {@link StarTreeDocumentBitSetUtil} + */ +public class StarTreeDocumentBitSetUtilTests extends OpenSearchTestCase { + + public void testWriteAndReadNullBitSets() throws IOException { + for (int k = 0; k < 10; k++) { + int randomArraySize = randomIntBetween(2, 256); + Long[] dims = new Long[randomArraySize]; + for (int i = 0; i < randomArraySize; i++) { + dims[i] = randomLong(); + } + testNullBasedOnBitset(dims); + } + } + + void testNullBasedOnBitset(Long[] dims) throws IOException { + Long[] dims1 = Arrays.copyOf(dims, dims.length); + int randomNullIndex1 = randomIntBetween(0, dims.length - 1); + int randomNullIndex2 = randomIntBetween(0, dims.length - 1); + dims[randomNullIndex1] = null; + dims[randomNullIndex2] = null; + Path basePath = createTempDir("OffHeapTests"); + FSDirectory fsDirectory = FSDirectory.open(basePath); + String TEST_FILE = "test_file"; + IndexOutput indexOutput = fsDirectory.createOutput(TEST_FILE, IOContext.DEFAULT); + StarTreeDocumentBitSetUtil.writeBitSet(dims, indexOutput); + indexOutput.close(); + + // test null value on read + IndexInput in = fsDirectory.openInput(TEST_FILE, IOContext.DEFAULT); + RandomAccessInput randomAccessInput = in.randomAccessSlice(0, in.length()); + StarTreeDocumentBitSetUtil.readAndSetNullBasedOnBitSet(randomAccessInput, 0, dims1); + assertNull(dims1[randomNullIndex1]); + assertNull(dims1[randomNullIndex2]); + in.close(); + + // test identity value on read + long randomLong = randomLong(); + Function identityValueSupplier = i -> randomLong; + in = fsDirectory.openInput(TEST_FILE, IOContext.DEFAULT); + + randomAccessInput = in.randomAccessSlice(0, in.length()); + StarTreeDocumentBitSetUtil.readAndSetIdentityValueBasedOnBitSet(randomAccessInput, 0, dims1, identityValueSupplier); + assertEquals(randomLong, (long) dims1[randomNullIndex1]); + assertEquals(randomLong, (long) dims1[randomNullIndex2]); + in.close(); + } +} From 14bd5e6313dbada9365f77716604d6e876a8f1a8 Mon Sep 17 00:00:00 2001 From: Bharathwaj G Date: Wed, 31 Jul 2024 23:13:05 +0530 Subject: [PATCH 4/7] Modularizing the code Signed-off-by: Bharathwaj G --- .../builder/AbstractDocumentsFileManager.java | 220 +++++++ .../builder/OffHeapStarTreeBuilder.java | 587 ++---------------- .../builder/SegmentDocsFileManager.java | 105 ++++ .../builder/StarTreeDocsFileManager.java | 294 +++++++++ .../startree/utils/ByteListBackedBitset.java | 63 ++ .../utils/StarTreeDocumentBitSetUtil.java | 66 +- .../utils/StarTreeDocumentsSorter.java | 66 ++ .../builder/OffHeapStarTreeBuilderTests.java | 51 +- .../StarTreeDocumentBitSetUtilTests.java | 7 +- .../utils/StarTreeDocumentsSorterTests.java | 101 +++ 10 files changed, 921 insertions(+), 639 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractDocumentsFileManager.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/SegmentDocsFileManager.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocsFileManager.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/ByteListBackedBitset.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorter.java create mode 100644 server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorterTests.java diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractDocumentsFileManager.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractDocumentsFileManager.java new file mode 100644 index 0000000000000..c662d90959f5c --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractDocumentsFileManager.java @@ -0,0 +1,220 @@ +/* + * 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.index.SegmentWriteState; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.RandomAccessInput; +import org.apache.lucene.store.TrackingDirectoryWrapper; +import org.apache.lucene.util.NumericUtils; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeDocument; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricAggregatorInfo; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericTypeConverters; +import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeDocumentBitSetUtil; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; + +/** + * Abstract class for managing star tree file operations. + * + * @opensearch.experimental + */ +@ExperimentalApi +public abstract class AbstractDocumentsFileManager implements Closeable { + private static final Logger logger = LogManager.getLogger(AbstractDocumentsFileManager.class); + protected final StarTreeField starTreeField; + protected final List metricAggregatorInfos; + protected final int numMetrics; + protected final TrackingDirectoryWrapper tmpDirectory; + protected final SegmentWriteState state; + protected int docSizeInBytes = -1; + + public AbstractDocumentsFileManager( + SegmentWriteState state, + StarTreeField starTreeField, + List metricAggregatorInfos + ) { + this.starTreeField = starTreeField; + this.tmpDirectory = new TrackingDirectoryWrapper(state.directory); + this.metricAggregatorInfos = metricAggregatorInfos; + this.state = state; + numMetrics = metricAggregatorInfos.size(); + } + + private void assertDocSizeInBytes(int numBytes) { + if (docSizeInBytes == -1) { + docSizeInBytes = numBytes; + } + assert docSizeInBytes == numBytes; + } + + /** + * Write the star tree document to file associated with dimensions and metrics + */ + protected int writeStarTreeDocument(StarTreeDocument starTreeDocument, IndexOutput output, boolean isAggregatedDoc) throws IOException { + int numBytes = writeDimensions(starTreeDocument, output); + numBytes += writeMetrics(starTreeDocument, output, isAggregatedDoc); + assertDocSizeInBytes(numBytes); + return numBytes; + } + + /** + * Write dimensions to file + */ + protected int writeDimensions(StarTreeDocument starTreeDocument, IndexOutput output) throws IOException { + int numBytes = 0; + for (int i = 0; i < starTreeDocument.dimensions.length; i++) { + output.writeLong(starTreeDocument.dimensions[i] == null ? 0L : starTreeDocument.dimensions[i]); + numBytes += Long.BYTES; + } + numBytes += StarTreeDocumentBitSetUtil.writeBitSet(starTreeDocument.dimensions, output); + return numBytes; + } + + /** + * Write star tree document metrics to file + */ + protected int writeMetrics(StarTreeDocument starTreeDocument, IndexOutput output, boolean isAggregatedDoc) throws IOException { + int numBytes = 0; + for (int i = 0; i < starTreeDocument.metrics.length; i++) { + switch (metricAggregatorInfos.get(i).getValueAggregators().getAggregatedValueType()) { + case LONG: + output.writeLong(starTreeDocument.metrics[i] == null ? 0L : (Long) starTreeDocument.metrics[i]); + numBytes += Long.BYTES; + break; + case DOUBLE: + if (isAggregatedDoc) { + long val = NumericUtils.doubleToSortableLong( + starTreeDocument.metrics[i] == null ? 0.0 : (Double) starTreeDocument.metrics[i] + ); + output.writeLong(val); + } else { + output.writeLong(starTreeDocument.metrics[i] == null ? 0L : (Long) starTreeDocument.metrics[i]); + } + numBytes += Long.BYTES; + break; + default: + throw new IllegalStateException("Unsupported metric type"); + } + } + numBytes += StarTreeDocumentBitSetUtil.writeBitSet(starTreeDocument.metrics, output); + return numBytes; + } + + /** + * Reads the star tree document from file with given offset + * + * @param input RandomAccessInput + * @param offset Offset in the file + * @param shouldReadAggregatedDocs boolean to indicate if aggregated star tree docs should be read + * @return StarTreeDocument + * @throws IOException IOException in case of I/O errors + */ + protected StarTreeDocument readStarTreeDocument(RandomAccessInput input, long offset, boolean shouldReadAggregatedDocs) + throws IOException { + int dimSize = starTreeField.getDimensionsOrder().size(); + Long[] dimensions = new Long[dimSize]; + long initialOffset = offset; + offset = readDimensions(dimensions, input, offset); + + Object[] metrics = new Object[numMetrics]; + offset = readMetrics(input, offset, numMetrics, metrics, shouldReadAggregatedDocs); + assert (offset - initialOffset) == docSizeInBytes; + return new StarTreeDocument(dimensions, metrics); + } + + /** + * Read dimensions from file + */ + protected long readDimensions(Long[] dimensions, RandomAccessInput input, long offset) throws IOException { + for (int i = 0; i < dimensions.length; i++) { + try { + dimensions[i] = input.readLong(offset); + } catch (Exception e) { + logger.error("Error reading dimension value at offset {} for dimension {}", offset, i); + throw e; + } + offset += Long.BYTES; + } + offset += StarTreeDocumentBitSetUtil.readBitSet(input, offset, dimensions, index -> null); + return offset; + } + + /** + * Read star tree metrics from file + */ + protected long readMetrics(RandomAccessInput input, long offset, int numMetrics, Object[] metrics, boolean shouldReadAggregatedDocs) + throws IOException { + for (int i = 0; i < numMetrics; i++) { + switch (metricAggregatorInfos.get(i).getValueAggregators().getAggregatedValueType()) { + case LONG: + metrics[i] = input.readLong(offset); + offset += Long.BYTES; + break; + case DOUBLE: + long val = input.readLong(offset); + if (shouldReadAggregatedDocs) { + metrics[i] = StarTreeNumericTypeConverters.sortableLongtoDouble(val); + } else { + metrics[i] = val; + } + offset += Long.BYTES; + break; + default: + throw new IllegalStateException("Unsupported metric type"); + } + } + offset += StarTreeDocumentBitSetUtil.readBitSet( + input, + offset, + metrics, + index -> metricAggregatorInfos.get(index).getValueAggregators().getIdentityMetricValue() + ); + return offset; + } + + /** + * Write star tree document to file + */ + public abstract void writeStarTreeDocument(StarTreeDocument starTreeDocument, boolean isAggregatedDoc) throws IOException; + + /** + * Read star tree document from file based on doc id + */ + public abstract StarTreeDocument readStarTreeDocument(int docId, boolean isMerge) throws IOException; + + /** + * Read star document dimensions from file based on doc id + */ + public abstract Long[] readDimensions(int docId) throws IOException; + + /** + * Read dimension value for given doc id and dimension id + */ + public abstract Long getDimensionValue(int docId, int dimensionId) throws IOException; + + /** + * Delete the temporary files created + */ + public void deleteFiles() { + for (String file : tmpDirectory.getCreatedFiles()) { + try { + tmpDirectory.deleteFile(file); + } catch (final IOException ignored) { + logger.error("Error deleting file {}", file); + } + } + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java index 2e9d6b7d449ed..30befce9520ad 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java @@ -12,21 +12,14 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.index.SegmentWriteState; import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.store.IndexInput; -import org.apache.lucene.store.IndexOutput; -import org.apache.lucene.store.RandomAccessInput; -import org.apache.lucene.store.TrackingDirectoryWrapper; -import org.apache.lucene.util.IntroSorter; -import org.apache.lucene.util.NumericUtils; import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.util.io.IOUtils; import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; import org.opensearch.index.compositeindex.datacube.Dimension; import org.opensearch.index.compositeindex.datacube.startree.StarTreeDocument; import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; -import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericTypeConverters; import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; -import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeDocumentBitSetUtil; +import org.opensearch.index.compositeindex.datacube.startree.utils.StarTreeDocumentsSorter; import org.opensearch.index.mapper.MapperService; import java.io.IOException; @@ -34,60 +27,20 @@ import java.util.Arrays; import java.util.Collections; import java.util.Iterator; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.function.Function; /** * Off-heap implementation of the star tree builder. - * - *

- * Segment documents are stored in a single file named 'segment.documents' for sorting and aggregation. A document ID array is created, - * and the document IDs in the array are swapped during sorting based on the actual segment document values in the file. - *

- * Star tree documents are stored in multiple 'star-tree.documents' files. The algorithm works as follows: - *

    - *
  1. Initially, aggregated documents are created based on the segment documents.
  2. - *
  3. Further, star tree documents are generated (e.g., in the {@code generateStarTreeDocumentsForStarNode} method) by reading the current - * aggregated documents and creating new aggregated star tree documents, which are appended to the 'star-tree.documents' files.
  4. - *
  5. This process is repeated until all combinations of star tree documents are generated.
  6. - *
- *

In cases where previously written star tree documents need to be read from the 'star-tree.documents' files, the current - * 'star-tree.documents' file is closed, and the values are read. Then, the derived values gets appended to a new 'star-tree.documents' file. - * This is necessary because Lucene maintains immutability of data, and an {@code IndexOutput} cannot be kept open while creating an - * {@code IndexInput} on the same file, as all file contents may not be visible in the reader. Therefore, the {@code IndexOutput} must be - * closed to ensure all data can be read before creating an {@code IndexInput}. Additionally, an {@code IndexOutput} cannot be reopened, - * so a new file is created for the new star tree documents. - *

The set of 'star-tree.documents' files is maintained, and a tracker array is used to keep track of the start document ID for each file. - * Once the number of files reaches a set threshold, the files are merged. - - @opensearch.experimental - **/ + * @opensearch.experimental + */ @ExperimentalApi public class OffHeapStarTreeBuilder extends BaseStarTreeBuilder { private static final Logger logger = LogManager.getLogger(OffHeapStarTreeBuilder.class); - private static final String SEGMENT_DOC_FILE_NAME = "segment.documents"; - private static final String STAR_TREE_DOC_FILE_NAME = "star-tree.documents"; - // TODO : Should this be via settings ? - private static final int DEFAULT_FILE_COUNT_MERGE_THRESHOLD = 5; - private final int fileCountMergeThreshold; - private final List starTreeDocumentOffsets; - private int numReadableStarTreeDocuments; - final IndexOutput segmentDocsFileOutput; - private IndexOutput starTreeDocsFileOutput; - private IndexInput starTreeDocsFileInput; - private IndexInput segmentDocsFileInput; - private RandomAccessInput segmentRandomInput; - private RandomAccessInput starTreeDocsFileRandomInput; - private final SegmentWriteState state; - private final LinkedHashMap fileToEndDocIdMap;// maintain order - private int starTreeFileCount = -1; - private int currentDocStartId = Integer.MAX_VALUE; - private int currBytes = 0; - private int docSizeInBytes = -1; - private final TrackingDirectoryWrapper tmpDirectory; + private final AbstractDocumentsFileManager starTreeDocumentFileManager; + private final AbstractDocumentsFileManager segmentDocumentFileManager; + private final StarTreeDocumentsSorter documentSorter; /** * Builds star tree based on star tree field configuration consisting of dimensions, metrics and star tree index @@ -98,49 +51,21 @@ public class OffHeapStarTreeBuilder extends BaseStarTreeBuilder { * @param mapperService helps to find the original type of the field */ protected OffHeapStarTreeBuilder(StarTreeField starTreeField, SegmentWriteState state, MapperService mapperService) throws IOException { - this(starTreeField, state, mapperService, DEFAULT_FILE_COUNT_MERGE_THRESHOLD); - } - - /** - * 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 state stores the segment write state - * @param mapperService helps to find the original type of the field - * @param fileThreshold threshold for number of files after which we merge the files - */ - protected OffHeapStarTreeBuilder(StarTreeField starTreeField, SegmentWriteState state, MapperService mapperService, int fileThreshold) - throws IOException { super(starTreeField, state, mapperService); - this.fileCountMergeThreshold = fileThreshold; - this.state = state; - this.tmpDirectory = new TrackingDirectoryWrapper(state.directory); - fileToEndDocIdMap = new LinkedHashMap<>(); + documentSorter = new StarTreeDocumentsSorter(); + segmentDocumentFileManager = new SegmentDocsFileManager(state, starTreeField, metricAggregatorInfos); try { - starTreeDocsFileOutput = createStarTreeDocumentsFileOutput(); - segmentDocsFileOutput = tmpDirectory.createTempOutput(SEGMENT_DOC_FILE_NAME, state.segmentSuffix, state.context); + starTreeDocumentFileManager = new StarTreeDocsFileManager(state, starTreeField, metricAggregatorInfos); } catch (IOException e) { - IOUtils.closeWhileHandlingException(starTreeDocsFileOutput); - IOUtils.close(this); + IOUtils.closeWhileHandlingException(segmentDocumentFileManager); throw e; } - starTreeDocumentOffsets = new ArrayList<>(); - } - /** - * Creates a new star tree document temporary file to store star tree documents. - */ - IndexOutput createStarTreeDocumentsFileOutput() throws IOException { - starTreeFileCount++; - return tmpDirectory.createTempOutput(STAR_TREE_DOC_FILE_NAME + starTreeFileCount, state.segmentSuffix, state.context); } @Override public void appendStarTreeDocument(StarTreeDocument starTreeDocument) throws IOException { - int bytes = writeStarTreeDocument(starTreeDocument, starTreeDocsFileOutput, true); - starTreeDocumentOffsets.add(currBytes); - currBytes += bytes; + starTreeDocumentFileManager.writeStarTreeDocument(starTreeDocument, true); } /** @@ -153,13 +78,8 @@ public void build(List starTreeValuesSubs) throws IOException { try { build(mergeStarTrees(starTreeValuesSubs)); } finally { - for (String file : tmpDirectory.getCreatedFiles()) { - try { - tmpDirectory.deleteFile(file); - } catch (final IOException ignored) { - logDeleteFileError(file); - } - } + starTreeDocumentFileManager.deleteFiles(); + segmentDocumentFileManager.deleteFiles(); } } @@ -177,7 +97,6 @@ public Iterator sortAndAggregateSegmentDocuments( ) throws IOException { // Write all dimensions for segment documents into the buffer, // and sort all documents using an int array - int documentBytesLength = 0; int[] sortedDocIds = new int[totalSegmentDocs]; for (int i = 0; i < totalSegmentDocs; i++) { sortedDocIds[i] = i; @@ -185,13 +104,14 @@ public Iterator sortAndAggregateSegmentDocuments( try { for (int i = 0; i < totalSegmentDocs; i++) { StarTreeDocument document = getSegmentStarTreeDocument(i, dimensionReaders, metricReaders); - documentBytesLength = writeStarTreeDocument(document, segmentDocsFileOutput, false); + segmentDocumentFileManager.writeStarTreeDocument(document, false); } - } finally { - segmentDocsFileOutput.close(); + } catch (IOException ex) { + segmentDocumentFileManager.close(); + throw ex; } // Create an iterator for aggregated documents - return sortAndReduceDocuments(sortedDocIds, totalSegmentDocs, documentBytesLength); + return sortAndReduceDocuments(sortedDocIds, totalSegmentDocs); } /** @@ -202,7 +122,6 @@ public Iterator sortAndAggregateSegmentDocuments( * @return iterator of star tree documents */ Iterator mergeStarTrees(List starTreeValuesSubs) throws IOException { - int docBytesLength = 0; int numDocs = 0; int[] docIds; try { @@ -225,9 +144,8 @@ Iterator mergeStarTrees(List starTreeValuesSub ); while (currentDocId < numSegmentDocs) { StarTreeDocument starTreeDocument = getStarTreeDocument(currentDocId, dimensionReaders, metricReaders); - int bytes = writeStarTreeDocument(starTreeDocument, segmentDocsFileOutput, true); + segmentDocumentFileManager.writeStarTreeDocument(starTreeDocument, true); numDocs++; - docBytesLength = bytes; currentDocId++; } } @@ -235,45 +153,48 @@ Iterator mergeStarTrees(List starTreeValuesSub for (int i = 0; i < numDocs; i++) { docIds[i] = i; } - } finally { - segmentDocsFileOutput.close(); + } catch (IOException ex) { + segmentDocumentFileManager.close(); + throw ex; } if (numDocs == 0) { return Collections.emptyIterator(); } - return sortAndReduceDocuments(docIds, numDocs, docBytesLength, true); + return sortAndReduceDocuments(docIds, numDocs, true); } /** * Sorts and reduces the star tree documents based on the dimensions during flush flow */ - private Iterator sortAndReduceDocuments(int[] docIds, int numDocs, int docBytesLength) throws IOException { - return sortAndReduceDocuments(docIds, numDocs, docBytesLength, false); + private Iterator sortAndReduceDocuments(int[] docIds, int numDocs) throws IOException { + return sortAndReduceDocuments(docIds, numDocs, false); } /** * Sorts and reduces the star tree documents based on the dimensions */ - private Iterator sortAndReduceDocuments(int[] sortedDocIds, int numDocs, int docBytesLength, boolean isMerge) - throws IOException { + private Iterator sortAndReduceDocuments(int[] sortedDocIds, int numDocs, boolean isMerge) throws IOException { try { - segmentDocsFileInput = tmpDirectory.openInput(segmentDocsFileOutput.getName(), state.context); - final long documentBytes = docBytesLength; - segmentRandomInput = segmentDocsFileInput.randomAccessSlice(0, segmentDocsFileInput.length()); if (sortedDocIds == null || sortedDocIds.length == 0) { logger.debug("Sorted doc ids array is null"); return Collections.emptyIterator(); } - sortDocumentsOffHeap(sortedDocIds, (index) -> (sortedDocIds[index] * documentBytes), -1, numDocs, segmentRandomInput); + final StarTreeDocument currentDocument = segmentDocumentFileManager.readStarTreeDocument(sortedDocIds[0], isMerge); + + documentSorter.sort(sortedDocIds, -1, numDocs, index -> { + try { + return segmentDocumentFileManager.readDimensions(sortedDocIds[index]); + } catch (IOException e) { + throw new RuntimeException("Sort failed : ", e); + } + }); // Create an iterator for aggregated documents - IndexInput finalSegmentDocsFileInput = segmentDocsFileInput; return new Iterator() { + StarTreeDocument tempCurrentDocument = currentDocument; boolean hasNext = true; - StarTreeDocument currentDocument = getSegmentStarTreeDocument(sortedDocIds[0], documentBytes, isMerge); - int docId = 1; @Override @@ -283,51 +204,42 @@ public boolean hasNext() { @Override public StarTreeDocument next() { - StarTreeDocument next = reduceSegmentStarTreeDocuments(null, currentDocument, isMerge); + StarTreeDocument next = reduceSegmentStarTreeDocuments(null, tempCurrentDocument, isMerge); while (docId < numDocs) { StarTreeDocument doc; try { - doc = getSegmentStarTreeDocument(sortedDocIds[docId++], documentBytes, isMerge); + doc = segmentDocumentFileManager.readStarTreeDocument(sortedDocIds[docId++], isMerge); } catch (IOException e) { throw new RuntimeException("Reducing documents failed ", e); } if (!Arrays.equals(doc.dimensions, next.dimensions)) { - currentDocument = doc; + tempCurrentDocument = doc; return next; } else { next = reduceSegmentStarTreeDocuments(next, doc, isMerge); } } hasNext = false; - IOUtils.closeWhileHandlingException(finalSegmentDocsFileInput); try { - tmpDirectory.deleteFile(segmentDocsFileOutput.getName()); - } catch (final IOException ignored) { - logDeleteFileError(segmentDocsFileOutput.getName()); + segmentDocumentFileManager.close(); + } catch (IOException ex) { + logger.error("Closing segment documents file failed", ex); } return next; } }; } catch (IOException ex) { - IOUtils.closeWhileHandlingException(segmentDocsFileInput); + segmentDocumentFileManager.close(); throw ex; } } - /** - * Get segment star tree document from the segment.documents file - */ - public StarTreeDocument getSegmentStarTreeDocument(int docID, long documentBytes, boolean isMerge) throws IOException { - return readStarTreeDocument(segmentRandomInput, docID * documentBytes, isMerge); - } - /** * Get star tree document for the given docId from the star-tree.documents file */ @Override public StarTreeDocument getStarTreeDocument(int docId) throws IOException { - ensureDocumentReadable(docId); - return readStarTreeDocument(starTreeDocsFileRandomInput, starTreeDocumentOffsets.get(docId), true); + return starTreeDocumentFileManager.readStarTreeDocument(docId, true); } // This should be only used for testing @@ -342,8 +254,7 @@ public List getStarTreeDocuments() throws IOException { @Override public Long getDimensionValue(int docId, int dimensionId) throws IOException { - ensureDocumentReadable(docId); - return starTreeDocsFileRandomInput.readLong((starTreeDocumentOffsets.get(docId) + ((long) dimensionId * Long.BYTES))); + return starTreeDocumentFileManager.getDimensionValue(docId, dimensionId); } /** @@ -358,25 +269,19 @@ public Long getDimensionValue(int docId, int dimensionId) throws IOException { @Override public Iterator generateStarTreeDocumentsForStarNode(int startDocId, int endDocId, int dimensionId) throws IOException { - // End doc id is not inclusive but start doc is inclusive - // Hence we need to check if buffer is readable till endDocId - 1 - ensureDocumentReadable(endDocId - 1); - // Sort all documents using an int array int numDocs = endDocId - startDocId; int[] sortedDocIds = new int[numDocs]; for (int i = 0; i < numDocs; i++) { sortedDocIds[i] = startDocId + i; } - - sortDocumentsOffHeap( - sortedDocIds, - (index) -> Long.valueOf(starTreeDocumentOffsets.get(sortedDocIds[index])), - dimensionId, - numDocs, - starTreeDocsFileRandomInput - ); - + documentSorter.sort(sortedDocIds, dimensionId, numDocs, index -> { + try { + return starTreeDocumentFileManager.readDimensions(sortedDocIds[index]); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); // Create an iterator for aggregated documents return new Iterator() { boolean hasNext = true; @@ -421,394 +326,10 @@ public StarTreeDocument next() { }; } - /** - * Sort documents based on the dimension values off heap using intro sorter. - */ - private void sortDocumentsOffHeap( - int[] sortedDocIds, - Function offsetSupplier, - int dimensionId, - int numDocs, - RandomAccessInput randomAccessInput - ) { - new IntroSorter() { - private Long[] dimensions; - - @Override - protected void swap(int i, int j) { - int temp = sortedDocIds[i]; - sortedDocIds[i] = sortedDocIds[j]; - sortedDocIds[j] = temp; - } - - @Override - protected void setPivot(int i) { - long offset = offsetSupplier.apply(i); - dimensions = new Long[starTreeField.getDimensionsOrder().size()]; - try { - readDimensions(dimensions, randomAccessInput, offset); - } catch (IOException e) { - throw new RuntimeException("Sort documents failed ", e); - } - } - - @Override - protected int comparePivot(int j) { - long offset = offsetSupplier.apply(j); - Long[] dimensionsFromOutput = new Long[starTreeField.getDimensionsOrder().size()]; - try { - readDimensions(dimensionsFromOutput, randomAccessInput, offset); - } catch (IOException e) { - throw new RuntimeException("Sort documents failed ", e); - } - for (int i = dimensionId + 1; i < dimensions.length; i++) { - Long dimension = dimensionsFromOutput[i]; - if (!Objects.equals(dimensions[i], dimension)) { - if (dimensions[i] == null && dimension == null) { - return 0; - } - if (dimension == null) { - return -1; - } - if (dimensions[i] == null) { - return 1; - } - return Long.compare(dimensions[i], dimension); - } - } - return 0; - } - }.sort(0, numDocs); - } - - /** - * Write the star tree document to file associated with dimensions and metrics - */ - int writeStarTreeDocument(StarTreeDocument starTreeDocument, IndexOutput output, boolean isAggregatedDoc) throws IOException { - int numBytes = writeDimensions(starTreeDocument, output); - numBytes += writeMetrics(starTreeDocument, output, isAggregatedDoc); - if (docSizeInBytes == -1) { - docSizeInBytes = numBytes; - } - assert docSizeInBytes == numBytes; - return numBytes; - } - - /** - * Write dimensions to file - */ - int writeDimensions(StarTreeDocument starTreeDocument, IndexOutput output) throws IOException { - int numBytes = 0; - for (int i = 0; i < starTreeDocument.dimensions.length; i++) { - output.writeLong(starTreeDocument.dimensions[i] == null ? 0L : starTreeDocument.dimensions[i]); - numBytes += Long.BYTES; - } - numBytes += StarTreeDocumentBitSetUtil.writeBitSet(starTreeDocument.dimensions, output); - return numBytes; - } - - /** - * Write star tree document metrics to file - */ - private int writeMetrics(StarTreeDocument starTreeDocument, IndexOutput output, boolean isAggregatedDoc) throws IOException { - int numBytes = 0; - for (int i = 0; i < starTreeDocument.metrics.length; i++) { - switch (metricAggregatorInfos.get(i).getValueAggregators().getAggregatedValueType()) { - case LONG: - output.writeLong(starTreeDocument.metrics[i] == null ? 0L : (Long) starTreeDocument.metrics[i]); - numBytes += Long.BYTES; - break; - case DOUBLE: - if (isAggregatedDoc) { - long val = NumericUtils.doubleToSortableLong( - starTreeDocument.metrics[i] == null ? 0.0 : (Double) starTreeDocument.metrics[i] - ); - output.writeLong(val); - } else { - output.writeLong(starTreeDocument.metrics[i] == null ? 0L : (Long) starTreeDocument.metrics[i]); - } - numBytes += Long.BYTES; - break; - default: - throw new IllegalStateException("Unsupported metric type"); - } - } - numBytes += StarTreeDocumentBitSetUtil.writeBitSet(starTreeDocument.metrics, output); - return numBytes; - } - - /** - * Reads the star tree document from file with given offset - * - * @param input RandomAccessInput - * @param offset Offset in the file - * @param shouldReadAggregatedDocs boolean to indicate if aggregated star tree docs should be read - * @return StarTreeDocument - * @throws IOException IOException in case of I/O errors - */ - private StarTreeDocument readStarTreeDocument(RandomAccessInput input, long offset, boolean shouldReadAggregatedDocs) - throws IOException { - int dimSize = starTreeField.getDimensionsOrder().size(); - Long[] dimensions = new Long[dimSize]; - long initialOffset = offset; - offset = readDimensions(dimensions, input, offset); - - Object[] metrics = new Object[numMetrics]; - offset = readMetrics(input, offset, numMetrics, metrics, shouldReadAggregatedDocs); - assert (offset - initialOffset) == docSizeInBytes; - return new StarTreeDocument(dimensions, metrics); - } - - /** - * Read dimensions from file - */ - long readDimensions(Long[] dimensions, RandomAccessInput input, long offset) throws IOException { - for (int i = 0; i < dimensions.length; i++) { - try { - dimensions[i] = input.readLong(offset); - } catch (Exception e) { - logger.error("Error reading dimension value at offset {} for dimension {}", offset, i); - throw e; - } - offset += Long.BYTES; - } - offset += StarTreeDocumentBitSetUtil.readAndSetNullBasedOnBitSet(input, offset, dimensions); - return offset; - } - - /** - * Read star tree metrics from file - */ - private long readMetrics(RandomAccessInput input, long offset, int numMetrics, Object[] metrics, boolean shouldReadAggregatedDocs) - throws IOException { - for (int i = 0; i < numMetrics; i++) { - switch (metricAggregatorInfos.get(i).getValueAggregators().getAggregatedValueType()) { - case LONG: - metrics[i] = input.readLong(offset); - offset += Long.BYTES; - break; - case DOUBLE: - long val = input.readLong(offset); - if (shouldReadAggregatedDocs) { - metrics[i] = StarTreeNumericTypeConverters.sortableLongtoDouble(val); - } else { - metrics[i] = val; - } - offset += Long.BYTES; - break; - default: - throw new IllegalStateException("Unsupported metric type"); - } - } - offset += StarTreeDocumentBitSetUtil.readAndSetIdentityValueBasedOnBitSet( - input, - offset, - metrics, - index -> metricAggregatorInfos.get(index).getValueAggregators().getIdentityMetricValue() - ); - return offset; - } - - /** - * Load the correct StarTreeDocuments file based on the docId - */ - private void ensureDocumentReadable(int docId) throws IOException { - ensureDocumentReadable(docId, true); - } - - /** - * Load the correct StarTreeDocuments file based on the docId - * - * @param docId requested doc id - * @param shouldCreateFileOutput this flag is used to indicate whether to create a new file output which is not needed during file format write operation - */ - private void ensureDocumentReadable(int docId, boolean shouldCreateFileOutput) throws IOException { - try { - if (docId >= currentDocStartId && docId < numReadableStarTreeDocuments) { - return; - } - IOUtils.closeWhileHandlingException(starTreeDocsFileInput); - starTreeDocsFileInput = null; - if (docId < numStarTreeDocs) { - loadPreviousStarTreeDocumentFile(docId); - } - if (starTreeDocsFileInput != null) { - return; - } - closeAndMaybeCreateNewFile(shouldCreateFileOutput); - int prevStartDocId = 0; - for (Map.Entry fileToEndDocId : fileToEndDocIdMap.entrySet()) { - if (docId <= fileToEndDocId.getValue() - 1) { - loadStarTreeDocumentFile(fileToEndDocId.getKey(), fileToEndDocId.getValue()); - break; - } - prevStartDocId = fileToEndDocId.getValue(); - } - this.currentDocStartId = prevStartDocId; - } catch (IOException ex) { - IOUtils.close(this); - throw ex; - } - } - - /** - * If docId is less then the numDocs , then we need to find a previous file associated with doc id - * The fileToByteSizeMap is in the following format - * file1 == 521 - * file2 == 780 - * which represents that file1 contains all docs till "520". - *

- * "prevStartDocId" tracks the "start doc id" of the range in the present 'star-tree.documents' file - * "numReadableStarTreeDocuments" tracks the "end doc id + 1" of the range of docs in the present file - *

- * IMPORTANT : This is case where the requested file is not the file which is being currently written to - */ - private void loadPreviousStarTreeDocumentFile(int docId) throws IOException { - int prevStartDocId = 0; - for (Map.Entry entry : fileToEndDocIdMap.entrySet()) { - if (docId < entry.getValue()) { - loadStarTreeDocumentFile(entry.getKey(), entry.getValue()); - break; - } - prevStartDocId = entry.getValue(); - } - this.currentDocStartId = prevStartDocId; - } - - /** - * Load the requested star-tree.documents file - */ - private void loadStarTreeDocumentFile(String fileName, int endDocId) throws IOException { - starTreeDocsFileInput = tmpDirectory.openInput(fileName, state.context); - starTreeDocsFileRandomInput = starTreeDocsFileInput.randomAccessSlice( - starTreeDocsFileInput.getFilePointer(), - starTreeDocsFileInput.length() - starTreeDocsFileInput.getFilePointer() - ); - numReadableStarTreeDocuments = endDocId; - } - - /** - * This case handles when the requested document ID is beyond the range of the currently open 'star-tree.documents' file. - * In this scenario, the following steps are taken: - * - * 1. Close the current 'star-tree.documents' file. - * 2. Create a new 'star-tree.documents' file if the operation involves appending new documents. - * If the operation is only for reading existing documents, a new file is not created. - */ - private void closeAndMaybeCreateNewFile(boolean shouldCreateFileForAppend) throws IOException { - if (starTreeDocsFileOutput != null) { - IOUtils.close(starTreeDocsFileOutput); - } - currBytes = 0; - if (starTreeDocsFileOutput != null) { - fileToEndDocIdMap.put(starTreeDocsFileOutput.getName(), numStarTreeDocs); - } - if (shouldCreateFileForAppend) { - starTreeDocsFileOutput = createStarTreeDocumentsFileOutput(); - if (fileToEndDocIdMap.size() >= fileCountMergeThreshold) { - mergeFiles(); - } - } - if (starTreeDocsFileRandomInput != null) { - starTreeDocsFileRandomInput = null; - } - } - - /** - * Merge temporary star tree files once the number of files reach threshold - */ - private void mergeFiles() throws IOException { - long st = System.currentTimeMillis(); - try (IndexOutput mergedOutput = createStarTreeDocumentsFileOutput()) { - long mergeBytes = mergeFilesToOutput(mergedOutput); - logger.debug( - "Created merge file : {} in : {} ms with size of : {} KB", - starTreeDocsFileOutput.getName(), - System.currentTimeMillis() - st, - mergeBytes / 1024 - ); - - deleteOldFiles(); - fileToEndDocIdMap.clear(); - fileToEndDocIdMap.put(mergedOutput.getName(), numStarTreeDocs); - resetStarTreeDocumentOffsets(); - } - } - - /** - * Merge all files to single IndexOutput - */ - private long mergeFilesToOutput(IndexOutput mergedOutput) throws IOException { - long mergeBytes = 0L; - for (Map.Entry entry : fileToEndDocIdMap.entrySet()) { - IndexInput input = tmpDirectory.openInput(entry.getKey(), state.context); - mergedOutput.copyBytes(input, input.length()); - mergeBytes += input.length(); - input.close(); - } - return mergeBytes; - } - - /** - * Delete the old startree.documents files - */ - private void deleteOldFiles() { - for (String fileName : fileToEndDocIdMap.keySet()) { - try { - tmpDirectory.deleteFile(fileName); - } catch (IOException ignored) { - logDeleteFileError(fileName); - } - } - } - - /** - * Reset the star tree document offsets based on the merged file - */ - private void resetStarTreeDocumentOffsets() { - int curr = 0; - for (int i = 0; i < starTreeDocumentOffsets.size(); i++) { - starTreeDocumentOffsets.set(i, curr); - curr += docSizeInBytes; - } - } - - /** - * Close the open segment files, star tree document files and associated data in/outputs. - * Delete all the temporary segment files and star tree document files - * - * @throws IOException IOException in case of I/O errors - */ @Override public void close() throws IOException { - try { - if (starTreeDocsFileOutput != null) { - IOUtils.closeWhileHandlingException(starTreeDocsFileOutput); - try { - tmpDirectory.deleteFile(starTreeDocsFileOutput.getName()); - } catch (IOException ignored) {} - } - } catch (Exception e) { - throw new RuntimeException(e); - } finally { - IOUtils.closeWhileHandlingException(starTreeDocsFileInput, segmentDocsFileInput, starTreeDocsFileOutput, segmentDocsFileOutput); - } - try { - if (this.segmentDocsFileOutput != null) { - // Delete all temporary segment document files - tmpDirectory.deleteFile(segmentDocsFileOutput.getName()); - } - } catch (IOException ignored) {} - // Delete all temporary star tree document files - for (String file : fileToEndDocIdMap.keySet()) { - try { - tmpDirectory.deleteFile(file); - } catch (IOException ignored) {} - } + starTreeDocumentFileManager.close(); + segmentDocumentFileManager.close(); super.close(); } - - private void logDeleteFileError(String file) { - logger.error("Error deleting file {}", file); - } } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/SegmentDocsFileManager.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/SegmentDocsFileManager.java new file mode 100644 index 0000000000000..7a3b7e4412f64 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/SegmentDocsFileManager.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.builder; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.RandomAccessInput; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.util.io.IOUtils; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeDocument; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricAggregatorInfo; + +import java.io.Closeable; +import java.io.IOException; +import java.util.List; + +/** + * Class for managing segment documents file. + * Segment documents are stored in a single file named 'segment.documents' for sorting and aggregation. A document ID array is created, + * and the document IDs in the array are swapped during sorting based on the actual segment document values in the file. + * + * @opensearch.experimental + */ +@ExperimentalApi +public class SegmentDocsFileManager extends AbstractDocumentsFileManager implements Closeable { + + private static final Logger logger = LogManager.getLogger(SegmentDocsFileManager.class); + private static final String SEGMENT_DOC_FILE_NAME = "segment.documents"; + private IndexInput segmentDocsFileInput; + private RandomAccessInput segmentRandomInput; + final IndexOutput segmentDocsFileOutput; + + public SegmentDocsFileManager(SegmentWriteState state, StarTreeField starTreeField, List metricAggregatorInfos) + throws IOException { + super(state, starTreeField, metricAggregatorInfos); + try { + segmentDocsFileOutput = tmpDirectory.createTempOutput(SEGMENT_DOC_FILE_NAME, state.segmentSuffix, state.context); + } catch (IOException e) { + IOUtils.close(this); + throw e; + } + } + + @Override + public void writeStarTreeDocument(StarTreeDocument starTreeDocument, boolean isAggregatedDoc) throws IOException { + writeStarTreeDocument(starTreeDocument, segmentDocsFileOutput, isAggregatedDoc); + } + + private void maybeInitializeSegmentInput() throws IOException { + try { + if (segmentDocsFileInput == null) { + IOUtils.closeWhileHandlingException(segmentDocsFileOutput); + segmentDocsFileInput = tmpDirectory.openInput(segmentDocsFileOutput.getName(), state.context); + segmentRandomInput = segmentDocsFileInput.randomAccessSlice(0, segmentDocsFileInput.length()); + } + } catch (IOException e) { + IOUtils.close(this); + throw e; + } + } + + @Override + public StarTreeDocument readStarTreeDocument(int docId, boolean isMerge) throws IOException { + maybeInitializeSegmentInput(); + return readStarTreeDocument(segmentRandomInput, (long) docId * docSizeInBytes, isMerge); + } + + @Override + public Long[] readDimensions(int docId) throws IOException { + maybeInitializeSegmentInput(); + Long[] dims = new Long[starTreeField.getDimensionsOrder().size()]; + readDimensions(dims, segmentRandomInput, (long) docId * docSizeInBytes); + return dims; + } + + @Override + public Long getDimensionValue(int docId, int dimensionId) throws IOException { + Long[] dims = readDimensions(docId); + return dims[dimensionId]; + } + + @Override + public void close() { + try { + if (this.segmentDocsFileOutput != null) { + IOUtils.closeWhileHandlingException(segmentDocsFileOutput); + tmpDirectory.deleteFile(segmentDocsFileOutput.getName()); + } + } catch (IOException ignored) {} catch (Exception e) { + throw new RuntimeException(e); + } finally { + IOUtils.closeWhileHandlingException(segmentDocsFileInput, segmentDocsFileOutput); + } + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocsFileManager.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocsFileManager.java new file mode 100644 index 0000000000000..9384baf0afb28 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocsFileManager.java @@ -0,0 +1,294 @@ +/* + * 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.index.SegmentWriteState; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.RandomAccessInput; +import org.opensearch.common.util.io.IOUtils; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeDocument; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricAggregatorInfo; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +/** + * Star tree document file manager. + * This class manages all the temporary files associated with off heap star tree builder. + *

+ * Star tree documents are stored in multiple 'star-tree.documents' files. The algorithm works as follows: + *

    + *
  1. Initially, aggregated documents are created based on the segment documents.
  2. + *
  3. Further, star tree documents are generated (e.g., in the {@code generateStarTreeDocumentsForStarNode} method) by reading the current + * aggregated documents and creating new aggregated star tree documents, which are appended to the 'star-tree.documents' files.
  4. + *
  5. This process is repeated until all combinations of star tree documents are generated.
  6. + *
+ *

In cases where previously written star tree documents need to be read from the 'star-tree.documents' files, the current + * 'star-tree.documents' file is closed, and the values are read. Then, the derived values gets appended to a new 'star-tree.documents' file. + * This is necessary because Lucene maintains immutability of data, and an {@code IndexOutput} cannot be kept open while creating an + * {@code IndexInput} on the same file, as all file contents may not be visible in the reader. Therefore, the {@code IndexOutput} must be + * closed to ensure all data can be read before creating an {@code IndexInput}. Additionally, an {@code IndexOutput} cannot be reopened, + * so a new file is created for the new star tree documents. + *

The set of 'star-tree.documents' files is maintained, and a tracker array is used to keep track of the start document ID for each file. + * Once the number of files reaches a set threshold, the files are merged. + * + */ +public class StarTreeDocsFileManager extends AbstractDocumentsFileManager implements Closeable { + private static final Logger logger = LogManager.getLogger(StarTreeDocsFileManager.class); + private static final String STAR_TREE_DOC_FILE_NAME = "star-tree.documents"; + public static final int DEFAULT_FILE_COUNT_MERGE_THRESHOLD = 5; + private IndexInput starTreeDocsFileInput; + private RandomAccessInput starTreeDocsFileRandomInput; + private IndexOutput starTreeDocsFileOutput; + private final Map fileToEndDocIdMap; + private final List starTreeDocumentOffsets = new ArrayList<>(); + private int currentFileStartDocId; + private int numReadableStarTreeDocuments; + private int starTreeFileCount = -1; + private int currBytes = 0; + private final int fileCountMergeThreshold; + private int numStarTreeDocs = 0; + + public StarTreeDocsFileManager(SegmentWriteState state, StarTreeField starTreeField, List metricAggregatorInfos) + throws IOException { + this(state, starTreeField, metricAggregatorInfos, DEFAULT_FILE_COUNT_MERGE_THRESHOLD); + } + + public StarTreeDocsFileManager( + SegmentWriteState state, + StarTreeField starTreeField, + List metricAggregatorInfos, + int fileCountThreshold + ) throws IOException { + super(state, starTreeField, metricAggregatorInfos); + fileToEndDocIdMap = new LinkedHashMap<>(); + try { + starTreeDocsFileOutput = createStarTreeDocumentsFileOutput(); + } catch (IOException e) { + IOUtils.closeWhileHandlingException(starTreeDocsFileOutput); + IOUtils.close(this); + throw e; + } + fileCountMergeThreshold = fileCountThreshold; + } + + /** + * Creates a new star tree document temporary file to store star tree documents. + */ + IndexOutput createStarTreeDocumentsFileOutput() throws IOException { + starTreeFileCount++; + return tmpDirectory.createTempOutput(STAR_TREE_DOC_FILE_NAME + starTreeFileCount, state.segmentSuffix, state.context); + } + + @Override + public void writeStarTreeDocument(StarTreeDocument starTreeDocument, boolean isAggregatedDoc) throws IOException { + assert isAggregatedDoc == true; + int numBytes = writeStarTreeDocument(starTreeDocument, starTreeDocsFileOutput, true); + addStarTreeDocumentOffset(numBytes); + numStarTreeDocs++; + } + + @Override + public StarTreeDocument readStarTreeDocument(int docId, boolean isAggregatedDoc) throws IOException { + assert isAggregatedDoc == true; + ensureDocumentReadable(docId); + return readStarTreeDocument(starTreeDocsFileRandomInput, starTreeDocumentOffsets.get(docId), true); + } + + @Override + public Long getDimensionValue(int docId, int dimensionId) throws IOException { + Long[] dims = readDimensions(docId); + return dims[dimensionId]; + } + + @Override + public Long[] readDimensions(int docId) throws IOException { + ensureDocumentReadable(docId); + Long[] dims = new Long[starTreeField.getDimensionsOrder().size()]; + readDimensions(dims, starTreeDocsFileRandomInput, starTreeDocumentOffsets.get(docId)); + return dims; + } + + private void addStarTreeDocumentOffset(int bytes) { + starTreeDocumentOffsets.add(currBytes); + currBytes += bytes; + if (docSizeInBytes == -1) { + docSizeInBytes = bytes; + } + assert docSizeInBytes == bytes; + } + + /** + * Load the correct StarTreeDocuments file based on the docId + */ + private void ensureDocumentReadable(int docId) throws IOException { + ensureDocumentReadable(docId, true); + } + + private void ensureDocumentReadable(int docId, boolean shouldCreateFileOutput) throws IOException { + try { + if (docId >= currentFileStartDocId && docId < numReadableStarTreeDocuments) { + return; + } + IOUtils.closeWhileHandlingException(starTreeDocsFileInput); + starTreeDocsFileInput = null; + if (docId < numStarTreeDocs) { + loadStarTreeDocumentFile(docId); + } + if (starTreeDocsFileInput != null) { + return; + } + closeAndMaybeCreateNewFile(shouldCreateFileOutput, numStarTreeDocs); + loadStarTreeDocumentFile(docId); + } catch (IOException ex) { + IOUtils.close(this); + throw ex; + } + } + + /** + * The fileToByteSizeMap is in the following format + * file1 == 521 + * file2 == 780 + * which represents that file1 contains all docs till "520". + *

+ * "currentFileStartDocId" tracks the "start doc id" of the range in the present 'star-tree.documents' file + * "numReadableStarTreeDocuments" tracks the "end doc id + 1" of the range of docs in the present file + */ + private void loadStarTreeDocumentFile(int docId) throws IOException { + int currentFileStartDocId = 0; + for (Map.Entry entry : fileToEndDocIdMap.entrySet()) { + if (docId < entry.getValue()) { + starTreeDocsFileInput = tmpDirectory.openInput(entry.getKey(), state.context); + starTreeDocsFileRandomInput = starTreeDocsFileInput.randomAccessSlice( + starTreeDocsFileInput.getFilePointer(), + starTreeDocsFileInput.length() - starTreeDocsFileInput.getFilePointer() + ); + numReadableStarTreeDocuments = entry.getValue(); + break; + } + currentFileStartDocId = entry.getValue(); + } + this.currentFileStartDocId = currentFileStartDocId; + } + + /** + * This case handles when the requested document ID is beyond the range of the currently open 'star-tree.documents' file. + * In this scenario, the following steps are taken: + *

+ * 1. Close the current 'star-tree.documents' file. + * 2. Create a new 'star-tree.documents' file if the operation involves appending new documents. + * If the operation is only for reading existing documents, a new file is not created. + */ + private void closeAndMaybeCreateNewFile(boolean shouldCreateFileForAppend, int numStarTreeDocs) throws IOException { + if (starTreeDocsFileOutput != null) { + IOUtils.close(starTreeDocsFileOutput); + } + currBytes = 0; + if (starTreeDocsFileOutput != null) { + fileToEndDocIdMap.put(starTreeDocsFileOutput.getName(), numStarTreeDocs); + } + if (shouldCreateFileForAppend) { + starTreeDocsFileOutput = createStarTreeDocumentsFileOutput(); + if (fileToEndDocIdMap.size() >= fileCountMergeThreshold) { + mergeFiles(numStarTreeDocs); + } + } + if (starTreeDocsFileRandomInput != null) { + starTreeDocsFileRandomInput = null; + } + } + + /** + * Merge temporary star tree files once the number of files reach threshold + */ + private void mergeFiles(int numStarTreeDocs) throws IOException { + long st = System.currentTimeMillis(); + try (IndexOutput mergedOutput = createStarTreeDocumentsFileOutput()) { + long mergeBytes = mergeFilesToOutput(mergedOutput); + logger.debug( + "Created merge file : {} in : {} ms with size of : {} KB", + starTreeDocsFileOutput.getName(), + System.currentTimeMillis() - st, + mergeBytes / 1024 + ); + + deleteOldFiles(); + fileToEndDocIdMap.clear(); + fileToEndDocIdMap.put(mergedOutput.getName(), numStarTreeDocs); + resetStarTreeDocumentOffsets(); + } + } + + /** + * Merge all files to single IndexOutput + */ + private long mergeFilesToOutput(IndexOutput mergedOutput) throws IOException { + long mergeBytes = 0L; + for (Map.Entry entry : fileToEndDocIdMap.entrySet()) { + IndexInput input = tmpDirectory.openInput(entry.getKey(), state.context); + mergedOutput.copyBytes(input, input.length()); + mergeBytes += input.length(); + input.close(); + } + return mergeBytes; + } + + /** + * Delete the old star-tree.documents files + */ + private void deleteOldFiles() { + for (String fileName : fileToEndDocIdMap.keySet()) { + try { + tmpDirectory.deleteFile(fileName); + } catch (IOException ignored) { + logger.error("Error deleting file {}", fileName); + } + } + } + + /** + * Reset the star tree document offsets based on the merged file + */ + private void resetStarTreeDocumentOffsets() { + int curr = 0; + for (int i = 0; i < starTreeDocumentOffsets.size(); i++) { + starTreeDocumentOffsets.set(i, curr); + curr += docSizeInBytes; + } + } + + @Override + public void close() { + try { + if (starTreeDocsFileOutput != null) { + IOUtils.closeWhileHandlingException(starTreeDocsFileOutput); + try { + tmpDirectory.deleteFile(starTreeDocsFileOutput.getName()); + } catch (IOException ignored) {} + } + } finally { + IOUtils.closeWhileHandlingException(starTreeDocsFileInput, starTreeDocsFileOutput); + } + // Delete all temporary star tree document files + for (String file : fileToEndDocIdMap.keySet()) { + try { + tmpDirectory.deleteFile(file); + } catch (IOException ignored) {} + } + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/ByteListBackedBitset.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/ByteListBackedBitset.java new file mode 100644 index 0000000000000..792832d751b56 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/ByteListBackedBitset.java @@ -0,0 +1,63 @@ +/* + * 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.store.IndexOutput; +import org.apache.lucene.store.RandomAccessInput; + +import java.io.IOException; + +/** + * This is used to store and retrieve null values in the bitset. + */ +public class ByteListBackedBitset { + private final byte[] byteArray; + + /** + * Constructor which uses an on heap list. This should be using during construction of the bitset. + */ + public ByteListBackedBitset(int capacity) { + byteArray = new byte[capacity]; + } + + /** + * Constructor which set the Lucene's IndexInput to read the bitset into a read-only buffer. + */ + public ByteListBackedBitset(RandomAccessInput in, long offset, int length) throws IOException { + byteArray = new byte[length]; + int i = 0; + while (i < length) { + byteArray[i] = in.readByte(offset + i); + i++; + } + } + + public void set(int index) { + int byteArrIndex = index >> 3; + byteArray[byteArrIndex] |= (byte) (1 << (index & 7)); + } + + public int write(IndexOutput output) throws IOException { + int numBytes = 0; + for (Byte bitSet : byteArray) { + output.writeByte(bitSet); + numBytes += Byte.BYTES; + } + return numBytes; + } + + public boolean get(int index) throws IOException { + int byteArrIndex = index >> 3; + return (byteArray[byteArrIndex] & (1 << (index & 7))) != 0; + } + + public int getCurrBytesRead() { + return byteArray.length; + } +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtil.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtil.java index 929a2cd9cea28..0efcd3a2c8cee 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtil.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtil.java @@ -12,8 +12,6 @@ import org.apache.lucene.store.RandomAccessInput; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; import java.util.function.Function; /** @@ -29,69 +27,31 @@ public class StarTreeDocumentBitSetUtil { * @throws IOException if an I/O error occurs while writing to the output stream */ public static int writeBitSet(Object[] array, IndexOutput output) throws IOException { - int numBytes = 0; - List nullBitSetList = new ArrayList<>(); - byte nullBitSet = 0; + int length = (array.length / 8) + (array.length % 8 == 0 ? 0 : 1); + ByteListBackedBitset bitset = new ByteListBackedBitset(length); for (int i = 0; i < array.length; i++) { - if (i % 8 == 0 && i > 0) { - nullBitSetList.add(nullBitSet); - nullBitSet = 0; - } if (array[i] == null) { - // Set the corresponding bit in dimensionNullBitSet to 1 (present) - nullBitSet |= (byte) (1 << (i % 8)); + bitset.set(i); } } - nullBitSetList.add(nullBitSet); - for (Byte bitSet : nullBitSetList) { - output.writeByte(bitSet); - numBytes += Byte.BYTES; - } - return numBytes; - } - - /** - * Set null values based on bitset. - */ - public static int readAndSetNullBasedOnBitSet(RandomAccessInput input, long offset, Object[] array) throws IOException { - int numBytes = 0; - byte nullDimensionsBitSet = input.readByte(offset + numBytes); - numBytes += Byte.BYTES; - for (int i = 0; i < array.length; i++) { - if (i > 0 && i % 8 == 0) { - nullDimensionsBitSet = input.readByte(offset + numBytes); - numBytes += Byte.BYTES; - } - boolean isElementNull = (nullDimensionsBitSet & (1L << (i % 8))) != 0; - if (isElementNull) { - array[i] = null; - } - } - return numBytes; + return bitset.write(output); } /** * Set identity values based on bitset. */ - public static int readAndSetIdentityValueBasedOnBitSet( - RandomAccessInput input, - long offset, - Object[] array, - Function identityValueSupplier - ) throws IOException { - int numBytes = 0; - byte nullDimensionsBitSet = input.readByte(offset + numBytes); - numBytes += Byte.BYTES; + public static int readBitSet(RandomAccessInput input, long offset, Object[] array, Function identityValueSupplier) + throws IOException { + ByteListBackedBitset bitset = new ByteListBackedBitset(input, offset, getLength(array)); for (int i = 0; i < array.length; i++) { - if (i > 0 && i % 8 == 0) { - nullDimensionsBitSet = input.readByte(offset + numBytes); - numBytes += Byte.BYTES; - } - boolean isElementNull = (nullDimensionsBitSet & (1L << (i % 8))) != 0; - if (isElementNull) { + if (bitset.get(i)) { array[i] = identityValueSupplier.apply(i); } } - return numBytes; + return bitset.getCurrBytesRead(); + } + + private static int getLength(Object[] array) { + return (array.length / 8) + (array.length % 8 == 0 ? 0 : 1); } } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorter.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorter.java new file mode 100644 index 0000000000000..18d908f2664e0 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorter.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.utils; + +import org.apache.lucene.util.IntroSorter; + +import java.util.Objects; +import java.util.function.IntFunction; + +/** + * Utility class for building star tree + */ +public class StarTreeDocumentsSorter { + /** + * Sort documents based on the dimension values off heap using intro sorter. + */ + public void sort( + final int[] sortedDocIds, + final int dimensionId, + final int numDocs, + final IntFunction dimensionsReader + ) { + new IntroSorter() { + private Long[] dimensions; + + @Override + protected void swap(int i, int j) { + int temp = sortedDocIds[i]; + sortedDocIds[i] = sortedDocIds[j]; + sortedDocIds[j] = temp; + } + + @Override + protected void setPivot(int i) { + dimensions = dimensionsReader.apply(i); + } + + @Override + protected int comparePivot(int j) { + Long[] currentDimensions = dimensionsReader.apply(j); + for (int i = dimensionId + 1; i < dimensions.length; i++) { + Long dimension = currentDimensions[i]; + if (!Objects.equals(dimensions[i], dimension)) { + if (dimensions[i] == null && dimension == null) { + return 0; + } + if (dimension == null) { + return -1; + } + if (dimensions[i] == null) { + return 1; + } + return Long.compare(dimensions[i], dimension); + } + } + return 0; + } + }.sort(0, numDocs); + } +} diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilderTests.java index e42b6b96706e7..92382b78f60c6 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilderTests.java @@ -9,17 +9,10 @@ package org.opensearch.index.compositeindex.datacube.startree.builder; import org.apache.lucene.index.SegmentWriteState; -import org.apache.lucene.store.FSDirectory; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.IndexInput; -import org.apache.lucene.store.IndexOutput; -import org.apache.lucene.store.RandomAccessInput; -import org.opensearch.index.compositeindex.datacube.startree.StarTreeDocument; import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; import org.opensearch.index.mapper.MapperService; import java.io.IOException; -import java.nio.file.Path; public class OffHeapStarTreeBuilderTests extends AbstractStarTreeBuilderTests { @Override @@ -28,48 +21,6 @@ public BaseStarTreeBuilder getStarTreeBuilder( SegmentWriteState segmentWriteState, MapperService mapperService ) throws IOException { - return new OffHeapStarTreeBuilder(starTreeField, segmentWriteState, mapperService, randomIntBetween(2, 6)); - } - - public void testDimensions() throws IOException { - for (int k = 0; k < 10; k++) { - int randomDimensionSize = randomIntBetween(2, 63); - Long[] dims = new Long[randomDimensionSize]; - for (int i = 0; i < randomDimensionSize; i++) { - dims[i] = randomLong(); - } - assertNullAndValuesInDims(dims); - } - } - - private void assertNullAndValuesInDims(Long[] dims) throws IOException { - int randomNullIndex1 = randomIntBetween(0, dims.length - 1); - int randomNullIndex2 = randomIntBetween(0, dims.length - 1); - dims[randomNullIndex1] = null; - dims[randomNullIndex2] = null; - Object[] metrics = new Object[64]; - StarTreeDocument doc = new StarTreeDocument(dims, metrics); - - Path basePath = createTempDir("OffHeapTests"); - FSDirectory fsDirectory = FSDirectory.open(basePath); - String TEST_FILE = "test_file"; - IndexOutput indexOutput = fsDirectory.createOutput(TEST_FILE, IOContext.DEFAULT); - OffHeapStarTreeBuilder builder = (OffHeapStarTreeBuilder) getStarTreeBuilder(compositeField, writeState, mapperService); - builder.writeDimensions(doc, indexOutput); - indexOutput.close(); - Long[] dims1 = new Long[dims.length]; - IndexInput in = fsDirectory.openInput(TEST_FILE, IOContext.DEFAULT); - RandomAccessInput randomAccessInput = in.randomAccessSlice(0, in.length()); - builder.readDimensions(dims1, randomAccessInput, 0); - for (int i = 0; i < dims.length; i++) { - if (i == randomNullIndex1 || i == randomNullIndex2) { - assertNull(dims1[i]); - } else { - assertEquals(dims[i], dims1[i]); - } - } - in.close(); - builder.close(); - fsDirectory.close(); + return new OffHeapStarTreeBuilder(starTreeField, segmentWriteState, mapperService); } } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtilTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtilTests.java index 3c882fa964bd9..7d1bd37246fae 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtilTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtilTests.java @@ -52,18 +52,19 @@ void testNullBasedOnBitset(Long[] dims) throws IOException { // test null value on read IndexInput in = fsDirectory.openInput(TEST_FILE, IOContext.DEFAULT); RandomAccessInput randomAccessInput = in.randomAccessSlice(0, in.length()); - StarTreeDocumentBitSetUtil.readAndSetNullBasedOnBitSet(randomAccessInput, 0, dims1); + Function identityValueSupplier = i -> null; + StarTreeDocumentBitSetUtil.readBitSet(randomAccessInput, 0, dims1, identityValueSupplier); assertNull(dims1[randomNullIndex1]); assertNull(dims1[randomNullIndex2]); in.close(); // test identity value on read long randomLong = randomLong(); - Function identityValueSupplier = i -> randomLong; + identityValueSupplier = i -> randomLong; in = fsDirectory.openInput(TEST_FILE, IOContext.DEFAULT); randomAccessInput = in.randomAccessSlice(0, in.length()); - StarTreeDocumentBitSetUtil.readAndSetIdentityValueBasedOnBitSet(randomAccessInput, 0, dims1, identityValueSupplier); + StarTreeDocumentBitSetUtil.readBitSet(randomAccessInput, 0, dims1, identityValueSupplier); assertEquals(randomLong, (long) dims1[randomNullIndex1]); assertEquals(randomLong, (long) dims1[randomNullIndex2]); in.close(); diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorterTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorterTests.java new file mode 100644 index 0000000000000..04e64d4f9df5b --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorterTests.java @@ -0,0 +1,101 @@ +/* + * 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.test.OpenSearchTestCase; +import org.junit.Before; + +import java.util.HashMap; +import java.util.Map; + +/** + * Tests for {@link StarTreeDocumentsSorter}. + */ +public class StarTreeDocumentsSorterTests extends OpenSearchTestCase { + private StarTreeDocumentsSorter sorter; + private Map testData; + + @Before + public void setUp() throws Exception { + super.setUp(); + sorter = new StarTreeDocumentsSorter(); + testData = new HashMap<>(); + testData.put(0, new Long[] { 1L, 2L, 3L }); + testData.put(1, new Long[] { 1L, 2L, 2L }); + testData.put(2, new Long[] { 1L, 1L, 3L }); + testData.put(3, new Long[] { 1L, 2L, null }); + testData.put(4, new Long[] { 1L, null, 3L }); + } + + public void testSortDocumentsOffHeap_FirstDimension() { + int[] sortedDocIds = { 0, 1, 2, 3, 4 }; + int dimensionId = -1; + int numDocs = 5; + + sorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(i)); + + assertArrayEquals(new int[] { 2, 1, 0, 3, 4 }, sortedDocIds); + } + + public void testSortDocumentsOffHeap_ThirdDimension() { + int[] sortedDocIds = { 0, 1, 2, 3, 4 }; + int dimensionId = 1; + int numDocs = 5; + + sorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(i)); + + assertArrayEquals(new int[] { 1, 0, 2, 4, 3 }, sortedDocIds); + } + + public void testSortDocumentsOffHeap_SingleElement() { + int[] sortedDocIds = { 0 }; + int dimensionId = -1; + int numDocs = 1; + + sorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(i)); + + assertArrayEquals(new int[] { 0 }, sortedDocIds); + } + + public void testSortDocumentsOffHeap_EmptyArray() { + int[] sortedDocIds = {}; + int dimensionId = -1; + int numDocs = 0; + + sorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(i)); + + assertArrayEquals(new int[] {}, sortedDocIds); + } + + public void testSortDocumentsOffHeap_SecondDimensionId() { + int[] sortedDocIds = { 0, 1, 2, 3, 4 }; + int dimensionId = 0; + int numDocs = 5; + + sorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(i)); + + assertArrayEquals(new int[] { 2, 1, 0, 3, 4 }, sortedDocIds); + } + + public void testSortDocumentsOffHeap_AllNulls() { + Map testData = new HashMap<>(); + testData.put(0, new Long[] { null, null, null }); + testData.put(1, new Long[] { null, null, null }); + testData.put(2, new Long[] { null, null, null }); + + int[] sortedDocIds = { 0, 1, 2 }; + int dimensionId = -1; + int numDocs = 3; + + sorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(i)); + + // The order should remain unchanged as all elements are equal (null) + assertArrayEquals(new int[] { 0, 1, 2 }, sortedDocIds); + } +} From 62d9e31b7b606115fbae2af7e8530fff533ae662 Mon Sep 17 00:00:00 2001 From: Bharathwaj G Date: Fri, 2 Aug 2024 12:19:26 +0530 Subject: [PATCH 5/7] Addressing comments and correcting close files Signed-off-by: Bharathwaj G --- .../composite/Composite99DocValuesWriter.java | 2 +- .../builder/AbstractDocumentsFileManager.java | 20 ++++++++++++---- .../startree/builder/BaseStarTreeBuilder.java | 2 -- .../builder/OffHeapStarTreeBuilder.java | 24 +++++++------------ .../builder/SegmentDocsFileManager.java | 8 +++---- .../builder/StarTreeDocsFileManager.java | 18 +++++++------- .../utils/StarTreeDocumentBitSetUtil.java | 3 +-- .../utils/StarTreeDocumentsSorter.java | 7 +----- 8 files changed, 39 insertions(+), 45 deletions(-) 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 b44ea2ae6cb52..6ed1a8c42e380 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 @@ -69,7 +69,7 @@ public Composite99DocValuesWriter(DocValuesConsumer delegate, SegmentWriteState compositeFieldSet.addAll(type.fields()); } // check if there are any composite fields which are part of the segment - segmentHasCompositeFields = !Collections.disjoint(segmentFieldSet, compositeFieldSet); + segmentHasCompositeFields = Collections.disjoint(segmentFieldSet, compositeFieldSet) == false; } @Override diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractDocumentsFileManager.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractDocumentsFileManager.java index c662d90959f5c..0f5601a68a59f 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractDocumentsFileManager.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractDocumentsFileManager.java @@ -208,13 +208,25 @@ protected long readMetrics(RandomAccessInput input, long offset, int numMetrics, /** * Delete the temporary files created */ - public void deleteFiles() { + public void deleteFiles(boolean success) throws IOException { + if (success) { + for (String file : tmpDirectory.getCreatedFiles()) { + tmpDirectory.deleteFile(file); + } + } else { + deleteFilesIgnoringException(); + } + + } + + /** + * Delete the temporary files created + */ + private void deleteFilesIgnoringException() throws IOException { for (String file : tmpDirectory.getCreatedFiles()) { try { tmpDirectory.deleteFile(file); - } catch (final IOException ignored) { - logger.error("Error deleting file {}", file); - } + } catch (final IOException ignored) {} // similar to IOUtils.deleteFilesWhileIgnoringExceptions } } } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java index 2a7d67bdc0022..e3743a1a36e72 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java @@ -73,7 +73,6 @@ public abstract class BaseStarTreeBuilder implements StarTreeBuilder { protected final TreeNode rootNode = getNewNode(); protected final StarTreeField starTreeField; - private final MapperService mapperService; private final SegmentWriteState state; static String NUM_SEGMENT_DOCS = "numSegmentDocs"; @@ -95,7 +94,6 @@ protected BaseStarTreeBuilder(StarTreeField starTreeField, SegmentWriteState sta this.skipStarNodeCreationForDimensions = new HashSet<>(); this.totalSegmentDocs = state.segmentInfo.maxDoc(); - this.mapperService = mapperService; this.state = state; Set skipStarNodeCreationForDimensions = starTreeFieldSpec.getSkipStarNodeCreationInDims(); diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java index 30befce9520ad..7a33bbab2d494 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java @@ -38,8 +38,8 @@ @ExperimentalApi public class OffHeapStarTreeBuilder extends BaseStarTreeBuilder { private static final Logger logger = LogManager.getLogger(OffHeapStarTreeBuilder.class); - private final AbstractDocumentsFileManager starTreeDocumentFileManager; - private final AbstractDocumentsFileManager segmentDocumentFileManager; + private final StarTreeDocsFileManager starTreeDocumentFileManager; + private final SegmentDocsFileManager segmentDocumentFileManager; private final StarTreeDocumentsSorter documentSorter; /** @@ -75,11 +75,13 @@ public void appendStarTreeDocument(StarTreeDocument starTreeDocument) throws IOE */ @Override public void build(List starTreeValuesSubs) throws IOException { + boolean success = false; try { build(mergeStarTrees(starTreeValuesSubs)); + success = true; } finally { - starTreeDocumentFileManager.deleteFiles(); - segmentDocumentFileManager.deleteFiles(); + starTreeDocumentFileManager.deleteFiles(success); + segmentDocumentFileManager.deleteFiles(success); } } @@ -111,7 +113,7 @@ public Iterator sortAndAggregateSegmentDocuments( throw ex; } // Create an iterator for aggregated documents - return sortAndReduceDocuments(sortedDocIds, totalSegmentDocs); + return sortAndReduceDocuments(sortedDocIds, totalSegmentDocs, false); } /** @@ -165,13 +167,6 @@ Iterator mergeStarTrees(List starTreeValuesSub return sortAndReduceDocuments(docIds, numDocs, true); } - /** - * Sorts and reduces the star tree documents based on the dimensions during flush flow - */ - private Iterator sortAndReduceDocuments(int[] docIds, int numDocs) throws IOException { - return sortAndReduceDocuments(docIds, numDocs, false); - } - /** * Sorts and reduces the star tree documents based on the dimensions */ @@ -229,7 +224,7 @@ public StarTreeDocument next() { } }; } catch (IOException ex) { - segmentDocumentFileManager.close(); + IOUtils.closeWhileHandlingException(segmentDocumentFileManager); throw ex; } } @@ -328,8 +323,7 @@ public StarTreeDocument next() { @Override public void close() throws IOException { - starTreeDocumentFileManager.close(); - segmentDocumentFileManager.close(); + IOUtils.closeWhileHandlingException(starTreeDocumentFileManager, segmentDocumentFileManager); super.close(); } } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/SegmentDocsFileManager.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/SegmentDocsFileManager.java index 7a3b7e4412f64..172314f7fb2f3 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/SegmentDocsFileManager.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/SegmentDocsFileManager.java @@ -46,7 +46,7 @@ public SegmentDocsFileManager(SegmentWriteState state, StarTreeField starTreeFie try { segmentDocsFileOutput = tmpDirectory.createTempOutput(SEGMENT_DOC_FILE_NAME, state.segmentSuffix, state.context); } catch (IOException e) { - IOUtils.close(this); + IOUtils.closeWhileHandlingException(this); throw e; } } @@ -64,7 +64,7 @@ private void maybeInitializeSegmentInput() throws IOException { segmentRandomInput = segmentDocsFileInput.randomAccessSlice(0, segmentDocsFileInput.length()); } } catch (IOException e) { - IOUtils.close(this); + IOUtils.closeWhileHandlingException(this); throw e; } } @@ -90,14 +90,12 @@ public Long getDimensionValue(int docId, int dimensionId) throws IOException { } @Override - public void close() { + public void close() throws IOException { try { if (this.segmentDocsFileOutput != null) { IOUtils.closeWhileHandlingException(segmentDocsFileOutput); tmpDirectory.deleteFile(segmentDocsFileOutput.getName()); } - } catch (IOException ignored) {} catch (Exception e) { - throw new RuntimeException(e); } finally { IOUtils.closeWhileHandlingException(segmentDocsFileInput, segmentDocsFileOutput); } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocsFileManager.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocsFileManager.java index 9384baf0afb28..83a93fc6aea95 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocsFileManager.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocsFileManager.java @@ -80,7 +80,7 @@ public StarTreeDocsFileManager( starTreeDocsFileOutput = createStarTreeDocumentsFileOutput(); } catch (IOException e) { IOUtils.closeWhileHandlingException(starTreeDocsFileOutput); - IOUtils.close(this); + IOUtils.closeWhileHandlingException(this); throw e; } fileCountMergeThreshold = fileCountThreshold; @@ -155,7 +155,7 @@ private void ensureDocumentReadable(int docId, boolean shouldCreateFileOutput) t closeAndMaybeCreateNewFile(shouldCreateFileOutput, numStarTreeDocs); loadStarTreeDocumentFile(docId); } catch (IOException ex) { - IOUtils.close(this); + IOUtils.closeWhileHandlingException(this); throw ex; } } @@ -251,13 +251,9 @@ private long mergeFilesToOutput(IndexOutput mergedOutput) throws IOException { /** * Delete the old star-tree.documents files */ - private void deleteOldFiles() { + private void deleteOldFiles() throws IOException { for (String fileName : fileToEndDocIdMap.keySet()) { - try { - tmpDirectory.deleteFile(fileName); - } catch (IOException ignored) { - logger.error("Error deleting file {}", fileName); - } + tmpDirectory.deleteFile(fileName); } } @@ -279,7 +275,7 @@ public void close() { IOUtils.closeWhileHandlingException(starTreeDocsFileOutput); try { tmpDirectory.deleteFile(starTreeDocsFileOutput.getName()); - } catch (IOException ignored) {} + } catch (IOException ignored) {} // similar to IOUtils.deleteFilesIgnoringExceptions } } finally { IOUtils.closeWhileHandlingException(starTreeDocsFileInput, starTreeDocsFileOutput); @@ -288,7 +284,9 @@ public void close() { for (String file : fileToEndDocIdMap.keySet()) { try { tmpDirectory.deleteFile(file); - } catch (IOException ignored) {} + } catch (IOException ignored) {} // similar to IOUtils.deleteFilesIgnoringExceptions } + starTreeDocumentOffsets.clear(); + fileToEndDocIdMap.clear(); } } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtil.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtil.java index 0efcd3a2c8cee..0a5c8028436a0 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtil.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtil.java @@ -27,8 +27,7 @@ public class StarTreeDocumentBitSetUtil { * @throws IOException if an I/O error occurs while writing to the output stream */ public static int writeBitSet(Object[] array, IndexOutput output) throws IOException { - int length = (array.length / 8) + (array.length % 8 == 0 ? 0 : 1); - ByteListBackedBitset bitset = new ByteListBackedBitset(length); + ByteListBackedBitset bitset = new ByteListBackedBitset(getLength(array)); for (int i = 0; i < array.length; i++) { if (array[i] == null) { bitset.set(i); diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorter.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorter.java index 18d908f2664e0..4ae315c5cd3a7 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorter.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorter.java @@ -20,12 +20,7 @@ public class StarTreeDocumentsSorter { /** * Sort documents based on the dimension values off heap using intro sorter. */ - public void sort( - final int[] sortedDocIds, - final int dimensionId, - final int numDocs, - final IntFunction dimensionsReader - ) { + public void sort(final int[] sortedDocIds, final int dimensionId, final int numDocs, final IntFunction dimensionsReader) { new IntroSorter() { private Long[] dimensions; From dccc30e8aea13591190943a91385fde4be99c28b Mon Sep 17 00:00:00 2001 From: Bharathwaj G Date: Fri, 2 Aug 2024 15:29:57 +0530 Subject: [PATCH 6/7] addressing review comments Signed-off-by: Bharathwaj G --- .../builder/AbstractDocumentsFileManager.java | 4 +- .../startree/builder/BaseStarTreeBuilder.java | 4 + .../builder/OffHeapStarTreeBuilder.java | 29 +- .../builder/StarTreeDocsFileManager.java | 14 +- .../utils/StarTreeDocumentsSorter.java | 7 +- .../datacube/startree/utils/TreeNode.java | 4 + .../builder/AbstractStarTreeBuilderTests.java | 287 +++++++++++++++++- .../utils/StarTreeDocumentsSorterTests.java | 122 +++++++- 8 files changed, 425 insertions(+), 46 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractDocumentsFileManager.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractDocumentsFileManager.java index 0f5601a68a59f..6a696ce2a2933 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractDocumentsFileManager.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractDocumentsFileManager.java @@ -53,7 +53,7 @@ public AbstractDocumentsFileManager( numMetrics = metricAggregatorInfos.size(); } - private void assertDocSizeInBytes(int numBytes) { + private void setDocSizeInBytes(int numBytes) { if (docSizeInBytes == -1) { docSizeInBytes = numBytes; } @@ -66,7 +66,7 @@ private void assertDocSizeInBytes(int numBytes) { protected int writeStarTreeDocument(StarTreeDocument starTreeDocument, IndexOutput output, boolean isAggregatedDoc) throws IOException { int numBytes = writeDimensions(starTreeDocument, output); numBytes += writeMetrics(starTreeDocument, output, isAggregatedDoc); - assertDocSizeInBytes(numBytes); + setDocSizeInBytes(numBytes); return numBytes; } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java index e3743a1a36e72..56bb46e83a9da 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java @@ -743,4 +743,8 @@ public void close() throws IOException { } abstract Iterator mergeStarTrees(List starTreeValues) throws IOException; + + public TreeNode getRootNode() { + return rootNode; + } } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java index 7a33bbab2d494..f63b0cb0cc77d 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OffHeapStarTreeBuilder.java @@ -23,6 +23,7 @@ import org.opensearch.index.mapper.MapperService; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -40,7 +41,6 @@ public class OffHeapStarTreeBuilder extends BaseStarTreeBuilder { private static final Logger logger = LogManager.getLogger(OffHeapStarTreeBuilder.class); private final StarTreeDocsFileManager starTreeDocumentFileManager; private final SegmentDocsFileManager segmentDocumentFileManager; - private final StarTreeDocumentsSorter documentSorter; /** * Builds star tree based on star tree field configuration consisting of dimensions, metrics and star tree index @@ -52,7 +52,6 @@ public class OffHeapStarTreeBuilder extends BaseStarTreeBuilder { */ protected OffHeapStarTreeBuilder(StarTreeField starTreeField, SegmentWriteState state, MapperService mapperService) throws IOException { super(starTreeField, state, mapperService); - documentSorter = new StarTreeDocumentsSorter(); segmentDocumentFileManager = new SegmentDocsFileManager(state, starTreeField, metricAggregatorInfos); try { starTreeDocumentFileManager = new StarTreeDocsFileManager(state, starTreeField, metricAggregatorInfos); @@ -176,16 +175,22 @@ private Iterator sortAndReduceDocuments(int[] sortedDocIds, in logger.debug("Sorted doc ids array is null"); return Collections.emptyIterator(); } - final StarTreeDocument currentDocument = segmentDocumentFileManager.readStarTreeDocument(sortedDocIds[0], isMerge); - - documentSorter.sort(sortedDocIds, -1, numDocs, index -> { - try { - return segmentDocumentFileManager.readDimensions(sortedDocIds[index]); - } catch (IOException e) { - throw new RuntimeException("Sort failed : ", e); + try { + StarTreeDocumentsSorter.sort(sortedDocIds, -1, numDocs, index -> { + try { + return segmentDocumentFileManager.readDimensions(sortedDocIds[index]); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }); + } catch (UncheckedIOException ex) { + // Unwrap UncheckedIOException and throw as IOException + if (ex.getCause() != null) { + throw ex.getCause(); } - }); - + throw ex; + } + final StarTreeDocument currentDocument = segmentDocumentFileManager.readStarTreeDocument(sortedDocIds[0], isMerge); // Create an iterator for aggregated documents return new Iterator() { StarTreeDocument tempCurrentDocument = currentDocument; @@ -270,7 +275,7 @@ public Iterator generateStarTreeDocumentsForStarNode(int start for (int i = 0; i < numDocs; i++) { sortedDocIds[i] = startDocId + i; } - documentSorter.sort(sortedDocIds, dimensionId, numDocs, index -> { + StarTreeDocumentsSorter.sort(sortedDocIds, dimensionId, numDocs, index -> { try { return starTreeDocumentFileManager.readDimensions(sortedDocIds[index]); } catch (IOException e) { diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocsFileManager.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocsFileManager.java index 83a93fc6aea95..3dec0b660cd48 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocsFileManager.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocsFileManager.java @@ -139,6 +139,11 @@ private void ensureDocumentReadable(int docId) throws IOException { ensureDocumentReadable(docId, true); } + /** + * Load the correct StarTreeDocuments file based on the docId + * "currentFileStartDocId" and "numReadableStarTreeDocuments" tracks the "start doc id" and "end doc id + 1" + * of the range in the current open 'star-tree.documents' file + */ private void ensureDocumentReadable(int docId, boolean shouldCreateFileOutput) throws IOException { try { if (docId >= currentFileStartDocId && docId < numReadableStarTreeDocuments) { @@ -162,12 +167,11 @@ private void ensureDocumentReadable(int docId, boolean shouldCreateFileOutput) t /** * The fileToByteSizeMap is in the following format - * file1 == 521 - * file2 == 780 - * which represents that file1 contains all docs till "520". + * file1 == 521 [ contains docs from 0 to 520 ] + * file2 == 780 [ contains docs from 521 to 779 ] *

- * "currentFileStartDocId" tracks the "start doc id" of the range in the present 'star-tree.documents' file - * "numReadableStarTreeDocuments" tracks the "end doc id + 1" of the range of docs in the present file + * This method loads the correct 'star-tree.documents' file based on the docId + * and updates the "currentFileStartDocId" and "numReadableStarTreeDocuments" */ private void loadStarTreeDocumentFile(int docId) throws IOException { int currentFileStartDocId = 0; diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorter.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorter.java index 4ae315c5cd3a7..7b1c63bc611ee 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorter.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorter.java @@ -20,7 +20,12 @@ public class StarTreeDocumentsSorter { /** * Sort documents based on the dimension values off heap using intro sorter. */ - public void sort(final int[] sortedDocIds, final int dimensionId, final int numDocs, final IntFunction dimensionsReader) { + public static void sort( + final int[] sortedDocIds, + final int dimensionId, + final int numDocs, + final IntFunction dimensionsReader + ) { new IntroSorter() { private Long[] dimensions; diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/TreeNode.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/TreeNode.java index 5cf737c61ab2d..a5d59a2602633 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/TreeNode.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/TreeNode.java @@ -62,4 +62,8 @@ public class TreeNode { * A map containing the child nodes of this star-tree node, keyed by their dimension id. */ public Map children; + + public long getDimensionValue() { + return dimensionValue; + } } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java index 616a6e10509c8..131d7444ff91c 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java @@ -55,6 +55,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Objects; @@ -355,19 +356,21 @@ public void test_sortAndAggregateStarTreeDocuments_nullMetricField() throws IOEx } } - public void test_sortAndAggregateStarTreeDocuments_nullDimensionField() throws IOException { + @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/14813") + public void test_sortAndAggregateStarTreeDocuments_nullAndMinusOneInDimensionField() throws IOException { int noOfStarTreeDocuments = 5; StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; // Setting second metric iterator as empty sorted numeric , indicating a metric field is null starTreeDocuments[0] = new StarTreeDocument(new Long[] { 2L, null, 3L, 4L }, new Double[] { 12.0, null, randomDouble() }); - starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 10.0, null, randomDouble() }); - starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 14.0, null, randomDouble() }); + starTreeDocuments[1] = new StarTreeDocument(new Long[] { null, 4L, 2L, 1L }, new Double[] { 10.0, null, randomDouble() }); + starTreeDocuments[2] = new StarTreeDocument(new Long[] { null, 4L, 2L, 1L }, new Double[] { 14.0, null, randomDouble() }); starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, null, 3L, 4L }, new Double[] { 9.0, null, randomDouble() }); - starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 11.0, null, randomDouble() }); + starTreeDocuments[4] = new StarTreeDocument(new Long[] { -1L, 4L, 2L, 1L }, new Double[] { 11.0, null, randomDouble() }); List inorderStarTreeDocuments = List.of( new StarTreeDocument(new Long[] { 2L, null, 3L, 4L }, new Object[] { 21.0, 0.0, 2L }), - new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Object[] { 35.0, 0.0, 3L }) + new StarTreeDocument(new Long[] { null, 4L, 2L, 1L }, new Object[] { 24.0, 0.0, 2L }), + new StarTreeDocument(new Long[] { -1L, 4L, 2L, 1L }, new Object[] { 11.0, 0.0, 1L }) ); Iterator expectedStarTreeDocumentIterator = inorderStarTreeDocuments.iterator(); @@ -388,8 +391,7 @@ public void test_sortAndAggregateStarTreeDocuments_nullDimensionField() throws I metricsIterators ); - while (segmentStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { - StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); + for (StarTreeDocument resultStarTreeDocument : builder.getStarTreeDocuments()) { StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); @@ -399,6 +401,8 @@ public void test_sortAndAggregateStarTreeDocuments_nullDimensionField() throws I assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); assertEquals(expectedStarTreeDocument.metrics[2], resultStarTreeDocument.metrics[2]); } + builder.build(segmentStarTreeDocumentIterator); + validateStarTree(builder.getRootNode(), 4, 1, builder.getStarTreeDocuments()); } public void test_sortAndAggregateStarTreeDocuments_nullDimensionsAndNullMetrics() throws IOException { @@ -448,6 +452,8 @@ public void test_sortAndAggregateStarTreeDocuments_nullDimensionsAndNullMetrics( assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); assertEquals(expectedStarTreeDocument.metrics[2], resultStarTreeDocument.metrics[2]); } + builder.build(segmentStarTreeDocumentIterator); + validateStarTree(builder.getRootNode(), 4, 1, builder.getStarTreeDocuments()); } public void test_sortAndAggregateStarTreeDocuments_emptyDimensions() throws IOException { @@ -597,6 +603,8 @@ public void test_sortAndAggregateStarTreeDocument_DoubleMaxAndDoubleMinMetrics() } assertEquals(inorderStarTreeDocuments.size(), numOfAggregatedDocuments); + builder.build(segmentStarTreeDocumentIterator); + validateStarTree(builder.getRootNode(), 3, 1, builder.getStarTreeDocuments()); } @@ -673,6 +681,7 @@ public void test_build_halfFloatMetrics() throws IOException { Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator(); assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); + builder.build(expectedStarTreeDocumentIterator); } public void test_build_floatMetrics() throws IOException { @@ -977,6 +986,7 @@ public void test_build_starTreeDataset() throws IOException { assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); } + validateStarTree(builder.getRootNode(), 3, 1, builder.getStarTreeDocuments()); } private static Map> getExpectedDimToValueMap() { @@ -1083,6 +1093,62 @@ public void testFlushFlow() throws IOException { assertEquals(1L, starTreeDocument.metrics[1]); } assertEquals(6, count); + builder.build(starTreeDocumentIterator); + validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); + } + + public void testFlushFlowDimsReverse() throws IOException { + List dimList = List.of(5L, 4L, 3L, 2L, 1L); + List docsWithField = List.of(0, 1, 2, 3, 4); + List dimList2 = List.of(5L, 4L, 3L, 2L, 1L, 0L); + List docsWithField2 = List.of(0, 1, 2, 3, 4, 5); + + List metricsList = List.of( + getLongFromDouble(50.0), + getLongFromDouble(40.0), + getLongFromDouble(30.0), + getLongFromDouble(20.0), + getLongFromDouble(10.0), + getLongFromDouble(0.0) + ); + List metricsWithField = List.of(0, 1, 2, 3, 4, 5); + + StarTreeField sf = getStarTreeFieldWithMultipleMetrics(); + SortedNumericDocValues d1sndv = getSortedNumericMock(dimList, docsWithField); + SortedNumericDocValues d2sndv = getSortedNumericMock(dimList2, docsWithField2); + SortedNumericDocValues m1sndv = getSortedNumericMock(metricsList, metricsWithField); + SortedNumericDocValues m2sndv = getSortedNumericMock(metricsList, metricsWithField); + + builder = getStarTreeBuilder(sf, getWriteState(6), mapperService); + SequentialDocValuesIterator[] dimDvs = { new SequentialDocValuesIterator(d1sndv), new SequentialDocValuesIterator(d2sndv) }; + Iterator starTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( + dimDvs, + List.of(new SequentialDocValuesIterator(m1sndv), new SequentialDocValuesIterator(m2sndv)) + ); + /** + * Asserting following dim / metrics [ dim1, dim2 / Sum [metric], count [metric] ] + [1, 1] | [10.0, 1] + [2, 2] | [20.0, 1] + [3, 3] | [30.0, 1] + [4, 4] | [40.0, 1] + [5, 5] | [50.0, 1] + [null, 0] | [0.0, 1] + */ + int count = 0; + while (starTreeDocumentIterator.hasNext()) { + count++; + StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); + if (starTreeDocument.dimensions[0] != null) { + assertEquals(count, (long) starTreeDocument.dimensions[0]); + } else { + assertEquals(6, count); + } + assertEquals(starTreeDocument.dimensions[1] * 10.0, starTreeDocument.metrics[0]); + assertEquals(1L, starTreeDocument.metrics[1]); + } + assertEquals(6, count); + builder.build(starTreeDocumentIterator); + validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); } public void testFlushFlowBuild() throws IOException { @@ -1149,7 +1215,7 @@ public void testFlushFlowBuild() throws IOException { starTreeDocument.metrics[0] ); } - builder.close(); + validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); } private static DocValuesProducer getDocValuesProducer(SortedNumericDocValues sndv) { @@ -1234,6 +1300,8 @@ public void testMergeFlowWithSum() throws IOException { ); } assertEquals(6, count); + builder.build(starTreeDocumentIterator); + validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); } public void testMergeFlowWithCount() throws IOException { @@ -1281,6 +1349,9 @@ public void testMergeFlowWithCount() throws IOException { assertEquals(starTreeDocument.dimensions[0] != null ? starTreeDocument.dimensions[0] * 2 : 4, starTreeDocument.metrics[0]); } assertEquals(6, count); + builder.build(starTreeDocumentIterator); + validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); + } private StarTreeValues getStarTreeValues( @@ -1363,6 +1434,8 @@ public void testMergeFlowWithDifferentDocsFromSegments() throws IOException { } } assertEquals(9, count); + builder.build(starTreeDocumentIterator); + validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); } public void testMergeFlowNumSegmentsDocs() throws IOException { @@ -1483,6 +1556,8 @@ public void testMergeFlowWithMissingDocs() throws IOException { assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); } assertEquals(10, count); + builder.build(starTreeDocumentIterator); + validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); } public void testMergeFlowWithMissingDocsWithZero() throws IOException { @@ -1544,6 +1619,8 @@ public void testMergeFlowWithMissingDocsWithZero() throws IOException { } } assertEquals(6, count); + builder.build(starTreeDocumentIterator); + validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); } public void testMergeFlowWithMissingDocsWithZeroComplexCase() throws IOException { @@ -1609,6 +1686,8 @@ public void testMergeFlowWithMissingDocsWithZeroComplexCase() throws IOException } } assertEquals(7, count); + builder.build(starTreeDocumentIterator); + validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); } public void testMergeFlowWithMissingDocsInSecondDim() throws IOException { @@ -1670,6 +1749,8 @@ public void testMergeFlowWithMissingDocsInSecondDim() throws IOException { } } assertEquals(10, count); + builder.build(starTreeDocumentIterator); + validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); } public void testMergeFlowWithDocsMissingAtTheEnd() throws IOException { @@ -1730,6 +1811,8 @@ public void testMergeFlowWithDocsMissingAtTheEnd() throws IOException { assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); } assertEquals(10, count); + builder.build(starTreeDocumentIterator); + validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); } public void testMergeFlowWithEmptyFieldsInOneSegment() throws IOException { @@ -1778,6 +1861,8 @@ public void testMergeFlowWithEmptyFieldsInOneSegment() throws IOException { assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); } assertEquals(6, count); + builder.build(starTreeDocumentIterator); + validateStarTree(builder.getRootNode(), 2, 1, builder.getStarTreeDocuments()); } public void testMergeFlowWithDuplicateDimensionValues() throws IOException { @@ -1881,7 +1966,7 @@ public void testMergeFlowWithDuplicateDimensionValues() throws IOException { count++; } assertEquals(401, count); - builder.close(); + validateStarTree(builder.getRootNode(), 4, sf.getStarTreeConfig().maxLeafDocs(), builder.getStarTreeDocuments()); } public void testMergeFlowWithMaxLeafDocs() throws IOException { @@ -1978,7 +2063,7 @@ public void testMergeFlowWithMaxLeafDocs() throws IOException { [null, null, null, null] | [2495000.0] */ assertEquals(635, starTreeDocuments.size()); - builder.close(); + validateStarTree(builder.getRootNode(), 4, sf.getStarTreeConfig().maxLeafDocs(), builder.getStarTreeDocuments()); } private StarTreeValues getStarTreeValues( @@ -2084,7 +2169,7 @@ public void testMergeFlowWithDuplicateDimensionValueWithMaxLeafDocs() throws IOE builder.build(builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2))); List starTreeDocuments = builder.getStarTreeDocuments(); assertEquals(401, starTreeDocuments.size()); - builder.close(); + validateStarTree(builder.getRootNode(), 4, sf.getStarTreeConfig().maxLeafDocs(), builder.getStarTreeDocuments()); } public static long getLongFromDouble(double value) { @@ -2195,7 +2280,7 @@ public void testMergeFlowWithMaxLeafDocsAndStarTreeNodesAssertion() throws IOExc } } assertEquals(1041, starTreeDocuments.size()); - builder.close(); + validateStarTree(builder.getRootNode(), 4, sf.getStarTreeConfig().maxLeafDocs(), builder.getStarTreeDocuments()); } private static StarTreeField getStarTreeField(int maxLeafDocs) { @@ -2351,11 +2436,183 @@ public void testMergeFlow() throws IOException { ... [999, 999, 999, 999] | [19980.0] */ - while (starTreeDocumentIterator.hasNext()) { - StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); + for (StarTreeDocument starTreeDocument : builder.getStarTreeDocuments()) { assertEquals(starTreeDocument.dimensions[0] * 20.0, starTreeDocument.metrics[0]); } - builder.close(); + builder.build(starTreeDocumentIterator); + + // Validate the star tree structure + validateStarTree(builder.getRootNode(), 4, 1, builder.getStarTreeDocuments()); + } + + private void validateStarTree(TreeNode root, int totalDimensions, int maxLeafDocuments, List starTreeDocuments) { + Queue queue = new LinkedList<>(); + queue.offer(new Object[] { root, false }); + while (!queue.isEmpty()) { + Object[] current = queue.poll(); + TreeNode node = (TreeNode) current[0]; + boolean currentIsStarNode = (boolean) current[1]; + + assertNotNull(node); + + // assert dimensions + if (node.dimensionId != TreeNode.ALL) { + assertTrue(node.dimensionId >= 0 && node.dimensionId < totalDimensions); + } + if (node.children != null && !node.children.isEmpty()) { + assertEquals(node.dimensionId + 1, node.childDimensionId); + assertTrue(node.childDimensionId < totalDimensions); + TreeNode starNode = null; + Object[] nonStarNodeCumulativeMetrics = getMetrics(starTreeDocuments); + for (Map.Entry entry : node.children.entrySet()) { + Long childDimensionValue = entry.getKey(); + TreeNode child = entry.getValue(); + Object[] currMetrics = getMetrics(starTreeDocuments); + if (!child.isStarNode) { + // Validate dimension values in documents + for (int i = child.startDocId; i < child.endDocId; i++) { + StarTreeDocument doc = starTreeDocuments.get(i); + int j = 0; + addMetrics(doc, currMetrics, j); + if (!child.isStarNode) { + Long dimension = doc.dimensions[child.dimensionId]; + assertEquals(childDimensionValue, dimension); + if (dimension != null) { + assertEquals(child.dimensionValue, (long) dimension); + } else { + // TODO : fix this ? + assertEquals(child.dimensionValue, TreeNode.ALL); + } + } + } + Object[] aggregatedMetrics = starTreeDocuments.get(child.aggregatedDocId).metrics; + int j = 0; + for (Object metric : currMetrics) { + /* + * TODO : refactor this to handle any data type + */ + if (metric instanceof Double) { + nonStarNodeCumulativeMetrics[j] = (double) nonStarNodeCumulativeMetrics[j] + (double) metric; + assertEquals((Double) metric, (Double) aggregatedMetrics[j], 0); + } else if (metric instanceof Long) { + nonStarNodeCumulativeMetrics[j] = (long) nonStarNodeCumulativeMetrics[j] + (long) metric; + assertEquals((long) metric, (long) aggregatedMetrics[j]); + } else if (metric instanceof Float) { + nonStarNodeCumulativeMetrics[j] = (float) nonStarNodeCumulativeMetrics[j] + (float) metric; + assertEquals((float) metric, (float) aggregatedMetrics[j], 0); + } + j++; + } + queue.offer(new Object[] { child, false }); + } else { + starNode = child; + } + } + // Add star node to queue + if (starNode != null) { + Object[] starNodeMetrics = getMetrics(starTreeDocuments); + for (int i = starNode.startDocId; i < starNode.endDocId; i++) { + StarTreeDocument doc = starTreeDocuments.get(i); + int j = 0; + addMetrics(doc, starNodeMetrics, j); + } + int j = 0; + Object[] aggregatedMetrics = starTreeDocuments.get(starNode.aggregatedDocId).metrics; + for (Object nonStarNodeCumulativeMetric : nonStarNodeCumulativeMetrics) { + assertEquals(nonStarNodeCumulativeMetric, starNodeMetrics[j]); + assertEquals(starNodeMetrics[j], aggregatedMetrics[j]); + /* + * TODO : refactor this to handle any data type + */ + if (nonStarNodeCumulativeMetric instanceof Double) { + assertEquals((double) nonStarNodeCumulativeMetric, (double) starNodeMetrics[j], 0); + assertEquals((double) nonStarNodeCumulativeMetric, (double) aggregatedMetrics[j], 0); + } else if (nonStarNodeCumulativeMetric instanceof Long) { + assertEquals((long) nonStarNodeCumulativeMetric, (long) starNodeMetrics[j]); + assertEquals((long) nonStarNodeCumulativeMetric, (long) aggregatedMetrics[j]); + } else if (nonStarNodeCumulativeMetric instanceof Float) { + assertEquals((float) nonStarNodeCumulativeMetric, (float) starNodeMetrics[j], 0); + assertEquals((float) nonStarNodeCumulativeMetric, (float) aggregatedMetrics[j], 0); + } + + j++; + } + assertEquals(-1L, starNode.dimensionValue); + queue.offer(new Object[] { starNode, true }); + } + } else { + assertTrue(node.endDocId - node.startDocId <= maxLeafDocuments); + } + + if (currentIsStarNode) { + StarTreeDocument prevDoc = null; + int docCount = 0; + int docId = node.startDocId; + int dimensionId = node.dimensionId; + + while (docId < node.endDocId) { + StarTreeDocument currentDoc = starTreeDocuments.get(docId); + docCount++; + + // Verify that the dimension at 'dimensionId' is set to STAR_IN_DOC_VALUES_INDEX + assertNull(currentDoc.dimensions[dimensionId]); + + // Verify sorting of documents + if (prevDoc != null) { + assertTrue(compareDocuments(prevDoc, currentDoc, dimensionId + 1, totalDimensions) <= 0); + } + prevDoc = currentDoc; + docId++; + } + + // Verify that the number of generated star documents matches the range in the star node + assertEquals(node.endDocId - node.startDocId, docCount); + } + } + } + + /** + * TODO : refactor this to handle any data type + */ + private static void addMetrics(StarTreeDocument doc, Object[] currMetrics, int j) { + for (Object metric : doc.metrics) { + if (metric instanceof Double) { + currMetrics[j] = (double) currMetrics[j] + (double) metric; + } else if (metric instanceof Long) { + currMetrics[j] = (long) currMetrics[j] + (long) metric; + } else if (metric instanceof Float) { + currMetrics[j] = (float) currMetrics[j] + (float) metric; + } + j++; + } + } + + private static Object[] getMetrics(List starTreeDocuments) { + Object[] nonStarNodeCumulativeMetrics = new Object[starTreeDocuments.get(0).metrics.length]; + for (int i = 0; i < nonStarNodeCumulativeMetrics.length; i++) { + if (starTreeDocuments.get(0).metrics[i] instanceof Long) { + nonStarNodeCumulativeMetrics[i] = 0L; + } else if (starTreeDocuments.get(0).metrics[i] instanceof Double) { + nonStarNodeCumulativeMetrics[i] = 0.0; + } else if (starTreeDocuments.get(0).metrics[i] instanceof Float) { + nonStarNodeCumulativeMetrics[i] = 0.0f; + } + } + return nonStarNodeCumulativeMetrics; + } + + private int compareDocuments(StarTreeDocument doc1, StarTreeDocument doc2, int startDim, int endDim) { + for (int i = startDim; i < endDim; i++) { + Long val1 = doc1.dimensions[i]; + Long val2 = doc2.dimensions[i]; + + if (!Objects.equals(val1, val2)) { + if (val1 == null) return 1; + if (val2 == null) return -1; + return Long.compare(val1, val2); + } + } + return 0; } Map getAttributes(int numSegmentDocs) { diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorterTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorterTests.java index 04e64d4f9df5b..b485ea1a4fe3e 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorterTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentsSorterTests.java @@ -8,27 +8,31 @@ package org.opensearch.index.compositeindex.datacube.startree.utils; +import org.opensearch.common.Randomness; import org.opensearch.test.OpenSearchTestCase; import org.junit.Before; +import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.Random; /** * Tests for {@link StarTreeDocumentsSorter}. */ public class StarTreeDocumentsSorterTests extends OpenSearchTestCase { - private StarTreeDocumentsSorter sorter; private Map testData; @Before public void setUp() throws Exception { super.setUp(); - sorter = new StarTreeDocumentsSorter(); testData = new HashMap<>(); - testData.put(0, new Long[] { 1L, 2L, 3L }); + testData.put(0, new Long[] { -1L, 2L, 3L }); testData.put(1, new Long[] { 1L, 2L, 2L }); - testData.put(2, new Long[] { 1L, 1L, 3L }); + testData.put(2, new Long[] { -1L, -1L, 3L }); testData.put(3, new Long[] { 1L, 2L, null }); testData.put(4, new Long[] { 1L, null, 3L }); } @@ -38,9 +42,9 @@ public void testSortDocumentsOffHeap_FirstDimension() { int dimensionId = -1; int numDocs = 5; - sorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(i)); + StarTreeDocumentsSorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(sortedDocIds[i])); - assertArrayEquals(new int[] { 2, 1, 0, 3, 4 }, sortedDocIds); + assertArrayEquals(new int[] { 2, 0, 1, 3, 4 }, sortedDocIds); } public void testSortDocumentsOffHeap_ThirdDimension() { @@ -48,7 +52,7 @@ public void testSortDocumentsOffHeap_ThirdDimension() { int dimensionId = 1; int numDocs = 5; - sorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(i)); + StarTreeDocumentsSorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(sortedDocIds[i])); assertArrayEquals(new int[] { 1, 0, 2, 4, 3 }, sortedDocIds); } @@ -58,7 +62,7 @@ public void testSortDocumentsOffHeap_SingleElement() { int dimensionId = -1; int numDocs = 1; - sorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(i)); + StarTreeDocumentsSorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(sortedDocIds[i])); assertArrayEquals(new int[] { 0 }, sortedDocIds); } @@ -68,7 +72,7 @@ public void testSortDocumentsOffHeap_EmptyArray() { int dimensionId = -1; int numDocs = 0; - sorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(i)); + StarTreeDocumentsSorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(sortedDocIds[i])); assertArrayEquals(new int[] {}, sortedDocIds); } @@ -78,7 +82,7 @@ public void testSortDocumentsOffHeap_SecondDimensionId() { int dimensionId = 0; int numDocs = 5; - sorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(i)); + StarTreeDocumentsSorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(sortedDocIds[i])); assertArrayEquals(new int[] { 2, 1, 0, 3, 4 }, sortedDocIds); } @@ -93,9 +97,105 @@ public void testSortDocumentsOffHeap_AllNulls() { int dimensionId = -1; int numDocs = 3; - sorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(i)); + StarTreeDocumentsSorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(sortedDocIds[i])); // The order should remain unchanged as all elements are equal (null) assertArrayEquals(new int[] { 0, 1, 2 }, sortedDocIds); } + + public void testSortDocumentsOffHeap_Negatives() { + Map testData = new HashMap<>(); + testData.put(0, new Long[] { -10L, 0L }); + testData.put(1, new Long[] { -9L, 0L }); + testData.put(2, new Long[] { -8L, 0L }); + testData.put(3, new Long[] { -7L, -0L }); + testData.put(4, new Long[] { -15L, -0L }); + + int[] sortedDocIds = { 0, 1, 2, 3, 4 }; + int dimensionId = -1; + int numDocs = 5; + + StarTreeDocumentsSorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(sortedDocIds[i])); + + // The order should remain unchanged as all elements are equal (null) + assertArrayEquals(new int[] { 4, 0, 1, 2, 3 }, sortedDocIds); + } + + public void testRandomSort() { + int i = 0; + while (i < 10) { + testRandomizedSort(); + i++; + } + } + + private void testRandomizedSort() { + + int numDocs = randomIntBetween(0, 1000); + Random random = Randomness.get(); + // skew more towards realistic number of dimensions + int numDimensions = random.nextBoolean() ? randomIntBetween(2, 10) : randomIntBetween(2, 100); + List testData = new ArrayList<>(); + // Generate random test data + for (int i = 0; i < numDocs; i++) { + Long[] dimensions = new Long[numDimensions]; + for (int j = 0; j < numDimensions; j++) { + if (random.nextFloat() < 0.5) { + dimensions[j] = random.nextBoolean() ? Long.valueOf(0L) : random.nextBoolean() ? -1L : null; + } else { + dimensions[j] = random.nextLong(); + } + } + testData.add(dimensions); + } + + int[] sortedDocIds = new int[numDocs]; + for (int i = 0; i < numDocs; i++) { + sortedDocIds[i] = i; + } + // sort dimensionId + 1 to numDimensions + // for example to start from dimension in 0th index, we need to pass -1 to sort method + int dimensionId = random.nextInt(numDimensions) - 1; + + // Sort using StarTreeDocumentsSorter + StarTreeDocumentsSorter.sort(sortedDocIds, dimensionId, numDocs, i -> testData.get(sortedDocIds[i])); + + // Verify the sorting + for (int i = 1; i < numDocs; i++) { + Long[] prev = testData.get(sortedDocIds[i - 1]); + Long[] curr = testData.get(sortedDocIds[i]); + boolean isCorrectOrder = true; + for (int j = dimensionId + 1; j < numDimensions; j++) { + int comparison = compareLongs(prev[j], curr[j]); + if (comparison < 0) { + break; + } else if (comparison > 0) { + isCorrectOrder = false; + break; + } + } + assertTrue( + "Sorting error when sorting from dimension index " + + dimensionId + + " Prev : " + + Arrays.toString(prev) + + " :: Curr : " + + Arrays.toString(curr), + isCorrectOrder + ); + } + } + + private int compareLongs(Long a, Long b) { + if (!Objects.equals(a, b)) { + if (a == null) { + return 1; + } else if (b == null) { + return -1; + } else { + return a.compareTo(b); + } + } + return 0; + } } From 7f637c796f9e59698bc032daf4710c56e218708b Mon Sep 17 00:00:00 2001 From: Bharathwaj G Date: Sun, 4 Aug 2024 19:45:01 +0530 Subject: [PATCH 7/7] addressing comments Signed-off-by: Bharathwaj G --- .../util/ByteArrayBackedBitset.java} | 35 +++++++++++++++---- .../builder/AbstractDocumentsFileManager.java | 13 ++++--- .../builder/SegmentDocsFileManager.java | 4 +-- .../builder/StarTreeDocsFileManager.java | 4 +-- .../utils/StarTreeDocumentBitSetUtil.java | 5 +-- 5 files changed, 41 insertions(+), 20 deletions(-) rename server/src/main/java/org/opensearch/{index/compositeindex/datacube/startree/utils/ByteListBackedBitset.java => common/util/ByteArrayBackedBitset.java} (56%) diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/ByteListBackedBitset.java b/server/src/main/java/org/opensearch/common/util/ByteArrayBackedBitset.java similarity index 56% rename from server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/ByteListBackedBitset.java rename to server/src/main/java/org/opensearch/common/util/ByteArrayBackedBitset.java index 792832d751b56..2d7948d414937 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/ByteListBackedBitset.java +++ b/server/src/main/java/org/opensearch/common/util/ByteArrayBackedBitset.java @@ -6,30 +6,31 @@ * compatible open source license. */ -package org.opensearch.index.compositeindex.datacube.startree.utils; +package org.opensearch.common.util; +import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.RandomAccessInput; import java.io.IOException; /** - * This is used to store and retrieve null values in the bitset. + * A bitset backed by a byte array. This will initialize and set bits in the byte array based on the index. */ -public class ByteListBackedBitset { +public class ByteArrayBackedBitset { private final byte[] byteArray; /** * Constructor which uses an on heap list. This should be using during construction of the bitset. */ - public ByteListBackedBitset(int capacity) { + public ByteArrayBackedBitset(int capacity) { byteArray = new byte[capacity]; } /** - * Constructor which set the Lucene's IndexInput to read the bitset into a read-only buffer. + * Constructor which set the Lucene's RandomAccessInput to read the bitset into a read-only buffer. */ - public ByteListBackedBitset(RandomAccessInput in, long offset, int length) throws IOException { + public ByteArrayBackedBitset(RandomAccessInput in, long offset, int length) throws IOException { byteArray = new byte[length]; int i = 0; while (i < length) { @@ -38,6 +39,23 @@ public ByteListBackedBitset(RandomAccessInput in, long offset, int length) throw } } + /** + * Constructor which set the Lucene's IndexInput to read the bitset into a read-only buffer. + */ + public ByteArrayBackedBitset(IndexInput in, int length) throws IOException { + byteArray = new byte[length]; + int i = 0; + while (i < length) { + byteArray[i] = in.readByte(); + i++; + } + } + + /** + * Sets the bit at the given index to 1. + * Each byte can indicate 8 bits, so the index is divided by 8 to get the byte array index. + * @param index the index to set the bit + */ public void set(int index) { int byteArrIndex = index >> 3; byteArray[byteArrIndex] |= (byte) (1 << (index & 7)); @@ -52,6 +70,11 @@ public int write(IndexOutput output) throws IOException { return numBytes; } + /** + * Retrieves whether the bit is set or not at the given index. + * @param index the index to look up for the bit + * @return true if bit is set, false otherwise + */ public boolean get(int index) throws IOException { int byteArrIndex = index >> 3; return (byteArray[byteArrIndex] & (1 << (index & 7))) != 0; diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractDocumentsFileManager.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractDocumentsFileManager.java index 6a696ce2a2933..78c49dbada6b2 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractDocumentsFileManager.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractDocumentsFileManager.java @@ -118,19 +118,18 @@ protected int writeMetrics(StarTreeDocument starTreeDocument, IndexOutput output * * @param input RandomAccessInput * @param offset Offset in the file - * @param shouldReadAggregatedDocs boolean to indicate if aggregated star tree docs should be read + * @param isAggregatedDoc boolean to indicate if aggregated star tree docs should be read * @return StarTreeDocument * @throws IOException IOException in case of I/O errors */ - protected StarTreeDocument readStarTreeDocument(RandomAccessInput input, long offset, boolean shouldReadAggregatedDocs) - throws IOException { + protected StarTreeDocument readStarTreeDocument(RandomAccessInput input, long offset, boolean isAggregatedDoc) throws IOException { int dimSize = starTreeField.getDimensionsOrder().size(); Long[] dimensions = new Long[dimSize]; long initialOffset = offset; offset = readDimensions(dimensions, input, offset); Object[] metrics = new Object[numMetrics]; - offset = readMetrics(input, offset, numMetrics, metrics, shouldReadAggregatedDocs); + offset = readMetrics(input, offset, numMetrics, metrics, isAggregatedDoc); assert (offset - initialOffset) == docSizeInBytes; return new StarTreeDocument(dimensions, metrics); } @@ -155,7 +154,7 @@ protected long readDimensions(Long[] dimensions, RandomAccessInput input, long o /** * Read star tree metrics from file */ - protected long readMetrics(RandomAccessInput input, long offset, int numMetrics, Object[] metrics, boolean shouldReadAggregatedDocs) + protected long readMetrics(RandomAccessInput input, long offset, int numMetrics, Object[] metrics, boolean isAggregatedDoc) throws IOException { for (int i = 0; i < numMetrics; i++) { switch (metricAggregatorInfos.get(i).getValueAggregators().getAggregatedValueType()) { @@ -165,7 +164,7 @@ protected long readMetrics(RandomAccessInput input, long offset, int numMetrics, break; case DOUBLE: long val = input.readLong(offset); - if (shouldReadAggregatedDocs) { + if (isAggregatedDoc) { metrics[i] = StarTreeNumericTypeConverters.sortableLongtoDouble(val); } else { metrics[i] = val; @@ -193,7 +192,7 @@ protected long readMetrics(RandomAccessInput input, long offset, int numMetrics, /** * Read star tree document from file based on doc id */ - public abstract StarTreeDocument readStarTreeDocument(int docId, boolean isMerge) throws IOException; + public abstract StarTreeDocument readStarTreeDocument(int docId, boolean isAggregatedDoc) throws IOException; /** * Read star document dimensions from file based on doc id diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/SegmentDocsFileManager.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/SegmentDocsFileManager.java index 172314f7fb2f3..fe94df57d9535 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/SegmentDocsFileManager.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/SegmentDocsFileManager.java @@ -70,9 +70,9 @@ private void maybeInitializeSegmentInput() throws IOException { } @Override - public StarTreeDocument readStarTreeDocument(int docId, boolean isMerge) throws IOException { + public StarTreeDocument readStarTreeDocument(int docId, boolean isAggregatedDoc) throws IOException { maybeInitializeSegmentInput(); - return readStarTreeDocument(segmentRandomInput, (long) docId * docSizeInBytes, isMerge); + return readStarTreeDocument(segmentRandomInput, (long) docId * docSizeInBytes, isAggregatedDoc); } @Override diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocsFileManager.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocsFileManager.java index 3dec0b660cd48..779ed77b0540a 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocsFileManager.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocsFileManager.java @@ -199,12 +199,10 @@ private void loadStarTreeDocumentFile(int docId) throws IOException { * If the operation is only for reading existing documents, a new file is not created. */ private void closeAndMaybeCreateNewFile(boolean shouldCreateFileForAppend, int numStarTreeDocs) throws IOException { - if (starTreeDocsFileOutput != null) { - IOUtils.close(starTreeDocsFileOutput); - } currBytes = 0; if (starTreeDocsFileOutput != null) { fileToEndDocIdMap.put(starTreeDocsFileOutput.getName(), numStarTreeDocs); + IOUtils.close(starTreeDocsFileOutput); } if (shouldCreateFileForAppend) { starTreeDocsFileOutput = createStarTreeDocumentsFileOutput(); diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtil.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtil.java index 0a5c8028436a0..a508e497adcdf 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtil.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/StarTreeDocumentBitSetUtil.java @@ -10,6 +10,7 @@ import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.RandomAccessInput; +import org.opensearch.common.util.ByteArrayBackedBitset; import java.io.IOException; import java.util.function.Function; @@ -27,7 +28,7 @@ public class StarTreeDocumentBitSetUtil { * @throws IOException if an I/O error occurs while writing to the output stream */ public static int writeBitSet(Object[] array, IndexOutput output) throws IOException { - ByteListBackedBitset bitset = new ByteListBackedBitset(getLength(array)); + ByteArrayBackedBitset bitset = new ByteArrayBackedBitset(getLength(array)); for (int i = 0; i < array.length; i++) { if (array[i] == null) { bitset.set(i); @@ -41,7 +42,7 @@ public static int writeBitSet(Object[] array, IndexOutput output) throws IOExcep */ public static int readBitSet(RandomAccessInput input, long offset, Object[] array, Function identityValueSupplier) throws IOException { - ByteListBackedBitset bitset = new ByteListBackedBitset(input, offset, getLength(array)); + ByteArrayBackedBitset bitset = new ByteArrayBackedBitset(input, offset, getLength(array)); for (int i = 0; i < array.length; i++) { if (bitset.get(i)) { array[i] = identityValueSupplier.apply(i);