diff --git a/pinot-core/src/test/java/org/apache/pinot/core/data/manager/TableIndexingTest.java b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/TableIndexingTest.java new file mode 100644 index 00000000000..fe4b7cc1e03 --- /dev/null +++ b/pinot-core/src/test/java/org/apache/pinot/core/data/manager/TableIndexingTest.java @@ -0,0 +1,711 @@ +/** + * 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.core.data.manager; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.JsonNodeFactory; +import com.fasterxml.jackson.databind.node.ObjectNode; +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import org.apache.commons.io.FileUtils; +import org.apache.pinot.core.data.manager.offline.ImmutableSegmentDataManager; +import org.apache.pinot.segment.local.indexsegment.immutable.ImmutableSegmentLoader; +import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; +import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; +import org.apache.pinot.segment.local.segment.readers.GenericRowRecordReader; +import org.apache.pinot.segment.local.utils.TableConfigUtils; +import org.apache.pinot.segment.spi.ImmutableSegment; +import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; +import org.apache.pinot.segment.spi.creator.SegmentVersion; +import org.apache.pinot.segment.spi.datasource.DataSource; +import org.apache.pinot.segment.spi.index.IndexService; +import org.apache.pinot.segment.spi.index.startree.StarTreeV2; +import org.apache.pinot.spi.config.table.FieldConfig; +import org.apache.pinot.spi.config.table.IndexingConfig; +import org.apache.pinot.spi.config.table.StarTreeIndexConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.config.table.TimestampConfig; +import org.apache.pinot.spi.config.table.TimestampIndexGranularity; +import org.apache.pinot.spi.data.ComplexFieldSpec; +import org.apache.pinot.spi.data.DimensionFieldSpec; +import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.FieldSpec.DataType; +import org.apache.pinot.spi.data.MetricFieldSpec; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.utils.JsonUtils; +import org.apache.pinot.spi.utils.builder.TableConfigBuilder; +import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.apache.pinot.util.TestUtils; +import org.jetbrains.annotations.NotNull; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + + +// Tests various combinations of field type, encoding, single/multi-value and index type +// and compares test case results with TableIndexingTest.csv +public class TableIndexingTest { + private static final File TEMP_DIR = new File(FileUtils.getTempDirectory(), "TableIndexingTest"); + private static final String TABLE_NAME = "mytable"; + private static final String OFFLINE_TABLE_NAME = TableNameBuilder.OFFLINE.tableNameWithType(TABLE_NAME); + public static final String COLUMN_NAME = "col"; + public static final String COLUMN_DAY_NAME = "$col$DAY"; + public static final String COLUMN_MONTH_NAME = "$col$MONTH"; + public static final String COLUMN_WEEK_NAME = "$col$WEEK"; + + private final ArrayList _schemas = new ArrayList<>(); + private TestCase[] _testCases; + private Map _testCaseMap; + private final List _allResults = new ArrayList<>(); + + @BeforeClass + public void setUp() + throws Exception { + TestUtils.ensureDirectoriesExistAndEmpty(TEMP_DIR); + createSchemas(); + createTestCases(); + readExpectedResults(); + } + + private void createTestCases() { + String[] indexTypes = { + "timestamp_index", "bloom_filter", "fst_index", "h3_index", "inverted_index", "json_index", + "native_text_index", "text_index", "range_index", "startree_index", "vector_index" + }; + + _testCases = new TestCase[_schemas.size() * indexTypes.length]; + _testCaseMap = new HashMap<>(); + + for (int i = 0; i < _schemas.size(); i++) { + for (int j = 0; j < indexTypes.length; j++) { + TestCase testCase = new TestCase(_schemas.get(i).getSchemaName(), i, indexTypes[j]); + _testCases[i * indexTypes.length + j] = testCase; + _testCaseMap.put(testCase, testCase); + } + } + } + + private void readExpectedResults() + throws IOException { + List expected = readExpectedFromFile(); + // parse csv lines, e.g. INT;sv;raw;timestamp_index;true; + for (int i = 1; i < expected.size(); i++) { + String line = expected.get(i); + if (line.isEmpty()) { + continue; + } + + int idx = line.indexOf(';'); + String dataType = line.substring(0, idx); + int cardIdx = line.indexOf(';', idx + 1); + String cardType = line.substring(idx + 1, cardIdx); + int encIdx = line.indexOf(';', cardIdx + 1); + String enc = line.substring(cardIdx + 1, encIdx); + int indexIdx = line.indexOf(';', encIdx + 1); + String indexType = line.substring(encIdx + 1, indexIdx); + int resIdx = line.indexOf(';', indexIdx + 1); + String result = line.substring(indexIdx + 1, resIdx); + String error = line.substring(resIdx + 1); + + String schemaName = enc + "_" + cardType + "_" + dataType; + TestCase testCase = _testCaseMap.get(new TestCase(schemaName, -1, indexType)); + if (testCase == null) { + throw new AssertionError("Expected testCase not found: " + testCase); + } else { + testCase._expectedSuccess = Boolean.valueOf(result); + testCase._expectedMessage = error; + } + } + } + + protected void createSchemas() { + for (DataType type : DataType.values()) { + if (type == DataType.UNKNOWN || type == DataType.LIST || type == DataType.MAP || type == DataType.STRUCT) { + continue; + } + + for (String encoding : List.of("raw", "dict")) { + if (type == DataType.BOOLEAN && "dict".equals(encoding)) { + // pinot doesn't support dictionary encoding for boolean type + continue; + } + + if (type == DataType.TIMESTAMP) { + //create separate tables for all data types + _schemas.add(new Schema.SchemaBuilder().setSchemaName(encoding + "_sv_" + type.name()) + .addDateTime(COLUMN_NAME, type, "1:MILLISECONDS:TIMESTAMP", "1:MILLISECONDS") + .build()); + + _schemas.add(new Schema.SchemaBuilder().setSchemaName(encoding + "_mv_" + type.name()) + .addDateTime(COLUMN_NAME, type, "1:MILLISECONDS:TIMESTAMP", "1:MILLISECONDS") + .build()); + } else { + _schemas.add(new Schema.SchemaBuilder().setSchemaName(encoding + "_sv_" + type.name()) + .addSingleValueDimension(COLUMN_NAME, type) + .build()); + //pinot doesn't support multi-values for big decimals, json and map + if (type != DataType.BIG_DECIMAL && type != DataType.JSON) { + _schemas.add(new Schema.SchemaBuilder().setSchemaName(encoding + "_mv_" + type.name()) + .addMultiValueDimension(COLUMN_NAME, type) + .build()); + } + } + } + } + + // add maps with all possible value data types + for (DataType type : List.of(DataType.STRING, DataType.INT, DataType.LONG, DataType.FLOAT, DataType.DOUBLE)) { + for (String encoding : List.of("raw", "dict")) { + Map children = new HashMap<>(); + children.put("key", new DimensionFieldSpec("key", DataType.STRING, true)); + children.put("value", + type == DataType.STRING ? new DimensionFieldSpec("value", type, true) : new MetricFieldSpec("value", type)); + + _schemas.add(new Schema.SchemaBuilder().setSchemaName(encoding + "_map_" + type.name()) + .addComplex(COLUMN_NAME, DataType.MAP, children) + .build()); + } + } + } + + static class TestCase { + String _schemaName; + String _indexType; + + int _schemaIndex; + Throwable _error; + + String _expectedMessage; + Boolean _expectedSuccess; + + public TestCase(String schemaName, int schemaIndex, String indexType) { + _schemaName = schemaName; + _indexType = indexType; + _schemaIndex = schemaIndex; + } + + @Override + public String toString() { + return _schemaName + "," + _indexType; + } + + @Override + public int hashCode() { + return 31 * _schemaName.hashCode() + _indexType.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (!(obj instanceof TestCase)) { + return false; + } + + TestCase other = (TestCase) obj; + return _schemaName.equals(other._schemaName) && _indexType.equals(other._indexType); + } + + String getErrorMessage() { + if (_error == null) { + return null; + } else { + return _error.getMessage().replaceAll("\n", " "); + } + } + } + + @Test(dataProvider = "fieldsAndIndexTypes") + public void testAddIndex(TestCase testCase) { + try { + // create schema copy to avoid side effects between test cases + // e.g. timestamp index creates additional virtual columns + Schema schema = Schema.fromString(_schemas.get(testCase._schemaIndex).toPrettyJsonString()); + String indexType = testCase._indexType; + String schemaName = schema.getSchemaName(); + + FieldSpec field = schema.getFieldSpecFor(COLUMN_NAME); + + TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE).setTableName(schema.getSchemaName()) + .setFieldConfigList(new ArrayList<>()) + .build(); + IndexingConfig idxCfg = tableConfig.getIndexingConfig(); + + FieldConfig.EncodingType encoding = + schemaName.startsWith("raw") ? FieldConfig.EncodingType.RAW : FieldConfig.EncodingType.DICTIONARY; + + List indexTypes = new ArrayList<>(); + Map properties = new HashMap<>(); + ObjectNode indexes = new ObjectNode(JsonNodeFactory.instance); + TimestampConfig tstmpConfig = null; + FieldConfig config = null; // table will be created from scratch for each run; + + switch (indexType) { + case "bloom_filter": + /* bloom filter. Maybe we should call it bloom filter index to be consistent ? + { + "tableName": "somePinotTable", + "fieldConfigList": [ + { + "name": "playerID", + "indexes": { + "bloom": {} + } + }, + ... + ], + ... + } */ + // no params + indexes.put("bloom", JsonUtils.newObjectNode()); + + break; + case "fst_index": + /* fst index / text index + "fieldConfigList":[ + { + "name":"text_col_1", + "encodingType":"DICTIONARY", + "indexType":"FST" + } + ] + */ + indexTypes.add(FieldConfig.IndexType.FST); + break; + case "h3_index": + /* geospatial - requires dictionary be disabled + { + "fieldConfigList": [ + { + "name": "location_st_point", + "encodingType":"RAW", // this actually disables the dictionary + "indexes": { + "h3": { + "resolutions": [13, 5, 6] + } + } + } + ], + ... + } + */ + JsonNode resolutions = JsonUtils.stringToJsonNode("{\"resolutions\": [13, 5, 6]}"); + indexes.put("h3", resolutions); + break; + case "inverted_index": + /* inverted index (bitmap or sorted). requires dictionary + -> new: + { + "fieldConfigList": [ + { + "name": "theColumnName", + "indexes": { + "inverted": {} + } + } + ], + } + old: + -> "tableIndexConfig": { "invertedIndexColumns": ["uuid"], */ + // no params, has to be dictionary + indexes.put("inverted", new ObjectNode(JsonNodeFactory.instance)); + break; + case "json_index": + /* json index (string or json column), should be no-dictionary + { + "fieldConfigList": [ + { + "name": "person", + "indexes": { + "json": {} + } + } + ], + ... + } */ + // no params, should be no dictionary, only string or json + indexes.put("json", new ObjectNode(JsonNodeFactory.instance)); + break; + case "native_text_index": + /* native text index + "fieldConfigList":[ + { + "name":"text_col_1", + "encodingType":"RAW", + "indexTypes": ["TEXT"], + "properties":{"fstType":"native"} + } + ] */ + indexTypes.add(FieldConfig.IndexType.TEXT); + properties.put("fstType", "native"); + break; + case "text_index": + /* text index + "fieldConfigList":[ + { + "name":"text_col_1", + "encodingType":"RAW", + "indexTypes":["TEXT"] + } + ] */ + indexTypes.add(FieldConfig.IndexType.TEXT); + break; + case "range_index": + /* range index (supported for dictionary encoded columns of any type as well as raw encoded columns + of a numeric type) + { + "tableIndexConfig": { + "rangeIndexColumns": [ + "column_name", + ... + ], + ... + } + } + */ + if (idxCfg.getRangeIndexColumns() == null) { + idxCfg.setRangeIndexColumns(new ArrayList<>()); + } + idxCfg.getRangeIndexColumns().add(field.getName()); + break; + case "startree_index": + /* star tree + "tableIndexConfig": { + "starTreeIndexConfigs": [{ + "dimensionsSplitOrder": [ + "Country", + "Browser", + "Locale" + ], + "skipStarNodeCreationForDimensions": [ + ], + "functionColumnPairs": [ + "SUM__Impressions" + ], + "maxLeafRecords": 1 + }], + ... + } + */ + if (idxCfg.getStarTreeIndexConfigs() == null) { + idxCfg.setStarTreeIndexConfigs(new ArrayList<>()); + } + StarTreeIndexConfig stIdxCfg = + new StarTreeIndexConfig(List.of(COLUMN_NAME), Collections.emptyList(), List.of("SUM__col"), + Collections.emptyList(), 1); + idxCfg.getStarTreeIndexConfigs().add(stIdxCfg); + + break; + case "timestamp_index": + /* timestamp index + { + "fieldConfigList": [ + { + "name": "ts", + "timestampConfig": { + "granularities": [ + "DAY", + "WEEK", + "MONTH" + ] + } + } */ + tstmpConfig = new TimestampConfig( + List.of(TimestampIndexGranularity.DAY, TimestampIndexGranularity.WEEK, TimestampIndexGranularity.MONTH)); + break; + case "vector_index": + /* vector + "fieldConfigList": [ + { + "encodingType": "RAW", + "indexType": "VECTOR", + "name": "embedding", + "properties": { + "vectorIndexType": "HNSW", + "vectorDimension": 1536, + "vectorDistanceFunction": "COSINE", + "version": 1 + } + } */ + indexTypes.add(FieldConfig.IndexType.VECTOR); + properties.put("vectorIndexType", "HNSW"); + properties.put("vectorDimension", "1536"); + properties.put("vectorDistanceFunction", "COSINE"); + properties.put("version", "1"); + break; + default: + throw new IllegalArgumentException("Unexpected index type " + indexType); + } + + config = + new FieldConfig(field.getName(), encoding, null, indexTypes, null, tstmpConfig, indexes, properties, null); + + tableConfig.getFieldConfigList().add(config); + + //ImmutableSegmentDataManager segmentDataManager = + Map> indexStats = + createImmutableSegment(tableConfig, schema, indexType, generateRows(schema)); + + if ("timestamp_index".equals(indexType)) { + // this index is built on virtual columns, not on the timestamp one + Assert.assertEquals(indexStats.get(COLUMN_DAY_NAME).get("range_index"), 1); + Assert.assertEquals(indexStats.get(COLUMN_WEEK_NAME).get("range_index"), 1); + Assert.assertEquals(indexStats.get(COLUMN_MONTH_NAME).get("range_index"), 1); + } else { + String expectedType; + if ("native_text_index".equals(indexType)) { + expectedType = "text_index"; + } else { + expectedType = indexType; + } + + Assert.assertEquals(indexStats.get(COLUMN_NAME).get(expectedType), 1); + } + } catch (Throwable t) { + testCase._error = t; + //throw t; + } finally { + _allResults.add(testCase); + } + + if (testCase._expectedSuccess == null) { + throw new AssertionError("No expected status found for test case: " + testCase); + } else if (testCase._expectedSuccess && testCase._error != null) { + throw new AssertionError("Expected success for test case: " + testCase + " but got error: " + testCase._error); + } else if (!testCase._expectedSuccess && !testCase.getErrorMessage().equals(testCase._expectedMessage)) { + throw new AssertionError( + "Expected error: \"" + testCase._expectedMessage + "\" for test case: " + testCase + " but got: \"" + + testCase.getErrorMessage() + " \""); + } + } + + @AfterClass + public void printSummary() { + StringBuilder summary = generateSummary(); + if (Boolean.parseBoolean(System.getProperty("PRINT_SUMMARY"))) { + System.out.println(summary); + } + } + + private @NotNull StringBuilder generateSummary() { + StringBuilder summary = new StringBuilder(); + summary.append("data_type;cardinality;encoding;index_type;success;error\n"); + for (TestCase test : _allResults) { + String tableName = test._schemaName; + + int fst = tableName.indexOf('_'); + int sec = tableName.lastIndexOf('_'); + String encoding = tableName.substring(0, fst); + String cardinality = tableName.substring(fst + 1, sec); + String type = tableName.substring(sec + 1); + + //@formatter:off + summary.append(type).append(';') + .append(cardinality).append(';') + .append(encoding).append(';') + .append(test._indexType).append(';') + .append(test._error == null).append(';'); + //@formatter:on + if (test._error != null) { + summary.append(test._error.getMessage().replaceAll("\n", " ")); + } + summary.append('\n'); + } + return summary; + } + + private @NotNull List readExpectedFromFile() + throws IOException { + URL resource = getClass().getClassLoader().getResource("TableIndexingTest.csv"); + File expectedFile = new File(TestUtils.getFileFromResourceUrl(resource)); + return Files.readAllLines(expectedFile.toPath()); + } + + @DataProvider(name = "fieldsAndIndexTypes") + public TestCase[] getFieldsAndIndexTypes() { + return _testCases; + } + + protected static List generateRows(Schema schema) { + ArrayList rows = new ArrayList<>(); + Random random = new Random(0); + FieldSpec col = schema.getFieldSpecFor(COLUMN_NAME); + + for (int i = 0; i < 10; i++) { + GenericRow row = new GenericRow(); + row.putValue(COLUMN_NAME, getValue(col, random)); + rows.add(row); + } + + return rows; + } + + private static Object getValue(FieldSpec fieldSpec, Random r) { + if (fieldSpec.isSingleValueField()) { + switch (fieldSpec.getDataType()) { + case INT: + return r.nextInt(); + case LONG: + case TIMESTAMP: + return r.nextLong(); + case FLOAT: + return r.nextFloat(); + case DOUBLE: + case BIG_DECIMAL: + return r.nextDouble(); + case BOOLEAN: + return r.nextBoolean(); + case STRING: + return "str" + r.nextInt(); + case BYTES: + return ByteBuffer.wrap(("bytes" + r.nextInt()).getBytes()); + case JSON: + return "{ \"field\": \"" + r.nextLong() + "\" }"; + case MAP: + DataType valueType = ((ComplexFieldSpec) fieldSpec).getChildFieldSpecs().get("value").getDataType(); + Object value; + switch (valueType) { + case STRING: + value = "str" + r.nextInt(); + break; + case INT: + value = r.nextInt(); + break; + case LONG: + value = r.nextLong(); + break; + case FLOAT: + value = r.nextFloat(); + break; + case DOUBLE: + value = r.nextDouble(); + break; + default: + throw new IllegalArgumentException("Unexpected map value type: " + valueType); + } + return Map.of("key", value); + default: + throw new IllegalArgumentException("Unexpected data type " + fieldSpec.getDataType()); + } + } else { + switch (fieldSpec.getDataType()) { + case INT: + return List.of(r.nextInt(), r.nextInt()); + case LONG: + case TIMESTAMP: + return List.of(r.nextLong(), r.nextLong()); + case FLOAT: + return List.of(r.nextFloat(), r.nextFloat()); + case DOUBLE: + return List.of(r.nextDouble(), r.nextDouble()); + case BOOLEAN: + return List.of(r.nextBoolean(), r.nextBoolean()); + case STRING: + return List.of("str" + r.nextInt(), "str" + r.nextInt()); + case BYTES: + return List.of(ByteBuffer.wrap(("bytes" + r.nextInt()).getBytes()), + ByteBuffer.wrap(("bytes" + r.nextInt()).getBytes())); + default: + throw new IllegalArgumentException("Unexpected data type " + fieldSpec.getDataType()); + } + } + } + + private static File createSegment(TableConfig tableConfig, Schema schema, String segmentName, List rows) + throws Exception { + // load each segment in separate directory + File dataDir = new File(TEMP_DIR, OFFLINE_TABLE_NAME + "_" + schema.getSchemaName()); + SegmentGeneratorConfig config = new SegmentGeneratorConfig(tableConfig, schema); + config.setOutDir(dataDir.getAbsolutePath()); + config.setSegmentName(segmentName); + config.setSegmentVersion(SegmentVersion.v3); + + SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl(); + driver.init(config, new GenericRowRecordReader(rows)); + driver.build(); + return new File(dataDir, segmentName); + } + + private static Map> createImmutableSegment(TableConfig tableConfig, Schema schema, + String segmentName, List rows) + throws Exception { + // validate here to get better error messages (segment creation doesn't check everything ) + TableConfigUtils.validate(tableConfig, schema); + ImmutableSegmentDataManager segmentDataManager = mock(ImmutableSegmentDataManager.class); + when(segmentDataManager.getSegmentName()).thenReturn(segmentName); + File indexDir = createSegment(tableConfig, schema, segmentName, rows); + + IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(tableConfig, schema); + ImmutableSegment segment = ImmutableSegmentLoader.load(indexDir, indexLoadingConfig); + + Map> map = new HashMap<>(); + addColumnIndexStats(segment, COLUMN_NAME, map); + + if (tableConfig.getFieldConfigList().get(0).getTimestampConfig() != null) { + addColumnIndexStats(segment, COLUMN_DAY_NAME, map); + addColumnIndexStats(segment, COLUMN_MONTH_NAME, map); + addColumnIndexStats(segment, COLUMN_WEEK_NAME, map); + } + + segment.destroy(); + + return map; + } + + private static void addColumnIndexStats(ImmutableSegment immutableSegment, String columnName, + Map> map) { + map.put(columnName, getColumnIndexStats(immutableSegment, columnName)); + } + + private static Map getColumnIndexStats(ImmutableSegment segment, String columnName) { + DataSource colDataSource = segment.getDataSource(columnName); + HashMap stats = new HashMap<>(); + IndexService.getInstance().getAllIndexes().forEach(idxType -> { + int count = colDataSource.getIndex(idxType) != null ? 1 : 0; + stats.merge(idxType.getId(), count, Integer::sum); + }); + int starTrees = 0; + if (segment.getStarTrees() != null) { + for (StarTreeV2 stree : segment.getStarTrees()) { + if (stree.getMetadata().getDimensionsSplitOrder().contains(columnName)) { + starTrees++; + } + } + } + stats.put("startree_index", starTrees); + return stats; + } +} diff --git a/pinot-core/src/test/resources/TableIndexingTest.csv b/pinot-core/src/test/resources/TableIndexingTest.csv new file mode 100644 index 00000000000..0c3891be1ce --- /dev/null +++ b/pinot-core/src/test/resources/TableIndexingTest.csv @@ -0,0 +1,485 @@ +data_type;cardinality;encoding;index_type;success;error +INT;sv;raw;timestamp_index;true; +INT;sv;raw;bloom_filter;true; +INT;sv;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +INT;sv;raw;h3_index;false;H3 index is currently only supported on BYTES columns +INT;sv;raw;inverted_index;false;Cannot create inverted index for raw index column: col +INT;sv;raw;json_index;false;Json index is currently only supported on STRING columns +INT;sv;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +INT;sv;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +INT;sv;raw;range_index;true; +INT;sv;raw;startree_index;false;Dimension: col does not have dictionary +INT;sv;raw;vector_index;false;Vector index is currently only supported on float array columns +INT;mv;raw;timestamp_index;true; +INT;mv;raw;bloom_filter;true; +INT;mv;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +INT;mv;raw;h3_index;false;H3 index is currently only supported on single-value columns +INT;mv;raw;inverted_index;false;Cannot create inverted index for raw index column: col +INT;mv;raw;json_index;false;Json index is currently only supported on single-value columns +INT;mv;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +INT;mv;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +INT;mv;raw;range_index;true; +INT;mv;raw;startree_index;false;Column Name col defined in StarTreeIndex Config must be a single value column +INT;mv;raw;vector_index;false;Vector index is currently only supported on float array columns +INT;sv;dict;timestamp_index;true; +INT;sv;dict;bloom_filter;true; +INT;sv;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +INT;sv;dict;h3_index;false;H3 index is currently only supported on BYTES columns +INT;sv;dict;inverted_index;true; +INT;sv;dict;json_index;false;Json index is currently only supported on STRING columns +INT;sv;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +INT;sv;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +INT;sv;dict;range_index;true; +INT;sv;dict;startree_index;true; +INT;sv;dict;vector_index;false;Vector index is currently only supported on float array columns +INT;mv;dict;timestamp_index;true; +INT;mv;dict;bloom_filter;true; +INT;mv;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +INT;mv;dict;h3_index;false;H3 index is currently only supported on single-value columns +INT;mv;dict;inverted_index;true; +INT;mv;dict;json_index;false;Json index is currently only supported on single-value columns +INT;mv;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +INT;mv;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +INT;mv;dict;range_index;true; +INT;mv;dict;startree_index;false;Column Name col defined in StarTreeIndex Config must be a single value column +INT;mv;dict;vector_index;false;Vector index is currently only supported on float array columns +LONG;sv;raw;timestamp_index;true; +LONG;sv;raw;bloom_filter;true; +LONG;sv;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +LONG;sv;raw;h3_index;false;H3 index is currently only supported on BYTES columns +LONG;sv;raw;inverted_index;false;Cannot create inverted index for raw index column: col +LONG;sv;raw;json_index;false;Json index is currently only supported on STRING columns +LONG;sv;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +LONG;sv;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +LONG;sv;raw;range_index;true; +LONG;sv;raw;startree_index;false;Dimension: col does not have dictionary +LONG;sv;raw;vector_index;false;Vector index is currently only supported on float array columns +LONG;mv;raw;timestamp_index;true; +LONG;mv;raw;bloom_filter;true; +LONG;mv;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +LONG;mv;raw;h3_index;false;H3 index is currently only supported on single-value columns +LONG;mv;raw;inverted_index;false;Cannot create inverted index for raw index column: col +LONG;mv;raw;json_index;false;Json index is currently only supported on single-value columns +LONG;mv;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +LONG;mv;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +LONG;mv;raw;range_index;true; +LONG;mv;raw;startree_index;false;Column Name col defined in StarTreeIndex Config must be a single value column +LONG;mv;raw;vector_index;false;Vector index is currently only supported on float array columns +LONG;sv;dict;timestamp_index;true; +LONG;sv;dict;bloom_filter;true; +LONG;sv;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +LONG;sv;dict;h3_index;false;H3 index is currently only supported on BYTES columns +LONG;sv;dict;inverted_index;true; +LONG;sv;dict;json_index;false;Json index is currently only supported on STRING columns +LONG;sv;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +LONG;sv;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +LONG;sv;dict;range_index;true; +LONG;sv;dict;startree_index;true; +LONG;sv;dict;vector_index;false;Vector index is currently only supported on float array columns +LONG;mv;dict;timestamp_index;true; +LONG;mv;dict;bloom_filter;true; +LONG;mv;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +LONG;mv;dict;h3_index;false;H3 index is currently only supported on single-value columns +LONG;mv;dict;inverted_index;true; +LONG;mv;dict;json_index;false;Json index is currently only supported on single-value columns +LONG;mv;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +LONG;mv;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +LONG;mv;dict;range_index;true; +LONG;mv;dict;startree_index;false;Column Name col defined in StarTreeIndex Config must be a single value column +LONG;mv;dict;vector_index;false;Vector index is currently only supported on float array columns +FLOAT;sv;raw;timestamp_index;true; +FLOAT;sv;raw;bloom_filter;true; +FLOAT;sv;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +FLOAT;sv;raw;h3_index;false;H3 index is currently only supported on BYTES columns +FLOAT;sv;raw;inverted_index;false;Cannot create inverted index for raw index column: col +FLOAT;sv;raw;json_index;false;Json index is currently only supported on STRING columns +FLOAT;sv;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +FLOAT;sv;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +FLOAT;sv;raw;range_index;true; +FLOAT;sv;raw;startree_index;false;Dimension: col does not have dictionary +FLOAT;sv;raw;vector_index;false;Vector index is currently only supported on float array columns +FLOAT;mv;raw;timestamp_index;true; +FLOAT;mv;raw;bloom_filter;true; +FLOAT;mv;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +FLOAT;mv;raw;h3_index;false;H3 index is currently only supported on single-value columns +FLOAT;mv;raw;inverted_index;false;Cannot create inverted index for raw index column: col +FLOAT;mv;raw;json_index;false;Json index is currently only supported on single-value columns +FLOAT;mv;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +FLOAT;mv;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +FLOAT;mv;raw;range_index;true; +FLOAT;mv;raw;startree_index;false;Column Name col defined in StarTreeIndex Config must be a single value column +FLOAT;mv;raw;vector_index;true; +FLOAT;sv;dict;timestamp_index;true; +FLOAT;sv;dict;bloom_filter;true; +FLOAT;sv;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +FLOAT;sv;dict;h3_index;false;H3 index is currently only supported on BYTES columns +FLOAT;sv;dict;inverted_index;true; +FLOAT;sv;dict;json_index;false;Json index is currently only supported on STRING columns +FLOAT;sv;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +FLOAT;sv;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +FLOAT;sv;dict;range_index;true; +FLOAT;sv;dict;startree_index;true; +FLOAT;sv;dict;vector_index;false;Vector index is currently only supported on float array columns +FLOAT;mv;dict;timestamp_index;true; +FLOAT;mv;dict;bloom_filter;true; +FLOAT;mv;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +FLOAT;mv;dict;h3_index;false;H3 index is currently only supported on single-value columns +FLOAT;mv;dict;inverted_index;true; +FLOAT;mv;dict;json_index;false;Json index is currently only supported on single-value columns +FLOAT;mv;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +FLOAT;mv;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +FLOAT;mv;dict;range_index;true; +FLOAT;mv;dict;startree_index;false;Column Name col defined in StarTreeIndex Config must be a single value column +FLOAT;mv;dict;vector_index;true; +DOUBLE;sv;raw;timestamp_index;true; +DOUBLE;sv;raw;bloom_filter;true; +DOUBLE;sv;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +DOUBLE;sv;raw;h3_index;false;H3 index is currently only supported on BYTES columns +DOUBLE;sv;raw;inverted_index;false;Cannot create inverted index for raw index column: col +DOUBLE;sv;raw;json_index;false;Json index is currently only supported on STRING columns +DOUBLE;sv;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +DOUBLE;sv;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +DOUBLE;sv;raw;range_index;true; +DOUBLE;sv;raw;startree_index;false;Dimension: col does not have dictionary +DOUBLE;sv;raw;vector_index;false;Vector index is currently only supported on float array columns +DOUBLE;mv;raw;timestamp_index;true; +DOUBLE;mv;raw;bloom_filter;true; +DOUBLE;mv;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +DOUBLE;mv;raw;h3_index;false;H3 index is currently only supported on single-value columns +DOUBLE;mv;raw;inverted_index;false;Cannot create inverted index for raw index column: col +DOUBLE;mv;raw;json_index;false;Json index is currently only supported on single-value columns +DOUBLE;mv;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +DOUBLE;mv;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +DOUBLE;mv;raw;range_index;true; +DOUBLE;mv;raw;startree_index;false;Column Name col defined in StarTreeIndex Config must be a single value column +DOUBLE;mv;raw;vector_index;false;Vector index is currently only supported on float array columns +DOUBLE;sv;dict;timestamp_index;true; +DOUBLE;sv;dict;bloom_filter;true; +DOUBLE;sv;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +DOUBLE;sv;dict;h3_index;false;H3 index is currently only supported on BYTES columns +DOUBLE;sv;dict;inverted_index;true; +DOUBLE;sv;dict;json_index;false;Json index is currently only supported on STRING columns +DOUBLE;sv;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +DOUBLE;sv;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +DOUBLE;sv;dict;range_index;true; +DOUBLE;sv;dict;startree_index;true; +DOUBLE;sv;dict;vector_index;false;Vector index is currently only supported on float array columns +DOUBLE;mv;dict;timestamp_index;true; +DOUBLE;mv;dict;bloom_filter;true; +DOUBLE;mv;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +DOUBLE;mv;dict;h3_index;false;H3 index is currently only supported on single-value columns +DOUBLE;mv;dict;inverted_index;true; +DOUBLE;mv;dict;json_index;false;Json index is currently only supported on single-value columns +DOUBLE;mv;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +DOUBLE;mv;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +DOUBLE;mv;dict;range_index;true; +DOUBLE;mv;dict;startree_index;false;Column Name col defined in StarTreeIndex Config must be a single value column +DOUBLE;mv;dict;vector_index;false;Vector index is currently only supported on float array columns +DECIMAL;sv_BIG;raw;timestamp_index;true; +DECIMAL;sv_BIG;raw;bloom_filter;true; +DECIMAL;sv_BIG;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +DECIMAL;sv_BIG;raw;h3_index;false;H3 index is currently only supported on BYTES columns +DECIMAL;sv_BIG;raw;inverted_index;false;Cannot create inverted index for raw index column: col +DECIMAL;sv_BIG;raw;json_index;false;Json index is currently only supported on STRING columns +DECIMAL;sv_BIG;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +DECIMAL;sv_BIG;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +DECIMAL;sv_BIG;raw;range_index;false;Unsupported data type: BIG_DECIMAL +DECIMAL;sv_BIG;raw;startree_index;false;Dimension: col does not have dictionary +DECIMAL;sv_BIG;raw;vector_index;false;Vector index is currently only supported on float array columns +DECIMAL;sv_BIG;dict;timestamp_index;true; +DECIMAL;sv_BIG;dict;bloom_filter;true; +DECIMAL;sv_BIG;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +DECIMAL;sv_BIG;dict;h3_index;false;H3 index is currently only supported on BYTES columns +DECIMAL;sv_BIG;dict;inverted_index;true; +DECIMAL;sv_BIG;dict;json_index;false;Json index is currently only supported on STRING columns +DECIMAL;sv_BIG;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +DECIMAL;sv_BIG;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +DECIMAL;sv_BIG;dict;range_index;true; +DECIMAL;sv_BIG;dict;startree_index;true; +DECIMAL;sv_BIG;dict;vector_index;false;Vector index is currently only supported on float array columns +BOOLEAN;sv;raw;timestamp_index;true; +BOOLEAN;sv;raw;bloom_filter;true; +BOOLEAN;sv;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +BOOLEAN;sv;raw;h3_index;false;H3 index is currently only supported on BYTES columns +BOOLEAN;sv;raw;inverted_index;false;Cannot create inverted index for raw index column: col +BOOLEAN;sv;raw;json_index;false;Json index is currently only supported on STRING columns +BOOLEAN;sv;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +BOOLEAN;sv;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +BOOLEAN;sv;raw;range_index;false;Unsupported data type BOOLEAN for range index +BOOLEAN;sv;raw;startree_index;false;Dimension: col does not have dictionary +BOOLEAN;sv;raw;vector_index;false;Vector index is currently only supported on float array columns +BOOLEAN;mv;raw;timestamp_index;false;Caught exception while reading data +BOOLEAN;mv;raw;bloom_filter;true; +BOOLEAN;mv;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +BOOLEAN;mv;raw;h3_index;false;H3 index is currently only supported on single-value columns +BOOLEAN;mv;raw;inverted_index;false;Cannot create inverted index for raw index column: col +BOOLEAN;mv;raw;json_index;false;Json index is currently only supported on single-value columns +BOOLEAN;mv;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +BOOLEAN;mv;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +BOOLEAN;mv;raw;range_index;false;Range index is not supported for columns of data type:BOOLEAN +BOOLEAN;mv;raw;startree_index;false;Column Name col defined in StarTreeIndex Config must be a single value column +BOOLEAN;mv;raw;vector_index;false;Vector index is currently only supported on float array columns +TIMESTAMP;sv;raw;timestamp_index;true; +TIMESTAMP;sv;raw;bloom_filter;true; +TIMESTAMP;sv;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +TIMESTAMP;sv;raw;h3_index;false;H3 index is currently only supported on BYTES columns +TIMESTAMP;sv;raw;inverted_index;false;Cannot create inverted index for raw index column: col +TIMESTAMP;sv;raw;json_index;false;Json index is currently only supported on STRING columns +TIMESTAMP;sv;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +TIMESTAMP;sv;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +TIMESTAMP;sv;raw;range_index;false;Unsupported data type TIMESTAMP for range index +TIMESTAMP;sv;raw;startree_index;false;Dimension: col does not have dictionary +TIMESTAMP;sv;raw;vector_index;false;Vector index is currently only supported on float array columns +TIMESTAMP;mv;raw;timestamp_index;true; +TIMESTAMP;mv;raw;bloom_filter;true; +TIMESTAMP;mv;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +TIMESTAMP;mv;raw;h3_index;false;H3 index is currently only supported on BYTES columns +TIMESTAMP;mv;raw;inverted_index;false;Cannot create inverted index for raw index column: col +TIMESTAMP;mv;raw;json_index;false;Json index is currently only supported on STRING columns +TIMESTAMP;mv;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +TIMESTAMP;mv;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +TIMESTAMP;mv;raw;range_index;false;Unsupported data type TIMESTAMP for range index +TIMESTAMP;mv;raw;startree_index;false;Dimension: col does not have dictionary +TIMESTAMP;mv;raw;vector_index;false;Vector index is currently only supported on float array columns +TIMESTAMP;sv;dict;timestamp_index;true; +TIMESTAMP;sv;dict;bloom_filter;true; +TIMESTAMP;sv;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +TIMESTAMP;sv;dict;h3_index;false;H3 index is currently only supported on BYTES columns +TIMESTAMP;sv;dict;inverted_index;true; +TIMESTAMP;sv;dict;json_index;false;Json index is currently only supported on STRING columns +TIMESTAMP;sv;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +TIMESTAMP;sv;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +TIMESTAMP;sv;dict;range_index;true; +TIMESTAMP;sv;dict;startree_index;true; +TIMESTAMP;sv;dict;vector_index;false;Vector index is currently only supported on float array columns +TIMESTAMP;mv;dict;timestamp_index;true; +TIMESTAMP;mv;dict;bloom_filter;true; +TIMESTAMP;mv;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +TIMESTAMP;mv;dict;h3_index;false;H3 index is currently only supported on BYTES columns +TIMESTAMP;mv;dict;inverted_index;true; +TIMESTAMP;mv;dict;json_index;false;Json index is currently only supported on STRING columns +TIMESTAMP;mv;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +TIMESTAMP;mv;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +TIMESTAMP;mv;dict;range_index;true; +TIMESTAMP;mv;dict;startree_index;true; +TIMESTAMP;mv;dict;vector_index;false;Vector index is currently only supported on float array columns +STRING;sv;raw;timestamp_index;false;Caught exception while reading data +STRING;sv;raw;bloom_filter;true; +STRING;sv;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +STRING;sv;raw;h3_index;false;H3 index is currently only supported on BYTES columns +STRING;sv;raw;inverted_index;false;Cannot create inverted index for raw index column: col +STRING;sv;raw;json_index;false;Column: col Unrecognized token 'str': was expecting (JSON String, Number, Array, Object or token 'null', 'true' or 'false') at [Source: REDACTED (`StreamReadFeature.INCLUDE_SOURCE_IN_LOCATION` disabled); line: 1, column: 15] +STRING;sv;raw;native_text_index;true; +STRING;sv;raw;text_index;true; +STRING;sv;raw;range_index;false;Unsupported data type: STRING +STRING;sv;raw;startree_index;false;Dimension: col does not have dictionary +STRING;sv;raw;vector_index;false;Vector index is currently only supported on float array columns +STRING;mv;raw;timestamp_index;false;Caught exception while reading data +STRING;mv;raw;bloom_filter;true; +STRING;mv;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +STRING;mv;raw;h3_index;false;H3 index is currently only supported on single-value columns +STRING;mv;raw;inverted_index;false;Cannot create inverted index for raw index column: col +STRING;mv;raw;json_index;false;Json index is currently only supported on single-value columns +STRING;mv;raw;native_text_index;true; +STRING;mv;raw;text_index;true; +STRING;mv;raw;range_index;false;Cannot get number of bytes for: STRING +STRING;mv;raw;startree_index;false;Column Name col defined in StarTreeIndex Config must be a single value column +STRING;mv;raw;vector_index;false;Vector index is currently only supported on float array columns +STRING;sv;dict;timestamp_index;false;Caught exception while reading data +STRING;sv;dict;bloom_filter;true; +STRING;sv;dict;fst_index;true; +STRING;sv;dict;h3_index;false;H3 index is currently only supported on BYTES columns +STRING;sv;dict;inverted_index;true; +STRING;sv;dict;json_index;false;Column: col Unrecognized token 'str': was expecting (JSON String, Number, Array, Object or token 'null', 'true' or 'false') at [Source: REDACTED (`StreamReadFeature.INCLUDE_SOURCE_IN_LOCATION` disabled); line: 1, column: 15] +STRING;sv;dict;native_text_index;true; +STRING;sv;dict;text_index;true; +STRING;sv;dict;range_index;true; +STRING;sv;dict;startree_index;false;class java.lang.String cannot be cast to class java.lang.Number (java.lang.String and java.lang.Number are in module java.base of loader 'bootstrap') +STRING;sv;dict;vector_index;false;Vector index is currently only supported on float array columns +STRING;mv;dict;timestamp_index;false;Caught exception while reading data +STRING;mv;dict;bloom_filter;true; +STRING;mv;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +STRING;mv;dict;h3_index;false;H3 index is currently only supported on single-value columns +STRING;mv;dict;inverted_index;true; +STRING;mv;dict;json_index;false;Json index is currently only supported on single-value columns +STRING;mv;dict;native_text_index;true; +STRING;mv;dict;text_index;true; +STRING;mv;dict;range_index;true; +STRING;mv;dict;startree_index;false;Column Name col defined in StarTreeIndex Config must be a single value column +STRING;mv;dict;vector_index;false;Vector index is currently only supported on float array columns +JSON;sv;raw;timestamp_index;false;Caught exception while reading data +JSON;sv;raw;bloom_filter;true; +JSON;sv;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +JSON;sv;raw;h3_index;false;H3 index is currently only supported on BYTES columns +JSON;sv;raw;inverted_index;false;Cannot create inverted index for raw index column: col +JSON;sv;raw;json_index;true; +JSON;sv;raw;native_text_index;false;expected [1] but found [0] +JSON;sv;raw;text_index;false;expected [1] but found [0] +JSON;sv;raw;range_index;false;Unsupported data type: JSON +JSON;sv;raw;startree_index;false;Dimension: col does not have dictionary +JSON;sv;raw;vector_index;false;Vector index is currently only supported on float array columns +JSON;sv;dict;timestamp_index;false;Caught exception while reading data +JSON;sv;dict;bloom_filter;true; +JSON;sv;dict;fst_index;true; +JSON;sv;dict;h3_index;false;H3 index is currently only supported on BYTES columns +JSON;sv;dict;inverted_index;true; +JSON;sv;dict;json_index;true; +JSON;sv;dict;native_text_index;false;expected [1] but found [0] +JSON;sv;dict;text_index;false;expected [1] but found [0] +JSON;sv;dict;range_index;true; +JSON;sv;dict;startree_index;false;class java.lang.String cannot be cast to class java.lang.Number (java.lang.String and java.lang.Number are in module java.base of loader 'bootstrap') +JSON;sv;dict;vector_index;false;Vector index is currently only supported on float array columns +BYTES;sv;raw;timestamp_index;false;Caught exception while reading data +BYTES;sv;raw;bloom_filter;false;Caught exception while reading data +BYTES;sv;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +BYTES;sv;raw;h3_index;false;Caught exception while reading data +BYTES;sv;raw;inverted_index;false;Caught exception while reading data +BYTES;sv;raw;json_index;false;Caught exception while reading data +BYTES;sv;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +BYTES;sv;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +BYTES;sv;raw;range_index;false;Caught exception while reading data +BYTES;sv;raw;startree_index;false;Caught exception while reading data +BYTES;sv;raw;vector_index;false;Caught exception while reading data +BYTES;mv;raw;timestamp_index;false;Caught exception while reading data +BYTES;mv;raw;bloom_filter;false;Caught exception while reading data +BYTES;mv;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +BYTES;mv;raw;h3_index;false;Caught exception while reading data +BYTES;mv;raw;inverted_index;false;Caught exception while reading data +BYTES;mv;raw;json_index;false;Caught exception while reading data +BYTES;mv;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +BYTES;mv;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +BYTES;mv;raw;range_index;false;Caught exception while reading data +BYTES;mv;raw;startree_index;false;Column Name col defined in StarTreeIndex Config must be a single value column +BYTES;mv;raw;vector_index;false;Caught exception while reading data +BYTES;sv;dict;timestamp_index;false;Caught exception while reading data +BYTES;sv;dict;bloom_filter;false;Caught exception while reading data +BYTES;sv;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +BYTES;sv;dict;h3_index;false;Caught exception while reading data +BYTES;sv;dict;inverted_index;false;Caught exception while reading data +BYTES;sv;dict;json_index;false;Caught exception while reading data +BYTES;sv;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +BYTES;sv;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +BYTES;sv;dict;range_index;false;Caught exception while reading data +BYTES;sv;dict;startree_index;false;Caught exception while reading data +BYTES;sv;dict;vector_index;false;Caught exception while reading data +BYTES;mv;dict;timestamp_index;false;Caught exception while reading data +BYTES;mv;dict;bloom_filter;false;Caught exception while reading data +BYTES;mv;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +BYTES;mv;dict;h3_index;false;Caught exception while reading data +BYTES;mv;dict;inverted_index;false;Caught exception while reading data +BYTES;mv;dict;json_index;false;Caught exception while reading data +BYTES;mv;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +BYTES;mv;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +BYTES;mv;dict;range_index;false;Caught exception while reading data +BYTES;mv;dict;startree_index;false;Column Name col defined in StarTreeIndex Config must be a single value column +BYTES;mv;dict;vector_index;false;Caught exception while reading data +STRING;map;raw;timestamp_index;false;Caught exception while reading data +STRING;map;raw;bloom_filter;true; +STRING;map;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +STRING;map;raw;h3_index;false;H3 index is currently only supported on BYTES columns +STRING;map;raw;inverted_index;false;Cannot create inverted index for raw index column: col +STRING;map;raw;json_index;false;Json index is currently only supported on STRING columns +STRING;map;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +STRING;map;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +STRING;map;raw;range_index;false;Unsupported data type: MAP +STRING;map;raw;startree_index;false;Dimension: col does not have dictionary +STRING;map;raw;vector_index;false;Vector index is currently only supported on float array columns +STRING;map;dict;timestamp_index;false;Caught exception while reading data +STRING;map;dict;bloom_filter;true; +STRING;map;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +STRING;map;dict;h3_index;false;H3 index is currently only supported on BYTES columns +STRING;map;dict;inverted_index;false;Cannot create inverted index for raw index column: col +STRING;map;dict;json_index;false;Json index is currently only supported on STRING columns +STRING;map;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +STRING;map;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +STRING;map;dict;range_index;false;Unsupported data type: MAP +STRING;map;dict;startree_index;false;Dimension: col does not have dictionary +STRING;map;dict;vector_index;false;Vector index is currently only supported on float array columns +INT;map;raw;timestamp_index;false;Caught exception while reading data +INT;map;raw;bloom_filter;true; +INT;map;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +INT;map;raw;h3_index;false;H3 index is currently only supported on BYTES columns +INT;map;raw;inverted_index;false;Cannot create inverted index for raw index column: col +INT;map;raw;json_index;false;Json index is currently only supported on STRING columns +INT;map;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +INT;map;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +INT;map;raw;range_index;false;Unsupported data type: MAP +INT;map;raw;startree_index;false;Dimension: col does not have dictionary +INT;map;raw;vector_index;false;Vector index is currently only supported on float array columns +INT;map;dict;timestamp_index;false;Caught exception while reading data +INT;map;dict;bloom_filter;true; +INT;map;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +INT;map;dict;h3_index;false;H3 index is currently only supported on BYTES columns +INT;map;dict;inverted_index;false;Cannot create inverted index for raw index column: col +INT;map;dict;json_index;false;Json index is currently only supported on STRING columns +INT;map;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +INT;map;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +INT;map;dict;range_index;false;Unsupported data type: MAP +INT;map;dict;startree_index;false;Dimension: col does not have dictionary +INT;map;dict;vector_index;false;Vector index is currently only supported on float array columns +LONG;map;raw;timestamp_index;false;Caught exception while reading data +LONG;map;raw;bloom_filter;true; +LONG;map;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +LONG;map;raw;h3_index;false;H3 index is currently only supported on BYTES columns +LONG;map;raw;inverted_index;false;Cannot create inverted index for raw index column: col +LONG;map;raw;json_index;false;Json index is currently only supported on STRING columns +LONG;map;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +LONG;map;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +LONG;map;raw;range_index;false;Unsupported data type: MAP +LONG;map;raw;startree_index;false;Dimension: col does not have dictionary +LONG;map;raw;vector_index;false;Vector index is currently only supported on float array columns +LONG;map;dict;timestamp_index;false;Caught exception while reading data +LONG;map;dict;bloom_filter;true; +LONG;map;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +LONG;map;dict;h3_index;false;H3 index is currently only supported on BYTES columns +LONG;map;dict;inverted_index;false;Cannot create inverted index for raw index column: col +LONG;map;dict;json_index;false;Json index is currently only supported on STRING columns +LONG;map;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +LONG;map;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +LONG;map;dict;range_index;false;Unsupported data type: MAP +LONG;map;dict;startree_index;false;Dimension: col does not have dictionary +LONG;map;dict;vector_index;false;Vector index is currently only supported on float array columns +FLOAT;map;raw;timestamp_index;false;Caught exception while reading data +FLOAT;map;raw;bloom_filter;true; +FLOAT;map;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +FLOAT;map;raw;h3_index;false;H3 index is currently only supported on BYTES columns +FLOAT;map;raw;inverted_index;false;Cannot create inverted index for raw index column: col +FLOAT;map;raw;json_index;false;Json index is currently only supported on STRING columns +FLOAT;map;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +FLOAT;map;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +FLOAT;map;raw;range_index;false;Unsupported data type: MAP +FLOAT;map;raw;startree_index;false;Dimension: col does not have dictionary +FLOAT;map;raw;vector_index;false;Vector index is currently only supported on float array columns +FLOAT;map;dict;timestamp_index;false;Caught exception while reading data +FLOAT;map;dict;bloom_filter;true; +FLOAT;map;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +FLOAT;map;dict;h3_index;false;H3 index is currently only supported on BYTES columns +FLOAT;map;dict;inverted_index;false;Cannot create inverted index for raw index column: col +FLOAT;map;dict;json_index;false;Json index is currently only supported on STRING columns +FLOAT;map;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +FLOAT;map;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +FLOAT;map;dict;range_index;false;Unsupported data type: MAP +FLOAT;map;dict;startree_index;false;Dimension: col does not have dictionary +FLOAT;map;dict;vector_index;false;Vector index is currently only supported on float array columns +DOUBLE;map;raw;timestamp_index;false;Caught exception while reading data +DOUBLE;map;raw;bloom_filter;true; +DOUBLE;map;raw;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +DOUBLE;map;raw;h3_index;false;H3 index is currently only supported on BYTES columns +DOUBLE;map;raw;inverted_index;false;Cannot create inverted index for raw index column: col +DOUBLE;map;raw;json_index;false;Json index is currently only supported on STRING columns +DOUBLE;map;raw;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +DOUBLE;map;raw;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +DOUBLE;map;raw;range_index;false;Unsupported data type: MAP +DOUBLE;map;raw;startree_index;false;Dimension: col does not have dictionary +DOUBLE;map;raw;vector_index;false;Vector index is currently only supported on float array columns +DOUBLE;map;dict;timestamp_index;false;Caught exception while reading data +DOUBLE;map;dict;bloom_filter;true; +DOUBLE;map;dict;fst_index;false;Cannot create FST index on column: col, it can only be applied to dictionary encoded single value string columns +DOUBLE;map;dict;h3_index;false;H3 index is currently only supported on BYTES columns +DOUBLE;map;dict;inverted_index;false;Cannot create inverted index for raw index column: col +DOUBLE;map;dict;json_index;false;Json index is currently only supported on STRING columns +DOUBLE;map;dict;native_text_index;false;Cannot create text index on column: col, it can only be applied to string columns +DOUBLE;map;dict;text_index;false;Cannot create text index on column: col, it can only be applied to string columns +DOUBLE;map;dict;range_index;false;Unsupported data type: MAP +DOUBLE;map;dict;startree_index;false;Dimension: col does not have dictionary +DOUBLE;map;dict;vector_index;false;Vector index is currently only supported on float array columns \ No newline at end of file diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentLoader.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentLoader.java index 9bb5e84a470..9a6046623d7 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentLoader.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentLoader.java @@ -132,12 +132,17 @@ public static ImmutableSegment load(File indexDir, IndexLoadingConfig indexLoadi } String segmentName = segmentMetadata.getName(); SegmentDirectoryLoaderContext segmentLoaderContext = - new SegmentDirectoryLoaderContext.Builder().setTableConfig(indexLoadingConfig.getTableConfig()) - .setSchema(schema).setInstanceId(indexLoadingConfig.getInstanceId()) - .setTableDataDir(indexLoadingConfig.getTableDataDir()).setSegmentName(segmentName) - .setSegmentCrc(segmentMetadata.getCrc()).setSegmentTier(indexLoadingConfig.getSegmentTier()) + new SegmentDirectoryLoaderContext.Builder() + .setTableConfig(indexLoadingConfig.getTableConfig()) + .setSchema(schema) + .setInstanceId(indexLoadingConfig.getInstanceId()) + .setTableDataDir(indexLoadingConfig.getTableDataDir()) + .setSegmentName(segmentName) + .setSegmentCrc(segmentMetadata.getCrc()) + .setSegmentTier(indexLoadingConfig.getSegmentTier()) .setInstanceTierConfigs(indexLoadingConfig.getInstanceTierConfigs()) - .setSegmentDirectoryConfigs(indexLoadingConfig.getSegmentDirectoryConfigs()).build(); + .setSegmentDirectoryConfigs(indexLoadingConfig.getSegmentDirectoryConfigs()) + .build(); SegmentDirectoryLoader segmentLoader = SegmentDirectoryLoaderRegistry.getSegmentDirectoryLoader(indexLoadingConfig.getSegmentDirectoryLoader()); SegmentDirectory segmentDirectory = segmentLoader.load(indexDir.toURI(), segmentLoaderContext); @@ -273,9 +278,14 @@ private static void preprocessSegment(File indexDir, String segmentName, String throws Exception { PinotConfiguration segmentDirectoryConfigs = indexLoadingConfig.getSegmentDirectoryConfigs(); SegmentDirectoryLoaderContext segmentLoaderContext = - new SegmentDirectoryLoaderContext.Builder().setTableConfig(indexLoadingConfig.getTableConfig()) - .setSchema(schema).setInstanceId(indexLoadingConfig.getInstanceId()).setSegmentName(segmentName) - .setSegmentCrc(segmentCrc).setSegmentDirectoryConfigs(segmentDirectoryConfigs).build(); + new SegmentDirectoryLoaderContext.Builder() + .setTableConfig(indexLoadingConfig.getTableConfig()) + .setSchema(schema) + .setInstanceId(indexLoadingConfig.getInstanceId()) + .setSegmentName(segmentName) + .setSegmentCrc(segmentCrc) + .setSegmentDirectoryConfigs(segmentDirectoryConfigs) + .build(); SegmentDirectory segmentDirectory = SegmentDirectoryLoaderRegistry.getDefaultSegmentDirectoryLoader().load(indexDir.toURI(), segmentLoaderContext); try (SegmentPreProcessor preProcessor = new SegmentPreProcessor(segmentDirectory, indexLoadingConfig, schema)) {