|
| 1 | +/* |
| 2 | + * SPDX-License-Identifier: Apache-2.0 |
| 3 | + * |
| 4 | + * The OpenSearch Contributors require contributions made to |
| 5 | + * this file be licensed under the Apache-2.0 license or a |
| 6 | + * compatible open source license. |
| 7 | + */ |
| 8 | + |
| 9 | +package org.opensearch.index.codec.composite; |
| 10 | + |
| 11 | +import org.apache.lucene.codecs.DocValuesConsumer; |
| 12 | +import org.apache.lucene.codecs.DocValuesProducer; |
| 13 | +import org.apache.lucene.index.FieldInfo; |
| 14 | +import org.apache.lucene.index.MergeState; |
| 15 | +import org.apache.lucene.index.SegmentWriteState; |
| 16 | +import org.opensearch.common.annotation.ExperimentalApi; |
| 17 | +import org.opensearch.index.mapper.CompositeMappedFieldType; |
| 18 | +import org.opensearch.index.mapper.MapperService; |
| 19 | +import org.opensearch.index.mapper.StarTreeMapper; |
| 20 | + |
| 21 | +import java.io.IOException; |
| 22 | +import java.util.HashMap; |
| 23 | +import java.util.HashSet; |
| 24 | +import java.util.Map; |
| 25 | +import java.util.Set; |
| 26 | +import java.util.concurrent.atomic.AtomicReference; |
| 27 | + |
| 28 | +/** |
| 29 | + * This class write the star tree index and star tree doc values |
| 30 | + * based on the doc values structures of the original index |
| 31 | + * |
| 32 | + * @opensearch.experimental |
| 33 | + */ |
| 34 | +@ExperimentalApi |
| 35 | +public class Composite99DocValuesWriter extends DocValuesConsumer { |
| 36 | + private final DocValuesConsumer delegate; |
| 37 | + private final SegmentWriteState state; |
| 38 | + private final MapperService mapperService; |
| 39 | + AtomicReference<MergeState> mergeState = new AtomicReference<>(); |
| 40 | + private final Set<CompositeMappedFieldType> compositeMappedFieldTypes; |
| 41 | + private final Set<String> compositeFieldSet; |
| 42 | + |
| 43 | + private final Map<String, DocValuesProducer> fieldProducerMap = new HashMap<>(); |
| 44 | + |
| 45 | + public Composite99DocValuesWriter(DocValuesConsumer delegate, SegmentWriteState segmentWriteState, MapperService mapperService) { |
| 46 | + |
| 47 | + this.delegate = delegate; |
| 48 | + this.state = segmentWriteState; |
| 49 | + this.mapperService = mapperService; |
| 50 | + this.compositeMappedFieldTypes = mapperService.getCompositeFieldTypes(); |
| 51 | + compositeFieldSet = new HashSet<>(); |
| 52 | + for (CompositeMappedFieldType type : compositeMappedFieldTypes) { |
| 53 | + compositeFieldSet.addAll(type.fields()); |
| 54 | + } |
| 55 | + } |
| 56 | + |
| 57 | + @Override |
| 58 | + public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { |
| 59 | + delegate.addNumericField(field, valuesProducer); |
| 60 | + } |
| 61 | + |
| 62 | + @Override |
| 63 | + public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { |
| 64 | + delegate.addBinaryField(field, valuesProducer); |
| 65 | + } |
| 66 | + |
| 67 | + @Override |
| 68 | + public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { |
| 69 | + delegate.addSortedField(field, valuesProducer); |
| 70 | + } |
| 71 | + |
| 72 | + @Override |
| 73 | + public void addSortedNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { |
| 74 | + delegate.addSortedNumericField(field, valuesProducer); |
| 75 | + // Perform this only during flush flow |
| 76 | + if (mergeState.get() == null) { |
| 77 | + createCompositeIndicesIfPossible(valuesProducer, field); |
| 78 | + } |
| 79 | + } |
| 80 | + |
| 81 | + @Override |
| 82 | + public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException { |
| 83 | + delegate.addSortedSetField(field, valuesProducer); |
| 84 | + } |
| 85 | + |
| 86 | + @Override |
| 87 | + public void close() throws IOException { |
| 88 | + delegate.close(); |
| 89 | + } |
| 90 | + |
| 91 | + private void createCompositeIndicesIfPossible(DocValuesProducer valuesProducer, FieldInfo field) throws IOException { |
| 92 | + if (compositeFieldSet.isEmpty()) return; |
| 93 | + if (compositeFieldSet.contains(field.name)) { |
| 94 | + fieldProducerMap.put(field.name, valuesProducer); |
| 95 | + compositeFieldSet.remove(field.name); |
| 96 | + } |
| 97 | + // we have all the required fields to build composite fields |
| 98 | + if (compositeFieldSet.isEmpty()) { |
| 99 | + for (CompositeMappedFieldType mappedType : compositeMappedFieldTypes) { |
| 100 | + if (mappedType instanceof StarTreeMapper.StarTreeFieldType) { |
| 101 | + // TODO : Call StarTree builder |
| 102 | + } |
| 103 | + } |
| 104 | + } |
| 105 | + } |
| 106 | + |
| 107 | + @Override |
| 108 | + public void merge(MergeState mergeState) throws IOException { |
| 109 | + this.mergeState.compareAndSet(null, mergeState); |
| 110 | + super.merge(mergeState); |
| 111 | + // TODO : handle merge star tree |
| 112 | + // mergeStarTreeFields(mergeState); |
| 113 | + } |
| 114 | +} |
0 commit comments