Skip to content

Commit b5917c5

Browse files
[Star tree] Moving compositeCodec to composite912Codec and dropping composite99Codec (#16227)
* Refactoring builder tests * composite912 codec refactor changes --------- Signed-off-by: Bharathwaj G <bharath78910@gmail.com>
1 parent 62b1537 commit b5917c5

19 files changed

+176
-241
lines changed

server/src/main/java/org/opensearch/index/codec/composite/CompositeCodecFactory.java

+1-2
Original file line numberDiff line numberDiff line change
@@ -13,7 +13,6 @@
1313
import org.apache.lucene.codecs.lucene912.Lucene912Codec;
1414
import org.opensearch.common.annotation.ExperimentalApi;
1515
import org.opensearch.index.codec.composite.composite912.Composite912Codec;
16-
import org.opensearch.index.codec.composite.composite99.Composite99Codec;
1716
import org.opensearch.index.mapper.MapperService;
1817

1918
import java.util.HashMap;
@@ -33,7 +32,7 @@
3332
public class CompositeCodecFactory {
3433

3534
// we can use this to track the latest composite codec
36-
public static final String COMPOSITE_CODEC = Composite99Codec.COMPOSITE_INDEX_CODEC_NAME;
35+
public static final String COMPOSITE_CODEC = Composite912Codec.COMPOSITE_INDEX_CODEC_NAME;
3736

3837
public CompositeCodecFactory() {}
3938

server/src/main/java/org/opensearch/index/codec/composite/LuceneDocValuesProducerFactory.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -11,7 +11,7 @@
1111
import org.apache.lucene.codecs.DocValuesProducer;
1212
import org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducerWrapper;
1313
import org.apache.lucene.index.SegmentReadState;
14-
import org.opensearch.index.codec.composite.composite99.Composite99Codec;
14+
import org.opensearch.index.codec.composite.composite912.Composite912Codec;
1515

1616
import java.io.IOException;
1717

@@ -39,7 +39,7 @@ public static DocValuesProducer getDocValuesProducerForCompositeCodec(
3939
) throws IOException {
4040

4141
switch (compositeCodec) {
42-
case Composite99Codec.COMPOSITE_INDEX_CODEC_NAME:
42+
case Composite912Codec.COMPOSITE_INDEX_CODEC_NAME:
4343
Lucene90DocValuesProducerWrapper lucene90DocValuesProducerWrapper = new Lucene90DocValuesProducerWrapper(
4444
state,
4545
dataCodec,

server/src/main/java/org/opensearch/index/codec/composite/composite912/Composite912Codec.java

+1-2
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,6 @@
1515
import org.apache.lucene.codecs.lucene912.Lucene912Codec;
1616
import org.opensearch.common.annotation.ExperimentalApi;
1717
import org.opensearch.index.codec.PerFieldMappingPostingFormatCodec;
18-
import org.opensearch.index.codec.composite.composite99.Composite99DocValuesFormat;
1918
import org.opensearch.index.mapper.MapperService;
2019

2120
/**
@@ -53,6 +52,6 @@ protected Composite912Codec(String name, Codec delegate, MapperService mapperSer
5352

5453
@Override
5554
public DocValuesFormat docValuesFormat() {
56-
return new Composite99DocValuesFormat(mapperService);
55+
return new Composite912DocValuesFormat(mapperService);
5756
}
5857
}

server/src/main/java/org/opensearch/index/codec/composite/composite99/Composite99DocValuesFormat.java server/src/main/java/org/opensearch/index/codec/composite/composite912/Composite912DocValuesFormat.java

+11-11
Original file line numberDiff line numberDiff line change
@@ -6,7 +6,7 @@
66
* compatible open source license.
77
*/
88

9-
package org.opensearch.index.codec.composite.composite99;
9+
package org.opensearch.index.codec.composite.composite912;
1010

1111
import org.apache.lucene.codecs.DocValuesConsumer;
1212
import org.apache.lucene.codecs.DocValuesFormat;
@@ -25,7 +25,7 @@
2525
* @opensearch.experimental
2626
*/
2727
@ExperimentalApi
28-
public class Composite99DocValuesFormat extends DocValuesFormat {
28+
public class Composite912DocValuesFormat extends DocValuesFormat {
2929
/**
3030
* Creates a new docvalues format.
3131
*
@@ -38,10 +38,10 @@ public class Composite99DocValuesFormat extends DocValuesFormat {
3838
private final MapperService mapperService;
3939

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

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

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

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

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

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

7070
// needed for SPI
71-
public Composite99DocValuesFormat() {
71+
public Composite912DocValuesFormat() {
7272
this(new Lucene90DocValuesFormat(), null);
7373
}
7474

75-
public Composite99DocValuesFormat(MapperService mapperService) {
75+
public Composite912DocValuesFormat(MapperService mapperService) {
7676
this(new Lucene90DocValuesFormat(), mapperService);
7777
}
7878

79-
public Composite99DocValuesFormat(DocValuesFormat delegate, MapperService mapperService) {
79+
public Composite912DocValuesFormat(DocValuesFormat delegate, MapperService mapperService) {
8080
super(delegate.getName());
8181
this.delegate = delegate;
8282
this.mapperService = mapperService;
8383
}
8484

8585
@Override
8686
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
87-
return new Composite99DocValuesWriter(delegate.fieldsConsumer(state), state, mapperService);
87+
return new Composite912DocValuesWriter(delegate.fieldsConsumer(state), state, mapperService);
8888
}
8989

9090
@Override
9191
public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
92-
return new Composite99DocValuesReader(delegate.fieldsProducer(state), state);
92+
return new Composite912DocValuesReader(delegate.fieldsProducer(state), state);
9393
}
9494
}

server/src/main/java/org/opensearch/index/codec/composite/composite99/Composite99DocValuesReader.java server/src/main/java/org/opensearch/index/codec/composite/composite912/Composite912DocValuesReader.java

+17-17
Original file line numberDiff line numberDiff line change
@@ -6,7 +6,7 @@
66
* compatible open source license.
77
*/
88

9-
package org.opensearch.index.codec.composite.composite99;
9+
package org.opensearch.index.codec.composite.composite912;
1010

1111
import org.apache.logging.log4j.LogManager;
1212
import org.apache.logging.log4j.Logger;
@@ -56,8 +56,8 @@
5656
* @opensearch.experimental
5757
*/
5858
@ExperimentalApi
59-
public class Composite99DocValuesReader extends DocValuesProducer implements CompositeIndexReader {
60-
private static final Logger logger = LogManager.getLogger(Composite99DocValuesReader.class);
59+
public class Composite912DocValuesReader extends DocValuesProducer implements CompositeIndexReader {
60+
private static final Logger logger = LogManager.getLogger(Composite912DocValuesReader.class);
6161

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

72-
public Composite99DocValuesReader(DocValuesProducer producer, SegmentReadState readState) throws IOException {
72+
public Composite912DocValuesReader(DocValuesProducer producer, SegmentReadState readState) throws IOException {
7373
this.delegate = producer;
7474
this.fields = new ArrayList<>();
7575

7676
String metaFileName = IndexFileNames.segmentFileName(
7777
readState.segmentInfo.name,
7878
readState.segmentSuffix,
79-
Composite99DocValuesFormat.META_EXTENSION
79+
Composite912DocValuesFormat.META_EXTENSION
8080
);
8181

8282
String dataFileName = IndexFileNames.segmentFileName(
8383
readState.segmentInfo.name,
8484
readState.segmentSuffix,
85-
Composite99DocValuesFormat.DATA_EXTENSION
85+
Composite912DocValuesFormat.DATA_EXTENSION
8686
);
8787

8888
boolean success = false;
@@ -92,9 +92,9 @@ public Composite99DocValuesReader(DocValuesProducer producer, SegmentReadState r
9292
dataIn = readState.directory.openInput(dataFileName, readState.context);
9393
CodecUtil.checkIndexHeader(
9494
dataIn,
95-
Composite99DocValuesFormat.DATA_CODEC_NAME,
96-
Composite99DocValuesFormat.VERSION_START,
97-
Composite99DocValuesFormat.VERSION_CURRENT,
95+
Composite912DocValuesFormat.DATA_CODEC_NAME,
96+
Composite912DocValuesFormat.VERSION_START,
97+
Composite912DocValuesFormat.VERSION_CURRENT,
9898
readState.segmentInfo.getId(),
9999
readState.segmentSuffix
100100
);
@@ -105,9 +105,9 @@ public Composite99DocValuesReader(DocValuesProducer producer, SegmentReadState r
105105
try {
106106
CodecUtil.checkIndexHeader(
107107
metaIn,
108-
Composite99DocValuesFormat.META_CODEC_NAME,
109-
Composite99DocValuesFormat.VERSION_START,
110-
Composite99DocValuesFormat.VERSION_CURRENT,
108+
Composite912DocValuesFormat.META_CODEC_NAME,
109+
Composite912DocValuesFormat.VERSION_START,
110+
Composite912DocValuesFormat.VERSION_CURRENT,
111111
readState.segmentInfo.getId(),
112112
readState.segmentSuffix
113113
);
@@ -190,12 +190,12 @@ public Composite99DocValuesReader(DocValuesProducer producer, SegmentReadState r
190190
// initialize star-tree doc values producer
191191

192192
compositeDocValuesProducer = LuceneDocValuesProducerFactory.getDocValuesProducerForCompositeCodec(
193-
Composite99Codec.COMPOSITE_INDEX_CODEC_NAME,
193+
Composite912Codec.COMPOSITE_INDEX_CODEC_NAME,
194194
this.readState,
195-
Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC,
196-
Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION,
197-
Composite99DocValuesFormat.META_DOC_VALUES_CODEC,
198-
Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION
195+
Composite912DocValuesFormat.DATA_DOC_VALUES_CODEC,
196+
Composite912DocValuesFormat.DATA_DOC_VALUES_EXTENSION,
197+
Composite912DocValuesFormat.META_DOC_VALUES_CODEC,
198+
Composite912DocValuesFormat.META_DOC_VALUES_EXTENSION
199199
);
200200

201201
} catch (Throwable t) {

server/src/main/java/org/opensearch/index/codec/composite/composite99/Composite99DocValuesWriter.java server/src/main/java/org/opensearch/index/codec/composite/composite912/Composite912DocValuesWriter.java

+20-20
Original file line numberDiff line numberDiff line change
@@ -6,7 +6,7 @@
66
* compatible open source license.
77
*/
88

9-
package org.opensearch.index.codec.composite.composite99;
9+
package org.opensearch.index.codec.composite.composite912;
1010

1111
import org.apache.lucene.codecs.CodecUtil;
1212
import org.apache.lucene.codecs.DocValuesConsumer;
@@ -55,14 +55,14 @@
5555
* @opensearch.experimental
5656
*/
5757
@ExperimentalApi
58-
public class Composite99DocValuesWriter extends DocValuesConsumer {
58+
public class Composite912DocValuesWriter extends DocValuesConsumer {
5959
private final DocValuesConsumer delegate;
6060
private final SegmentWriteState state;
6161
private final MapperService mapperService;
6262
AtomicReference<MergeState> mergeState = new AtomicReference<>();
6363
private final Set<CompositeMappedFieldType> compositeMappedFieldTypes;
6464
private final Set<String> compositeFieldSet;
65-
private DocValuesConsumer composite99DocValuesConsumer;
65+
private DocValuesConsumer compositeDocValuesConsumer;
6666

6767
public IndexOutput dataOut;
6868
public IndexOutput metaOut;
@@ -72,7 +72,7 @@ public class Composite99DocValuesWriter extends DocValuesConsumer {
7272

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

75-
public Composite99DocValuesWriter(DocValuesConsumer delegate, SegmentWriteState segmentWriteState, MapperService mapperService)
75+
public Composite912DocValuesWriter(DocValuesConsumer delegate, SegmentWriteState segmentWriteState, MapperService mapperService)
7676
throws IOException {
7777

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

104-
this.composite99DocValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec(
104+
this.compositeDocValuesConsumer = LuceneDocValuesConsumerFactory.getDocValuesConsumerForCompositeCodec(
105105
consumerWriteState,
106-
Composite99DocValuesFormat.DATA_DOC_VALUES_CODEC,
107-
Composite99DocValuesFormat.DATA_DOC_VALUES_EXTENSION,
108-
Composite99DocValuesFormat.META_DOC_VALUES_CODEC,
109-
Composite99DocValuesFormat.META_DOC_VALUES_EXTENSION
106+
Composite912DocValuesFormat.DATA_DOC_VALUES_CODEC,
107+
Composite912DocValuesFormat.DATA_DOC_VALUES_EXTENSION,
108+
Composite912DocValuesFormat.META_DOC_VALUES_CODEC,
109+
Composite912DocValuesFormat.META_DOC_VALUES_EXTENSION
110110
);
111111

112112
String dataFileName = IndexFileNames.segmentFileName(
113113
this.state.segmentInfo.name,
114114
this.state.segmentSuffix,
115-
Composite99DocValuesFormat.DATA_EXTENSION
115+
Composite912DocValuesFormat.DATA_EXTENSION
116116
);
117117
dataOut = this.state.directory.createOutput(dataFileName, this.state.context);
118118
CodecUtil.writeIndexHeader(
119119
dataOut,
120-
Composite99DocValuesFormat.DATA_CODEC_NAME,
121-
Composite99DocValuesFormat.VERSION_CURRENT,
120+
Composite912DocValuesFormat.DATA_CODEC_NAME,
121+
Composite912DocValuesFormat.VERSION_CURRENT,
122122
this.state.segmentInfo.getId(),
123123
this.state.segmentSuffix
124124
);
125125

126126
String metaFileName = IndexFileNames.segmentFileName(
127127
this.state.segmentInfo.name,
128128
this.state.segmentSuffix,
129-
Composite99DocValuesFormat.META_EXTENSION
129+
Composite912DocValuesFormat.META_EXTENSION
130130
);
131131
metaOut = this.state.directory.createOutput(metaFileName, this.state.context);
132132
CodecUtil.writeIndexHeader(
133133
metaOut,
134-
Composite99DocValuesFormat.META_CODEC_NAME,
135-
Composite99DocValuesFormat.VERSION_CURRENT,
134+
Composite912DocValuesFormat.META_CODEC_NAME,
135+
Composite912DocValuesFormat.VERSION_CURRENT,
136136
this.state.segmentInfo.getId(),
137137
this.state.segmentSuffix
138138
);
@@ -197,12 +197,12 @@ public void close() throws IOException {
197197
success = true;
198198
} finally {
199199
if (success) {
200-
IOUtils.close(dataOut, metaOut, composite99DocValuesConsumer);
200+
IOUtils.close(dataOut, metaOut, compositeDocValuesConsumer);
201201
} else {
202-
IOUtils.closeWhileHandlingException(dataOut, metaOut, composite99DocValuesConsumer);
202+
IOUtils.closeWhileHandlingException(dataOut, metaOut, compositeDocValuesConsumer);
203203
}
204204
metaOut = dataOut = null;
205-
composite99DocValuesConsumer = null;
205+
compositeDocValuesConsumer = null;
206206
}
207207
}
208208

@@ -224,7 +224,7 @@ private void createCompositeIndicesIfPossible(DocValuesProducer valuesProducer,
224224
for (CompositeMappedFieldType mappedType : compositeMappedFieldTypes) {
225225
if (mappedType instanceof StarTreeMapper.StarTreeFieldType) {
226226
try (StarTreesBuilder starTreesBuilder = new StarTreesBuilder(state, mapperService, fieldNumberAcrossCompositeFields)) {
227-
starTreesBuilder.build(metaOut, dataOut, fieldProducerMap, composite99DocValuesConsumer);
227+
starTreesBuilder.build(metaOut, dataOut, fieldProducerMap, compositeDocValuesConsumer);
228228
}
229229
}
230230
}
@@ -313,7 +313,7 @@ private void mergeStarTreeFields(MergeState mergeState) throws IOException {
313313
}
314314
}
315315
try (StarTreesBuilder starTreesBuilder = new StarTreesBuilder(state, mapperService, fieldNumberAcrossCompositeFields)) {
316-
starTreesBuilder.buildDuringMerge(metaOut, dataOut, starTreeSubsPerField, composite99DocValuesConsumer);
316+
starTreesBuilder.buildDuringMerge(metaOut, dataOut, starTreeSubsPerField, compositeDocValuesConsumer);
317317
}
318318
}
319319

server/src/main/java/org/opensearch/index/codec/composite/composite99/Composite99Codec.java

-51
This file was deleted.

server/src/main/java/org/opensearch/index/codec/composite/composite99/package-info.java

-12
This file was deleted.

0 commit comments

Comments
 (0)