This is an automated email from the ASF dual-hosted git repository. tingchen pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/pinot.git
The following commit(s) were added to refs/heads/master by this push: new 957a1cc058 Enable `noRawDataForTextIndex` for the mutable segment when re-use mutable index is enabled (#13776) 957a1cc058 is described below commit 957a1cc05857b96eb89b7e985ffd182bebda8d25 Author: Christopher Peck <27231838+itschrisp...@users.noreply.github.com> AuthorDate: Wed Aug 28 14:12:43 2024 -0700 Enable `noRawDataForTextIndex` for the mutable segment when re-use mutable index is enabled (#13776) * Use noRawDataForTextIndex for the mutable segment when reuse is true * license --- .../indexsegment/mutable/MutableSegmentImpl.java | 23 +- .../dictionary/SameValueMutableDictionary.java | 131 +++++++++++ .../impl/forward/SameValueMutableForwardIndex.java | 241 +++++++++++++++++++++ .../segment/index/text/TextIndexConfigBuilder.java | 10 + .../converter/RealtimeSegmentConverterTest.java | 68 ++++-- .../apache/pinot/spi/config/table/FieldConfig.java | 3 + 6 files changed, 455 insertions(+), 21 deletions(-) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java index 070e7d1bd0..f37ee5b96d 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImpl.java @@ -52,6 +52,8 @@ import org.apache.pinot.segment.local.dedup.PartitionDedupMetadataManager; import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentConfig; import org.apache.pinot.segment.local.realtime.impl.RealtimeSegmentStatsHistory; import org.apache.pinot.segment.local.realtime.impl.dictionary.BaseOffHeapMutableDictionary; +import org.apache.pinot.segment.local.realtime.impl.dictionary.SameValueMutableDictionary; +import org.apache.pinot.segment.local.realtime.impl.forward.SameValueMutableForwardIndex; import org.apache.pinot.segment.local.realtime.impl.nullvalue.MutableNullValueVector; import org.apache.pinot.segment.local.segment.index.datasource.ImmutableDataSource; import org.apache.pinot.segment.local.segment.index.datasource.MutableDataSource; @@ -359,6 +361,22 @@ public class MutableSegmentImpl implements MutableSegment { String sourceColumn = columnAggregatorPair.getLeft(); ValueAggregator valueAggregator = columnAggregatorPair.getRight(); + // TODO this can be removed after forward index contents no longer depends on text index configs + // If the raw value is provided, use it for the forward/dictionary index of this column by wrapping the + // already created MutableIndex with a SameValue implementation. This optimization can only be done when + // the mutable index is being reused + Object rawValueForTextIndex = indexConfigs.getConfig(StandardIndexes.text()).getRawValueForTextIndex(); + boolean reuseMutableIndex = indexConfigs.getConfig(StandardIndexes.text()).isReuseMutableIndex(); + if (rawValueForTextIndex != null && reuseMutableIndex) { + if (dictionary == null) { + MutableIndex forwardIndex = mutableIndexes.get(StandardIndexes.forward()); + mutableIndexes.put(StandardIndexes.forward(), + new SameValueMutableForwardIndex(rawValueForTextIndex, (MutableForwardIndex) forwardIndex)); + } else { + dictionary = new SameValueMutableDictionary(rawValueForTextIndex, dictionary); + } + } + _indexContainerMap.put(column, new IndexContainer(fieldSpec, partitionFunction, partitions, new ValuesInfo(), mutableIndexes, dictionary, nullValueVector, sourceColumn, valueAggregator)); @@ -998,8 +1016,9 @@ public class MutableSegmentImpl implements MutableSegment { RealtimeSegmentStatsHistory.SegmentStats segmentStats = new RealtimeSegmentStatsHistory.SegmentStats(); for (Map.Entry<String, IndexContainer> entry : _indexContainerMap.entrySet()) { String column = entry.getKey(); - BaseOffHeapMutableDictionary dictionary = (BaseOffHeapMutableDictionary) entry.getValue()._dictionary; - if (dictionary != null) { + // Skip stat collection for SameValueMutableDictionary + if (entry.getValue()._dictionary instanceof BaseOffHeapMutableDictionary) { + BaseOffHeapMutableDictionary dictionary = (BaseOffHeapMutableDictionary) entry.getValue()._dictionary; RealtimeSegmentStatsHistory.ColumnStats columnStats = new RealtimeSegmentStatsHistory.ColumnStats(); columnStats.setCardinality(dictionary.length()); columnStats.setAvgColumnSize(dictionary.getAvgValueSize()); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/dictionary/SameValueMutableDictionary.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/dictionary/SameValueMutableDictionary.java new file mode 100644 index 0000000000..a13f466155 --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/dictionary/SameValueMutableDictionary.java @@ -0,0 +1,131 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.segment.local.realtime.impl.dictionary; + +import it.unimi.dsi.fastutil.ints.IntSet; +import java.io.IOException; +import java.math.BigDecimal; +import org.apache.pinot.segment.spi.index.mutable.MutableDictionary; +import org.apache.pinot.spi.data.FieldSpec; + +/** + * SameValueMutableDictionary is used to wrap any MutableDictionary, but store the same value. This is done to + * allow noRawDataForTextIndex config to work with mutable indexes. + */ +public class SameValueMutableDictionary implements MutableDictionary { + + private final Object _actualValue; + private final Object[] _actualValues; + private final MutableDictionary _delegate; + + public SameValueMutableDictionary(Object value, MutableDictionary delegate) { + _actualValue = value; + _actualValues = new Object[]{value}; + _delegate = delegate; + } + + public int index(Object value) { + return _delegate.index(_actualValue); + } + + public int[] index(Object[] values) { + return _delegate.index(_actualValues); + } + + @Override + public FieldSpec.DataType getValueType() { + return _delegate.getValueType(); + } + + @Override + public int length() { + return _delegate.length(); + } + + @Override + public int indexOf(String stringValue) { + return _delegate.index(stringValue); + } + + @Override + public IntSet getDictIdsInRange(String lower, String upper, boolean includeLower, boolean includeUpper) { + return _delegate.getDictIdsInRange(lower, upper, includeLower, includeUpper); + } + + @Override + public int compare(int dictId1, int dictId2) { + return _delegate.compare(dictId1, dictId2); + } + + @Override + public String getMinVal() { + return _actualValue.toString(); + } + + @Override + public String getMaxVal() { + return _actualValue.toString(); + } + + @Override + public Object getSortedValues() { + return _delegate.getSortedValues(); + } + + @Override + public Object get(int dictId) { + return _actualValue; + } + + @Override + public int getIntValue(int dictId) { + return Integer.parseInt(_actualValue.toString()); + } + + @Override + public long getLongValue(int dictId) { + return Long.parseLong(_actualValue.toString()); + } + + @Override + public float getFloatValue(int dictId) { + return Float.parseFloat(_actualValue.toString()); + } + + @Override + public double getDoubleValue(int dictId) { + return Double.parseDouble(_actualValue.toString()); + } + + @Override + public BigDecimal getBigDecimalValue(int dictId) { + return BigDecimal.valueOf(getDoubleValue(dictId)); + } + + @Override + public String getStringValue(int dictId) { + return _actualValue.toString(); + } + + @Override + public void close() + throws IOException { + _delegate.close(); + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/SameValueMutableForwardIndex.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/SameValueMutableForwardIndex.java new file mode 100644 index 0000000000..efafa0d9ec --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/forward/SameValueMutableForwardIndex.java @@ -0,0 +1,241 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.segment.local.realtime.impl.forward; + +import java.io.IOException; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import org.apache.pinot.segment.spi.index.mutable.MutableForwardIndex; +import org.apache.pinot.spi.data.FieldSpec; + + +/** + * SameValueMutableForwardIndex is used to wrap any MutableForwardIndex, but store the same value. This is done to + * allow noRawDataForTextIndex config to work with mutable indexes. + */ +public class SameValueMutableForwardIndex implements MutableForwardIndex { + + private final Object _actualValue; + private final Object[] _actualValues; + private final MutableForwardIndex _delegate; + + public SameValueMutableForwardIndex(Object actualValue, MutableForwardIndex delegate) { + _actualValue = actualValue; + _actualValues = new Object[]{actualValue}; + _delegate = delegate; + } + + @Override + public int getLengthOfShortestElement() { + return _actualValue.toString().length(); + } + + @Override + public int getLengthOfLongestElement() { + return _actualValue.toString().length(); + } + + @Override + public boolean isDictionaryEncoded() { + return _delegate.isDictionaryEncoded(); + } + + @Override + public boolean isSingleValue() { + return _delegate.isSingleValue(); + } + + @Override + public FieldSpec.DataType getStoredType() { + return _delegate.getStoredType(); + } + + // Single-value methods + @Override + public int getDictId(int docId) { + return _delegate.getDictId(docId); + } + + @Override + public void readDictIds(int[] docIds, int length, int[] dictIdBuffer) { + _delegate.readDictIds(docIds, length, dictIdBuffer); + } + + @Override + public int getInt(int docId) { + return _delegate.getInt(docId); + } + + @Override + public long getLong(int docId) { + return _delegate.getLong(docId); + } + + @Override + public float getFloat(int docId) { + return _delegate.getFloat(docId); + } + + @Override + public double getDouble(int docId) { + return _delegate.getDouble(docId); + } + + @Override + public String getString(int docId) { + return _actualValue.toString(); + } + + @Override + public byte[] getBytes(int docId) { + return _actualValue.toString().getBytes(); + } + + @Override + public void setDictId(int docId, int dictId) { + _delegate.setDictId(docId, dictId); + } + + @Override + public void setInt(int docId, int value) { + _delegate.setInt(docId, value); + } + + @Override + public void setLong(int docId, long value) { + _delegate.setLong(docId, value); + } + + @Override + public void setFloat(int docId, float value) { + _delegate.setFloat(docId, value); + } + + @Override + public void setDouble(int docId, double value) { + _delegate.setDouble(docId, value); + } + + @Override + public void setString(int docId, String value) { + _delegate.setString(docId, value); + } + + @Override + public void setBytes(int docId, byte[] value) { + _delegate.setBytes(docId, value); + } + + // Multi-value methods + @Override + public int getDictIdMV(int docId, int[] dictIdBuffer) { + return _delegate.getDictIdMV(docId, dictIdBuffer); + } + + @Override + public int[] getDictIdMV(int docId) { + return _delegate.getDictIdMV(docId); + } + + @Override + public int[] getIntMV(int docId) { + return _delegate.getIntMV(docId); + } + + @Override + public long[] getLongMV(int docId) { + return _delegate.getLongMV(docId); + } + + @Override + public float[] getFloatMV(int docId) { + return _delegate.getFloatMV(docId); + } + + @Override + public double[] getDoubleMV(int docId) { + return _delegate.getDoubleMV(docId); + } + + @Override + public String[] getStringMV(int docId) { + return _delegate.getStringMV(docId); + } + + @Override + public byte[][] getBytesMV(int docId) { + return _delegate.getBytesMV(docId); + } + + @Override + public int getNumValuesMV(int docId) { + return _delegate.getNumValuesMV(docId); + } + + @Override + public void setDictIdMV(int docId, int[] dictIds) { + _delegate.setDictIdMV(docId, dictIds); + } + + @Override + public void setIntMV(int docId, int[] values) { + _delegate.setIntMV(docId, values); + } + + @Override + public void setLongMV(int docId, long[] values) { + _delegate.setLongMV(docId, values); + } + + @Override + public void setFloatMV(int docId, float[] values) { + _delegate.setFloatMV(docId, values); + } + + @Override + public void setDoubleMV(int docId, double[] values) { + _delegate.setDoubleMV(docId, values); + } + + @Override + public void setStringMV(int docId, String[] values) { + _delegate.setStringMV(docId, values); + } + + @Override + public void setBytesMV(int docId, byte[][] values) { + _delegate.setBytesMV(docId, values); + } + + @Override + public void close() + throws IOException { + _delegate.close(); + } + + @Override + public void add(@Nonnull Object value, int dictId, int docId) { + _delegate.add(_actualValue, dictId, docId); + } + + @Override + public void add(@Nonnull Object[] values, @Nullable int[] dictIds, int docId) { + _delegate.add(_actualValues, dictIds, docId); + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/text/TextIndexConfigBuilder.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/text/TextIndexConfigBuilder.java index 99516fe511..33735fdf7a 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/text/TextIndexConfigBuilder.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/text/TextIndexConfigBuilder.java @@ -89,6 +89,16 @@ public class TextIndexConfigBuilder extends TextIndexConfig.AbstractBuilder { _luceneQueryParserClass = textIndexProperties.get(FieldConfig.TEXT_INDEX_LUCENE_QUERY_PARSER_CLASS); } + if (textIndexProperties.get(FieldConfig.TEXT_INDEX_LUCENE_REUSE_MUTABLE_INDEX) != null) { + _reuseMutableIndex = + Boolean.parseBoolean(textIndexProperties.get(FieldConfig.TEXT_INDEX_LUCENE_REUSE_MUTABLE_INDEX)); + } + + if (textIndexProperties.get(FieldConfig.TEXT_INDEX_LUCENE_NRT_CACHING_DIRECTORY_BUFFER_SIZE) != null) { + _luceneNRTCachingDirectoryMaxBufferSizeMB = + Integer.parseInt(textIndexProperties.get(FieldConfig.TEXT_INDEX_LUCENE_NRT_CACHING_DIRECTORY_BUFFER_SIZE)); + } + for (Map.Entry<String, String> entry : textIndexProperties.entrySet()) { if (entry.getKey().equalsIgnoreCase(FieldConfig.TEXT_FST_TYPE)) { _fstType = FSTType.NATIVE; diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverterTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverterTest.java index 2618930bf6..e2dd41dbb9 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverterTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/realtime/converter/RealtimeSegmentConverterTest.java @@ -449,49 +449,69 @@ public class RealtimeSegmentConverterTest { @DataProvider public static Object[][] reuseParams() { List<Boolean> enabledColumnMajorSegmentBuildParams = Arrays.asList(false, true); - List<String> sortedColumnParams = Arrays.asList(null, STRING_COLUMN1); + List<String> sortedColumnParams = Arrays.asList(null, LONG_COLUMN1); List<Boolean> reuseMutableIndex = Arrays.asList(true, false); List<Integer> luceneNRTCachingDirectoryMaxBufferSizeMB = Arrays.asList(0, 5); + List<String> rawValueForTextIndexParams = Arrays.asList(null, "n"); + List<DictionaryIndexConfig> dictionaryIndexConfigs = + Arrays.asList(DictionaryIndexConfig.DISABLED, DictionaryIndexConfig.DEFAULT); return enabledColumnMajorSegmentBuildParams.stream().flatMap(columnMajor -> sortedColumnParams.stream().flatMap( - sortedColumn -> reuseMutableIndex.stream().flatMap( - reuseIndex -> luceneNRTCachingDirectoryMaxBufferSizeMB.stream() - .map(cacheSize -> new Object[]{columnMajor, sortedColumn, reuseIndex, cacheSize})))) - .toArray(Object[][]::new); + sortedColumn -> dictionaryIndexConfigs.stream().flatMap( + dictionaryIndexConfig -> rawValueForTextIndexParams.stream().flatMap( + rawValueForTextIndex -> reuseMutableIndex.stream().flatMap( + reuseIndex -> luceneNRTCachingDirectoryMaxBufferSizeMB.stream().map(cacheSize -> new Object[]{ + columnMajor, sortedColumn, reuseIndex, cacheSize, rawValueForTextIndex, dictionaryIndexConfig + })))))).toArray(Object[][]::new); } // Test the realtime segment conversion of a table with an index that reuses mutable index artifacts during conversion @Test(dataProvider = "reuseParams") public void testSegmentBuilderWithReuse(boolean columnMajorSegmentBuilder, String sortedColumn, - boolean reuseMutableIndex, int luceneNRTCachingDirectoryMaxBufferSizeMB) + boolean reuseMutableIndex, int luceneNRTCachingDirectoryMaxBufferSizeMB, String rawValueForTextIndex, + DictionaryIndexConfig dictionaryIndexConfig) throws Exception { - File tmpDir = new File(TMP_DIR, "tmp_" + System.currentTimeMillis()); + File tmpDir = new File(TMP_DIR, "tmp_" + System.nanoTime()); + + Map<String, String> fieldConfigColumnProperties = new HashMap<>(); + fieldConfigColumnProperties.put(FieldConfig.TEXT_INDEX_LUCENE_REUSE_MUTABLE_INDEX, + String.valueOf(reuseMutableIndex)); + fieldConfigColumnProperties.put(FieldConfig.TEXT_INDEX_USE_AND_FOR_MULTI_TERM_QUERIES, "true"); + if (rawValueForTextIndex != null) { + fieldConfigColumnProperties.put(FieldConfig.TEXT_INDEX_RAW_VALUE, rawValueForTextIndex); + } FieldConfig textIndexFieldConfig = new FieldConfig.Builder(STRING_COLUMN1).withEncodingType(FieldConfig.EncodingType.RAW) - .withIndexTypes(Collections.singletonList(FieldConfig.IndexType.TEXT)).build(); + .withIndexTypes(Collections.singletonList(FieldConfig.IndexType.TEXT)) + .withProperties(fieldConfigColumnProperties).build(); List<FieldConfig> fieldConfigList = Collections.singletonList(textIndexFieldConfig); TableConfig tableConfig = new TableConfigBuilder(TableType.REALTIME).setTableName("testTable").setTimeColumnName(DATE_TIME_COLUMN) - .setInvertedIndexColumns(Lists.newArrayList(STRING_COLUMN1)) - .setSortedColumn(sortedColumn).setColumnMajorSegmentBuilderEnabled(columnMajorSegmentBuilder) - .setFieldConfigList(fieldConfigList).build(); + .setInvertedIndexColumns(Lists.newArrayList(LONG_COLUMN1)) + .setNoDictionaryColumns(Lists.newArrayList(STRING_COLUMN1)) + .setSortedColumn(sortedColumn) + .setColumnMajorSegmentBuilderEnabled(columnMajorSegmentBuilder) + .setFieldConfigList(fieldConfigList) + .build(); Schema schema = new Schema.SchemaBuilder().addSingleValueDimension(STRING_COLUMN1, FieldSpec.DataType.STRING) - .addDateTime(DATE_TIME_COLUMN, FieldSpec.DataType.LONG, "1:MILLISECONDS:EPOCH", "1:MILLISECONDS").build(); + .addSingleValueDimension(LONG_COLUMN1, FieldSpec.DataType.LONG) + .addDateTime(DATE_TIME_COLUMN, FieldSpec.DataType.LONG, "1:MILLISECONDS:EPOCH", "1:MILLISECONDS") + .build(); String tableNameWithType = tableConfig.getTableName(); String segmentName = "testTable__0__0__123456"; IndexingConfig indexingConfig = tableConfig.getIndexingConfig(); - TextIndexConfig textIndexConfig = new TextIndexConfigBuilder() - .withUseANDForMultiTermQueries(false) - .withReuseMutableIndex(reuseMutableIndex) + TextIndexConfig textIndexConfig = + new TextIndexConfigBuilder().withUseANDForMultiTermQueries(false).withReuseMutableIndex(reuseMutableIndex) .withLuceneNRTCachingDirectoryMaxBufferSizeMB(luceneNRTCachingDirectoryMaxBufferSizeMB) - .build(); + .withRawValueForTextIndex(rawValueForTextIndex).build(); RealtimeSegmentConfig.Builder realtimeSegmentConfigBuilder = new RealtimeSegmentConfig.Builder().setTableNameWithType(tableNameWithType).setSegmentName(segmentName) .setStreamName(tableNameWithType).setSchema(schema).setTimeColumnName(DATE_TIME_COLUMN).setCapacity(1000) - .setIndex(Sets.newHashSet(STRING_COLUMN1), StandardIndexes.inverted(), IndexConfig.ENABLED) + .setIndex(Sets.newHashSet(LONG_COLUMN1), StandardIndexes.inverted(), IndexConfig.ENABLED) .setIndex(Sets.newHashSet(STRING_COLUMN1), StandardIndexes.text(), textIndexConfig) + .setIndex(Sets.newHashSet(STRING_COLUMN1), StandardIndexes.dictionary(), dictionaryIndexConfig) .setFieldConfigList(fieldConfigList).setSegmentZKMetadata(getSegmentZKMetadata(segmentName)) .setOffHeap(true).setMemoryManager(new DirectMemoryManager(segmentName)) .setStatsHistory(RealtimeSegmentStatsHistory.deserialzeFrom(new File(tmpDir, "stats"))) @@ -561,8 +581,15 @@ public class RealtimeSegmentConverterTest { } segmentFile.getRecord(docId, readRow); - assertEquals(readRow.getValue(STRING_COLUMN1), row.getValue(STRING_COLUMN1)); - assertEquals(readRow.getValue(DATE_TIME_COLUMN), row.getValue(DATE_TIME_COLUMN)); + + // if rawValueForTextIndex is set and mutable index is reused, the forward index should return the dummy value + if (rawValueForTextIndex != null && reuseMutableIndex) { + assertEquals(readRow.getValue(STRING_COLUMN1), rawValueForTextIndex); + assertEquals(readRow.getValue(DATE_TIME_COLUMN), row.getValue(DATE_TIME_COLUMN)); + } else { + assertEquals(readRow.getValue(STRING_COLUMN1), row.getValue(STRING_COLUMN1)); + assertEquals(readRow.getValue(DATE_TIME_COLUMN), row.getValue(DATE_TIME_COLUMN)); + } docId += 1; } @@ -575,6 +602,8 @@ public class RealtimeSegmentConverterTest { assertEquals(textIndexReader.getDocIds("str-8"), ImmutableRoaringBitmap.bitmapOf(7)); assertEquals(textIndexReader.getDocIds("str-4"), ImmutableRoaringBitmap.bitmapOf(3)); } + + mutableSegmentImpl.destroy(); } private List<GenericRow> generateTestData() { @@ -608,6 +637,7 @@ public class RealtimeSegmentConverterTest { GenericRow row = new GenericRow(); row.putValue(STRING_COLUMN1, "str" + (i - 8)); row.putValue(DATE_TIME_COLUMN, 1697814309L + i); + row.putValue(LONG_COLUMN1, 8L - i); rows.add(row); } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java index f6e6f3f99d..6f67ad6c67 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/FieldConfig.java @@ -68,6 +68,9 @@ public class FieldConfig extends BaseJsonConfig { public static final String DEFAULT_FORWARD_INDEX_DISABLED = Boolean.FALSE.toString(); public static final String TEXT_INDEX_ENABLE_PREFIX_SUFFIX_PHRASE_QUERIES = "enablePrefixSuffixMatchingInPhraseQueries"; + public static final String TEXT_INDEX_LUCENE_REUSE_MUTABLE_INDEX = "reuseMutableIndex"; + public static final String TEXT_INDEX_LUCENE_NRT_CACHING_DIRECTORY_BUFFER_SIZE = + "luceneNRTCachingDirectoryMaxBufferSizeMB"; private final String _name; private final EncodingType _encodingType; --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org