Skip to content

Commit

Permalink
[Star tree] Moving compositeCodec to composite912Codec and dropping c…
Browse files Browse the repository at this point in the history
…omposite99Codec (#16227)

* Refactoring builder tests
* composite912 codec refactor changes

---------

Signed-off-by: Bharathwaj G <[email protected]>
  • Loading branch information
bharath-techie authored Oct 9, 2024
1 parent 62b1537 commit b5917c5
Show file tree
Hide file tree
Showing 19 changed files with 176 additions and 241 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@
import org.apache.lucene.codecs.lucene912.Lucene912Codec;
import org.opensearch.common.annotation.ExperimentalApi;
import org.opensearch.index.codec.composite.composite912.Composite912Codec;
import org.opensearch.index.codec.composite.composite99.Composite99Codec;
import org.opensearch.index.mapper.MapperService;

import java.util.HashMap;
Expand All @@ -33,7 +32,7 @@
public class CompositeCodecFactory {

// we can use this to track the latest composite codec
public static final String COMPOSITE_CODEC = Composite99Codec.COMPOSITE_INDEX_CODEC_NAME;
public static final String COMPOSITE_CODEC = Composite912Codec.COMPOSITE_INDEX_CODEC_NAME;

public CompositeCodecFactory() {}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,7 @@
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducerWrapper;
import org.apache.lucene.index.SegmentReadState;
import org.opensearch.index.codec.composite.composite99.Composite99Codec;
import org.opensearch.index.codec.composite.composite912.Composite912Codec;

import java.io.IOException;

Expand Down Expand Up @@ -39,7 +39,7 @@ public static DocValuesProducer getDocValuesProducerForCompositeCodec(
) throws IOException {

switch (compositeCodec) {
case Composite99Codec.COMPOSITE_INDEX_CODEC_NAME:
case Composite912Codec.COMPOSITE_INDEX_CODEC_NAME:
Lucene90DocValuesProducerWrapper lucene90DocValuesProducerWrapper = new Lucene90DocValuesProducerWrapper(
state,
dataCodec,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@
import org.apache.lucene.codecs.lucene912.Lucene912Codec;
import org.opensearch.common.annotation.ExperimentalApi;
import org.opensearch.index.codec.PerFieldMappingPostingFormatCodec;
import org.opensearch.index.codec.composite.composite99.Composite99DocValuesFormat;
import org.opensearch.index.mapper.MapperService;

/**
Expand Down Expand Up @@ -53,6 +52,6 @@ protected Composite912Codec(String name, Codec delegate, MapperService mapperSer

@Override
public DocValuesFormat docValuesFormat() {
return new Composite99DocValuesFormat(mapperService);
return new Composite912DocValuesFormat(mapperService);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@
* compatible open source license.
*/

package org.opensearch.index.codec.composite.composite99;
package org.opensearch.index.codec.composite.composite912;

import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
Expand All @@ -25,7 +25,7 @@
* @opensearch.experimental
*/
@ExperimentalApi
public class Composite99DocValuesFormat extends DocValuesFormat {
public class Composite912DocValuesFormat extends DocValuesFormat {
/**
* Creates a new docvalues format.
*
Expand All @@ -38,10 +38,10 @@ public class Composite99DocValuesFormat extends DocValuesFormat {
private final MapperService mapperService;

/** Data codec name for Composite Doc Values Format */
public static final String DATA_CODEC_NAME = "Composite99FormatData";
public static final String DATA_CODEC_NAME = "Composite912FormatData";

/** Meta codec name for Composite Doc Values Format */
public static final String META_CODEC_NAME = "Composite99FormatMeta";
public static final String META_CODEC_NAME = "Composite912FormatMeta";

/** Filename extension for the composite index data */
public static final String DATA_EXTENSION = "cid";
Expand All @@ -50,10 +50,10 @@ public class Composite99DocValuesFormat extends DocValuesFormat {
public static final String META_EXTENSION = "cim";

/** Data doc values codec name for Composite Doc Values Format */
public static final String DATA_DOC_VALUES_CODEC = "Composite99DocValuesData";
public static final String DATA_DOC_VALUES_CODEC = "Composite912DocValuesData";

/** Meta doc values codec name for Composite Doc Values Format */
public static final String META_DOC_VALUES_CODEC = "Composite99DocValuesMetadata";
public static final String META_DOC_VALUES_CODEC = "Composite912DocValuesMetadata";

/** Filename extension for the composite index data doc values */
public static final String DATA_DOC_VALUES_EXTENSION = "cidvd";
Expand All @@ -68,27 +68,27 @@ public class Composite99DocValuesFormat extends DocValuesFormat {
public static final int VERSION_CURRENT = VERSION_START;

// needed for SPI
public Composite99DocValuesFormat() {
public Composite912DocValuesFormat() {
this(new Lucene90DocValuesFormat(), null);
}

public Composite99DocValuesFormat(MapperService mapperService) {
public Composite912DocValuesFormat(MapperService mapperService) {
this(new Lucene90DocValuesFormat(), mapperService);
}

public Composite99DocValuesFormat(DocValuesFormat delegate, MapperService mapperService) {
public Composite912DocValuesFormat(DocValuesFormat delegate, MapperService mapperService) {
super(delegate.getName());
this.delegate = delegate;
this.mapperService = mapperService;
}

@Override
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new Composite99DocValuesWriter(delegate.fieldsConsumer(state), state, mapperService);
return new Composite912DocValuesWriter(delegate.fieldsConsumer(state), state, mapperService);
}

@Override
public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
return new Composite99DocValuesReader(delegate.fieldsProducer(state), state);
return new Composite912DocValuesReader(delegate.fieldsProducer(state), state);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@
* compatible open source license.
*/

package org.opensearch.index.codec.composite.composite99;
package org.opensearch.index.codec.composite.composite912;

import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
Expand Down Expand Up @@ -56,8 +56,8 @@
* @opensearch.experimental
*/
@ExperimentalApi
public class Composite99DocValuesReader extends DocValuesProducer implements CompositeIndexReader {
private static final Logger logger = LogManager.getLogger(Composite99DocValuesReader.class);
public class Composite912DocValuesReader extends DocValuesProducer implements CompositeIndexReader {
private static final Logger logger = LogManager.getLogger(Composite912DocValuesReader.class);

private final DocValuesProducer delegate;
private IndexInput dataIn;
Expand All @@ -69,20 +69,20 @@ public class Composite99DocValuesReader extends DocValuesProducer implements Com
private final List<CompositeIndexFieldInfo> compositeFieldInfos = new ArrayList<>();
private SegmentReadState readState;

public Composite99DocValuesReader(DocValuesProducer producer, SegmentReadState readState) throws IOException {
public Composite912DocValuesReader(DocValuesProducer producer, SegmentReadState readState) throws IOException {
this.delegate = producer;
this.fields = new ArrayList<>();

String metaFileName = IndexFileNames.segmentFileName(
readState.segmentInfo.name,
readState.segmentSuffix,
Composite99DocValuesFormat.META_EXTENSION
Composite912DocValuesFormat.META_EXTENSION
);

String dataFileName = IndexFileNames.segmentFileName(
readState.segmentInfo.name,
readState.segmentSuffix,
Composite99DocValuesFormat.DATA_EXTENSION
Composite912DocValuesFormat.DATA_EXTENSION
);

boolean success = false;
Expand All @@ -92,9 +92,9 @@ public Composite99DocValuesReader(DocValuesProducer producer, SegmentReadState r
dataIn = readState.directory.openInput(dataFileName, readState.context);
CodecUtil.checkIndexHeader(
dataIn,
Composite99DocValuesFormat.DATA_CODEC_NAME,
Composite99DocValuesFormat.VERSION_START,
Composite99DocValuesFormat.VERSION_CURRENT,
Composite912DocValuesFormat.DATA_CODEC_NAME,
Composite912DocValuesFormat.VERSION_START,
Composite912DocValuesFormat.VERSION_CURRENT,
readState.segmentInfo.getId(),
readState.segmentSuffix
);
Expand All @@ -105,9 +105,9 @@ public Composite99DocValuesReader(DocValuesProducer producer, SegmentReadState r
try {
CodecUtil.checkIndexHeader(
metaIn,
Composite99DocValuesFormat.META_CODEC_NAME,
Composite99DocValuesFormat.VERSION_START,
Composite99DocValuesFormat.VERSION_CURRENT,
Composite912DocValuesFormat.META_CODEC_NAME,
Composite912DocValuesFormat.VERSION_START,
Composite912DocValuesFormat.VERSION_CURRENT,
readState.segmentInfo.getId(),
readState.segmentSuffix
);
Expand Down Expand Up @@ -190,12 +190,12 @@ public Composite99DocValuesReader(DocValuesProducer producer, SegmentReadState r
// initialize star-tree doc values producer

compositeDocValuesProducer = LuceneDocValuesProducerFactory.getDocValuesProducerForCompositeCodec(
Composite99Codec.COMPOSITE_INDEX_CODEC_NAME,
Composite912Codec.COMPOSITE_INDEX_CODEC_NAME,
this.readState,
Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC,
Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION,
Composite99DocValuesFormat.META_DOC_VALUES_CODEC,
Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION
Composite912DocValuesFormat.DATA_DOC_VALUES_CODEC,
Composite912DocValuesFormat.DATA_DOC_VALUES_EXTENSION,
Composite912DocValuesFormat.META_DOC_VALUES_CODEC,
Composite912DocValuesFormat.META_DOC_VALUES_EXTENSION
);

} catch (Throwable t) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@
* compatible open source license.
*/

package org.opensearch.index.codec.composite.composite99;
package org.opensearch.index.codec.composite.composite912;

import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesConsumer;
Expand Down Expand Up @@ -55,14 +55,14 @@
* @opensearch.experimental
*/
@ExperimentalApi
public class Composite99DocValuesWriter extends DocValuesConsumer {
public class Composite912DocValuesWriter extends DocValuesConsumer {
private final DocValuesConsumer delegate;
private final SegmentWriteState state;
private final MapperService mapperService;
AtomicReference<MergeState> mergeState = new AtomicReference<>();
private final Set<CompositeMappedFieldType> compositeMappedFieldTypes;
private final Set<String> compositeFieldSet;
private DocValuesConsumer composite99DocValuesConsumer;
private DocValuesConsumer compositeDocValuesConsumer;

public IndexOutput dataOut;
public IndexOutput metaOut;
Expand All @@ -72,7 +72,7 @@ public class Composite99DocValuesWriter extends DocValuesConsumer {

private final Map<String, DocValuesProducer> fieldProducerMap = new HashMap<>();

public Composite99DocValuesWriter(DocValuesConsumer delegate, SegmentWriteState segmentWriteState, MapperService mapperService)
public Composite912DocValuesWriter(DocValuesConsumer delegate, SegmentWriteState segmentWriteState, MapperService mapperService)
throws IOException {

this.delegate = delegate;
Expand Down Expand Up @@ -101,38 +101,38 @@ public Composite99DocValuesWriter(DocValuesConsumer delegate, SegmentWriteState
// so that all the fields are sparse numeric doc values and not dense numeric doc values
SegmentWriteState consumerWriteState = getSegmentWriteState(segmentWriteState);

this.composite99DocValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec(
this.compositeDocValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec(
consumerWriteState,
Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC,
Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION,
Composite99DocValuesFormat.META_DOC_VALUES_CODEC,
Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION
Composite912DocValuesFormat.DATA_DOC_VALUES_CODEC,
Composite912DocValuesFormat.DATA_DOC_VALUES_EXTENSION,
Composite912DocValuesFormat.META_DOC_VALUES_CODEC,
Composite912DocValuesFormat.META_DOC_VALUES_EXTENSION
);

String dataFileName = IndexFileNames.segmentFileName(
this.state.segmentInfo.name,
this.state.segmentSuffix,
Composite99DocValuesFormat.DATA_EXTENSION
Composite912DocValuesFormat.DATA_EXTENSION
);
dataOut = this.state.directory.createOutput(dataFileName, this.state.context);
CodecUtil.writeIndexHeader(
dataOut,
Composite99DocValuesFormat.DATA_CODEC_NAME,
Composite99DocValuesFormat.VERSION_CURRENT,
Composite912DocValuesFormat.DATA_CODEC_NAME,
Composite912DocValuesFormat.VERSION_CURRENT,
this.state.segmentInfo.getId(),
this.state.segmentSuffix
);

String metaFileName = IndexFileNames.segmentFileName(
this.state.segmentInfo.name,
this.state.segmentSuffix,
Composite99DocValuesFormat.META_EXTENSION
Composite912DocValuesFormat.META_EXTENSION
);
metaOut = this.state.directory.createOutput(metaFileName, this.state.context);
CodecUtil.writeIndexHeader(
metaOut,
Composite99DocValuesFormat.META_CODEC_NAME,
Composite99DocValuesFormat.VERSION_CURRENT,
Composite912DocValuesFormat.META_CODEC_NAME,
Composite912DocValuesFormat.VERSION_CURRENT,
this.state.segmentInfo.getId(),
this.state.segmentSuffix
);
Expand Down Expand Up @@ -197,12 +197,12 @@ public void close() throws IOException {
success = true;
} finally {
if (success) {
IOUtils.close(dataOut, metaOut, composite99DocValuesConsumer);
IOUtils.close(dataOut, metaOut, compositeDocValuesConsumer);
} else {
IOUtils.closeWhileHandlingException(dataOut, metaOut, composite99DocValuesConsumer);
IOUtils.closeWhileHandlingException(dataOut, metaOut, compositeDocValuesConsumer);
}
metaOut = dataOut = null;
composite99DocValuesConsumer = null;
compositeDocValuesConsumer = null;
}
}

Expand All @@ -224,7 +224,7 @@ private void createCompositeIndicesIfPossible(DocValuesProducer valuesProducer,
for (CompositeMappedFieldType mappedType : compositeMappedFieldTypes) {
if (mappedType instanceof StarTreeMapper.StarTreeFieldType) {
try (StarTreesBuilder starTreesBuilder = new StarTreesBuilder(state, mapperService, fieldNumberAcrossCompositeFields)) {
starTreesBuilder.build(metaOut, dataOut, fieldProducerMap, composite99DocValuesConsumer);
starTreesBuilder.build(metaOut, dataOut, fieldProducerMap, compositeDocValuesConsumer);
}
}
}
Expand Down Expand Up @@ -313,7 +313,7 @@ private void mergeStarTreeFields(MergeState mergeState) throws IOException {
}
}
try (StarTreesBuilder starTreesBuilder = new StarTreesBuilder(state, mapperService, fieldNumberAcrossCompositeFields)) {
starTreesBuilder.buildDuringMerge(metaOut, dataOut, starTreeSubsPerField, composite99DocValuesConsumer);
starTreesBuilder.buildDuringMerge(metaOut, dataOut, starTreeSubsPerField, compositeDocValuesConsumer);
}
}

Expand Down

This file was deleted.

This file was deleted.

Loading

0 comments on commit b5917c5

Please sign in to comment.