diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datablock/BaseDataBlock.java b/pinot-common/src/main/java/org/apache/pinot/common/datablock/BaseDataBlock.java index 9b643b088dfa..c264c75f647a 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/datablock/BaseDataBlock.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/datablock/BaseDataBlock.java @@ -33,6 +33,7 @@ import org.apache.pinot.common.response.ProcessingException; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.HashUtil; +import org.apache.pinot.common.utils.MapUtils; import org.apache.pinot.common.utils.RoaringBitmapUtils; import org.apache.pinot.spi.accounting.ThreadResourceUsageProvider; import org.apache.pinot.spi.utils.BigDecimalUtils; @@ -338,6 +339,14 @@ public String[] getStringArray(int rowId, int colId) { return strings; } + @Override + public Map getMap(int rowId, int colId) { + int size = positionOffsetInVariableBufferAndGetLength(rowId, colId); + ByteBuffer buffer = _variableSizeData.slice(); + buffer.limit(size); + return MapUtils.deserializeMap(buffer); + } + @Nullable @Override public CustomObject getCustomObject(int rowId, int colId) { diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datablock/DataBlock.java b/pinot-common/src/main/java/org/apache/pinot/common/datablock/DataBlock.java index 768d6cdb885d..be34163c52d4 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/datablock/DataBlock.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/datablock/DataBlock.java @@ -75,6 +75,8 @@ byte[] toBytes() String[] getStringArray(int rowId, int colId); + Map getMap(int rowId, int colId); + CustomObject getCustomObject(int rowId, int colId); @Nullable diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTable.java b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTable.java index 890e29634942..e11b632c169c 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTable.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTable.java @@ -77,6 +77,9 @@ byte[] toBytes() String[] getStringArray(int rowId, int colId); + @Nullable + Map getMap(int rowId, int colId); + @Nullable CustomObject getCustomObject(int rowId, int colId); diff --git a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableImplV4.java b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableImplV4.java index a39a7d7e9f37..c866cc5253eb 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableImplV4.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/datatable/DataTableImplV4.java @@ -34,6 +34,7 @@ import org.apache.pinot.common.response.ProcessingException; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.HashUtil; +import org.apache.pinot.common.utils.MapUtils; import org.apache.pinot.common.utils.RoaringBitmapUtils; import org.apache.pinot.spi.accounting.ThreadResourceUsageProvider; import org.apache.pinot.spi.trace.Tracing; @@ -317,6 +318,18 @@ public String[] getStringArray(int rowId, int colId) { return strings; } + @Nullable + @Override + public Map getMap(int rowId, int colId) { + int size = positionOffsetInVariableBufferAndGetLength(rowId, colId); + if (size == 0) { + return null; + } + ByteBuffer buffer = _variableSizeData.slice(); + buffer.limit(size); + return MapUtils.deserializeMap(buffer); + } + @Nullable @Override public CustomObject getCustomObject(int rowId, int colId) { diff --git a/pinot-common/src/main/java/org/apache/pinot/common/function/TransformFunctionType.java b/pinot-common/src/main/java/org/apache/pinot/common/function/TransformFunctionType.java index 27299ac290e1..eced2a0198e9 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/function/TransformFunctionType.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/function/TransformFunctionType.java @@ -211,6 +211,12 @@ public enum TransformFunctionType { VECTOR_DIMS("vectorDims", ReturnTypes.INTEGER, OperandTypes.ARRAY), VECTOR_NORM("vectorNorm", ReturnTypes.DOUBLE, OperandTypes.ARRAY), + // MAP Functions + ITEM("item", + ReturnTypes.cascade(opBinding -> opBinding.getOperandType(0).getComponentType(), + SqlTypeTransforms.FORCE_NULLABLE), + OperandTypes.family(List.of(SqlTypeFamily.MAP, SqlTypeFamily.STRING))), + // Trigonometry SIN("sin"), COS("cos"), diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/DataSchema.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/DataSchema.java index f6c1bc398ccd..3eb3a081d39a 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/DataSchema.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/DataSchema.java @@ -206,6 +206,7 @@ public enum ColumnDataType { TIMESTAMP(LONG, NullValuePlaceHolder.LONG), STRING(NullValuePlaceHolder.STRING), JSON(STRING, NullValuePlaceHolder.STRING), + MAP(null), BYTES(NullValuePlaceHolder.INTERNAL_BYTES), OBJECT(null), INT_ARRAY(NullValuePlaceHolder.INT_ARRAY), @@ -494,6 +495,7 @@ public Serializable convertAndFormat(Object value) { return new Timestamp((long) value).toString(); case STRING: case JSON: + case MAP: return value.toString(); case BYTES: return ((ByteArray) value).toHexString(); @@ -676,6 +678,8 @@ public static ColumnDataType fromDataTypeSV(DataType dataType) { return STRING; case JSON: return JSON; + case MAP: + return MAP; case BYTES: return BYTES; case UNKNOWN: diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/MapUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/MapUtils.java new file mode 100644 index 000000000000..6c81d3e9961b --- /dev/null +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/MapUtils.java @@ -0,0 +1,105 @@ +/** + * 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.common.utils; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static java.nio.charset.StandardCharsets.UTF_8; + + +public class MapUtils { + + private static final Logger LOGGER = LoggerFactory.getLogger(MapUtils.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private MapUtils() { + } + + public static byte[] serializeMap(Map map) + throws JsonProcessingException { + int size = map.size(); + + // Directly return the size (0) for empty map + if (size == 0) { + return new byte[Integer.BYTES]; + } + + // Besides the value bytes, we store: size, length for each key, length for each value + long bufferSize = (1 + 2 * (long) size) * Integer.BYTES; + byte[][] keyBytesArray = new byte[size][]; + byte[][] valueBytesArray = new byte[size][]; + + int index = 0; + for (Map.Entry entry : map.entrySet()) { + byte[] keyBytes = entry.getKey().getBytes(UTF_8); + bufferSize += keyBytes.length; + keyBytesArray[index] = keyBytes; + byte[] valueBytes = OBJECT_MAPPER.writeValueAsBytes(entry.getValue()); + bufferSize += valueBytes.length; + valueBytesArray[index++] = valueBytes; + } + Preconditions.checkState(bufferSize <= Integer.MAX_VALUE, "Buffer size exceeds 2GB"); + byte[] bytes = new byte[(int) bufferSize]; + ByteBuffer byteBuffer = ByteBuffer.wrap(bytes); + byteBuffer.putInt(size); + for (int i = 0; i < index; i++) { + byte[] keyBytes = keyBytesArray[i]; + byteBuffer.putInt(keyBytes.length); + byteBuffer.put(keyBytes); + byte[] valueBytes = valueBytesArray[i]; + byteBuffer.putInt(valueBytes.length); + byteBuffer.put(valueBytes); + } + return bytes; + } + + public static Map deserializeMap(ByteBuffer byteBuffer) { + int size = byteBuffer.getInt(); + if (size == 0) { + return Map.of(); + } + + Map map = new java.util.HashMap<>(size); + for (int i = 0; i < size; i++) { + int keyLength = byteBuffer.getInt(); + byte[] keyBytes = new byte[keyLength]; + byteBuffer.get(keyBytes); + String key = new String(keyBytes, UTF_8); + int valueLength = byteBuffer.getInt(); + byte[] valueBytes = new byte[valueLength]; + byteBuffer.get(valueBytes); + Object value = null; + try { + value = OBJECT_MAPPER.readValue(valueBytes, Object.class); + } catch (IOException e) { + LOGGER.error("Caught exception while deserializing value for key: {}", key, e); + } + map.put(key, value); + } + return map; + } +} diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java index 67b0229c9e44..0eedef5ecbe8 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/PinotDataType.java @@ -23,6 +23,7 @@ import java.sql.Timestamp; import java.util.Base64; import java.util.Collection; +import java.util.Map; import org.apache.commons.lang3.ArrayUtils; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; import org.apache.pinot.spi.data.FieldSpec; @@ -816,6 +817,24 @@ public Object convert(Object value, PinotDataType sourceType) { } }, + MAP { + @Override + public Object convert(Object value, PinotDataType sourceType) { + switch (sourceType) { + case OBJECT: + if (value instanceof Map) { + return value; + } else { + throw new UnsupportedOperationException(String.format("Cannot convert '%s' (Class of value: '%s') to MAP", + sourceType, value.getClass())); + } + default: + throw new UnsupportedOperationException(String.format("Cannot convert '%s' (Class of value: '%s') to MAP", + sourceType, value.getClass())); + } + } + }, + BYTE_ARRAY { @Override public byte[] toBytes(Object value) { @@ -1468,6 +1487,11 @@ public static PinotDataType getPinotDataTypeForIngestion(FieldSpec fieldSpec) { return fieldSpec.isSingleValueField() ? STRING : STRING_ARRAY; case BYTES: return fieldSpec.isSingleValueField() ? BYTES : BYTES_ARRAY; + case MAP: + if (fieldSpec.isSingleValueField()) { + return MAP; + } + throw new IllegalStateException("There is no multi-value type for MAP"); default: throw new UnsupportedOperationException( "Unsupported data type: " + dataType + " in field: " + fieldSpec.getName()); diff --git a/pinot-core/src/main/java/org/apache/pinot/core/common/DataFetcher.java b/pinot-core/src/main/java/org/apache/pinot/core/common/DataFetcher.java index 6ed1822d83cb..1c4c312c7449 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/common/DataFetcher.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/common/DataFetcher.java @@ -421,6 +421,11 @@ void readStringValues(int[] docIds, int length, String[] valueBuffer) { valueBuffer[i] = BytesUtils.toHexString(_reader.getBytes(docIds[i], readerContext)); } break; + case MAP: + for (int i = 0; i < length; i++) { + valueBuffer[i] = _reader.getString(docIds[i], readerContext); + } + break; default: throw new IllegalStateException(); } diff --git a/pinot-core/src/main/java/org/apache/pinot/core/common/RowBasedBlockValueFetcher.java b/pinot-core/src/main/java/org/apache/pinot/core/common/RowBasedBlockValueFetcher.java index 21c6d34f0490..4bc3713b78e2 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/common/RowBasedBlockValueFetcher.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/common/RowBasedBlockValueFetcher.java @@ -65,6 +65,8 @@ private ValueFetcher createFetcher(BlockValSet blockValSet) { return new BigDecimalValueFetcher(blockValSet.getBigDecimalValuesSV()); case STRING: return new StringSingleValueFetcher(blockValSet.getStringValuesSV()); + case MAP: + return new StringSingleValueFetcher(blockValSet.getStringValuesSV()); case BYTES: return new BytesValueFetcher(blockValSet.getBytesValuesSV()); case UNKNOWN: diff --git a/pinot-core/src/main/java/org/apache/pinot/core/common/datablock/DataBlockBuilder.java b/pinot-core/src/main/java/org/apache/pinot/core/common/datablock/DataBlockBuilder.java index 98e2f32c4a99..5f4b900c04fe 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/common/datablock/DataBlockBuilder.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/common/datablock/DataBlockBuilder.java @@ -25,6 +25,7 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.util.List; +import java.util.Map; import javax.annotation.Nullable; import org.apache.commons.io.output.UnsynchronizedByteArrayOutputStream; import org.apache.pinot.common.CustomObject; @@ -34,6 +35,7 @@ import org.apache.pinot.common.datablock.RowDataBlock; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; +import org.apache.pinot.common.utils.MapUtils; import org.apache.pinot.common.utils.RoaringBitmapUtils; import org.apache.pinot.core.common.ObjectSerDeUtils; import org.apache.pinot.spi.utils.BigDecimalUtils; @@ -171,6 +173,11 @@ public static RowDataBlock buildFromRows(List rows, DataSchema dataSch setColumn(rowBuilder, byteBuffer, (String[]) value); break; + // Map column + case MAP: + setColumn(rowBuilder, byteBuffer, (Map) value); + break; + // Special intermediate result for aggregation function case OBJECT: setColumn(rowBuilder, byteBuffer, value); @@ -343,6 +350,18 @@ public static ColumnarDataBlock buildFromColumns(List columns, DataSch } break; + // Map column + case MAP: + for (int rowId = 0; rowId < numRows; rowId++) { + value = column[rowId]; + if (value == null) { + nullBitmaps[colId].add(rowId); + value = nullPlaceholders[colId]; + } + setColumn(columnarBuilder, byteBuffer, (Map) value); + } + break; + // Special intermediate result for aggregation function case OBJECT: for (int rowId = 0; rowId < numRows; rowId++) { @@ -413,6 +432,19 @@ private static void setColumn(DataBlockBuilder builder, ByteBuffer byteBuffer, B builder._variableSizeDataByteArrayOutputStream.write(bytes); } + private static void setColumn(DataBlockBuilder builder, ByteBuffer byteBuffer, @Nullable Map value) + throws IOException { + byteBuffer.putInt(builder._variableSizeDataByteArrayOutputStream.size()); + if (value == null) { + byteBuffer.putInt(0); + builder._variableSizeDataOutputStream.writeInt(CustomObject.NULL_TYPE_VALUE); + } else { + byte[] bytes = MapUtils.serializeMap(value); + byteBuffer.putInt(bytes.length); + builder._variableSizeDataByteArrayOutputStream.write(bytes); + } + } + // TODO: Move ser/de into AggregationFunction interface private static void setColumn(DataBlockBuilder builder, ByteBuffer byteBuffer, @Nullable Object value) throws IOException { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/common/datatable/BaseDataTableBuilder.java b/pinot-core/src/main/java/org/apache/pinot/core/common/datatable/BaseDataTableBuilder.java index c0a8ff8ea1cb..be6358139282 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/common/datatable/BaseDataTableBuilder.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/common/datatable/BaseDataTableBuilder.java @@ -23,10 +23,12 @@ import java.io.IOException; import java.math.BigDecimal; import java.nio.ByteBuffer; +import java.util.Map; import javax.annotation.Nullable; import org.apache.pinot.common.CustomObject; import org.apache.pinot.common.datatable.DataTableUtils; import org.apache.pinot.common.utils.DataSchema; +import org.apache.pinot.common.utils.MapUtils; import org.apache.pinot.core.common.ObjectSerDeUtils; import org.apache.pinot.spi.utils.BigDecimalUtils; @@ -96,6 +98,20 @@ public void setColumn(int colId, BigDecimal value) _variableSizeDataByteArrayOutputStream.write(bytes); } + @Override + public void setColumn(int colId, @Nullable Map value) + throws IOException { + _currentRowDataByteBuffer.position(_columnOffsets[colId]); + _currentRowDataByteBuffer.putInt(_variableSizeDataByteArrayOutputStream.size()); + if (value == null) { + _currentRowDataByteBuffer.putInt(0); + } else { + byte[] bytes = MapUtils.serializeMap(value); + _currentRowDataByteBuffer.putInt(bytes.length); + _variableSizeDataByteArrayOutputStream.write(bytes); + } + } + @Override public void setColumn(int colId, @Nullable Object value) throws IOException { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/common/datatable/DataTableBuilder.java b/pinot-core/src/main/java/org/apache/pinot/core/common/datatable/DataTableBuilder.java index cd4d9d300390..2e9d04ca9a1c 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/common/datatable/DataTableBuilder.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/common/datatable/DataTableBuilder.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.math.BigDecimal; +import java.util.Map; import javax.annotation.Nullable; import org.apache.pinot.common.datatable.DataTable; import org.apache.pinot.spi.annotations.InterfaceAudience; @@ -62,6 +63,9 @@ void setColumn(int colId, BigDecimal value) void setColumn(int colId, ByteArray value) throws IOException; + void setColumn(int colId, @Nullable Map value) + throws IOException; + // TODO: Move ser/de into AggregationFunction interface void setColumn(int colId, @Nullable Object value) throws IOException; diff --git a/pinot-core/src/main/java/org/apache/pinot/core/map/MapUtils.java b/pinot-core/src/main/java/org/apache/pinot/core/map/MapUtils.java new file mode 100644 index 000000000000..80c887e9bd95 --- /dev/null +++ b/pinot-core/src/main/java/org/apache/pinot/core/map/MapUtils.java @@ -0,0 +1,74 @@ +/** + * 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.map; + +import java.util.Map; +import org.apache.pinot.common.request.context.ExpressionContext; +import org.apache.pinot.segment.local.segment.index.map.MapDataSource; +import org.apache.pinot.segment.spi.IndexSegment; +import org.apache.pinot.segment.spi.datasource.DataSource; + + +public class MapUtils { + private MapUtils() { + } + + /** + * In the current model of integration between Map columns and the Pinot query engine, when an Item operation is + * applied to a map column (e.g., `myMap['foo']`) we create a new DataSource that treats that expression as if it + * were a column. In other words, the Query Engine treats a Key within a Map column just as it would a user + * defined Column. In order for this to work, we must map Item operations to unique column names and then map + * those unique column names to a Data Source. This function handles traversing a query expression, finding any + * Map Item operations, constructing the unique internal column and mapping it to the appropriate Key Data Source. + * + * @param indexSegment + * @param dataSourceMap - the Caller's mapping from column names to Data Source for that column. This function will + * add Key's to this mapping. + * @param expression - The expression to analyze for Map Item operations. + */ + public static void addMapItemOperationsToDataSourceMap(IndexSegment indexSegment, + Map dataSourceMap, ExpressionContext expression) { + if (expression.getType() == ExpressionContext.Type.FUNCTION) { + if (expression.getFunction().getFunctionName().equals("item")) { + String columnOp = expression.getFunction().getArguments().get(0).toString(); + String key = expression.getFunction().getArguments().get(1).getLiteral().getStringValue(); + + dataSourceMap.put(constructKeyDataSourceIdentifier(columnOp, key), + ((MapDataSource) indexSegment.getDataSource(columnOp)).getKeyDataSource(key)); + } else { + // Iterate over the operands and check if any of them are Map Item operations + expression.getFunction().getArguments().forEach( + arg -> addMapItemOperationsToDataSourceMap(indexSegment, dataSourceMap, arg)); + } + } + } + + /** + * Constructs the internal identifier for DataSources that represent the values of a specific key within a Map + * column. + * + * @param column + * @param key + * @return + */ + public static String constructKeyDataSourceIdentifier(String column, String key) { + return String.format("map_col__%s.%s", column, key); + } +} diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/filter/ExpressionFilterOperator.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/filter/ExpressionFilterOperator.java index 7c0fe3b829c1..4c57b5e0ae8e 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/operator/filter/ExpressionFilterOperator.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/filter/ExpressionFilterOperator.java @@ -29,6 +29,7 @@ import org.apache.pinot.common.utils.HashUtil; import org.apache.pinot.core.common.BlockDocIdSet; import org.apache.pinot.core.common.Operator; +import org.apache.pinot.core.map.MapUtils; import org.apache.pinot.core.operator.ColumnContext; import org.apache.pinot.core.operator.dociditerators.ExpressionScanDocIdIterator; import org.apache.pinot.core.operator.docidsets.ExpressionDocIdSet; @@ -66,6 +67,7 @@ public ExpressionFilterOperator(IndexSegment segment, QueryContext queryContext, _dataSourceMap.put(column, dataSource); columnContextMap.put(column, ColumnContext.fromDataSource(dataSource)); }); + MapUtils.addMapItemOperationsToDataSourceMap(segment, _dataSourceMap, lhs); _transformFunction = TransformFunctionFactory.get(lhs, columnContextMap, _queryContext); _predicateType = predicate.getType(); if (_predicateType == Predicate.Type.IS_NULL || _predicateType == Predicate.Type.IS_NOT_NULL) { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/MapItemTransformFunction.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/MapItemTransformFunction.java new file mode 100644 index 000000000000..c4b35958242e --- /dev/null +++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/MapItemTransformFunction.java @@ -0,0 +1,139 @@ +/** + * 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.operator.transform.function; + +import com.google.common.base.Preconditions; +import java.util.List; +import java.util.Map; +import org.apache.pinot.core.map.MapUtils; +import org.apache.pinot.core.operator.ColumnContext; +import org.apache.pinot.core.operator.blocks.ValueBlock; +import org.apache.pinot.core.operator.transform.TransformResultMetadata; +import org.apache.pinot.segment.local.segment.index.map.MapDataSource; +import org.apache.pinot.segment.spi.datasource.DataSource; +import org.apache.pinot.segment.spi.index.reader.Dictionary; +import org.apache.pinot.spi.data.FieldSpec; + + +/** + * Evaluates myMap['foo'] + */ +public class MapItemTransformFunction { + public static class MapItemFunction extends BaseTransformFunction { + public static final String FUNCTION_NAME = "map_item"; + String _column; + String _key; + String _keyDataSourceId; + TransformFunction _mapValue; + TransformFunction _keyValue; + Dictionary _keyDictionary; + private TransformResultMetadata _resultMetadata; + + public MapItemFunction() { + _column = null; + _key = null; + _keyDataSourceId = null; + } + + @Override + public void init(List arguments, Map columnContextMap) { + super.init(arguments, columnContextMap); + // Should be exactly 2 arguments (map value expression and key expression + if (arguments.size() != 2) { + throw new IllegalArgumentException("Exactly 1 argument is required for Vector transform function"); + } + + // Check if the second operand (the key) is a string literal, if it is then we can directly construct the + // MapDataSource which will pre-compute the Key ID. + + _mapValue = arguments.get(0); + Preconditions.checkArgument(_mapValue instanceof IdentifierTransformFunction, "Map Item: Left operand" + + "must be an identifier"); + _column = ((IdentifierTransformFunction) _mapValue).getColumnName(); + if (_column == null) { + throw new IllegalArgumentException("Map Item: left operand resolved to a null column name"); + } + + _keyValue = arguments.get(1); + Preconditions.checkArgument(_keyValue instanceof LiteralTransformFunction, "Map Item: Right operand" + + "must be a literal"); + _key = ((LiteralTransformFunction) arguments.get(1)).getStringLiteral(); + Preconditions.checkArgument(_key != null, "Map Item: Right operand" + + "must be a string literal"); + + _keyDataSourceId = MapUtils.constructKeyDataSourceIdentifier(_column, _key); + + // The metadata about the values that this operation will resolve to is determined by the type of teh data + // under they key, not by the Map column. So we need to look up the Key's Metadata. + MapDataSource mapDS = (MapDataSource) columnContextMap.get(_column).getDataSource(); + if (mapDS == null) { + // This should _always_ be a Map Data Source. + throw new RuntimeException("The left operand for a MAP ITEM operation must resolve to a Map Data Source"); + } + + DataSource keyDS = mapDS.getKeyDataSource(_key); + FieldSpec.DataType keyType = keyDS.getDataSourceMetadata().getDataType().getStoredType(); + _keyDictionary = keyDS.getDictionary(); + _resultMetadata = + new TransformResultMetadata(keyType, keyDS.getDataSourceMetadata().isSingleValue(), + _keyDictionary != null); + } + + @Override + public String getName() { + return FUNCTION_NAME; + } + + @Override + public TransformResultMetadata getResultMetadata() { + return new TransformResultMetadata(_resultMetadata.getDataType().getStoredType(), true, + _resultMetadata.hasDictionary()); + } + + @Override + public Dictionary getDictionary() { + return _keyDictionary; + } + + @Override + public int[] transformToDictIdsSV(ValueBlock valueBlock) { + return transformToIntValuesSV(valueBlock); + } + + @Override + public int[] transformToIntValuesSV(ValueBlock valueBlock) { + return valueBlock.getBlockValueSet(_keyDataSourceId).getIntValuesSV(); + } + + @Override + public long[] transformToLongValuesSV(ValueBlock valueBlock) { + return valueBlock.getBlockValueSet(_keyDataSourceId).getLongValuesSV(); + } + + @Override + public double[] transformToDoubleValuesSV(ValueBlock valueBlock) { + return valueBlock.getBlockValueSet(_keyDataSourceId).getDoubleValuesSV(); + } + + @Override + public String[] transformToStringValuesSV(ValueBlock valueBlock) { + return valueBlock.getBlockValueSet(_keyDataSourceId).getStringValuesSV(); + } + } +} diff --git a/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/TransformFunctionFactory.java b/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/TransformFunctionFactory.java index 3067cdc5f59d..71a29f227578 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/TransformFunctionFactory.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/operator/transform/function/TransformFunctionFactory.java @@ -239,7 +239,8 @@ private static Map> createRegistry() typeToImplementation.put(TransformFunctionType.L2_DISTANCE, L2DistanceTransformFunction.class); typeToImplementation.put(TransformFunctionType.VECTOR_DIMS, VectorDimsTransformFunction.class); typeToImplementation.put(TransformFunctionType.VECTOR_NORM, VectorNormTransformFunction.class); - + // Map functions + typeToImplementation.put(TransformFunctionType.ITEM, MapItemTransformFunction.MapItemFunction.class); Map> registry = new HashMap<>(HashUtil.getHashMapCapacity(typeToImplementation.size())); for (Map.Entry> entry : typeToImplementation.entrySet()) { diff --git a/pinot-core/src/main/java/org/apache/pinot/core/plan/ProjectPlanNode.java b/pinot-core/src/main/java/org/apache/pinot/core/plan/ProjectPlanNode.java index 6bc763009084..5cc0ee4684ea 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/plan/ProjectPlanNode.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/plan/ProjectPlanNode.java @@ -26,6 +26,7 @@ import javax.annotation.Nullable; import org.apache.pinot.common.request.context.ExpressionContext; import org.apache.pinot.common.utils.HashUtil; +import org.apache.pinot.core.map.MapUtils; import org.apache.pinot.core.operator.BaseProjectOperator; import org.apache.pinot.core.operator.DocIdSetOperator; import org.apache.pinot.core.operator.ProjectionOperator; @@ -67,15 +68,28 @@ public ProjectPlanNode(SegmentContext segmentContext, QueryContext queryContext, @Override public BaseProjectOperator run() { Set projectionColumns = new HashSet<>(); + boolean hasNonIdentifierExpression = false; for (ExpressionContext expression : _expressions) { expression.getColumns(projectionColumns); + if (expression.getType() != ExpressionContext.Type.IDENTIFIER) { hasNonIdentifierExpression = true; } } Map dataSourceMap = new HashMap<>(HashUtil.getHashMapCapacity(projectionColumns.size())); projectionColumns.forEach(column -> dataSourceMap.put(column, _indexSegment.getDataSource(column))); + + // TODO(ERICH): if the expression type is an item op with map col then create a MapDataSource and pass the key + for (ExpressionContext expression : _expressions) { + MapUtils.addMapItemOperationsToDataSourceMap(_indexSegment, dataSourceMap, expression); + } + + if (_queryContext.getFilter() != null && _queryContext.getFilter().getPredicate() != null) { + MapUtils.addMapItemOperationsToDataSourceMap(_indexSegment, dataSourceMap, + _queryContext.getFilter().getPredicate().getLhs()); + } + // NOTE: Skip creating DocIdSetOperator when maxDocsPerCall is 0 (for selection query with LIMIT 0) DocIdSetOperator docIdSetOperator = _maxDocsPerCall > 0 ? new DocIdSetPlanNode(_segmentContext, _queryContext, _maxDocsPerCall, diff --git a/pinot-core/src/main/java/org/apache/pinot/core/query/selection/SelectionOperatorUtils.java b/pinot-core/src/main/java/org/apache/pinot/core/query/selection/SelectionOperatorUtils.java index 9d4bc6dd8bd5..995983ed8877 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/query/selection/SelectionOperatorUtils.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/query/selection/SelectionOperatorUtils.java @@ -434,6 +434,9 @@ public static DataTable getDataTableFromRows(Collection rows, DataSche case STRING_ARRAY: dataTableBuilder.setColumn(i, (String[]) columnValue); break; + case MAP: + dataTableBuilder.setColumn(i, (String) columnValue); + break; default: throw new IllegalStateException( @@ -493,6 +496,9 @@ public static Object[] extractRowFromDataTable(DataTable dataTable, int rowId) { case UNKNOWN: row[i] = null; break; + case MAP: + row[i] = dataTable.getString(rowId, i); + break; // Multi-value column case INT_ARRAY: diff --git a/pinot-core/src/main/java/org/apache/pinot/core/util/DataBlockExtractUtils.java b/pinot-core/src/main/java/org/apache/pinot/core/util/DataBlockExtractUtils.java index 445881527788..c8136386aa42 100644 --- a/pinot-core/src/main/java/org/apache/pinot/core/util/DataBlockExtractUtils.java +++ b/pinot-core/src/main/java/org/apache/pinot/core/util/DataBlockExtractUtils.java @@ -94,6 +94,9 @@ private static Object extractValue(DataBlock dataBlock, ColumnDataType storedTyp case STRING_ARRAY: return dataBlock.getStringArray(rowId, colId); + case MAP: + return dataBlock.getMap(rowId, colId); + // Special intermediate result for aggregation function case OBJECT: return ObjectSerDeUtils.deserialize(dataBlock.getCustomObject(rowId, colId)); diff --git a/pinot-core/src/test/java/org/apache/pinot/core/common/datablock/DataBlockTestUtils.java b/pinot-core/src/test/java/org/apache/pinot/core/common/datablock/DataBlockTestUtils.java index a42a3ff4a396..4ceb24006234 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/common/datablock/DataBlockTestUtils.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/common/datablock/DataBlockTestUtils.java @@ -21,6 +21,7 @@ import java.math.BigDecimal; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Random; import org.apache.commons.lang3.RandomStringUtils; import org.apache.pinot.common.datablock.DataBlock; @@ -127,6 +128,20 @@ public static Object[] getRandomRow(DataSchema dataSchema, int nullPercentile) { } row[colId] = timestampArray; break; + case MAP: + length = RANDOM.nextInt(ARRAY_SIZE); + Map map = new java.util.HashMap<>(); + for (int i = 0; i < length; i++) { + int mapSize = RANDOM.nextInt(20); + for (int j = 0; j < mapSize; j++) { + map.put("0", RANDOM.nextInt()); + map.put("1", RANDOM.nextLong()); + map.put("2", RANDOM.nextDouble()); + map.put("3", RandomStringUtils.random(RANDOM.nextInt(20))); + } + } + row[colId] = map; + break; case UNKNOWN: row[colId] = null; break; @@ -173,6 +188,8 @@ public static Object getElement(DataBlock dataBlock, int rowId, int colId, Colum return dataBlock.getDoubleArray(rowId, colId); case STRING_ARRAY: return dataBlock.getStringArray(rowId, colId); + case MAP: + return dataBlock.getMap(rowId, colId); case UNKNOWN: return null; default: diff --git a/pinot-core/src/test/java/org/apache/pinot/core/common/datatable/DataTableSerDeTest.java b/pinot-core/src/test/java/org/apache/pinot/core/common/datatable/DataTableSerDeTest.java index a539f8d542c4..be9b5ea7dbb9 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/common/datatable/DataTableSerDeTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/common/datatable/DataTableSerDeTest.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.math.BigDecimal; import java.util.Arrays; +import java.util.Map; import java.util.Random; import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.StringUtils; @@ -68,6 +69,7 @@ public class DataTableSerDeTest { private static final int[][] BOOLEAN_ARRAYS = new int[NUM_ROWS][]; private static final long[][] TIMESTAMP_ARRAYS = new long[NUM_ROWS][]; private static final String[][] STRING_ARRAYS = new String[NUM_ROWS][]; + private static final Map[] MAPS = new Map[NUM_ROWS]; @Test(dataProvider = "versionProvider") public void testException(int dataTableVersion) @@ -434,6 +436,21 @@ private void fillDataTableWithRandomData(DataTableBuilder dataTableBuilder, STRING_ARRAYS[rowId] = stringArray; dataTableBuilder.setColumn(colId, stringArray); break; + case MAP: + length = RANDOM.nextInt(20); + Map map = new java.util.HashMap<>(); + for (int i = 0; i < length; i++) { + int mapSize = RANDOM.nextInt(20); + for (int j = 0; j < mapSize; j++) { + map.put("0", RANDOM.nextInt()); + map.put("1", RANDOM.nextLong()); + map.put("2", RANDOM.nextDouble()); + map.put("3", RandomStringUtils.random(RANDOM.nextInt(20))); + } + } + MAPS[rowId] = map; + dataTableBuilder.setColumn(colId, map); + break; case UNKNOWN: dataTableBuilder.setColumn(colId, (Object) null); break; @@ -535,6 +552,9 @@ private void verifyDataIsSame(DataTable newDataTable, DataSchema.ColumnDataType[ Assert.assertTrue(Arrays.equals(newDataTable.getStringArray(rowId, colId), STRING_ARRAYS[rowId]), ERROR_MESSAGE); break; + case MAP: + Assert.assertEquals(newDataTable.getMap(rowId, colId), MAPS[rowId], ERROR_MESSAGE); + break; case UNKNOWN: Object nulValue = newDataTable.getCustomObject(rowId, colId); Assert.assertNull(nulValue, ERROR_MESSAGE); diff --git a/pinot-core/src/test/java/org/apache/pinot/core/function/FunctionRegistryTest.java b/pinot-core/src/test/java/org/apache/pinot/core/function/FunctionRegistryTest.java index cf2fd6e8c29e..4dd7e5dd21cd 100644 --- a/pinot-core/src/test/java/org/apache/pinot/core/function/FunctionRegistryTest.java +++ b/pinot-core/src/test/java/org/apache/pinot/core/function/FunctionRegistryTest.java @@ -47,7 +47,7 @@ public class FunctionRegistryTest { TransformFunctionType.ARRAY_AVERAGE, TransformFunctionType.ARRAY_MIN, TransformFunctionType.ARRAY_MAX, TransformFunctionType.ARRAY_SUM, TransformFunctionType.VALUE_IN, TransformFunctionType.IN_ID_SET, TransformFunctionType.GROOVY, TransformFunctionType.CLP_DECODE, TransformFunctionType.CLP_ENCODED_VARS_MATCH, - TransformFunctionType.ST_POLYGON, TransformFunctionType.ST_AREA); + TransformFunctionType.ST_POLYGON, TransformFunctionType.ST_AREA, TransformFunctionType.ITEM); private static final EnumSet IGNORED_FILTER_KINDS = EnumSet.of( // Special filter functions without implementation FilterKind.TEXT_MATCH, FilterKind.TEXT_CONTAINS, FilterKind.JSON_MATCH, FilterKind.VECTOR_SIMILARITY, diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/fun/PinotOperatorTable.java b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/fun/PinotOperatorTable.java index fd6f9e6d8766..8402242d685e 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/fun/PinotOperatorTable.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/calcite/sql/fun/PinotOperatorTable.java @@ -210,7 +210,7 @@ public static PinotOperatorTable instance() { SqlStdOperatorTable.MINUTE, SqlStdOperatorTable.SECOND, - SqlStdOperatorTable.ITEM, + // SqlStdOperatorTable.ITEM, SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR, SqlStdOperatorTable.LISTAGG ); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java index ff49a36c8dc1..14a367659080 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/indexsegment/immutable/ImmutableSegmentImpl.java @@ -33,6 +33,7 @@ import org.apache.pinot.common.utils.HashUtil; import org.apache.pinot.segment.local.dedup.PartitionDedupMetadataManager; import org.apache.pinot.segment.local.segment.index.datasource.ImmutableDataSource; +import org.apache.pinot.segment.local.segment.index.map.ImmutableMapDataSource; import org.apache.pinot.segment.local.segment.readers.PinotSegmentColumnReader; import org.apache.pinot.segment.local.segment.readers.PinotSegmentRecordReader; import org.apache.pinot.segment.local.startree.v2.store.StarTreeIndexContainer; @@ -54,6 +55,7 @@ import org.apache.pinot.segment.spi.index.startree.StarTreeV2; import org.apache.pinot.segment.spi.store.SegmentDirectory; import org.apache.pinot.segment.spi.store.SegmentDirectoryPaths; +import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.readers.GenericRow; import org.roaringbitmap.buffer.ImmutableRoaringBitmap; import org.roaringbitmap.buffer.MutableRoaringBitmap; @@ -89,7 +91,12 @@ public ImmutableSegmentImpl(SegmentDirectory segmentDirectory, SegmentMetadataIm for (Map.Entry entry : segmentMetadata.getColumnMetadataMap().entrySet()) { String colName = entry.getKey(); - _dataSources.put(colName, new ImmutableDataSource(entry.getValue(), _indexContainerMap.get(colName))); + + if (entry.getValue().getDataType().getStoredType() == FieldSpec.DataType.MAP) { + _dataSources.put(colName, new ImmutableMapDataSource(entry.getValue(), _indexContainerMap.get(colName))); + } else { + _dataSources.put(colName, new ImmutableDataSource(entry.getValue(), _indexContainerMap.get(colName))); + } } } 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 847f589f2a4d..9ad06061d6ca 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 @@ -55,6 +55,7 @@ import org.apache.pinot.segment.local.segment.index.datasource.ImmutableDataSource; import org.apache.pinot.segment.local.segment.index.datasource.MutableDataSource; import org.apache.pinot.segment.local.segment.index.dictionary.DictionaryIndexType; +import org.apache.pinot.segment.local.segment.index.map.MutableMapDataSource; import org.apache.pinot.segment.local.segment.readers.PinotSegmentColumnReader; import org.apache.pinot.segment.local.segment.readers.PinotSegmentRecordReader; import org.apache.pinot.segment.local.segment.virtualcolumn.VirtualColumnContext; @@ -387,7 +388,8 @@ private boolean isNullable(FieldSpec fieldSpec) { private void addMutableIndex(Map mutableIndexes, IndexType indexType, MutableIndexContext context, FieldIndexConfigs indexConfigs) { - MutableIndex mutableIndex = indexType.createMutableIndex(context, indexConfigs.getConfig(indexType)); + C indexConfig = indexConfigs.getConfig(indexType); + MutableIndex mutableIndex = indexType.createMutableIndex(context, indexConfig); if (mutableIndex != null) { mutableIndexes.put(indexType, mutableIndex); } @@ -401,6 +403,10 @@ private void addMutableIndex(Map> getKeys() { + return _mapIndex.getKeys(); + } + + @Override + public Object getMinValueForKey(String key) { + return _mapDataSource.getKeyDataSource(key).getDataSourceMetadata().getMinValue(); + } + + @Override + public Object getMaxValueForKey(String key) { + return _mapDataSource.getKeyDataSource(key).getDataSourceMetadata().getMaxValue(); + } + + @Override + public int getLengthOfShortestElementForKey(String key) { + return ((MutableForwardIndex) (_mapDataSource.getKeyDataSource(key).getForwardIndex())) + .getLengthOfShortestElement(); + } + + @Override + public int getLengthOfLargestElementForKey(String key) { + return ((MutableForwardIndex) (_mapDataSource.getKeyDataSource(key).getForwardIndex())).getLengthOfLongestElement(); + } + + @Override + public boolean isSortedForKey(String key) { + return _mapDataSource.getKeyDataSource(key).getDataSourceMetadata().isSorted(); + } + + @Override + public int getTotalNumberOfEntriesForKey(String key) { + return _mapDataSource.getKeyDataSource(key).getDataSourceMetadata().getNumDocs(); + } + + @Override + public Object getMinValue() { + return _dataSourceMetadata.getMinValue(); + } + + @Override + public Object getMaxValue() { + return _dataSourceMetadata.getMaxValue(); + } + + @Override + public Object getUniqueValuesSet() { + return null; + } + + @Override + public int getCardinality() { + return UNKNOWN_CARDINALITY; + } + + @Override + public int getLengthOfShortestElement() { + return _mapIndex.getLengthOfShortestElement(); + } + + @Override + public int getLengthOfLargestElement() { + return _mapIndex.getLengthOfLongestElement(); + } + + @Override + public boolean isSorted() { + return false; + } + + @Override + public int getTotalNumberOfEntries() { + return _dataSourceMetadata.getNumDocs(); + } + + @Override + public int getMaxNumberOfMultiValues() { + return _dataSourceMetadata.getMaxNumValuesPerMVEntry(); + } + + @Override + public PartitionFunction getPartitionFunction() { + return _dataSourceMetadata.getPartitionFunction(); + } + + @Override + public int getNumPartitions() { + PartitionFunction partitionFunction = _dataSourceMetadata.getPartitionFunction(); + if (partitionFunction != null) { + return partitionFunction.getNumPartitions(); + } else { + return 0; + } + } + + @Override + public Map getPartitionFunctionConfig() { + PartitionFunction partitionFunction = _dataSourceMetadata.getPartitionFunction(); + return partitionFunction != null ? partitionFunction.getFunctionConfig() : null; + } + + @Override + public Set getPartitions() { + return _dataSourceMetadata.getPartitions(); + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/stats/RealtimeSegmentStatsContainer.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/stats/RealtimeSegmentStatsContainer.java index e9b76af49928..a65a81f6265d 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/stats/RealtimeSegmentStatsContainer.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/converter/stats/RealtimeSegmentStatsContainer.java @@ -21,10 +21,12 @@ import java.util.HashMap; import java.util.Map; import javax.annotation.Nullable; +import org.apache.pinot.segment.local.segment.index.map.MutableMapDataSource; import org.apache.pinot.segment.spi.MutableSegment; import org.apache.pinot.segment.spi.creator.ColumnStatistics; import org.apache.pinot.segment.spi.creator.SegmentPreIndexStatsContainer; import org.apache.pinot.segment.spi.datasource.DataSource; +import org.apache.pinot.spi.data.FieldSpec; /** @@ -40,7 +42,11 @@ public RealtimeSegmentStatsContainer(MutableSegment mutableSegment, @Nullable in // Create all column statistics for (String columnName : mutableSegment.getPhysicalColumnNames()) { DataSource dataSource = mutableSegment.getDataSource(columnName); - if (dataSource.getDictionary() != null) { + if (dataSource.getDataSourceMetadata().getDataType().getStoredType() == FieldSpec.DataType.MAP) { + // Map columns are fundamentally different than scalar columns, so we use a different stats collector + _columnStatisticsMap + .put(columnName, new MutableMapColStatistics((MutableMapDataSource) dataSource)); + } else if (dataSource.getDictionary() != null) { _columnStatisticsMap .put(columnName, new MutableColumnStatistics(mutableSegment.getDataSource(columnName), sortedDocIds)); } else { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/map/MutableMapIndexImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/map/MutableMapIndexImpl.java new file mode 100644 index 000000000000..30ab00d0e907 --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/realtime/impl/map/MutableMapIndexImpl.java @@ -0,0 +1,596 @@ +/** + * 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.map; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pinot.common.utils.PinotDataType; +import org.apache.pinot.segment.spi.ColumnMetadata; +import org.apache.pinot.segment.spi.index.FieldIndexConfigs; +import org.apache.pinot.segment.spi.index.ForwardIndexConfig; +import org.apache.pinot.segment.spi.index.IndexReader; +import org.apache.pinot.segment.spi.index.IndexType; +import org.apache.pinot.segment.spi.index.StandardIndexes; +import org.apache.pinot.segment.spi.index.mutable.MutableForwardIndex; +import org.apache.pinot.segment.spi.index.mutable.MutableIndex; +import org.apache.pinot.segment.spi.index.mutable.MutableMapIndex; +import org.apache.pinot.segment.spi.index.mutable.provider.MutableIndexContext; +import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext; +import org.apache.pinot.segment.spi.memory.PinotDataBufferMemoryManager; +import org.apache.pinot.spi.config.table.MapIndexConfig; +import org.apache.pinot.spi.data.DimensionFieldSpec; +import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.utils.ByteArray; +import org.apache.pinot.spi.utils.JsonUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.pinot.spi.data.FieldSpec.*; +import static org.apache.pinot.spi.data.FieldSpec.DataType.BYTES; + + +/** + * Dynamically typed Dense map column. This structure will allow for a "partially" dynamically typed map value + * to be created where different keys may have different types. The type of the key is determined when the + * key is first added to the index. + * + * Note, that this means that the type of a key can change across segments. + */ +public class MutableMapIndexImpl implements MutableMapIndex { + private static final Logger LOGGER = LoggerFactory.getLogger(MutableMapIndexImpl.class); + private final ConcurrentHashMap _keyIndexes; + private final ConcurrentHashMap> _minValue; + private final ConcurrentHashMap> _maxValue; + private final int _maxKeys; + private final boolean _offHeap; + private final int _capacity; + private final boolean _isDictionary; + private final PinotDataBufferMemoryManager _memoryManager; + private final File _consumerDir; + private final String _segmentName; + private final boolean _dynamicallyCreateKeys; + + public MutableMapIndexImpl(MapIndexConfig config, PinotDataBufferMemoryManager memoryManager, int capacity, + boolean offHeap, boolean isDictionary, String consumerDir, String segmentName) { + _maxKeys = config != null ? config.getMaxKeys() : 100; + _keyIndexes = new ConcurrentHashMap<>(); + _memoryManager = memoryManager; + _capacity = capacity; + _offHeap = offHeap; + _isDictionary = isDictionary; + _consumerDir = consumerDir != null ? new File(consumerDir) : null; + _segmentName = segmentName; + _minValue = new ConcurrentHashMap<>(); + _maxValue = new ConcurrentHashMap<>(); + _dynamicallyCreateKeys = config != null && config.getDynamicallyCreateDenseKeys(); + + + if (config != null && config.getDenseKeys() != null) { + if (_maxKeys < config.getDenseKeys().size()) { + throw new RuntimeException("The number of predefined keys exceeds the maximum number of keys"); + } + + for (FieldSpec keySpec : config.getDenseKeys()) { + getKeyIndex(keySpec.getName(), keySpec.getDataType().getStoredType(), 0); + } + } + LOGGER.info("Creating Mutable Map Dense Column. Max Keys: {}, Capacity: {}, offHeap: {}, " + + "isDictionary: {}, consumerDir: {}, Segment name: {}", + _maxKeys, _capacity, _offHeap, _isDictionary, _consumerDir, _segmentName); + } + + /** + * Adds a single map value to the Index. + * + * This will iterate over each Key-Value pair in value and will add the Key and Value to the index for + * docIde. When adding a Key-Value pair (K and V), this will check to see if this is the first + * time K has appeared in the index: if it is, then the type of V will be used to dynamically determine + * the type of the key. If the key is already in the index, then this will check that the type of V matches + * the already determined type for K. + * + * @param mapValue A nonnull map value to be added to the index. + * @param docId The document id of the given row. A non-negative value. + */ + @Override + public void add(Map mapValue, int docId) { + assert mapValue != null; + + if (_dynamicallyCreateKeys) { + addMissingKeys(mapValue, docId); + } + + // Iterate over the KV pairs in the document + // for (Map.Entry entry : mapValue.entrySet()) { + for (String indexKey : _keyIndexes.keySet()) { + //String key = entry.getKey(); + String key = indexKey; + //Object val = entry.getValue(); + Object val = mapValue.get(key); + if (val == null) { + val = getNullValue(_keyIndexes.get(key).getStoredType()); + } + FieldSpec.DataType valType = convertToDataType(PinotDataType.getSingleValueType(val.getClass())); + + // Get the index for the key + MutableForwardIndex keyIndex = getKeyIndex(key, valType, docId); + assert keyIndex != null; // the key should always exist because we are iterating over the key index set + + // Add the value to the index + keyIndex.add(val, -1, docId); + + Comparable comparable; + if (valType == BYTES) { + comparable = new ByteArray((byte[]) val); + } else { + comparable = (Comparable) val; + } + + if (!_minValue.containsKey(key)) { + _minValue.put(key, comparable); + } else if (comparable.compareTo(_minValue.get(key)) < 0) { + _minValue.put(key, comparable); + } + + if (!_maxValue.containsKey(key)) { + _maxValue.put(key, comparable); + } else if (comparable.compareTo(_maxValue.get(key)) > 0) { + _maxValue.put(key, comparable); + } + } + } + + private void addMissingKeys(Map mapValue, int docId) { + // Iterate over the keys in the map value and if they do not already exist then add them + for (Map.Entry entry: mapValue.entrySet()) { + if (!_keyIndexes.containsKey(entry.getKey())) { + // Add the key as an index + FieldSpec.DataType valType = convertToDataType(PinotDataType.getSingleValueType(entry.getValue().getClass())); + getKeyIndex(entry.getKey(), valType, docId); + } + } + } + + private Object getNullValue(DataType type) { + switch (type) { + case INT: + return DEFAULT_DIMENSION_NULL_VALUE_OF_INT; + case LONG: + return DEFAULT_DIMENSION_NULL_VALUE_OF_LONG; + case FLOAT: + return DEFAULT_DIMENSION_NULL_VALUE_OF_FLOAT; + case DOUBLE: + return DEFAULT_DIMENSION_NULL_VALUE_OF_DOUBLE; + case BIG_DECIMAL: + return DEFAULT_DIMENSION_NULL_VALUE_OF_BIG_DECIMAL; + case BOOLEAN: + return DEFAULT_DIMENSION_NULL_VALUE_OF_BOOLEAN; + case TIMESTAMP: + return DEFAULT_DIMENSION_NULL_VALUE_OF_TIMESTAMP; + case STRING: + return DEFAULT_DIMENSION_NULL_VALUE_OF_STRING; + case JSON: + return DEFAULT_DIMENSION_NULL_VALUE_OF_JSON; + default: + throw new UnsupportedOperationException(String.format("MAP does not support type: %s", type)); + } + } + + @Override + public void add(Map[] values, int[] docIds) { + assert values.length == docIds.length; + + for (int i = 0; i < values.length; i++) { + add(values[i], docIds[i]); + } + } + + @Override + public Comparable getMinValueForKey(String key) { + return _minValue.get(key); + } + + @Override + public Comparable getMaxValueForKey(String key) { + return _maxValue.get(key); + } + + @Override + public Set> getKeys() { + HashSet> keys = new HashSet<>(); + + for (Map.Entry key : _keyIndexes.entrySet()) { + keys.add(new ImmutablePair<>(key.getKey(), key.getValue().getStoredType())); + } + + return keys; + } + + @Override + public FieldSpec getKeySpec(String key) { + if (_keyIndexes.containsKey(key)) { + FieldSpec keySpec = new DimensionFieldSpec(); + keySpec.setDataType(getStoredType(key)); + keySpec.setSingleValueField(true); + keySpec.setName(key); + keySpec.setNullable(true); + return keySpec; + } else { + return null; + } + } + + @Override + public Map getKeyIndexes(String key) { + if (_keyIndexes.containsKey(key)) { + HashMap indexes = new HashMap<>(); + indexes.put(StandardIndexes.forward(), _keyIndexes.get(key)); + return indexes; + } else { + return null; + } + } + + private MutableForwardIndex getKeyIndex(String key, FieldSpec.DataType type, int docIdOffset) { + // Check to see if the index exists + MutableForwardIndex keyIndex = _keyIndexes.get(key); + if (keyIndex != null) { + // If it does, then check to see if the type of the index matches the type of the value that is being added + if (keyIndex.getStoredType().equals(type)) { + return keyIndex; + } else { + // If the types do not match, throw an exception, if the types do match then return the index + throw new RuntimeException( + String.format("Attempting to write a value of type %s to a key of type %s", + type.toString(), + keyIndex.getStoredType().toString())); + } + } else { + // If the key does not have an index, then create an index for the given value + MutableForwardIndex idx = createKeyIndex(key, type, docIdOffset); + if (idx != null) { + _keyIndexes.put(key, idx); + } + return idx; + } + } + + MutableForwardIndex createKeyIndex(String key, FieldSpec.DataType type, int docIdOffset) { + if (_keyIndexes.size() >= _maxKeys) { + LOGGER.warn(String.format("Maximum number of keys exceed: %d", _maxKeys)); + return null; + } + + LOGGER.info("Creating new Dense Column for key {} with type {}", key, type); + + FieldSpec fieldSpec = new DimensionFieldSpec(key, type, true); + MutableIndexContext context = + MutableIndexContext.builder().withFieldSpec(fieldSpec).withMemoryManager(_memoryManager) + .withDictionary(_isDictionary).withCapacity(_capacity).offHeap(_offHeap).withSegmentName(_segmentName) + .withConsumerDir(_consumerDir) + // TODO: judging by the MutableSegmentImpl this would be -1 but should double check + .withFixedLengthBytes(-1).build(); + FieldIndexConfigs indexConfig = FieldIndexConfigs.EMPTY; + MutableForwardIndex idx = createMutableForwardIndex(StandardIndexes.forward(), context, indexConfig); + return new DenseColumn(idx, docIdOffset); + } + + private MutableForwardIndex createMutableForwardIndex(IndexType indexType, + MutableIndexContext context, FieldIndexConfigs indexConfigs) { + return (MutableForwardIndex) indexType.createMutableIndex(context, + indexConfigs.getConfig(StandardIndexes.forward())); + } + + FieldSpec.DataType convertToDataType(PinotDataType ty) { + switch (ty) { + case BOOLEAN: + return FieldSpec.DataType.BOOLEAN; + case SHORT: + case INTEGER: + return FieldSpec.DataType.INT; + case LONG: + return FieldSpec.DataType.LONG; + case FLOAT: + return FieldSpec.DataType.FLOAT; + case DOUBLE: + return FieldSpec.DataType.DOUBLE; + case BIG_DECIMAL: + return FieldSpec.DataType.BIG_DECIMAL; + case TIMESTAMP: + return FieldSpec.DataType.TIMESTAMP; + case STRING: + return FieldSpec.DataType.STRING; + default: + throw new UnsupportedOperationException(); + } + } + + @Override + public IndexReader getKeyReader(String key, IndexType type) { + return _keyIndexes.get(key); + } + + @Override + public void close() + throws IOException { + // Iterate over each index and close them + for (MutableForwardIndex idx : _keyIndexes.values()) { + idx.close(); + } + } + + @Override + public FieldSpec.DataType getStoredType(String key) { + return _keyIndexes.get(key).getStoredType(); + } + + @Override + public ColumnMetadata getKeyMetadata(String key) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isDictionaryEncoded() { + return false; + } + + @Override + public boolean isSingleValue() { + return true; + } + + @Override + public FieldSpec.DataType getStoredType() { + return FieldSpec.DataType.MAP; + } + + @Override + public int getLengthOfShortestElement() { + return 0; + } + + @Override + public int getLengthOfLongestElement() { + return 0; + } + + @Override + public Map getMap(int docId, ForwardIndexReaderContext context) { + Map mapValue = new HashMap<>(); + + for (Map.Entry entry : _keyIndexes.entrySet()) { + String key = entry.getKey(); + MutableForwardIndex keyIndex = entry.getValue(); + + switch (keyIndex.getStoredType()) { + case INT: { + int value = keyIndex.getInt(docId, context); + mapValue.put(key, value); + break; + } + case LONG: { + long value = keyIndex.getLong(docId, context); + mapValue.put(key, value); + break; + } + case FLOAT: { + float value = keyIndex.getFloat(docId, context); + mapValue.put(key, value); + break; + } + case DOUBLE: { + double value = keyIndex.getDouble(docId, context); + mapValue.put(key, value); + break; + } + case STRING: { + String value = keyIndex.getString(docId, context); + mapValue.put(key, value); + break; + } + case BIG_DECIMAL: + case BOOLEAN: + case TIMESTAMP: + case JSON: + case BYTES: + case STRUCT: + case LIST: + case MAP: + case UNKNOWN: + default: + throw new UnsupportedOperationException(); + } + } + + return mapValue; + } + + @Override + public String getString(int docId, ForwardIndexReaderContext context) { + // TODO(ERICH): should the exceptions from getMap be caught here and Null returned or bubbled up? + Map mapValue = getMap(docId, context); + try { + return JsonUtils.objectToString(mapValue); + } catch (Exception ex) { + LOGGER.error("Failed to serialize MAP value to JSON String. Map Value: '{}'", mapValue, ex); + } + + return ""; + } + + /** + * A wrapper class around a Dense Mutable Forward index. This is necessary because a dense forward index may have + * a Doc ID Offset. + */ + private static class DenseColumn implements MutableForwardIndex { + // A key may be added to the index after the first document. In which case, when the Forward index for that key + // is created, the docIds for this index will not begin with 0, but they will be stored in the index with docId + // 0. This value will track the offset that will be used to account for this. + private final int _firstDocId; + private final MutableForwardIndex _idx; + + public DenseColumn(MutableForwardIndex idx, int firstDocId) { + _idx = idx; + _firstDocId = firstDocId; + } + + /** + * Adjusts the Requested Document ID by the ID Offset of this column so that it indexes into the internal + * column correctly. + * + * @param docId + * @return + */ + private int getInternalDocId(int docId) { + return docId - _firstDocId; + } + + @Override + public int getLengthOfShortestElement() { + return _idx.getLengthOfShortestElement(); + } + + @Override + public int getLengthOfLongestElement() { + return _idx.getLengthOfLongestElement(); + } + + @Override + public boolean isDictionaryEncoded() { + return _idx.isDictionaryEncoded(); + } + + @Override + public boolean isSingleValue() { + return false; + } + + @Override + public FieldSpec.DataType getStoredType() { + return _idx.getStoredType(); + } + + @Override + public void add(@Nonnull Object value, int dictId, int docId) { + // Account for the docId offset that will happen when new columns are added after the segment has started + int adjustedDocId = getInternalDocId(docId); + _idx.add(value, dictId, adjustedDocId); + } + + @Override + public void add(@Nonnull Object[] value, @Nullable int[] dictIds, int docId) { + throw new UnsupportedOperationException("Multivalues are not yet supported in Maps"); + } + + @Override + public float getFloat(int docId) { + int adjustedDocId = getInternalDocId(docId); + if (adjustedDocId < 0) { + return DEFAULT_DIMENSION_NULL_VALUE_OF_FLOAT; + } + + return _idx.getFloat(adjustedDocId); + } + + @Override + public float getFloat(int docId, ForwardIndexReaderContext context) { + int adjustedDocId = getInternalDocId(docId); + if (adjustedDocId < 0) { + return DEFAULT_DIMENSION_NULL_VALUE_OF_FLOAT; + } + + return _idx.getFloat(adjustedDocId, context); + } + + @Override + public double getDouble(int docId) { + int adjustedDocId = getInternalDocId(docId); + if (adjustedDocId < 0) { + return DEFAULT_DIMENSION_NULL_VALUE_OF_DOUBLE; + } + + return _idx.getDouble(adjustedDocId); + } + + @Override + public double getDouble(int docId, ForwardIndexReaderContext context) { + int adjustedDocId = getInternalDocId(docId); + if (adjustedDocId < 0) { + return DEFAULT_DIMENSION_NULL_VALUE_OF_DOUBLE; + } + + return _idx.getDouble(adjustedDocId, context); + } + + @Override + public int getInt(int docId) { + int adjustedDocId = getInternalDocId(docId); + if (adjustedDocId < 0) { + return DEFAULT_DIMENSION_NULL_VALUE_OF_INT; + } + + return _idx.getInt(adjustedDocId); + } + + @Override + public int getInt(int docId, ForwardIndexReaderContext context) { + int adjustedDocId = getInternalDocId(docId); + if (adjustedDocId < 0) { + return DEFAULT_DIMENSION_NULL_VALUE_OF_INT; + } + + return _idx.getInt(adjustedDocId, context); + } + + @Override + public String getString(int docId) { + int adjustedDocId = getInternalDocId(docId); + if (adjustedDocId < 0) { + return DEFAULT_DIMENSION_NULL_VALUE_OF_STRING; + } + + return _idx.getString(adjustedDocId); + } + + @Override + public String getString(int docId, ForwardIndexReaderContext context) { + int adjustedDocId = getInternalDocId(docId); + if (adjustedDocId < 0) { + return DEFAULT_DIMENSION_NULL_VALUE_OF_STRING; + } + + return _idx.getString(adjustedDocId, context); + } + + @Override + public void close() + throws IOException { + _idx.close(); + } + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/DataTypeTransformer.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/DataTypeTransformer.java index 1ed69c9065c3..83b966024002 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/DataTypeTransformer.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/DataTypeTransformer.java @@ -72,7 +72,7 @@ public GenericRow transform(GenericRow record) { } PinotDataType dest = entry.getValue(); - if (dest != PinotDataType.JSON) { + if (dest != PinotDataType.JSON && dest != PinotDataType.MAP) { value = standardize(column, value, dest.isSingleValue()); } @@ -88,6 +88,9 @@ public GenericRow transform(GenericRow record) { // Multi-value column Object[] values = (Object[]) value; source = PinotDataType.getMultiValueType(values[0].getClass()); + } else if (value instanceof Map) { + // Single-value column + source = PinotDataType.getSingleValueType(value.getClass()); // Change to set type to MAP? } else { // Single-value column source = PinotDataType.getSingleValueType(value.getClass()); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java index 0fb078ce52fd..347ce99a9963 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentColumnarIndexCreator.java @@ -161,6 +161,7 @@ public void init(SegmentGeneratorConfig segmentCreationSpec, SegmentIndexCreatio .onHeap(segmentCreationSpec.isOnHeap()) .withForwardIndexDisabled(forwardIndexDisabled) .withTextCommitOnClose(true) + .withColumnStatistics(columnIndexCreationInfo.getColumnStatistics()) .withImmutableToMutableIdMap(immutableToMutableIdMap) .withRealtimeConversion(segmentCreationSpec.isRealtimeConversion()) .withConsumerDir(segmentCreationSpec.getConsumerDir()) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java index fe404da09a93..361d30edaa4b 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/SegmentIndexCreationDriverImpl.java @@ -93,7 +93,6 @@ // TODO: Check resource leaks public class SegmentIndexCreationDriverImpl implements SegmentIndexCreationDriver { private static final Logger LOGGER = LoggerFactory.getLogger(SegmentIndexCreationDriverImpl.class); - private SegmentGeneratorConfig _config; private RecordReader _recordReader; private SegmentPreIndexStatsContainer _segmentStats; diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/map/DenseMapHeader.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/map/DenseMapHeader.java new file mode 100644 index 000000000000..9cd2f72c6142 --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/map/DenseMapHeader.java @@ -0,0 +1,425 @@ +/** + * 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.segment.creator.impl.map; + +import java.io.IOException; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pinot.segment.spi.ColumnMetadata; +import org.apache.pinot.segment.spi.index.IndexService; +import org.apache.pinot.segment.spi.index.IndexType; +import org.apache.pinot.segment.spi.index.metadata.ColumnMetadataImpl; +import org.apache.pinot.segment.spi.memory.PinotDataBuffer; +import org.apache.pinot.spi.data.DimensionFieldSpec; +import org.apache.pinot.spi.data.FieldSpec; + + +/** + * Dense Map index header. This is a sub-header of the Map Index header and stores information needed to read + * a dense map representation from the on disk file format. + * + * The subheader representation is: + * + * |------------------------| + * | Length of map index identifier (i32) | + * | ----------------------| + * | char array containing the map index identifier | + * | ----------------------| + * | Number of Keys (i32) | + * | -----------------------------| + * | |------------------------| | + * | | Len Key Name (i32) | | + * | | Key Name (byte array) | | + * | | Doc Id offset (i32) | | + * | | Number of Indices (i32)| | + * | | < For each Index > | | + * | | Len Index Name (i32) | | + * | | Index Name (byte array)| | + * | | Idx Offset (i64) | | + * | | < End For each Index > | | + * | | Column Metadata | | + * | |------------------------| | + * | | + * | | + * | | + */ +public class DenseMapHeader { + static final String ID = "dense_map_index"; + final List _denseKeyMetadata; + public DenseMapHeader() { + _denseKeyMetadata = new LinkedList<>(); + } + + DenseMapHeader(List keys) { + _denseKeyMetadata = keys; + } + + public DenseKeyMetadata getKey(String key) { + Optional result = _denseKeyMetadata.stream().filter(m -> m._key.equals(key)).findFirst(); + return result.orElse(null); + } + + public List getKeys() { + return _denseKeyMetadata; + } + + public long write(MapIndexHeader.MapHeaderWriter writer, long offset) throws IOException { + // Write the type of index this is + offset = writer.putString(offset, ID); + + // Write the number of keys in this index + offset = writer.putInt(offset, _denseKeyMetadata.size()); + + // For each key, write teh metadata fo the key + for (DenseKeyMetadata keyMetadata : _denseKeyMetadata) { + offset = keyMetadata.write(writer, offset); + } + + return offset; + } + + public static Pair read(PinotDataBuffer buffer, int offset) + throws IOException { + // read the type of index this is + Pair indexTypeResult = MapIndexHeader.readString(buffer, offset); + String type = indexTypeResult.getLeft(); + assert type.equals(ID); + + offset = indexTypeResult.getRight(); + + // read the number of keys in this index + int numKeys = buffer.getInt(offset); + offset += Integer.BYTES; + + // For each key, read the key metadata + List keys = new LinkedList<>(); + for (int i = 0; i < numKeys; i++) { + Pair result = DenseKeyMetadata.read(buffer, offset); + offset = result.getRight(); + keys.add(result.getLeft()); + } + + DenseMapHeader md = new DenseMapHeader(keys); + + return new ImmutablePair<>(md, offset); + } + + public void addKey(String key, ColumnMetadata metadata, List> indexes, int docIdOffset) { + _denseKeyMetadata.add(new DenseKeyMetadata(key, metadata, indexes, docIdOffset)); + } + + public boolean equals(Object obj) { + if (obj instanceof DenseMapHeader) { + return _denseKeyMetadata.equals(((DenseMapHeader) obj)._denseKeyMetadata); + } else { + return false; + } + } + + public int hashCode() { + return Objects.hash(_denseKeyMetadata); + } + + public static class DenseKeyMetadata { + final HashMap, Long> _keyOffsetPosition; + final int _docIdOffset; + final String _key; + final Map, Long> _indexOffsets; + static final long PLACEHOLDER = 0xDEADBEEFDEADBEEFL; + final ColumnMetadata _columnMetadata; + + public DenseKeyMetadata(String key, ColumnMetadata metadata, + Map, Long> indexOffsets, int docIdOffset) { + _keyOffsetPosition = new HashMap<>(); + _docIdOffset = docIdOffset; + _key = key; + _columnMetadata = metadata; + _indexOffsets = indexOffsets; + } + + public DenseKeyMetadata(String key, ColumnMetadata metadata, List> indexes, int docIdOffset) { + _keyOffsetPosition = new HashMap<>(); + _docIdOffset = docIdOffset; + _key = key; + _columnMetadata = metadata; + _indexOffsets = new HashMap<>(); + for (IndexType index : indexes) { + _indexOffsets.put(index, PLACEHOLDER); + } + } + + public String getName() { + return _key; + } + + public long getIndexOffset(IndexType type) { + if (_indexOffsets.containsKey(type)) { + return _indexOffsets.get(type); + } else { + throw new RuntimeException(String.format( + "Attempting to read an index ('%s') that does not exist for the key ('%s')", type, _key)); + } + } + + public ColumnMetadata getColumnMetadata() { + return _columnMetadata; + } + + public long write(MapIndexHeader.MapHeaderWriter writer, long offset) throws IOException { + // Write key name + offset = writer.putString(offset, _key); + + // Write the doc id offset + offset = writer.putInt(offset, _docIdOffset); + + // Write number of indexes + offset = writer.putInt(offset, _indexOffsets.size()); + + // For each index write the type of index then the offset of the index + for (IndexType index : _indexOffsets.keySet()) { + offset = writer.putString(offset, index.getId()); + + _keyOffsetPosition.put(index, offset); + offset = writer.putLong(offset, _indexOffsets.get(index)); + } + + // Write the column metadata + offset = writeColumnMetadata(writer, offset); + return offset; + } + + public void setIndexOffset(MapIndexHeader.MapHeaderWriter writer, IndexType index, long offset) { + long indexOffsetCell = _keyOffsetPosition.get(index); + writer.putLong(indexOffsetCell, offset); + _indexOffsets.put(index, offset); + } + + public static Pair read(PinotDataBuffer buffer, int offset) throws IOException { + // Read the key name + Pair keyResult = MapIndexHeader.readString(buffer, offset); + offset = keyResult.getRight(); + String key = keyResult.getLeft(); + + // Read the doc id offset + int docIdOffset = buffer.getInt(offset); + offset += Integer.BYTES; + + // Read the number of indexes + int numIndexes = buffer.getInt(offset); + offset += Integer.BYTES; + + // Read each index + HashMap, Long> indexOffsets = new HashMap<>(); + for (int i = 0; i < numIndexes; i++) { + Pair indexIdResult = MapIndexHeader.readString(buffer, offset); + offset = indexIdResult.getRight(); + long indexOffset = buffer.getLong(offset); + offset += Long.BYTES; + + indexOffsets.put(IndexService.getInstance().get(indexIdResult.getLeft()), indexOffset); + } + + // read the column metadata + Pair colMDResult = readColumnMetadata(buffer, offset, key); + offset = colMDResult.getRight(); + + return new ImmutablePair<>(new DenseKeyMetadata( + key, + colMDResult.getLeft(), + indexOffsets, + docIdOffset + ), offset); + } + + public long writeColumnMetadata(MapIndexHeader.MapHeaderWriter writer, long metadataOffset) { + final ColumnMetadata metadata = _columnMetadata; + long offset = metadataOffset; + + // Write the Data type of the key + offset = writer.putString(offset, metadata.getDataType().name()); + + // Write is single value + offset = writer.putByte(offset, (byte) (metadata.isSingleValue() ? 1 : 0)); + + // Write Has Dictionary + offset = writer.putByte(offset, (byte) (metadata.hasDictionary() ? 1 : 0)); + + // Write Is Sorted + offset = writer.putByte(offset, (byte) (metadata.isSorted() ? 1 : 0)); + + // Write total docs + offset = writer.putInt(offset, metadata.getTotalDocs()); + + // Write Cardinality + offset = writer.putInt(offset, metadata.getCardinality()); + + // Write Column Max Length + offset = writer.putInt(offset, metadata.getColumnMaxLength()); + + // Write bits per element + offset = writer.putInt(offset, metadata.getBitsPerElement()); + + // Write max number of multi values + offset = writer.putInt(offset, metadata.getMaxNumberOfMultiValues()); + + // Write total number of entries + offset = writer.putInt(offset, metadata.getTotalNumberOfEntries()); + + // Write the min value (this is dynamic in size so we need to store a size followed by value) + offset = writer.putValue(offset, metadata.getDataType(), metadata.getMinValue()); + + // Write the max value + offset = writer.putValue(offset, metadata.getDataType(), metadata.getMaxValue()); + + // Write minmax invalid + offset = writer.putByte(offset, (byte) 1); + + // Write Number of indexes + offset = writer.putInt(offset, metadata.getIndexSizeMap().size()); + + // Write index size data + for (Map.Entry, Long> indexInfo : metadata.getIndexSizeMap().entrySet()) { + offset = writer.putString(offset, indexInfo.getKey().getId()); + + final Long indexSize = indexInfo.getValue(); + offset = writer.putLong(offset, indexSize); + } + + return offset; + } + + public boolean equals(Object obj) { + if (obj instanceof DenseKeyMetadata) { + DenseKeyMetadata b = (DenseKeyMetadata) obj; + return _key.equals(b._key) + && _columnMetadata.equals(b._columnMetadata) + && _indexOffsets.equals(b._indexOffsets) + && _docIdOffset == b._docIdOffset; + } else { + return false; + } + } + + public int hashCode() { + return Objects.hash(_key, _columnMetadata, _indexOffsets, _docIdOffset); + } + } + + public static Pair readColumnMetadata(PinotDataBuffer buffer, + final int mdOffset, String name) { + int offset = mdOffset; + + // Data type + Pair typeResult = MapIndexHeader.readString(buffer, offset); + final FieldSpec.DataType dataType = FieldSpec.DataType.valueOf(typeResult.getLeft()); + offset = typeResult.getRight(); + + // Read is single value + final byte isSingleValue = buffer.getByte(offset); + offset += Byte.BYTES; + + // Read Has Dictionary + final byte hasDictionary = buffer.getByte(offset); + offset += Byte.BYTES; + + // Read Is Sorted + final byte isSorted = buffer.getByte(offset); + offset += Byte.BYTES; + + // Read total docs + final int totalDocs = buffer.getInt(offset); + offset += Integer.BYTES; + + // Read Cardinality + final int card = buffer.getInt(offset); + offset += Integer.BYTES; + + // Read Column Max Length + final int maxColLength = buffer.getInt(offset); + offset += Integer.BYTES; + + // Read bits per element + final int bitsPerElement = buffer.getInt(offset); + offset += Integer.BYTES; + + // Read max number of multi values + final int maxNumMVs = buffer.getInt(offset); + offset += Integer.BYTES; + + // Read total number of entries + final int totalNumEntries = buffer.getInt(offset); + offset += Integer.BYTES; + + // Read the min value (this is dynamic in size so we need to store a size followed by value) + Pair, Integer> valueResult = MapIndexHeader.readValue(buffer, offset, dataType); + Comparable minValue = valueResult.getLeft(); + offset = valueResult.getRight(); + // Read the max value + valueResult = MapIndexHeader.readValue(buffer, offset, dataType); + Comparable maxValue = valueResult.getLeft(); + offset = valueResult.getRight(); + + // Read min/max invalid + final byte isMinMaxInvalid = buffer.getByte(offset); + offset += Byte.BYTES; + + // Read Number of indexes + int numIndexes = buffer.getInt(offset); + offset += Integer.BYTES; + + // Read index size data + HashMap, Long> indexSizeMap = new HashMap<>(); + for (int i = 0; i < numIndexes; i++) { + // Type code + Pair indexTypeResult = MapIndexHeader.readString(buffer, offset); + final String indexTypeId = indexTypeResult.getLeft(); + offset = indexTypeResult.getRight(); + + // Index Size + final long indexSize = buffer.getLong(offset); + offset += Long.BYTES; + + indexSizeMap.put(IndexService.getInstance().get(indexTypeId), indexSize); + } + + ColumnMetadataImpl.Builder builder = + ColumnMetadataImpl.builder() + .setTotalDocs(totalDocs) + .setMinMaxValueInvalid(isMinMaxInvalid == 1) + .setColumnMaxLength(maxColLength) + .setSorted(isSorted == 1) + .setHasDictionary(hasDictionary == 1) + .setBitsPerElement(bitsPerElement) + .setCardinality(card) + .setFieldSpec(new DimensionFieldSpec(name, dataType, isSingleValue == 1)) + .setTotalNumberOfEntries(totalNumEntries) + .setMinValue(minValue) + .setMaxValue(maxValue); + builder.setIndexSizeMap(indexSizeMap); + return new ImmutablePair<>( + builder.build(), + offset); + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/map/MapIndexCreator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/map/MapIndexCreator.java new file mode 100644 index 000000000000..725860b5c5bb --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/map/MapIndexCreator.java @@ -0,0 +1,566 @@ +/** + * 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.segment.creator.impl.map; + +import com.google.common.collect.Maps; +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.ByteOrder; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.stream.Collectors; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pinot.common.utils.PinotDataType; +import org.apache.pinot.segment.local.segment.index.loader.defaultcolumn.DefaultColumnStatistics; +import org.apache.pinot.segment.spi.ColumnMetadata; +import org.apache.pinot.segment.spi.creator.ColumnIndexCreationInfo; +import org.apache.pinot.segment.spi.creator.ColumnStatistics; +import org.apache.pinot.segment.spi.creator.IndexCreationContext; +import org.apache.pinot.segment.spi.creator.MapColumnStatistics; +import org.apache.pinot.segment.spi.index.FieldIndexConfigs; +import org.apache.pinot.segment.spi.index.ForwardIndexConfig; +import org.apache.pinot.segment.spi.index.IndexCreator; +import org.apache.pinot.segment.spi.index.IndexService; +import org.apache.pinot.segment.spi.index.IndexType; +import org.apache.pinot.segment.spi.index.StandardIndexes; +import org.apache.pinot.segment.spi.index.metadata.ColumnMetadataImpl; +import org.apache.pinot.segment.spi.memory.PinotDataBuffer; +import org.apache.pinot.spi.config.table.IndexConfig; +import org.apache.pinot.spi.config.table.MapIndexConfig; +import org.apache.pinot.spi.data.DimensionFieldSpec; +import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.utils.ByteArray; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.pinot.segment.spi.V1Constants.Indexes.MAP_FORWARD_INDEX_FILE_EXTENSION; +import static org.apache.pinot.spi.data.FieldSpec.DataType; + + +/** + * Creates the durable representation of a map index. Metadata about the Map Column can be passed through via + * the IndexCreationContext. + */ +public final class MapIndexCreator implements org.apache.pinot.segment.spi.index.creator.MapIndexCreator { + private static final Logger LOGGER = LoggerFactory.getLogger(MapIndexCreator.class); + //output file which will hold the range index + private final String _tmpMapIndexDir; + private final String _indexDir; + private final Map, IndexCreator>> _creatorsByKeyAndIndex; + private final TreeMap _keyIndexCreationInfoMap = new TreeMap<>(); + private final Map _denseKeySpecs; + private final Set _denseKeys; + private final int _totalDocs; + private final Map _keyStats = new HashMap<>(); + private final Map _denseKeyMetadata = new HashMap<>(); + private final String _columnName; + private int _docsWritten; + private final IndexCreationContext _context; + private final MapColumnStatistics _mapColumnStatistics; + private final boolean _dynamicallyCreateDenseKeys; + + /** + * + * @param context The Index Creation Context, used for configuring many of the options for index creation. + * @param columnName name of the column + * @throws IOException + */ + public MapIndexCreator(IndexCreationContext context, String columnName, MapIndexConfig config) + throws IOException { + // The Dense map column is composed of other indexes, so we'll store those index in a subdirectory + // Then when those indexes are created, they are created in this column's subdirectory. + _indexDir = context.getIndexDir().getPath(); + _mapColumnStatistics = (MapColumnStatistics) context.getColumnStatistics(); + _tmpMapIndexDir = String.format("%s/tmp_%s/", _indexDir, columnName + MAP_FORWARD_INDEX_FILE_EXTENSION); + final File denseKeyDir = new File(_tmpMapIndexDir); + try { + if (!denseKeyDir.mkdirs()) { + LOGGER.error("Failed to create directory: {}", denseKeyDir); + } + } catch (Exception ex) { + LOGGER.error("Exception while creating temporary directory: '{}'", denseKeyDir, ex); + } + + config = config != null ? config : new MapIndexConfig(); + _context = context; + _totalDocs = context.getTotalDocs(); + _denseKeySpecs = new HashMap<>(config.getMaxKeys()); + _dynamicallyCreateDenseKeys = config.getDynamicallyCreateDenseKeys(); + _denseKeys = new HashSet<>(); + _columnName = columnName; + _docsWritten = 0; + _creatorsByKeyAndIndex = Maps.newHashMapWithExpectedSize(_denseKeySpecs.size()); + + if (config.getDenseKeys() != null) { + // If the user has pre-configured as certain keys then start by creating the writers for those keys + for (DimensionFieldSpec key : config.getDenseKeys()) { + addKeyWriter(key.getName(), key.getDataType().getStoredType()); + } + } + + if (_dynamicallyCreateDenseKeys) { + // If the user has enabled dynamic key creation, then iterate through the map column statistics to create + // key writers + if (_mapColumnStatistics != null && _mapColumnStatistics.getKeys() != null) { + for (Pair key : _mapColumnStatistics.getKeys()) { + addKeyWriter(key.getLeft(), key.getRight()); + } + } + } + } + + private void addKeyWriter(String keyName, DataType type) { + FieldSpec keySpec = createKeyFieldSpec(keyName, type); + _denseKeySpecs.put(keyName, keySpec); + _denseKeys.add(keySpec.getName()); + + ColumnStatistics stats = buildKeyStats(keyName); + _keyStats.put(keyName, stats); + + ColumnMetadata keyMetadata = generateColumnMetadataForKey(keySpec); + _denseKeyMetadata.put(keyName, keyMetadata); + + ColumnIndexCreationInfo creationInfo = buildIndexCreationInfoForKey(keySpec); + _keyIndexCreationInfoMap.put(keyName, creationInfo); + + Map, IndexCreator> creatorsByIndex = createIndexCreatorsForKey(keySpec); + _creatorsByKeyAndIndex.put(keySpec.getName(), creatorsByIndex); + } + + private FieldSpec createKeyFieldSpec(String name, DataType type) { + FieldSpec keySpec = new DimensionFieldSpec(); + keySpec.setName(name); + keySpec.setDataType(type); + keySpec.setNullable(true); + keySpec.setSingleValueField(true); + keySpec.setDefaultNullValue(null); // Sets the default default null value + + return keySpec; + } + + private ColumnStatistics buildKeyStats(String key) { + Object minValue = _mapColumnStatistics.getMinValueForKey(key); + Object maxValue = _mapColumnStatistics.getMaxValueForKey(key); + + // The length of the shortest and longest should be at least the length of the null value + int lengthShortest = Math.max(_mapColumnStatistics.getLengthOfShortestElementForKey(key), 4); + int lengthLongest = Math.max(_mapColumnStatistics.getLengthOfLargestElementForKey(key), 4); + + return new DefaultColumnStatistics(minValue, maxValue, null, + false, _totalDocs, 0, lengthShortest, lengthLongest); + } + + private Map, IndexCreator> createIndexCreatorsForKey(FieldSpec keySpec) { + boolean dictEnabledColumn = + false; //createDictionaryForColumn(columnIndexCreationInfo, segmentCreationSpec, fieldSpec); + ColumnIndexCreationInfo columnIndexCreationInfo = _keyIndexCreationInfoMap.get(keySpec.getName()); + + FieldIndexConfigs keyConfig = getKeyIndexConfig(columnIndexCreationInfo); + IndexCreationContext.Common keyContext = + getKeyIndexContext(dictEnabledColumn, keySpec, columnIndexCreationInfo, _context); + Map, IndexCreator> creatorsByIndex = + Maps.newHashMapWithExpectedSize(IndexService.getInstance().getAllIndexes().size()); + for (IndexType index : IndexService.getInstance().getAllIndexes()) { + if (index.getIndexBuildLifecycle() != IndexType.BuildLifecycle.DURING_SEGMENT_CREATION) { + continue; + } + + try { + tryCreateIndexCreator(creatorsByIndex, index, keyContext, keyConfig); + } catch (Exception e) { + LOGGER.error("An exception happened while creating IndexCreator for key '{}' for index '{}'", keySpec.getName(), + index.getId(), e); + } + } + + return creatorsByIndex; + } + + private FieldIndexConfigs getKeyIndexConfig(ColumnIndexCreationInfo columnIndexCreationInfo) { + FieldIndexConfigs.Builder builder = new FieldIndexConfigs.Builder(); + // Sorted columns treat the 'forwardIndexDisabled' flag as a no-op + // ForwardIndexConfig fwdConfig = config.getConfig(StandardIndexes.forward()); + + ForwardIndexConfig fwdConfig = new ForwardIndexConfig.Builder() + // TODO (make configurable): .withCompressionCodec(FieldConfig.CompressionCodec.PASS_THROUGH) + //.withCompressionCodec(FieldConfig.CompressionCodec.PASS_THROUGH) + .withDeriveNumDocsPerChunk(true) + .build(); + // TODO(What's this for?) if (!fwdConfig.isEnabled() && columnIndexCreationInfo.isSorted()) { + builder.add(StandardIndexes.forward(), new ForwardIndexConfig.Builder(fwdConfig).build()); + //} + // Initialize inverted index creator; skip creating inverted index if sorted + if (columnIndexCreationInfo.isSorted()) { + builder.undeclare(StandardIndexes.inverted()); + } + return builder.build(); + } + + private IndexCreationContext.Common getKeyIndexContext(boolean dictEnabledColumn, FieldSpec keySpec, + ColumnIndexCreationInfo columnIndexCreationInfo, IndexCreationContext context) { + File denseKeyDir = new File(_tmpMapIndexDir); + return IndexCreationContext.builder() + .withIndexDir(denseKeyDir) + .withDictionary(dictEnabledColumn) + .withFieldSpec(keySpec) + .withTotalDocs(_totalDocs) + .withColumnIndexCreationInfo(columnIndexCreationInfo) + .withMaxRowLengthInBytes(_keyStats.get(keySpec.getName()).getMaxRowLengthInBytes()) + .withLengthOfLongestEntry(_keyStats.get(keySpec.getName()).getLengthOfLargestElement()) + .withMinValue((Comparable) _keyStats.get(keySpec.getName()).getMinValue()) + .withMaxValue((Comparable) _keyStats.get(keySpec.getName()).getMaxValue()) + .withOptimizedDictionary(false) + .onHeap(context.isOnHeap()) + .withForwardIndexDisabled(false) + .withTextCommitOnClose(true) + .build(); + } + + ColumnIndexCreationInfo buildIndexCreationInfoForKey(FieldSpec keySpec) { + // TODO: Does this need to do anything with variable length columns? Search for varLengthDictionaryColumns + String keyName = keySpec.getName(); + DataType storedType = keySpec.getDataType().getStoredType(); + ColumnStatistics columnProfile = null; + try { + columnProfile = _keyStats.get(keyName); + } catch (Exception ex) { + LOGGER.error("Failed to get profile for key: '{}'", keyName, ex); + } + boolean useVarLengthDictionary = false; + //shouldUseVarLengthDictionary(columnName, varLengthDictionaryColumns, storedType, columnProfile); + Object defaultNullValue = keySpec.getDefaultNullValue(); + if (storedType == DataType.BYTES) { + defaultNullValue = new ByteArray((byte[]) defaultNullValue); + } + boolean createDictionary = false; + //!rawIndexCreationColumns.contains(keyName) && !rawIndexCompressionTypeKeys.contains(keyName); + return new ColumnIndexCreationInfo(columnProfile, createDictionary, useVarLengthDictionary, false/*isAutoGenerated*/, + defaultNullValue); + } + + private void tryCreateIndexCreator(Map, IndexCreator> creatorsByIndex, + IndexType index, IndexCreationContext.Common context, FieldIndexConfigs fieldIndexConfigs) + throws Exception { + C config = fieldIndexConfigs.getConfig(index); + if (config.isEnabled()) { + creatorsByIndex.put(index, index.createIndexCreator(context, config)); + } + } + + @Override + public void seal() + throws IOException { + for (Map.Entry, IndexCreator>> keysInMap : _creatorsByKeyAndIndex.entrySet()) { + for (IndexCreator keyIdxCreator : keysInMap.getValue().values()) { + keyIdxCreator.close(); + keyIdxCreator.seal(); + } + } + + // Stitch the index files together + mergeKeyFiles(); + } + + @Override + public boolean isDictionaryEncoded() { + return false; + } + + @Override + public boolean isSingleValue() { + return true; + } + + @Override + public DataType getValueType() { + return DataType.MAP; + } + + private void mergeKeyFiles() { + File mergedIndexFile = new File(_indexDir, _columnName + MAP_FORWARD_INDEX_FILE_EXTENSION); + + try { + // Construct the Header for the merged index + long totalIndexLength = 0; + + // Compute the total size of the indexes + HashMap indexSizes = new HashMap<>(); + Map keyFiles = new HashMap<>(_denseKeys.size()); + for (String key : _denseKeys) { + File keyFile = getFileFor(key, StandardIndexes.forward()); + final long indexSize = Files.size(keyFile.toPath()); + indexSizes.put(key, indexSize); + totalIndexLength += indexSize; + keyFiles.put(key, keyFile); + _denseKeyMetadata.get(key).getIndexSizeMap().put(StandardIndexes.forward(), indexSize); + } + + // Construct the Header + MapIndexHeader header = new MapIndexHeader(); + DenseMapHeader denseHeader = new DenseMapHeader(); + for (String key : _denseKeys) { + // Write with a placeholder offset, it will be updated when the indexes are merged + //header.addKey(key, _denseKeyMetadata.get(key), 0xDEADBEEF); + denseHeader.addKey(key, _denseKeyMetadata.get(key), List.of(StandardIndexes.forward()), 0); + } + header.addMapIndex(denseHeader); + + // Create an output buffer for writing to (see the V2 to V3 conversion logic for what to do here) + final long totalFileSize = header.size() + totalIndexLength; + long offset = 0; + PinotDataBuffer buffer = + PinotDataBuffer.mapFile(mergedIndexFile, false, offset, totalFileSize, ByteOrder.BIG_ENDIAN, + allocationContext(mergedIndexFile, "writing")); + + // Write the header to the buffer + MapIndexHeader.PinotDataBufferWriter writer = new MapIndexHeader.PinotDataBufferWriter(buffer); + offset = header.write(writer, offset); + + // Iterate over each key and find the index and write the index to a file + for (String key : _denseKeys) { + File keyFile = keyFiles.get(key); + try (FileChannel denseKeyFileChannel = new RandomAccessFile(keyFile, "r").getChannel()) { + long indexSize = denseKeyFileChannel.size(); + try (PinotDataBuffer keyBuffer = PinotDataBuffer.mapFile(keyFile, true, 0, indexSize, ByteOrder.BIG_ENDIAN, + allocationContext(keyFile, "reading"))) { + keyBuffer.copyTo(0, buffer, offset, indexSize); + denseHeader.getKey(key).setIndexOffset(writer, StandardIndexes.forward(), offset); + offset += indexSize; + } catch (Exception ex) { + LOGGER.error("Error mapping PinotDataBuffer for '{}'", keyFile, ex); + } + } catch (Exception ex) { + LOGGER.error("Error opening dense key file '{}': ", keyFile, ex); + } + } + + // Delete the index files + buffer.close(); + + deleteIntermediateFiles(keyFiles); + } catch (Exception ex) { + LOGGER.error("Exception while merging dense key indexes: ", ex); + } + } + + private void deleteIntermediateFiles(Map files) { + for (File file : files.values()) { + try { + if (!file.delete()) { + LOGGER.error("Failed to delete file '{}'. Reason is unknown", file); + } + } catch (Exception ex) { + LOGGER.error("Failed to delete intermediate file '{}'", file, ex); + } + } + + File tmpDir = new File(_tmpMapIndexDir); + try { + // Delete the temporary directory + if (!tmpDir.delete()) { + LOGGER.error("Failed to delete directory '{}'", tmpDir); + } + } catch (Exception ex) { + LOGGER.error("Failed to delete temporary directory: '{}'", tmpDir, ex); + } + } + + private ColumnMetadata generateColumnMetadataForKey(FieldSpec keySpec) { + ColumnMetadataImpl.Builder builder = new ColumnMetadataImpl.Builder(); + HashMap, Long> indexSizeMap = new HashMap<>(); + + Comparable minValue = getPlaceholderValue(keySpec.getDataType()); + Comparable maxValue = getPlaceholderValue(keySpec.getDataType()); + builder + .setFieldSpec(keySpec) + .setMinValue(minValue) + .setMaxValue(maxValue) + .setMinMaxValueInvalid(true) + .setCardinality(0) + .setSorted(false) + .setHasDictionary(false) + .setColumnMaxLength(10) + .setBitsPerElement(10) + .setAutoGenerated(false) + .setTotalDocs(10); + builder.setIndexSizeMap(indexSizeMap); + return builder.build(); + } + + private void backFillKey(String key, Object value) throws IOException { + for (int i = 0; i < _docsWritten; i++) { + try { + // Iterate over each key in the dictionary and if it exists in the record write a value, otherwise write + // the null value + for (Map.Entry, IndexCreator> indexes : _creatorsByKeyAndIndex.get(key).entrySet()) { + indexes.getValue().add(value, -1); // TODO: Add in dictionary encoding support + } + } catch (IOException ioe) { + LOGGER.error("Error writing to dense key '{}': ", key, ioe); + throw ioe; + } catch (Exception e) { + LOGGER.error("Error getting dense key '{}': ", key, e); + } + } + } + + private void dynamicallyAddKeys(Map mapValue) { + for (Map.Entry entry : mapValue.entrySet()) { + // Check if the key exists already + String keyName = entry.getKey(); + if (!_keyIndexCreationInfoMap.containsKey(keyName)) { + // If the key does not exist then create a writer for it + DataType valType = convertToDataType(PinotDataType.getSingleValueType(entry.getValue().getClass())); + addKeyWriter(keyName, valType); + + // Backfill the index + try { + backFillKey(keyName, _denseKeySpecs.get(keyName).getDefaultNullValue()); + } catch (Exception ex) { + LOGGER.error("Failed to write to key '{}'", keyName, ex); + } + } + } + } + + @Override + public void add(Map mapValue) { + if (_dynamicallyCreateDenseKeys) { + dynamicallyAddKeys(mapValue); + } + + // Iterate over every dense key in this map + for (String key : _denseKeys) { + FieldSpec denseKey = _denseKeySpecs.get(key); + String keyName = denseKey.getName(); + + // Get the value of the key from the input map and write to each index + Object value = mapValue.get(keyName); + + // If the value is NULL or the value's type does not match the key's index type then + // Write the default value to the index + if (value == null) { + value = _keyIndexCreationInfoMap.get(keyName).getDefaultNullValue(); + } else { + DataType valType = convertToDataType(PinotDataType.getSingleValueType(value.getClass())); + if (!valType.equals(denseKey.getDataType())) { + LOGGER.warn("Type mismatch, expected '{}' but got '{}'", denseKey.getDataType(), valType); + value = _keyIndexCreationInfoMap.get(keyName).getDefaultNullValue(); + } + } + + // Get the type of the value to check that it matches the Dense Key's type + try { + // Iterate over each key in the dictionary and if it exists in the record write a value, otherwise write + // the null value + for (Map.Entry, IndexCreator> indexes : _creatorsByKeyAndIndex.get(keyName).entrySet()) { + indexes.getValue().add(value, -1); // TODO: Add in dictionary encoding support + } + } catch (IOException ioe) { + LOGGER.error("Error writing to dense key '{}': ", keyName, ioe); + } catch (Exception e) { + LOGGER.error("Error getting dense key '{}': ", keyName, e); + } + } + + _docsWritten++; + } + + public void close() + throws IOException { + } + + File getFileFor(String column, IndexType indexType) { + List candidates = getFilesFor(column, indexType); + if (candidates.isEmpty()) { + throw new RuntimeException("No file candidates for index " + indexType + " and column " + column); + } + + return candidates.stream().filter(File::exists).findAny().orElse(candidates.get(0)); + } + + private List getFilesFor(String key, IndexType indexType) { + return indexType.getFileExtensions(null).stream() + .map(fileExtension -> new File(_tmpMapIndexDir, key + fileExtension)).collect(Collectors.toList()); + } + + private String allocationContext(File f, String context) { + return this.getClass().getSimpleName() + "." + f.toString() + "." + context; + } + + static FieldSpec.DataType convertToDataType(PinotDataType ty) { + // TODO: I've been told that we already have a function to do this, so find that function and replace this + switch (ty) { + case BOOLEAN: + return FieldSpec.DataType.BOOLEAN; + case SHORT: + case INTEGER: + return FieldSpec.DataType.INT; + case LONG: + return FieldSpec.DataType.LONG; + case FLOAT: + return FieldSpec.DataType.FLOAT; + case DOUBLE: + return FieldSpec.DataType.DOUBLE; + case BIG_DECIMAL: + return FieldSpec.DataType.BIG_DECIMAL; + case TIMESTAMP: + return FieldSpec.DataType.TIMESTAMP; + case STRING: + return FieldSpec.DataType.STRING; + default: + throw new UnsupportedOperationException(); + } + } + + private static Comparable getPlaceholderValue(DataType type) { + // TODO(ERICH): is this still needed? + switch (type) { + case INT: + return 0; + case LONG: + return 0L; + case FLOAT: + return 0.0F; + case DOUBLE: + return 0.0D; + case BOOLEAN: + return false; + case STRING: + return "null"; + case BIG_DECIMAL: + case TIMESTAMP: + case JSON: + case BYTES: + case STRUCT: + case MAP: + case LIST: + case UNKNOWN: + default: + throw new UnsupportedOperationException(); + } + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/map/MapIndexHeader.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/map/MapIndexHeader.java new file mode 100644 index 000000000000..7acfbb22a3a8 --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/map/MapIndexHeader.java @@ -0,0 +1,313 @@ +/** + * 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.segment.creator.impl.map; + +import java.io.IOException; +import java.util.LinkedList; +import java.util.List; +import java.util.Objects; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pinot.segment.spi.memory.PinotDataBuffer; +import org.apache.pinot.spi.data.FieldSpec; + +public class MapIndexHeader { + final static int VERSION = 1; + private final List _mapIndexes; + + public MapIndexHeader() { + _mapIndexes = new LinkedList<>(); + } + + MapIndexHeader(List indexes) { + _mapIndexes = indexes; + } + + public void addMapIndex(DenseMapHeader mapIndex) { + _mapIndexes.add(mapIndex); + } + + public DenseMapHeader getMapIndex() { + return _mapIndexes.get(0); + } + + public long write(PinotDataBuffer buffer, long offset) throws IOException { + PinotDataBufferWriter writer = new PinotDataBufferWriter(buffer); + return write(writer, offset); + } + + public long size() throws IOException { + HeaderSizeComputer size = new HeaderSizeComputer(); + + write(size, 0); + + return size.size(); + } + + public long write(MapHeaderWriter writer, long offset) throws IOException { + // Write the header version + offset = writer.putInt(offset, VERSION); + + // Write the number of map indexes + offset = writer.putInt(offset, _mapIndexes.size()); + + // Iterate over each map index and write it + for (DenseMapHeader mapMeta : _mapIndexes) { + offset = mapMeta.write(writer, offset); + } + + return offset; + } + + public static Pair read(PinotDataBuffer buffer, int offset) + throws IOException { + // read the version + int version = buffer.getInt(offset); + offset += Integer.BYTES; + assert version == 1; + + // read the number of indexes + int numIndexes = buffer.getInt(offset); + offset += Integer.BYTES; + + // Iterate over the indexes and create the appropriate index + List indexes = new LinkedList<>(); + for (int i = 0; i < numIndexes; i++) { + Pair result = DenseMapHeader.read(buffer, offset); + offset = result.getRight(); + indexes.add(result.getLeft()); + } + + return new ImmutablePair<>(new MapIndexHeader(indexes), offset); + } + + public static Pair readString(PinotDataBuffer buffer, final int stringOffset) { + int offset = stringOffset; + // Read the length of the string + final int len = buffer.getInt(offset); + offset += Integer.BYTES; + + // Read the string + char[] chars = new char[len]; + for (int i = 0; i < len; i++) { + char ch = buffer.getChar(offset); + offset += Character.BYTES; + chars[i] = ch; + } + + return new ImmutablePair<>(new String(chars), offset); + } + + public static Pair, Integer> readValue(PinotDataBuffer buffer, + final int valueOffset, FieldSpec.DataType type) { + Comparable value = null; + int offset = valueOffset; + switch (type) { + case INT: + value = buffer.getInt(offset); + offset += Integer.BYTES; + break; + case LONG: + value = buffer.getLong(offset); + offset += Long.BYTES; + break; + case FLOAT: + value = buffer.getFloat(offset); + offset += Float.BYTES; + break; + case DOUBLE: + value = buffer.getDouble(offset); + offset += Double.BYTES; + break; + case BOOLEAN: + value = buffer.getByte(offset) == 1; + offset += Byte.BYTES; + break; + case STRING: + Pair result = readString(buffer, offset); + value = result.getLeft(); + offset = result.getRight(); + break; + case BIG_DECIMAL: + case TIMESTAMP: + case JSON: + case BYTES: + case STRUCT: + case MAP: + case LIST: + case UNKNOWN: + default: + throw new UnsupportedOperationException(); + } + + return new ImmutablePair<>(value, offset); + } + + public boolean equals(Object obj) { + if (obj instanceof MapIndexHeader) { + return _mapIndexes.equals(((MapIndexHeader) obj)._mapIndexes); + } else { + return false; + } + } + + public int hashCode() { + return Objects.hash(_mapIndexes); + } + + public interface MapHeaderWriter { + long putInt(long offset, int value); + long putLong(long offset, long value); + long putString(long offset, String value); + long putByte(long offset, byte value); + long putFloat(long offset, float value); + long putDouble(long offset, double value); + default long putValue(long offset, FieldSpec.DataType type, Comparable value) { + switch (type) { + case INT: + offset = putInt(offset, ((Integer) value)); + break; + case LONG: + offset = putLong(offset, ((Long) value)); + break; + case FLOAT: + offset = putFloat(offset, ((Float) value)); + break; + case DOUBLE: + offset = putDouble(offset, ((Double) value)); + break; + case STRING: + offset = putString(offset, (String) value); + break; + case BOOLEAN: + offset = putByte(offset, (byte) (((Boolean) value) ? 1 : 0)); + break; + case BIG_DECIMAL: + case TIMESTAMP: + case JSON: + case BYTES: + case STRUCT: + case MAP: + case LIST: + case UNKNOWN: + default: + throw new UnsupportedOperationException(); + } + + return offset; + } + } + + public static class PinotDataBufferWriter implements MapHeaderWriter { + private final PinotDataBuffer _buffer; + + public PinotDataBufferWriter(PinotDataBuffer buffer) { + _buffer = buffer; + } + + @Override + public long putInt(long offset, int value) { + _buffer.putInt(offset, value); + return offset + Integer.BYTES; + } + + @Override + public long putLong(long offset, long value) { + _buffer.putLong(offset, value); + return offset + Long.BYTES; + } + + @Override + public long putString(long offset, String value) { + char[] chars = value.toCharArray(); + _buffer.putInt(offset, chars.length); + offset += Integer.BYTES; + + for (char aChar : chars) { + _buffer.putChar(offset, aChar); + offset += Character.BYTES; + } + + return offset; + } + + @Override + public long putByte(long offset, byte value) { + _buffer.putByte(offset, value); + return offset + Byte.BYTES; + } + + @Override + public long putFloat(long offset, float value) { + _buffer.putFloat(offset, value); + return offset + Float.BYTES; + } + + @Override + public long putDouble(long offset, double value) { + _buffer.putDouble(offset, value); + return offset + Double.BYTES; + } + } + + public static class HeaderSizeComputer implements MapHeaderWriter { + private long _size = 0; + + public long size() { + return _size; + } + + @Override + public long putInt(long offset, int value) { + _size += Integer.BYTES; + return offset + Integer.BYTES; + } + + @Override + public long putLong(long offset, long value) { + _size += Long.BYTES; + return offset + Long.BYTES; + } + + @Override + public long putString(long offset, String value) { + _size += Integer.BYTES; + _size += value.length() * Character.BYTES; + return offset + Integer.BYTES + value.length() * Character.BYTES; + } + + @Override + public long putByte(long offset, byte value) { + _size += Byte.BYTES; + return offset + Byte.BYTES; + } + + @Override + public long putFloat(long offset, float value) { + _size += Float.BYTES; + return offset + Float.BYTES; + } + + @Override + public long putDouble(long offset, double value) { + _size += Double.BYTES; + return offset + Double.BYTES; + } + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/MapColumnPreIndexStatsCollector.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/MapColumnPreIndexStatsCollector.java new file mode 100644 index 000000000000..d550048b6c73 --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/MapColumnPreIndexStatsCollector.java @@ -0,0 +1,286 @@ +/** + * 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.segment.creator.impl.stats; + +import it.unimi.dsi.fastutil.objects.ObjectOpenHashSet; +import java.util.HashMap; +import java.util.Map; +import org.apache.pinot.common.utils.PinotDataType; +import org.apache.pinot.segment.spi.creator.StatsCollectorConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.data.DimensionFieldSpec; +import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.utils.builder.TableConfigBuilder; + + +/** + * Extension of {@link AbstractColumnStatisticsCollector} for Map column type. + * + * The Map column type is different than other columns in that it is essentially recursive. It contains keys + * and those keys are analogous to columns and, as such, have Key level statistics. So, this class keeps track of + * Map column level statistics _and_ Key level statistics. The Key Level statistics can then be used during + * the creation of the Immutable Segment to make decisions about how keys will be stored or what Map data structure + * to use. + * + * Assumptions that are made: + * 1. Each key has a single type for the value's associated with it across all documents. + * 2. At this point in the Pinot process, the type consistency of a key should already be enforced, so if a + * heterogenous value types for a key are encountered will constructing the Map statistics it can be raised as a + * fault. + */ +public class MapColumnPreIndexStatsCollector extends AbstractColumnStatisticsCollector { + private ObjectOpenHashSet _keys = new ObjectOpenHashSet<>(INITIAL_HASH_SET_SIZE); + private final HashMap _keyStats; + private String[] _sortedValues; + private int _minLength = Integer.MAX_VALUE; + private int _maxLength = 0; + private int _maxRowLength = 0; + private String _minValue = null; + private String _maxValue = null; + private boolean _sealed = false; + private final String _column; + + public MapColumnPreIndexStatsCollector(String column, StatsCollectorConfig statsCollectorConfig) { + super(column, statsCollectorConfig); + super._sorted = false; + _keyStats = new HashMap<>(); + _column = column; + } + + public AbstractColumnStatisticsCollector getKeyStatistics(String key) { + return _keyStats.get(key); + } + + @Override + public void collect(Object entry) { + assert !_sealed; + + if (entry instanceof Map) { + final Map mapValue = (Map) entry; + + for (Map.Entry mapValueEntry : mapValue.entrySet()) { + final String key = mapValueEntry.getKey(); + + // Record statistics about the key + int length = key.length() * Character.BYTES; + if (_keys.add(key)) { + if (isPartitionEnabled()) { + updatePartition(key); + } + if (_minValue == null) { + _minValue = key; + } else { + if (key.compareTo(_minValue) < 0) { + _minValue = key; + } + } + if (_maxValue == null) { + _maxValue = key; + } else { + if (key.compareTo(_maxValue) > 0) { + _maxValue = key; + } + } + _minLength = Math.min(_minLength, length); + _maxLength = Math.max(_maxLength, length); + _maxRowLength = _maxLength; + } + + // Record statistics about the value within the key + AbstractColumnStatisticsCollector keyStats = getOrCreateKeyStatsCollector(key, mapValueEntry.getValue()); + keyStats.collect(mapValueEntry.getValue()); + } + _totalNumberOfEntries++; + } else { + throw new UnsupportedOperationException(); + } + } + + @Override + public String getMinValue() { + if (_sealed) { + return _minValue; + } + throw new IllegalStateException("you must seal the collector first before asking for min value"); + } + + @Override + public String getMaxValue() { + if (_sealed) { + return _maxValue; + } + throw new IllegalStateException("you must seal the collector first before asking for max value"); + } + + @Override + public String[] getUniqueValuesSet() { + if (_sealed) { + return _sortedValues; + } + throw new IllegalStateException("you must seal the collector first before asking for unique values set"); + } + + @Override + public int getLengthOfShortestElement() { + return _minLength; + } + + @Override + public int getLengthOfLargestElement() { + if (_sealed) { + return _maxLength; + } + throw new IllegalStateException("you must seal the collector first before asking for longest value"); + } + + @Override + public int getMaxRowLengthInBytes() { + return _maxRowLength; + } + + @Override + public int getCardinality() { + if (_sealed) { + return _sortedValues.length; + } + throw new IllegalStateException("you must seal the collector first before asking for cardinality"); + } + + @Override + public void seal() { + if (!_sealed) { + _sortedValues = _keys.stream().sorted().toArray(String[]::new); + _keys = null; + + // Iterate through every key stats collector and seal them + for (AbstractColumnStatisticsCollector keyStatsCollector : _keyStats.values()) { + keyStatsCollector.seal(); + } + + _sealed = true; + } + } + + /** + * Create a Stats Collector for each Key in the Map. + * + * NOTE: this could raise an issue if there are millions of keys with very few values (Sparse keys, in other words). + * So a less memory intensive option may be better for this. + * + * @param key + * @param value + * @return + */ + private AbstractColumnStatisticsCollector getOrCreateKeyStatsCollector(String key, Object value) { + // Check if the key stats collector exists just return it + if (!_keyStats.containsKey(key)) { + // Get the type of the value + PinotDataType type = PinotDataType.getSingleValueType(value.getClass()); + TableConfig tableConfig = new TableConfigBuilder(TableType.OFFLINE) + .setTableName(_column) + .build(); + Schema keySchema = new Schema.SchemaBuilder() + .setSchemaName(key) + .addField(new DimensionFieldSpec(key, convertToDataType(type), false)) + .build(); + StatsCollectorConfig config = new StatsCollectorConfig(tableConfig, keySchema, null); + + AbstractColumnStatisticsCollector keyStatsCollector = null; + switch (type) { + case INTEGER: + keyStatsCollector = new IntColumnPreIndexStatsCollector(key, config); + break; + case LONG: + keyStatsCollector = new LongColumnPreIndexStatsCollector(key, config); + break; + case FLOAT: + keyStatsCollector = new FloatColumnPreIndexStatsCollector(key, config); + break; + case DOUBLE: + keyStatsCollector = new DoubleColumnPreIndexStatsCollector(key, config); + break; + case BIG_DECIMAL: + keyStatsCollector = new BigDecimalColumnPreIndexStatsCollector(key, config); + break; + case STRING: + keyStatsCollector = new StringColumnPreIndexStatsCollector(key, config); + break; + case TIMESTAMP: + case BOOLEAN: + case BYTE: + case CHARACTER: + case SHORT: + case JSON: + case BYTES: + case OBJECT: + case MAP: + case BYTE_ARRAY: + case CHARACTER_ARRAY: + case SHORT_ARRAY: + case PRIMITIVE_INT_ARRAY: + case INTEGER_ARRAY: + case PRIMITIVE_LONG_ARRAY: + case LONG_ARRAY: + case PRIMITIVE_FLOAT_ARRAY: + case FLOAT_ARRAY: + case PRIMITIVE_DOUBLE_ARRAY: + case DOUBLE_ARRAY: + case BOOLEAN_ARRAY: + case TIMESTAMP_ARRAY: + case STRING_ARRAY: + case BYTES_ARRAY: + case COLLECTION: + case OBJECT_ARRAY: + default: + throw new UnsupportedOperationException(String.format("MAP column does not yet support '%s'", type)); + } + + _keyStats.put(key, keyStatsCollector); + } + + return _keyStats.get(key); + } + + static FieldSpec.DataType convertToDataType(PinotDataType ty) { + // TODO: I've been told that we already have a function to do this, so find that function and replace this + switch (ty) { + case BOOLEAN: + return FieldSpec.DataType.BOOLEAN; + case SHORT: + case INTEGER: + return FieldSpec.DataType.INT; + case LONG: + return FieldSpec.DataType.LONG; + case FLOAT: + return FieldSpec.DataType.FLOAT; + case DOUBLE: + return FieldSpec.DataType.DOUBLE; + case BIG_DECIMAL: + return FieldSpec.DataType.BIG_DECIMAL; + case TIMESTAMP: + return FieldSpec.DataType.TIMESTAMP; + case STRING: + return FieldSpec.DataType.STRING; + default: + throw new UnsupportedOperationException(); + } + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/SegmentPreIndexStatsCollectorImpl.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/SegmentPreIndexStatsCollectorImpl.java index 3f50a1f85944..45c75e0c490f 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/SegmentPreIndexStatsCollectorImpl.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/creator/impl/stats/SegmentPreIndexStatsCollectorImpl.java @@ -71,6 +71,9 @@ public void init() { case BYTES: _columnStatsCollectorMap.put(column, new BytesColumnPredIndexStatsCollector(column, _statsCollectorConfig)); break; + case MAP: + _columnStatsCollectorMap.put(column, new MapColumnPreIndexStatsCollector(column, _statsCollectorConfig)); + break; default: throw new IllegalStateException("Unsupported data type: " + fieldSpec.getDataType()); } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexCreatorFactory.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexCreatorFactory.java index cdb7f0e12716..5a4fbbea7f05 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexCreatorFactory.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexCreatorFactory.java @@ -30,6 +30,7 @@ import org.apache.pinot.segment.local.segment.creator.impl.fwd.SingleValueSortedForwardIndexCreator; import org.apache.pinot.segment.local.segment.creator.impl.fwd.SingleValueUnsortedForwardIndexCreator; import org.apache.pinot.segment.local.segment.creator.impl.fwd.SingleValueVarByteRawIndexCreator; +import org.apache.pinot.segment.local.segment.creator.impl.map.MapIndexCreator; import org.apache.pinot.segment.spi.compression.ChunkCompressionType; import org.apache.pinot.segment.spi.compression.DictIdCompressionType; import org.apache.pinot.segment.spi.creator.IndexCreationContext; @@ -51,6 +52,10 @@ public static ForwardIndexCreator createIndexCreator(IndexCreationContext contex String columnName = fieldSpec.getName(); int numTotalDocs = context.getTotalDocs(); + if (fieldSpec.getDataType().getStoredType() == DataType.MAP) { + return new MapIndexCreator(context, columnName, indexConfig.getMapIndexConfig()); + } + if (context.hasDictionary()) { // Dictionary enabled columns int cardinality = context.getCardinality(); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java index db815761d9ea..3563775864d1 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexReaderFactory.java @@ -32,6 +32,7 @@ import org.apache.pinot.segment.local.segment.index.readers.forward.VarByteChunkForwardIndexReaderV4; import org.apache.pinot.segment.local.segment.index.readers.forward.VarByteChunkMVForwardIndexReader; import org.apache.pinot.segment.local.segment.index.readers.forward.VarByteChunkSVForwardIndexReader; +import org.apache.pinot.segment.local.segment.index.readers.map.ImmutableMapIndexReader; import org.apache.pinot.segment.local.segment.index.readers.sorted.SortedIndexReaderImpl; import org.apache.pinot.segment.spi.ColumnMetadata; import org.apache.pinot.segment.spi.index.ForwardIndexConfig; @@ -68,7 +69,9 @@ protected ForwardIndexReader createIndexReader(PinotDataBuffer dataBuffer, Colum } public static ForwardIndexReader createIndexReader(PinotDataBuffer dataBuffer, ColumnMetadata metadata) { - if (metadata.hasDictionary()) { + if (metadata.getDataType().getStoredType() == DataType.MAP) { + return new ImmutableMapIndexReader(dataBuffer); + } else if (metadata.hasDictionary()) { if (metadata.isSingleValue()) { if (metadata.isSorted()) { return new SortedIndexReaderImpl(dataBuffer, metadata.getCardinality()); @@ -100,6 +103,10 @@ public static ForwardIndexReader createIndexReader(PinotDataBuffer dataBuffer, C public static ForwardIndexReader createRawIndexReader(PinotDataBuffer dataBuffer, DataType storedType, boolean isSingleValue) { int version = dataBuffer.getInt(0); + if (storedType == DataType.MAP) { + return new ImmutableMapIndexReader(dataBuffer); + } + if (isSingleValue && storedType.isFixedWidth()) { return version == FixedBytePower2ChunkSVForwardIndexReader.VERSION ? new FixedBytePower2ChunkSVForwardIndexReader(dataBuffer, storedType) diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java index 4172a6bdbb57..02f9efa396a6 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/forward/ForwardIndexType.java @@ -33,6 +33,7 @@ import org.apache.pinot.segment.local.realtime.impl.forward.FixedByteMVMutableForwardIndex; import org.apache.pinot.segment.local.realtime.impl.forward.FixedByteSVMutableForwardIndex; import org.apache.pinot.segment.local.realtime.impl.forward.VarByteSVMutableForwardIndex; +import org.apache.pinot.segment.local.realtime.impl.map.MutableMapIndexImpl; import org.apache.pinot.segment.local.segment.index.loader.ConfigurableFromIndexLoadingConfig; import org.apache.pinot.segment.local.segment.index.loader.ForwardIndexHandler; import org.apache.pinot.segment.local.segment.index.loader.IndexLoadingConfig; @@ -59,6 +60,7 @@ import org.apache.pinot.segment.spi.store.SegmentDirectory; import org.apache.pinot.spi.config.table.FieldConfig; import org.apache.pinot.spi.config.table.FieldConfig.CompressionCodec; +import org.apache.pinot.spi.config.table.MapIndexConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.data.FieldSpec; import org.apache.pinot.spi.data.Schema; @@ -109,7 +111,10 @@ public Map fromIndexLoadingConfig(IndexLoadingConfig } } } - forwardIndexConfig = new ForwardIndexConfig.Builder().withCompressionCodec(compressionCodec).build(); + MapIndexConfig mapIndexConfig = indexLoadingConfig.getMapIndexConfigs().get(column); + forwardIndexConfig = new ForwardIndexConfig.Builder() + .withMapIndexConfig(mapIndexConfig) + .withCompressionCodec(compressionCodec).build(); } else { forwardIndexConfig = ForwardIndexConfig.DISABLED; } @@ -201,7 +206,10 @@ protected IndexReaderFactory createReaderFactory() { public String getFileExtension(ColumnMetadata columnMetadata) { if (columnMetadata.isSingleValue()) { - if (!columnMetadata.hasDictionary()) { + if (columnMetadata.getDataType() != null + && columnMetadata.getDataType().getStoredType() == FieldSpec.DataType.MAP) { + return V1Constants.Indexes.MAP_FORWARD_INDEX_FILE_EXTENSION; + } else if (!columnMetadata.hasDictionary()) { return V1Constants.Indexes.RAW_SV_FORWARD_INDEX_FILE_EXTENSION; } else if (columnMetadata.isSorted()) { return V1Constants.Indexes.SORTED_SV_FORWARD_INDEX_FILE_EXTENSION; @@ -265,6 +273,14 @@ public MutableIndex createMutableIndex(MutableIndexContext context, ForwardIndex String column = context.getFieldSpec().getName(); String segmentName = context.getSegmentName(); FieldSpec.DataType storedType = context.getFieldSpec().getDataType().getStoredType(); + + if (storedType == FieldSpec.DataType.MAP) { + MapIndexConfig mapConfig = config.getMapIndexConfig(); + return new MutableMapIndexImpl(mapConfig, context.getMemoryManager(), + context.getCapacity(), context.isOffHeap(), false, context.getConsumerDir().getPath(), + context.getSegmentName()); + } + int fixedLengthBytes = context.getFixedLengthBytes(); boolean isSingleValue = context.getFieldSpec().isSingleValueField(); if (!context.hasDictionary()) { diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java index ac32a0a3545d..fe252f3ddeb0 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/ForwardIndexHandler.java @@ -39,6 +39,7 @@ import org.apache.pinot.segment.local.segment.creator.impl.stats.FloatColumnPreIndexStatsCollector; import org.apache.pinot.segment.local.segment.creator.impl.stats.IntColumnPreIndexStatsCollector; import org.apache.pinot.segment.local.segment.creator.impl.stats.LongColumnPreIndexStatsCollector; +import org.apache.pinot.segment.local.segment.creator.impl.stats.MapColumnPreIndexStatsCollector; import org.apache.pinot.segment.local.segment.creator.impl.stats.StringColumnPreIndexStatsCollector; import org.apache.pinot.segment.local.segment.index.dictionary.DictionaryIndexType; import org.apache.pinot.segment.local.segment.index.forward.ForwardIndexType; @@ -1019,6 +1020,9 @@ private AbstractColumnStatisticsCollector getStatsCollector(String column, Field case BIG_DECIMAL: statsCollector = new BigDecimalColumnPreIndexStatsCollector(column, statsCollectorConfig); break; + case MAP: + statsCollector = new MapColumnPreIndexStatsCollector(column, statsCollectorConfig); + break; default: throw new IllegalStateException("Unsupported storedType=" + storedType.toString() + " for column=" + column); } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java index b711a899c3d0..48f2fc3893f5 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/IndexLoadingConfig.java @@ -18,7 +18,9 @@ */ package org.apache.pinot.segment.local.segment.index.loader; +import com.fasterxml.jackson.databind.JsonNode; import com.google.common.annotations.VisibleForTesting; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -34,11 +36,13 @@ import org.apache.commons.lang3.StringUtils; import org.apache.pinot.common.utils.config.TableConfigUtils; import org.apache.pinot.segment.local.segment.index.column.PhysicalColumnIndexContainer; +import org.apache.pinot.segment.local.segment.index.forward.ForwardIndexType; import org.apache.pinot.segment.local.segment.index.loader.columnminmaxvalue.ColumnMinMaxValueGeneratorMode; import org.apache.pinot.segment.spi.creator.SegmentVersion; import org.apache.pinot.segment.spi.index.ColumnConfigDeserializer; import org.apache.pinot.segment.spi.index.FieldIndexConfigs; import org.apache.pinot.segment.spi.index.FieldIndexConfigsUtil; +import org.apache.pinot.segment.spi.index.ForwardIndexConfig; import org.apache.pinot.segment.spi.index.IndexConfigDeserializer; import org.apache.pinot.segment.spi.index.IndexType; import org.apache.pinot.segment.spi.index.RangeIndexConfig; @@ -53,6 +57,7 @@ import org.apache.pinot.spi.config.table.IndexConfig; import org.apache.pinot.spi.config.table.IndexingConfig; import org.apache.pinot.spi.config.table.JsonIndexConfig; +import org.apache.pinot.spi.config.table.MapIndexConfig; import org.apache.pinot.spi.config.table.StarTreeIndexConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.data.DimensionFieldSpec; @@ -60,6 +65,7 @@ import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.env.PinotConfiguration; import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.JsonUtils; import org.apache.pinot.spi.utils.ReadMode; import org.apache.pinot.spi.utils.TimestampIndexUtils; import org.slf4j.Logger; @@ -84,6 +90,7 @@ public class IndexLoadingConfig { private Set _fstIndexColumns = new HashSet<>(); private FSTType _fstIndexType = FSTType.LUCENE; private Map _jsonIndexConfigs = new HashMap<>(); + private Map _mapIndexConfigs = new HashMap<>(); private Map _h3IndexConfigs = new HashMap<>(); private Map _vectorIndexConfigs = new HashMap<>(); private Set _noDictionaryColumns = new HashSet<>(); // TODO: replace this by _noDictionaryConfig. @@ -216,6 +223,7 @@ private void extractFromTableConfigAndSchema(TableConfig tableConfig, @Nullable } extractCompressionConfigs(tableConfig); + extractMapConfigs(tableConfig); extractTextIndexColumnsFromTableConfig(tableConfig); extractFSTIndexColumnsFromTableConfig(tableConfig); extractH3IndexConfigsFromTableConfig(tableConfig); @@ -360,6 +368,29 @@ private void extractCompressionConfigs(TableConfig tableConfig) { } } + /** + * Extracts configurations for forward indexes on Map type columns. + * @param tableConfig table config + */ + private void extractMapConfigs(TableConfig tableConfig) { + List fieldConfigList = tableConfig.getFieldConfigList(); + if (fieldConfigList == null) { + return; + } + + for (FieldConfig fieldConfig : fieldConfigList) { + String column = fieldConfig.getName(); + JsonNode jsonNode = fieldConfig.getIndexes().get(ForwardIndexType.INDEX_DISPLAY_NAME); + if (jsonNode != null) { + try { + ForwardIndexConfig config = JsonUtils.jsonNodeToObject(jsonNode, ForwardIndexConfig.class); + _mapIndexConfigs.put(column, config.getMapIndexConfig()); + } catch (IOException ioe) { + } + } + } + } + /** * Text index creation info for each column is specified * using {@link FieldConfig} model of indicating per column @@ -542,6 +573,10 @@ public Map getJsonIndexConfigs() { return unmodifiable(_jsonIndexConfigs); } + public Map getMapIndexConfigs() { + return unmodifiable(_mapIndexConfigs); + } + public Map getH3IndexConfigs() { return unmodifiable(_h3IndexConfigs); } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/columnminmaxvalue/ColumnMinMaxValueGenerator.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/columnminmaxvalue/ColumnMinMaxValueGenerator.java index d1cff095bc5e..12411ffd7146 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/columnminmaxvalue/ColumnMinMaxValueGenerator.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/columnminmaxvalue/ColumnMinMaxValueGenerator.java @@ -23,6 +23,7 @@ import java.math.BigDecimal; import java.util.ArrayList; import java.util.List; +import java.util.Set; import org.apache.commons.configuration2.PropertiesConfiguration; import org.apache.commons.lang3.StringUtils; import org.apache.pinot.segment.local.segment.creator.impl.SegmentColumnarIndexCreator; @@ -371,9 +372,30 @@ private void addColumnMinMaxValueWithoutDictionary(ColumnMetadata columnMetadata maxValue = new ByteArray(max); break; } - default: - throw new IllegalStateException("Unsupported data type: " + dataType + " for column: " + columnName); - } + case MAP: + String min = null; + String max = null; + if (isSingleValue) { + for (int docId = 0; docId < numDocs; docId++) { + Set keys = rawIndexReader.getMap(docId, readerContext).keySet(); + for (String key : keys) { + if (min == null || StringUtils.compare(min, key) > 0) { + min = key; + } + if (max == null || StringUtils.compare(max, key) < 0) { + max = key; + } + } + } + minValue = min; + maxValue = max; + } else { + throw new UnsupportedOperationException("MAP columns cannot be Multi Value"); + } + break; + default: + throw new IllegalStateException("Unsupported data type: " + dataType + " for column: " + columnName); + } SegmentColumnarIndexCreator.addColumnMinMaxValueInfo(_segmentProperties, columnName, minValue, maxValue, storedType); } diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java index cec1ea14b631..fd53225de227 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/BaseDefaultColumnHandler.java @@ -939,7 +939,7 @@ private ForwardIndexCreator getForwardIndexCreator(FieldSpec fieldSpec, ColumnIn forwardIndexConfig = fieldIndexConfig.getConfig(new ForwardIndexPlugin().getIndexType()); } if (forwardIndexConfig == null) { - forwardIndexConfig = new ForwardIndexConfig(false, null, null, null, null, null); + forwardIndexConfig = new ForwardIndexConfig(false, null, null, null, null, null, null); } return ForwardIndexCreatorFactory.createIndexCreator(indexCreationContext, forwardIndexConfig); diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/DefaultColumnStatistics.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/DefaultColumnStatistics.java index 1e7acb95aaf9..6ad8054f708f 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/DefaultColumnStatistics.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/loader/defaultcolumn/DefaultColumnStatistics.java @@ -30,20 +30,27 @@ public class DefaultColumnStatistics implements ColumnStatistics { private final Object _maxValue; private final Object _uniqueValuesSet; private final int _cardinality = 1; - private final int _lengthOfShortestElement = -1; - private final int _lengthOfLargestElement = -1; + private final int _lengthOfShortestElement; + private final int _lengthOfLargestElement; private final boolean _isSorted; private final int _totalNumberOfEntries; private final int _maxNumberOfMultiValues; public DefaultColumnStatistics(Object minValue, Object maxValue, Object uniqueValuesSet, boolean isSorted, int totalNumberOfEntries, int maxNumberOfMultiValues) { + this(minValue, maxValue, uniqueValuesSet, isSorted, totalNumberOfEntries, maxNumberOfMultiValues, -1, -1); + } + + public DefaultColumnStatistics(Object minValue, Object maxValue, Object uniqueValuesSet, boolean isSorted, + int totalNumberOfEntries, int maxNumberOfMultiValues, int lengthOfShortestElement, int lengthOfLargestElement) { _minValue = minValue; _maxValue = maxValue; _uniqueValuesSet = uniqueValuesSet; _isSorted = isSorted; _totalNumberOfEntries = totalNumberOfEntries; _maxNumberOfMultiValues = maxNumberOfMultiValues; + _lengthOfShortestElement = lengthOfShortestElement; + _lengthOfLargestElement = lengthOfLargestElement; } @Override diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/map/ImmutableMapDataSource.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/map/ImmutableMapDataSource.java new file mode 100644 index 000000000000..eacb89010ad9 --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/map/ImmutableMapDataSource.java @@ -0,0 +1,152 @@ +/** + * 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.segment.index.map; + +import java.util.Map; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.pinot.segment.local.segment.index.datasource.ImmutableDataSource; +import org.apache.pinot.segment.spi.ColumnMetadata; +import org.apache.pinot.segment.spi.datasource.DataSource; +import org.apache.pinot.segment.spi.datasource.DataSourceMetadata; +import org.apache.pinot.segment.spi.index.IndexReader; +import org.apache.pinot.segment.spi.index.IndexType; +import org.apache.pinot.segment.spi.index.column.ColumnIndexContainer; +import org.apache.pinot.segment.spi.index.reader.MapIndexReader; +import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.spi.data.FieldSpec; + + +/** + * The {@code MutableDataSource} class is the data source for a map type column in the mutable segment. + */ +@SuppressWarnings("rawtypes") +public class ImmutableMapDataSource extends MapDataSource { + public ImmutableMapDataSource(ColumnMetadata columnMetadata, ColumnIndexContainer columnIndexContainer) { + super(new ImmutableMapDataSourceMetadata(columnMetadata), columnIndexContainer); + } + + public DataSource getKeyDataSource(String key) { + MapIndexReader mir = (MapIndexReader) getForwardIndex(); + assert mir != null; + Map indexes = mir.getKeyIndexes(key); + + if (indexes == null) { + // The key does not exist in the map + return new NullDataSource(key); + } + + try (ColumnIndexContainer indexContainer = new ColumnIndexContainer.FromMap(indexes)) { + ColumnMetadata keyMeta = mir.getKeyMetadata(key); + return new ImmutableDataSource(keyMeta, indexContainer); + } catch (Exception ex) { + // TODO(ERICH): what to do here? + } + + return null; + } + + private static class ImmutableMapDataSourceMetadata implements DataSourceMetadata { + final FieldSpec _fieldSpec; + final int _numDocs; + final int _numValues; + final int _maxNumValuesPerMVEntry; + final int _cardinality; + final PartitionFunction _partitionFunction; + final Set _partitions; + final Comparable _minValue; + final Comparable _maxValue; + + ImmutableMapDataSourceMetadata(ColumnMetadata columnMetadata) { + _fieldSpec = columnMetadata.getFieldSpec(); + _numDocs = columnMetadata.getTotalDocs(); + _numValues = columnMetadata.getTotalNumberOfEntries(); + if (_fieldSpec.isSingleValueField()) { + _maxNumValuesPerMVEntry = -1; + } else { + _maxNumValuesPerMVEntry = columnMetadata.getMaxNumberOfMultiValues(); + } + _minValue = columnMetadata.getMinValue(); + _maxValue = columnMetadata.getMaxValue(); + _partitionFunction = columnMetadata.getPartitionFunction(); + _partitions = columnMetadata.getPartitions(); + _cardinality = columnMetadata.getCardinality(); + } + + @Override + public FieldSpec getFieldSpec() { + return _fieldSpec; + } + + @Override + public boolean isSorted() { + // NOTE: Mutable data source is never sorted + return false; + } + + @Override + public int getNumDocs() { + return _numDocs; + } + + @Override + public int getNumValues() { + return _numValues; + } + + @Override + public int getMaxNumValuesPerMVEntry() { + return _maxNumValuesPerMVEntry; + } + + @Nullable + @Override + public Comparable getMinValue() { + return _minValue; + } + + @Override + public Comparable getMaxValue() { + return _maxValue; + } + + @Nullable + @Override + public PartitionFunction getPartitionFunction() { + return _partitionFunction; + } + + @Nullable + @Override + public Set getPartitions() { + return _partitions; + } + + @Override + public int getCardinality() { + return _cardinality; + } + + @Override + public int getMaxRowLengthInBytes() { + throw new UnsupportedOperationException(); + } + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/map/MapDataSource.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/map/MapDataSource.java new file mode 100644 index 000000000000..12ab3b603d4e --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/map/MapDataSource.java @@ -0,0 +1,50 @@ +/** + * 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.segment.index.map; + +import javax.annotation.Nullable; +import org.apache.pinot.segment.local.segment.index.datasource.BaseDataSource; +import org.apache.pinot.segment.spi.datasource.DataSource; +import org.apache.pinot.segment.spi.datasource.DataSourceMetadata; +import org.apache.pinot.segment.spi.index.column.ColumnIndexContainer; +import org.apache.pinot.segment.spi.index.reader.Dictionary; + + +public abstract class MapDataSource extends BaseDataSource { + public MapDataSource(DataSourceMetadata dataSourceMetadata, ColumnIndexContainer indexContainer) { + super(dataSourceMetadata, indexContainer); + } + + /** + * Get the Data Source representation of a single key within this map column. + * + * @param key + * @return + */ + public DataSource getKeyDataSource(String key) { + throw new UnsupportedOperationException(); + } + + @Nullable + @Override + public Dictionary getDictionary() { + return null; + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/map/MutableMapDataSource.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/map/MutableMapDataSource.java new file mode 100644 index 000000000000..4afb710e44e6 --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/map/MutableMapDataSource.java @@ -0,0 +1,201 @@ +/** + * 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.segment.index.map; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.pinot.segment.local.segment.index.datasource.MutableDataSource; +import org.apache.pinot.segment.spi.datasource.DataSource; +import org.apache.pinot.segment.spi.datasource.DataSourceMetadata; +import org.apache.pinot.segment.spi.index.IndexType; +import org.apache.pinot.segment.spi.index.StandardIndexes; +import org.apache.pinot.segment.spi.index.column.ColumnIndexContainer; +import org.apache.pinot.segment.spi.index.mutable.MutableDictionary; +import org.apache.pinot.segment.spi.index.mutable.MutableForwardIndex; +import org.apache.pinot.segment.spi.index.mutable.MutableIndex; +import org.apache.pinot.segment.spi.index.mutable.MutableMapIndex; +import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.spi.data.FieldSpec; + +/** + * The {@code MutableDataSource} class is the data source for a map type column in the mutable segment. + */ +@SuppressWarnings("rawtypes") +public class MutableMapDataSource extends MapDataSource { + Map _mutableIndexes; + private final Map _keyDataSources; + + public MutableMapDataSource(FieldSpec fieldSpec, int numDocs, int numValues, int maxNumValuesPerMVEntry, + int cardinality, @Nullable PartitionFunction partitionFunction, @Nullable Set partitions, + @Nullable Comparable minValue, @Nullable Comparable maxValue, Map mutableIndexes, + MutableDictionary dictionary, int maxRowLengthInBytes) { + super(new MutableMapDataSourceMetadata(fieldSpec, numDocs, numValues, maxNumValuesPerMVEntry, cardinality, + partitionFunction, partitions, minValue, maxValue, maxRowLengthInBytes), + new ColumnIndexContainer.FromMap.Builder() + .withAll(mutableIndexes) + .build()); + _mutableIndexes = mutableIndexes; + _keyDataSources = new HashMap<>(); + } + + public DataSource getKeyDataSource(String key) { + if (_keyDataSources.containsKey(key)) { + return _keyDataSources.get(key); + } + + MutableMapDataSourceMetadata md = (MutableMapDataSourceMetadata) getDataSourceMetadata(); + + MutableMapIndex mpi = (MutableMapIndex) _mutableIndexes.get(StandardIndexes.forward()); + + assert mpi != null; + Map indexes = mpi.getKeyIndexes(key); + DataSource keyDatasource = null; + + if (indexes != null) { + FieldSpec keyFS = mpi.getKeySpec(key); + Comparable minValue = mpi.getMinValueForKey(key); + Comparable maxValue = mpi.getMaxValueForKey(key); + // Get the length of the longest entry + MutableForwardIndex fwdIdx = (MutableForwardIndex) indexes.get(StandardIndexes.forward()); + int lengthLongestEntry = fwdIdx.getLengthOfLongestElement(); + + keyDatasource = new MutableDataSource( + keyFS, + // TODO: We need to get KEY level metadata for these values not Map level metadata + md._numDocs, + md._numValues, + md._maxNumValuesPerMVEntry, + md._cardinality, + md._partitionFunction, + md._partitions, + minValue, + maxValue, + indexes, + null, + null, + lengthLongestEntry + ); + } else { + // If the key does not exist then resolve to the Integer Null value + keyDatasource = new NullDataSource(key); + } + _keyDataSources.put(key, keyDatasource); + + return keyDatasource; + } + + private static class MutableMapDataSourceMetadata implements DataSourceMetadata { + final FieldSpec _fieldSpec; + final int _numDocs; + final int _numValues; + final int _maxNumValuesPerMVEntry; + final int _cardinality; + final PartitionFunction _partitionFunction; + final Set _partitions; + final Comparable _minValue; + final Comparable _maxValue; + final int _maxRowLengthInBytes; + final Set _denseKeys; + + + MutableMapDataSourceMetadata(FieldSpec fieldSpec, int numDocs, int numValues, int maxNumValuesPerMVEntry, + int cardinality, @Nullable PartitionFunction partitionFunction, @Nullable Set partitions, + @Nullable Comparable minValue, @Nullable Comparable maxValue, int maxRowLengthInBytes) { + _fieldSpec = fieldSpec; + _numDocs = numDocs; + _numValues = numValues; + _maxNumValuesPerMVEntry = maxNumValuesPerMVEntry; + if (partitionFunction != null) { + _partitionFunction = partitionFunction; + _partitions = partitions; + } else { + _partitionFunction = null; + _partitions = null; + } + _minValue = minValue; + _maxValue = maxValue; + _cardinality = cardinality; + _maxRowLengthInBytes = maxRowLengthInBytes; + _denseKeys = new HashSet<>(); + } + + @Override + public FieldSpec getFieldSpec() { + return _fieldSpec; + } + + @Override + public boolean isSorted() { + // NOTE: Mutable data source is never sorted + return false; + } + + @Override + public int getNumDocs() { + return _numDocs; + } + + @Override + public int getNumValues() { + return _numValues; + } + + @Override + public int getMaxNumValuesPerMVEntry() { + return _maxNumValuesPerMVEntry; + } + + @Nullable + @Override + public Comparable getMinValue() { + return _minValue; + } + + @Override + public Comparable getMaxValue() { + return _maxValue; + } + + @Nullable + @Override + public PartitionFunction getPartitionFunction() { + return _partitionFunction; + } + + @Nullable + @Override + public Set getPartitions() { + return _partitions; + } + + @Override + public int getCardinality() { + return _cardinality; + } + + @Override + public int getMaxRowLengthInBytes() { + return _maxRowLengthInBytes; + } + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/map/NullDataSource.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/map/NullDataSource.java new file mode 100644 index 000000000000..160c3e939a86 --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/map/NullDataSource.java @@ -0,0 +1,246 @@ +/** + * 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.segment.index.map; + +import java.util.Map; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.pinot.segment.spi.datasource.DataSource; +import org.apache.pinot.segment.spi.datasource.DataSourceMetadata; +import org.apache.pinot.segment.spi.index.IndexReader; +import org.apache.pinot.segment.spi.index.IndexType; +import org.apache.pinot.segment.spi.index.StandardIndexes; +import org.apache.pinot.segment.spi.index.column.ColumnIndexContainer; +import org.apache.pinot.segment.spi.index.reader.BloomFilterReader; +import org.apache.pinot.segment.spi.index.reader.Dictionary; +import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader; +import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext; +import org.apache.pinot.segment.spi.index.reader.H3IndexReader; +import org.apache.pinot.segment.spi.index.reader.InvertedIndexReader; +import org.apache.pinot.segment.spi.index.reader.JsonIndexReader; +import org.apache.pinot.segment.spi.index.reader.NullValueVectorReader; +import org.apache.pinot.segment.spi.index.reader.RangeIndexReader; +import org.apache.pinot.segment.spi.index.reader.TextIndexReader; +import org.apache.pinot.segment.spi.index.reader.VectorIndexReader; +import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.spi.data.DimensionFieldSpec; +import org.apache.pinot.spi.data.FieldSpec; + + +/** + * If a key does not exist in a Map Column, then the Map Data Source will return this NulLDataSource. + * The NullDataSource represents an INT column where every document has the Default Null Value. Semantically, + * this means that if a key is not in a Map column, then the value will always resolve to "Null". + */ +public class NullDataSource implements DataSource { + private final NullDataSourceMetadata _md; + private final ColumnIndexContainer _indexes; + + public NullDataSource(String name) { + _md = new NullDataSourceMetadata(name); + _indexes = new ColumnIndexContainer.FromMap(Map.of(StandardIndexes.forward(), new NullForwardIndex())); + } + + @Override + public DataSourceMetadata getDataSourceMetadata() { + return _md; + } + + @Override + public R getIndex(IndexType type) { + return type.getIndexReader(_indexes); + } + + @Override + public ForwardIndexReader getForwardIndex() { + return getIndex(StandardIndexes.forward()); + } + + @Nullable + @Override + public Dictionary getDictionary() { + return getIndex(StandardIndexes.dictionary()); + } + + @Nullable + @Override + public InvertedIndexReader getInvertedIndex() { + return getIndex(StandardIndexes.inverted()); + } + + @Nullable + @Override + public RangeIndexReader getRangeIndex() { + return getIndex(StandardIndexes.range()); + } + + @Nullable + @Override + public TextIndexReader getTextIndex() { + return getIndex(StandardIndexes.text()); + } + + @Nullable + @Override + public TextIndexReader getFSTIndex() { + return getIndex(StandardIndexes.fst()); + } + + @Nullable + @Override + public JsonIndexReader getJsonIndex() { + return getIndex(StandardIndexes.json()); + } + + @Nullable + @Override + public H3IndexReader getH3Index() { + return getIndex(StandardIndexes.h3()); + } + + @Nullable + @Override + public BloomFilterReader getBloomFilter() { + return getIndex(StandardIndexes.bloomFilter()); + } + + @Nullable + @Override + public NullValueVectorReader getNullValueVector() { + return getIndex(StandardIndexes.nullValueVector()); + } + + @Nullable + @Override + public VectorIndexReader getVectorIndex() { + return getIndex(StandardIndexes.vector()); + } + + public static class NullDataSourceMetadata implements DataSourceMetadata { + String _name; + + NullDataSourceMetadata(String name) { + _name = name; + } + + @Override + public FieldSpec getFieldSpec() { + return new DimensionFieldSpec(_name, FieldSpec.DataType.INT, true); + } + + @Override + public boolean isSorted() { + return false; + } + + @Override + public int getNumDocs() { + return 0; + } + + @Override + public int getNumValues() { + return 0; + } + + @Override + public int getMaxNumValuesPerMVEntry() { + return 0; + } + + @Nullable + @Override + public Comparable getMinValue() { + return FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_INT; + } + + @Nullable + @Override + public Comparable getMaxValue() { + return FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_INT; + } + + @Nullable + @Override + public PartitionFunction getPartitionFunction() { + return null; + } + + @Nullable + @Override + public Set getPartitions() { + return null; + } + + @Override + public int getCardinality() { + return 1; + } + } + + public class NullForwardIndex implements ForwardIndexReader { + NullForwardIndex() { + } + + @Override + public boolean isDictionaryEncoded() { + return false; + } + + @Override + public boolean isSingleValue() { + return false; + } + + @Override + public FieldSpec.DataType getStoredType() { + return FieldSpec.DataType.INT; + } + + @Override + public int getInt(int docId, ForwardIndexReaderContext context) { + return FieldSpec.DEFAULT_DIMENSION_NULL_VALUE_OF_INT; + } + + @Override + public long getLong(int docId, ForwardIndexReaderContext context) { + throw new UnsupportedOperationException(); + } + + @Override + public float getFloat(int docId, ForwardIndexReaderContext context) { + throw new UnsupportedOperationException(); + } + + @Override + public double getDouble(int docId, ForwardIndexReaderContext context) { + throw new UnsupportedOperationException(); + } + + @Override + public String getString(int docId, ForwardIndexReaderContext context) { + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + } + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/map/ImmutableMapIndexReader.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/map/ImmutableMapIndexReader.java new file mode 100644 index 000000000000..0b7bffc04ed0 --- /dev/null +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/index/readers/map/ImmutableMapIndexReader.java @@ -0,0 +1,237 @@ +/** + * 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.segment.index.readers.map; + +import com.google.common.base.Preconditions; +import java.util.HashMap; +import java.util.Map; +import javax.annotation.Nullable; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pinot.segment.local.segment.creator.impl.map.DenseMapHeader; +import org.apache.pinot.segment.local.segment.creator.impl.map.MapIndexHeader; +import org.apache.pinot.segment.local.segment.index.forward.ForwardIndexReaderFactory; +import org.apache.pinot.segment.spi.ColumnMetadata; +import org.apache.pinot.segment.spi.compression.ChunkCompressionType; +import org.apache.pinot.segment.spi.index.IndexReader; +import org.apache.pinot.segment.spi.index.IndexType; +import org.apache.pinot.segment.spi.index.StandardIndexes; +import org.apache.pinot.segment.spi.index.creator.MapIndexCreator; +import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader; +import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext; +import org.apache.pinot.segment.spi.index.reader.MapIndexReader; +import org.apache.pinot.segment.spi.memory.PinotDataBuffer; +import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.utils.JsonUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Reader for map index. + * + * The Header for the Map Index + * - Information that is needed: + * 1. The number of dense columns in the set + * 2. The number of sparse columns in the set + * 3. The offset of each column within the data buffer + * 4. The type of each column + * 5. The index type of each column + * 6. The name of the key for each dense column + * + * Header Layout + * | Version Number + * | Number of Dense Keys + * | Offset of Dense Key Metadata + * | Offset of Sparse Key Metadata + * | Number of Sparse Indexes + * | (Offset, Dense Key Name, Type) Or do we store the metadata for each Keylumn + * | .... | + * | (Sparse column offset, sparse column metadata) + * | .... | + * | ...Actual Data... | + * + * Quesitons: + * 1. How can I a read a forward index from this data buffer? + * 2. How do I read index metadata from this buffer? + */ +public class ImmutableMapIndexReader implements MapIndexReader { + private static final Logger LOGGER = LoggerFactory.getLogger(ImmutableMapIndexReader.class); + // NOTE: Use long type for _numDocs to comply with the RoaringBitmap APIs. + protected final PinotDataBuffer _dataBuffer; + private MapIndexHeader _header; + private final HashMap, ForwardIndexReaderContext>> + _keyIndexes = new HashMap<>(); + + public ImmutableMapIndexReader(PinotDataBuffer dataBuffer) { + int version = dataBuffer.getInt(0); + Preconditions.checkState(version == MapIndexCreator.VERSION_1, + "Unsupported map index version: %s. Valid versions are {}", version, MapIndexCreator.VERSION_1); + _dataBuffer = dataBuffer; + try { + _header = MapIndexHeader.read(_dataBuffer, 0).getLeft(); + } catch (Exception ex) { + LOGGER.error("Error while reading header for map index", ex); + _header = null; + } + loadKeyIndexes(); + } + + @Override + public void close() { + // NOTE: DO NOT close the PinotDataBuffer here because it is tracked by the caller and might be reused later. The + // caller is responsible of closing the PinotDataBuffer. + } + + @Override + public IndexReader getKeyReader(String key, IndexType type) { + Preconditions.checkArgument(type.getId().equals(StandardIndexes.forward().getId()), + "Currently the Map type only supports Forward Indexes"); + // Get the offset for the key index + DenseMapHeader.DenseKeyMetadata keyMetadata = _header.getMapIndex().getKey(key); + if (keyMetadata != null) { + long offset = keyMetadata.getIndexOffset(StandardIndexes.forward()); + long size = keyMetadata.getColumnMetadata().getIndexSizeMap().get(type); + PinotDataBuffer indexBuffer = _dataBuffer.view(offset, offset + size); + return ForwardIndexReaderFactory.createIndexReader(indexBuffer, keyMetadata.getColumnMetadata()); + } else { + return null; + } + } + + @Override + public Map getKeyIndexes(String key) { + IndexReader fwdIdx = getKeyReader(key, StandardIndexes.forward()); + if (fwdIdx != null) { + return Map.of(StandardIndexes.forward(), getKeyReader(key, StandardIndexes.forward())); + } else { + return null; + } + } + + @Override + public FieldSpec.DataType getStoredType(String key) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnMetadata getKeyMetadata(String key) { + return _header.getMapIndex().getKey(key).getColumnMetadata(); + } + + @Override + public boolean isDictionaryEncoded() { + return false; + } + + @Override + public boolean isSingleValue() { + return true; + } + + @Override + public FieldSpec.DataType getStoredType() { + return FieldSpec.DataType.MAP; + } + + @Nullable + @Override + public ChunkCompressionType getCompressionType() { + return ChunkCompressionType.PASS_THROUGH; + } + + @Override + public Map getMap(int docId, ForwardIndexReaderContext mapContext) { + Map mapValue = new HashMap<>(); + + for (DenseMapHeader.DenseKeyMetadata keyMeta : _header.getMapIndex().getKeys()) { + String key = keyMeta.getName(); + Pair, ForwardIndexReaderContext> keyIndex = _keyIndexes.get(key); + + try { + switch (keyIndex.getLeft().getStoredType()) { + case INT: { + int value = keyIndex.getLeft().getInt(docId, keyIndex.getRight()); + mapValue.put(key, value); + break; + } + case LONG: { + long value = keyIndex.getLeft().getLong(docId, keyIndex.getRight()); + mapValue.put(key, value); + break; + } + case FLOAT: { + float value = keyIndex.getLeft().getFloat(docId, keyIndex.getRight()); + mapValue.put(key, value); + break; + } + case DOUBLE: { + double value = keyIndex.getLeft().getDouble(docId, keyIndex.getRight()); + mapValue.put(key, value); + break; + } + case STRING: { + String value = keyIndex.getLeft().getString(docId, keyIndex.getRight()); + mapValue.put(key, value); + break; + } + case BIG_DECIMAL: + case BOOLEAN: + case TIMESTAMP: + case JSON: + case BYTES: + case STRUCT: + case LIST: + case MAP: + case UNKNOWN: + default: + throw new UnsupportedOperationException(); + } + } catch (Exception ex) { + LOGGER.error("Exception caught while reading from key '{}'", key, ex); + throw ex; + } + } + return mapValue; + } + + @Override + public String getString(int docId, ForwardIndexReaderContext context) { + try { + Map mapValue = getMap(docId, context); + return JsonUtils.objectToString(mapValue); + } catch (Exception ex) { + LOGGER.error("Failed to serialize MAP value to JSON String", ex); + } + + return ""; + } + + private void loadKeyIndexes() { + // Iterate over each key in the header and load the index for that key along with the key's context + for (DenseMapHeader.DenseKeyMetadata keyMeta : _header.getMapIndex().getKeys()) { + String key = keyMeta.getName(); + ForwardIndexReader keyIndex = + (ForwardIndexReader) getKeyReader(key, StandardIndexes.forward()); + ForwardIndexReaderContext keyContext = keyIndex.createContext(); + + _keyIndexes.put(key, new ImmutablePair<>(keyIndex, keyContext)); + } + } +} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/readers/PinotSegmentColumnReader.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/readers/PinotSegmentColumnReader.java index aa6ffceb5178..93bc517dff63 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/readers/PinotSegmentColumnReader.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/segment/readers/PinotSegmentColumnReader.java @@ -151,6 +151,8 @@ public Object getValue(int docId) { return _forwardIndexReader.getString(docId, _forwardIndexReaderContext); case BYTES: return _forwardIndexReader.getBytes(docId, _forwardIndexReaderContext); + case MAP: + return _forwardIndexReader.getMap(docId, _forwardIndexReaderContext); default: throw new IllegalStateException("Unsupported SV type: " + storedType); } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplIngestionAggregationTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplIngestionAggregationTest.java index 9f5cec50a850..768172e07dba 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplIngestionAggregationTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/indexsegment/mutable/MutableSegmentImplIngestionAggregationTest.java @@ -58,7 +58,7 @@ public class MutableSegmentImplIngestionAggregationTest { private static Schema.SchemaBuilder getSchemaBuilder() { return new Schema.SchemaBuilder().setSchemaName("testSchema") .addSingleValueDimension(DIMENSION_1, FieldSpec.DataType.INT) - .addSingleValueDimension(DIMENSION_2, FieldSpec.DataType.STRING) + .addSingleValueDimension(DIMENSION_2, FieldSpec.DataType.JSON) .addDateTime(TIME_COLUMN1, FieldSpec.DataType.INT, "1:DAYS:EPOCH", "1:DAYS") .addDateTime(TIME_COLUMN2, FieldSpec.DataType.INT, "1:HOURS:EPOCH", "1:HOURS"); } @@ -95,8 +95,10 @@ public void testSameSrcDifferentAggregations() } GenericRow reuse = new GenericRow(); + ArrayList results = new ArrayList<>(); for (int docId = 0; docId < expectedMax.size(); docId++) { GenericRow row = mutableSegmentImpl.getRecord(docId, reuse); + results.add(row.copy()); String key = buildKey(row); Assert.assertEquals(row.getValue(m2), expectedMin.get(key), key); Assert.assertEquals(row.getValue(m1), expectedMax.get(key), key); @@ -263,8 +265,15 @@ private String buildKey(GenericRow row) { private GenericRow getRow(Random random, Integer multiplicationFactor) { GenericRow row = new GenericRow(); - row.putValue(DIMENSION_1, random.nextInt(2 * multiplicationFactor)); - row.putValue(DIMENSION_2, STRING_VALUES.get(random.nextInt(STRING_VALUES.size()))); + row.putValue(DIMENSION_1, 5); + //row.putValue(DIMENSION_1, random.nextInt(2 * multiplicationFactor)); + + if (random.nextInt() % 2 == 0) { + row.putValue(DIMENSION_2, String.format("{\"a\":\"foo\", \"b\":\"bar\"}")); + } else { + row.putValue(DIMENSION_2, String.format("{\"b\":\"bar\", \"a\":\"foo\"}")); + } + row.putValue(TIME_COLUMN1, random.nextInt(2 * multiplicationFactor)); row.putValue(TIME_COLUMN2, random.nextInt(2 * multiplicationFactor)); 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 2618930bf699..23a8c1f678ae 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 @@ -48,6 +48,7 @@ import org.apache.pinot.segment.spi.ColumnMetadata; import org.apache.pinot.segment.spi.creator.SegmentVersion; import org.apache.pinot.segment.spi.index.DictionaryIndexConfig; +import org.apache.pinot.segment.spi.index.ForwardIndexConfig; import org.apache.pinot.segment.spi.index.StandardIndexes; import org.apache.pinot.segment.spi.index.TextIndexConfig; import org.apache.pinot.segment.spi.index.column.ColumnIndexContainer; @@ -57,6 +58,7 @@ import org.apache.pinot.spi.config.table.FieldConfig; import org.apache.pinot.spi.config.table.IndexConfig; import org.apache.pinot.spi.config.table.IndexingConfig; +import org.apache.pinot.spi.config.table.MapIndexConfig; import org.apache.pinot.spi.config.table.SegmentZKPropsConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; @@ -77,7 +79,6 @@ public class RealtimeSegmentConverterTest { - private static final String STRING_COLUMN1 = "string_col1"; private static final String STRING_COLUMN2 = "string_col2"; private static final String STRING_COLUMN3 = "string_col3"; @@ -88,6 +89,7 @@ public class RealtimeSegmentConverterTest { private static final String LONG_COLUMN4 = "long_col4"; private static final String MV_INT_COLUMN = "mv_col"; private static final String DATE_TIME_COLUMN = "date_time_col"; + private static final String MAP_COLUMN = "map_col"; private static final File TMP_DIR = new File(FileUtils.getTempDirectory(), RealtimeSegmentConverterTest.class.getName()); @@ -412,6 +414,111 @@ public void test10RecordsIndexedColumnMajorSegmentBuilder() testSegment(rows, indexDir, tableConfig, segmentMetadata); } + @Test + public void testMapColumnIndexed() + throws Exception { + File tmpDir = new File(TMP_DIR, "tmp_" + System.currentTimeMillis()); + TableConfig tableConfig = + new TableConfigBuilder(TableType.OFFLINE).setTableName("testTable") + .setTimeColumnName(DATE_TIME_COLUMN) + .setNoDictionaryColumns(List.of(MAP_COLUMN)) + .setColumnMajorSegmentBuilderEnabled(true) + .build(); + + Schema schema = new Schema.SchemaBuilder() + .addSingleValueDimension(MAP_COLUMN, FieldSpec.DataType.MAP) + .addDateTime(DATE_TIME_COLUMN, FieldSpec.DataType.LONG, "1:MILLISECONDS:EPOCH", "1:MILLISECONDS") + .build(); + + String tableNameWithType = tableConfig.getTableName(); + String segmentName = "testTable__0__0__123456map"; + IndexingConfig indexingConfig = tableConfig.getIndexingConfig(); + + MapIndexConfig mapConfig = new MapIndexConfig(); + mapConfig.setDynamicallyCreateDenseKeys(true); + RealtimeSegmentConfig.Builder realtimeSegmentConfigBuilder = + new RealtimeSegmentConfig + .Builder() + .setTableNameWithType(tableNameWithType).setSegmentName(segmentName) + .setStreamName(tableNameWithType).setSchema(schema).setTimeColumnName(DATE_TIME_COLUMN).setCapacity(1000) + .setAvgNumMultiValues(3) + .setSegmentZKMetadata(getSegmentZKMetadata(segmentName)).setOffHeap(true) + .setMemoryManager(new DirectMemoryManager(segmentName)) + .setIndex(MAP_COLUMN, StandardIndexes.forward(), + new ForwardIndexConfig + .Builder(ForwardIndexConfig.DEFAULT) + .withDeriveNumDocsPerChunk(true) + .withMapIndexConfig(mapConfig) + .build()) + .setStatsHistory(RealtimeSegmentStatsHistory.deserialzeFrom(new File(tmpDir, "stats"))) + .setConsumerDir(new File(tmpDir, "consumerDir").getAbsolutePath()); + + // create mutable segment impl + MutableSegmentImpl mutableSegmentImpl = new MutableSegmentImpl(realtimeSegmentConfigBuilder.build(), null); + List rows = generateTestData(); + + for (GenericRow row : rows) { + mutableSegmentImpl.index(row, null); + } + + File outputDir = new File(tmpDir, "outputDir"); + SegmentZKPropsConfig segmentZKPropsConfig = new SegmentZKPropsConfig(); + segmentZKPropsConfig.setStartOffset("1"); + segmentZKPropsConfig.setEndOffset("100"); + ColumnIndicesForRealtimeTable cdc = + new ColumnIndicesForRealtimeTable(null /*indexingConfig.getSortedColumn().get(0)*/, + indexingConfig.getInvertedIndexColumns(), null, null, + indexingConfig.getNoDictionaryColumns(), indexingConfig.getVarLengthDictionaryColumns()); + RealtimeSegmentConverter converter = + new RealtimeSegmentConverter(mutableSegmentImpl, segmentZKPropsConfig, outputDir.getAbsolutePath(), schema, + tableNameWithType, tableConfig, segmentName, cdc, false); + converter.build(SegmentVersion.v3, null); + + File indexDir = new File(outputDir, segmentName); + SegmentMetadataImpl segmentMetadata = new SegmentMetadataImpl(indexDir); + assertEquals(segmentMetadata.getVersion(), SegmentVersion.v3); + assertEquals(segmentMetadata.getTotalDocs(), rows.size()); + assertEquals(segmentMetadata.getTimeColumn(), DATE_TIME_COLUMN); + assertEquals(segmentMetadata.getTimeUnit(), TimeUnit.MILLISECONDS); + + long expectedStartTime = (long) rows.get(0).getValue(DATE_TIME_COLUMN); + assertEquals(segmentMetadata.getStartTime(), expectedStartTime); + long expectedEndTime = (long) rows.get(rows.size() - 1).getValue(DATE_TIME_COLUMN); + assertEquals(segmentMetadata.getEndTime(), expectedEndTime); + + assertTrue(segmentMetadata.getAllColumns().containsAll(schema.getColumnNames())); + assertEquals(segmentMetadata.getStartOffset(), "1"); + assertEquals(segmentMetadata.getEndOffset(), "100"); + + testSegmentWithMap(rows, indexDir, tableConfig, segmentMetadata); + } + + private void testSegmentWithMap(List rows, File indexDir, + TableConfig tableConfig, SegmentMetadataImpl segmentMetadata) + throws IOException { + SegmentLocalFSDirectory segmentDir = new SegmentLocalFSDirectory(indexDir, segmentMetadata, ReadMode.mmap); + SegmentDirectory.Reader segmentReader = segmentDir.createReader(); + + Map indexContainerMap = new HashMap<>(); + Map columnMetadataMap = segmentMetadata.getColumnMetadataMap(); + IndexLoadingConfig indexLoadingConfig = new IndexLoadingConfig(null, tableConfig); + for (Map.Entry entry : columnMetadataMap.entrySet()) { + indexContainerMap.put(entry.getKey(), + new PhysicalColumnIndexContainer(segmentReader, entry.getValue(), indexLoadingConfig)); + } + ImmutableSegmentImpl segmentFile = new ImmutableSegmentImpl(segmentDir, segmentMetadata, indexContainerMap, null); + + GenericRow readRow = new GenericRow(); + int docId = 0; + for (GenericRow row : rows) { + segmentFile.getRecord(docId, readRow); + assertEquals(readRow.getValue(DATE_TIME_COLUMN), row.getValue(DATE_TIME_COLUMN)); + assertEquals(readRow.getValue(MAP_COLUMN), row.getValue(MAP_COLUMN)); + + docId += 1; + } + } + private void testSegment(List rows, File indexDir, TableConfig tableConfig, SegmentMetadataImpl segmentMetadata) throws IOException { @@ -590,6 +697,7 @@ private List generateTestData() { row.putValue(LONG_COLUMN2, 66L + i); row.putValue(LONG_COLUMN3, 65L + i); row.putValue(LONG_COLUMN4, 64L + i); + row.putValue(MAP_COLUMN, Map.of("foo", i, "barry", 10000 + i)); List intList = new ArrayList<>(); intList.add(100 + i); intList.add(200 + i); diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java index a678be426984..5da2778dfed1 100644 --- a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/creator/DictionariesTest.java @@ -45,6 +45,7 @@ import org.apache.pinot.segment.local.segment.creator.impl.stats.FloatColumnPreIndexStatsCollector; import org.apache.pinot.segment.local.segment.creator.impl.stats.IntColumnPreIndexStatsCollector; import org.apache.pinot.segment.local.segment.creator.impl.stats.LongColumnPreIndexStatsCollector; +import org.apache.pinot.segment.local.segment.creator.impl.stats.MapColumnPreIndexStatsCollector; import org.apache.pinot.segment.local.segment.creator.impl.stats.StringColumnPreIndexStatsCollector; import org.apache.pinot.segment.local.segment.index.readers.BigDecimalDictionary; import org.apache.pinot.segment.local.segment.index.readers.DoubleDictionary; @@ -356,6 +357,65 @@ public void testBigDecimalColumnPreIndexStatsCollector() { Assert.assertFalse(statsCollector.isSorted()); } + @Test + public void testMapColumnPreIndexStatsCollector() { + AbstractColumnStatisticsCollector statsCollector = buildStatsCollector("column1", DataType.MAP, true); + statsCollector.collect(Map.of("a", 1, "b", 2)); + Assert.assertFalse(statsCollector.isSorted()); + statsCollector.collect(Map.of("foo", 1, "bar", 2)); + Assert.assertFalse(statsCollector.isSorted()); + statsCollector.collect(Map.of("zaz", 1, "fizz", 2)); + Assert.assertFalse(statsCollector.isSorted()); + statsCollector.collect(Map.of("foo", 5, "bar", -3)); + Assert.assertFalse(statsCollector.isSorted()); + statsCollector.collect(Map.of("aStr", "hello", "bStr", "ABCDEFG")); + Assert.assertFalse(statsCollector.isSorted()); + statsCollector.collect(Map.of("aStr", "zyzzyzxyx", "bStr", "zzz")); + Assert.assertFalse(statsCollector.isSorted()); + statsCollector.seal(); + Assert.assertEquals(statsCollector.getCardinality(), 8); + Assert.assertEquals((String) statsCollector.getMinValue(), "a"); + Assert.assertEquals((String) statsCollector.getMaxValue(), "zaz"); + Assert.assertFalse(statsCollector.isSorted()); + + MapColumnPreIndexStatsCollector mapStats = (MapColumnPreIndexStatsCollector) statsCollector; + AbstractColumnStatisticsCollector aStats = mapStats.getKeyStatistics("a"); + Assert.assertNotNull(aStats); + Assert.assertEquals(aStats.getMinValue(), 1); + Assert.assertEquals(aStats.getMaxValue(), 1); + + AbstractColumnStatisticsCollector bStats = mapStats.getKeyStatistics("b"); + Assert.assertNotNull(bStats); + Assert.assertEquals(bStats.getMinValue(), 2); + Assert.assertEquals(bStats.getMaxValue(), 2); + + AbstractColumnStatisticsCollector fooStats = mapStats.getKeyStatistics("foo"); + Assert.assertNotNull(fooStats); + Assert.assertEquals(fooStats.getMinValue(), 1); + Assert.assertEquals(fooStats.getMaxValue(), 5); + + AbstractColumnStatisticsCollector barStats = mapStats.getKeyStatistics("bar"); + Assert.assertNotNull(barStats); + Assert.assertEquals(barStats.getMinValue(), -3); + Assert.assertEquals(barStats.getMaxValue(), 2); + + AbstractColumnStatisticsCollector aStrStats = mapStats.getKeyStatistics("aStr"); + Assert.assertNotNull(aStrStats); + Assert.assertEquals(aStrStats.getMinValue(), "hello"); + Assert.assertEquals(aStrStats.getMaxValue(), "zyzzyzxyx"); + Assert.assertEquals(aStrStats.getLengthOfLargestElement(), "zyzzyzxyx".length()); + Assert.assertEquals(aStrStats.getLengthOfShortestElement(), -1); + Assert.assertEquals(aStrStats.getMaxRowLengthInBytes(), "zyzzyzxyx".length()); + + AbstractColumnStatisticsCollector bStrStats = mapStats.getKeyStatistics("bStr"); + Assert.assertNotNull(bStrStats); + Assert.assertEquals(bStrStats.getMinValue(), "ABCDEFG"); + Assert.assertEquals(bStrStats.getMaxValue(), "zzz"); + Assert.assertEquals(bStrStats.getLengthOfLargestElement(), "ABCDEFG".length()); + Assert.assertEquals(bStrStats.getLengthOfShortestElement(), -1); + Assert.assertEquals(bStrStats.getMaxRowLengthInBytes(), "ABCDEFG".length()); + } + @Test public void testStringColumnPreIndexStatsCollectorForRandomString() { AbstractColumnStatisticsCollector statsCollector = buildStatsCollector("column1", DataType.STRING); @@ -530,6 +590,8 @@ private AbstractColumnStatisticsCollector buildStatsCollector(String column, Dat return new StringColumnPreIndexStatsCollector(column, statsCollectorConfig); case BYTES: return new BytesColumnPredIndexStatsCollector(column, statsCollectorConfig); + case MAP: + return new MapColumnPreIndexStatsCollector(column, statsCollectorConfig); default: throw new IllegalArgumentException("Illegal data type for stats builder: " + dataType); } diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/MutableMapIndexImplTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/MutableMapIndexImplTest.java new file mode 100644 index 000000000000..07d4880a386e --- /dev/null +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/forward/mutable/MutableMapIndexImplTest.java @@ -0,0 +1,309 @@ +/** + * 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.segment.index.forward.mutable; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import org.apache.pinot.segment.local.io.writer.impl.DirectMemoryManager; +import org.apache.pinot.segment.local.realtime.impl.map.MutableMapIndexImpl; +import org.apache.pinot.segment.spi.index.StandardIndexes; +import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader; +import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext; +import org.apache.pinot.segment.spi.memory.PinotDataBufferMemoryManager; +import org.apache.pinot.spi.config.table.MapIndexConfig; +import org.apache.pinot.spi.data.DimensionFieldSpec; +import org.apache.pinot.spi.data.FieldSpec; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + + +public class MutableMapIndexImplTest { + private PinotDataBufferMemoryManager _memoryManager; + + @BeforeClass + public void setUp() { + _memoryManager = new DirectMemoryManager(MutableMapIndexImplTest.class.getName()); + } + + @AfterClass + public void tearDown() + throws Exception { + _memoryManager.close(); + } + + @Test + public void testAddOneKeyIntegerValue() + throws IOException { + MapIndexConfig config = + new MapIndexConfig(false, 100, List.of( + new DimensionFieldSpec("a", FieldSpec.DataType.INT, true)), false); + + MutableMapIndexImpl mdc = new MutableMapIndexImpl(config, _memoryManager, 1000, false, + false, null, "test-segment"); + HashMap data = new HashMap<>(); + data.put("a", 1); + mdc.add(data, 0); + + data = new HashMap<>(); + data.put("a", 2); + mdc.add(data, 1); + + ForwardIndexReader reader = (ForwardIndexReader) mdc.getKeyReader("a", StandardIndexes.forward()); + ForwardIndexReaderContext ctx = reader.createContext(); + int result = reader.getInt(0, ctx); + Assert.assertEquals(result, 1); + + result = reader.getInt(1, ctx); + Assert.assertEquals(result, 2); + } + + @Test + public void testAddOneKeyIntegerValueDynamicKeys() + throws IOException { + MapIndexConfig config = + new MapIndexConfig(false, 100, List.of( + new DimensionFieldSpec("a", FieldSpec.DataType.INT, true)), true); + + MutableMapIndexImpl mdc = new MutableMapIndexImpl(config, _memoryManager, 1000, false, + false, null, "test-segment"); + HashMap data = new HashMap<>(); + data.put("a", 1); + mdc.add(data, 0); + + data = new HashMap<>(); + data.put("a", 2); + mdc.add(data, 1); + + data = new HashMap<>(); + data.put("a", 5); + data.put("b", "hello"); + mdc.add(data, 2); + + { + ForwardIndexReader reader = (ForwardIndexReader) mdc.getKeyReader("a", StandardIndexes.forward()); + ForwardIndexReaderContext ctx = reader.createContext(); + int result = reader.getInt(0, ctx); + Assert.assertEquals(result, 1); + + result = reader.getInt(1, ctx); + Assert.assertEquals(result, 2); + } + { + ForwardIndexReader reader = (ForwardIndexReader) mdc.getKeyReader("b", StandardIndexes.forward()); + ForwardIndexReaderContext ctx = reader.createContext(); + String result = reader.getString(2, ctx); + Assert.assertEquals(result, "hello"); + + result = reader.getString(0, ctx); + Assert.assertEquals(result, "null"); + result = reader.getString(1, ctx); + Assert.assertEquals(result, "null"); + } + } + + @Test(expectedExceptions = RuntimeException.class) + public void testNumberOfPredefinedKeysExceedsMaxKeys() { + MapIndexConfig config = + new MapIndexConfig(false, 2, + List.of( + new DimensionFieldSpec("a", FieldSpec.DataType.INT, true), + new DimensionFieldSpec("b", FieldSpec.DataType.INT, true), + new DimensionFieldSpec("c", FieldSpec.DataType.INT, true) + ), + true); + + MutableMapIndexImpl mdc = new MutableMapIndexImpl(config, _memoryManager, 1000, false, + false, null, "test-segment"); + } + + @Test + public void testAddOneKeyIntegerValueDynamicKeysKeyLimit() + throws IOException { + MapIndexConfig config = + new MapIndexConfig(false, 2, List.of( + new DimensionFieldSpec("a", FieldSpec.DataType.INT, true)), true); + + MutableMapIndexImpl mdc = new MutableMapIndexImpl(config, _memoryManager, 1000, false, + false, null, "test-segment"); + HashMap data = new HashMap<>(); + data.put("a", 1); + mdc.add(data, 0); + + data = new HashMap<>(); + data.put("a", 2); + mdc.add(data, 1); + + data = new HashMap<>(); + data.put("a", 5); + data.put("b", "hello"); + mdc.add(data, 2); + + data = new HashMap<>(); + data.put("a", 5); + data.put("b", "world"); + data.put("c", 2.0D); + mdc.add(data, 3); + + { + ForwardIndexReader reader = (ForwardIndexReader) mdc.getKeyReader("a", StandardIndexes.forward()); + ForwardIndexReaderContext ctx = reader.createContext(); + int result = reader.getInt(0, ctx); + Assert.assertEquals(result, 1); + + result = reader.getInt(1, ctx); + Assert.assertEquals(result, 2); + } + { + ForwardIndexReader reader = (ForwardIndexReader) mdc.getKeyReader("b", StandardIndexes.forward()); + ForwardIndexReaderContext ctx = reader.createContext(); + String result = reader.getString(2, ctx); + Assert.assertEquals(result, "hello"); + result = reader.getString(3, ctx); + Assert.assertEquals(result, "world"); + + result = reader.getString(0, ctx); + Assert.assertEquals(result, "null"); + result = reader.getString(1, ctx); + Assert.assertEquals(result, "null"); + } + { + ForwardIndexReader reader = (ForwardIndexReader) mdc.getKeyReader("c", StandardIndexes.forward()); + Assert.assertNull(reader); + } + } + + @Test + public void testAddOneKeyStringValue() + throws IOException { + MapIndexConfig config = + new MapIndexConfig(false, 100, List.of( + new DimensionFieldSpec("a", FieldSpec.DataType.STRING, true)), + false); + MutableMapIndexImpl mdc = new MutableMapIndexImpl(config, _memoryManager, 1000, false, + false, null, "test-segment"); + HashMap data = new HashMap<>(); + data.put("a", "hello"); + mdc.add(data, 0); + + data = new HashMap<>(); + data.put("a", "world"); + mdc.add(data, 1); + + ForwardIndexReader reader = (ForwardIndexReader) mdc.getKeyReader("a", StandardIndexes.forward()); + ForwardIndexReaderContext ctx = reader.createContext(); + String result = reader.getString(0, ctx); + Assert.assertEquals(result, "hello"); + + result = reader.getString(1, ctx); + Assert.assertEquals(result, "world"); + } + + @Test + public void testAddOneKeyDoubleValue() + throws IOException { + MapIndexConfig config = + new MapIndexConfig(false, 100, List.of( + new DimensionFieldSpec("a", FieldSpec.DataType.DOUBLE, true)), + false); + MutableMapIndexImpl mdc = new MutableMapIndexImpl(config, _memoryManager, 1000, false, + false, null, "test-segment"); + HashMap data = new HashMap<>(); + double value1 = 2.0; + data.put("a", value1); + mdc.add(data, 0); + + data = new HashMap<>(); + double value2 = 3.0; + data.put("a", value2); + mdc.add(data, 1); + + ForwardIndexReader reader = (ForwardIndexReader) mdc.getKeyReader("a", StandardIndexes.forward()); + ForwardIndexReaderContext ctx = reader.createContext(); + double result = reader.getDouble(0, ctx); + Assert.assertEquals(result, value1); + + result = reader.getDouble(1, ctx); + Assert.assertEquals(result, value2); + } + + @Test + public void testAddTwoKeyIntegerValue() + throws IOException { + MapIndexConfig config = + new MapIndexConfig(false, 100, List.of( + new DimensionFieldSpec("a", FieldSpec.DataType.INT, true), + new DimensionFieldSpec("b", FieldSpec.DataType.INT, true) + ), + false); + MutableMapIndexImpl mdc = new MutableMapIndexImpl(config, _memoryManager, 1000, false, + false, null, "test-segment"); + HashMap data = new HashMap<>(); + data.put("a", 1); + mdc.add(data, 0); + + data = new HashMap<>(); + data.put("b", 2); + mdc.add(data, 1); + + ForwardIndexReader aReader = (ForwardIndexReader) mdc.getKeyReader("a", StandardIndexes.forward()); + ForwardIndexReaderContext ctx = aReader.createContext(); + int result = aReader.getInt(0, ctx); + Assert.assertEquals(result, 1); + + ForwardIndexReader bReader = (ForwardIndexReader) mdc.getKeyReader("b", StandardIndexes.forward()); + result = bReader.getInt(1, ctx); + Assert.assertEquals(result, 2); + } + + @Test + public void testExceedMaxKeys() + throws IOException { + MapIndexConfig config = + new MapIndexConfig(false, 100, List.of( + new DimensionFieldSpec("a", FieldSpec.DataType.INT, true), + new DimensionFieldSpec("b", FieldSpec.DataType.INT, true), + new DimensionFieldSpec("c", FieldSpec.DataType.INT, true), + new DimensionFieldSpec("d", FieldSpec.DataType.INT, true), + new DimensionFieldSpec("e", FieldSpec.DataType.INT, true) + ), + false); + MutableMapIndexImpl mdc = new MutableMapIndexImpl(config, _memoryManager, 1000, false, + false, null, "test-segment"); + HashMap data = new HashMap<>(); + data.put("a", 1); + data.put("b", 1); + data.put("c", 1); + data.put("d", 1); + data.put("e", 1); + mdc.add(data, 0); + Assert.assertNotNull(mdc.getKeyIndexes("a")); + Assert.assertNotNull(mdc.getKeyIndexes("b")); + Assert.assertNotNull(mdc.getKeyIndexes("c")); + Assert.assertNotNull(mdc.getKeyIndexes("d")); + Assert.assertNotNull(mdc.getKeyIndexes("e")); + + data = new HashMap<>(); + data.put("f", 2); + mdc.add(data, 1); + Assert.assertNull(mdc.getKeyIndexes("f")); + } +} diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/map/HeaderTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/map/HeaderTest.java new file mode 100644 index 000000000000..35fb88af82a9 --- /dev/null +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/map/HeaderTest.java @@ -0,0 +1,407 @@ +/** + * 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.segment.index.map; + +import java.io.IOException; +import java.nio.ByteOrder; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pinot.segment.local.segment.creator.impl.map.DenseMapHeader; +import org.apache.pinot.segment.local.segment.creator.impl.map.MapIndexHeader; +import org.apache.pinot.segment.spi.ColumnMetadata; +import org.apache.pinot.segment.spi.index.IndexType; +import org.apache.pinot.segment.spi.index.StandardIndexes; +import org.apache.pinot.segment.spi.index.metadata.ColumnMetadataImpl; +import org.apache.pinot.segment.spi.memory.ByteBufferPinotBufferFactory; +import org.apache.pinot.segment.spi.memory.PinotDataBuffer; +import org.apache.pinot.spi.data.DimensionFieldSpec; +import org.apache.pinot.spi.data.FieldSpec; +import org.testng.Assert; +import org.testng.annotations.Test; + + +public class HeaderTest { + @Test + public void testSize() { + MapIndexHeader.HeaderSizeComputer writer = new MapIndexHeader.HeaderSizeComputer(); + + long offset = writer.putByte(0L, (byte) 1); + Assert.assertEquals(writer.size(), 1); + + offset = writer.putInt(offset, 5); + Assert.assertEquals(writer.size(), 5); + + offset = writer.putLong(offset, 5L); + Assert.assertEquals(writer.size(), 13); + + offset = writer.putFloat(offset, 1.0F); + Assert.assertEquals(writer.size(), 17); + + offset = writer.putDouble(offset, 1.0D); + Assert.assertEquals(writer.size(), 25); + + offset = writer.putString(offset, "Hello"); + Assert.assertEquals(writer.size(), 25 + 4 + 10); + + offset = writer.putValue(offset, FieldSpec.DataType.STRING, "Test"); + Assert.assertEquals(writer.size(), 39 + 4 + 8); + + offset = writer.putValue(offset, FieldSpec.DataType.INT, 1); + Assert.assertEquals(writer.size(), 51 + 4); + + offset = writer.putValue(offset, FieldSpec.DataType.LONG, 1L); + Assert.assertEquals(writer.size(), 55 + 8); + + offset = writer.putValue(offset, FieldSpec.DataType.FLOAT, 1.0F); + Assert.assertEquals(writer.size(), 63 + 4); + + offset = writer.putValue(offset, FieldSpec.DataType.DOUBLE, 1.0D); + Assert.assertEquals(writer.size(), 67 + 8); + } + + @Test + public void testWritingColumnMetadataSize() { + ColumnMetadata meta = createIntKeyMetadata(new DimensionFieldSpec("test", FieldSpec.DataType.INT, true)); + DenseMapHeader.DenseKeyMetadata dk = new DenseMapHeader.DenseKeyMetadata( + "test", + meta, + Map.of(StandardIndexes.forward(), -1L), + 5 + ); + + /* Size computation + + Data Type: 4 + 6 | 10 + Is SV: 1 | 11 + has dict : 1 | 12 + is sorted: 1 | 13 + total docs: 4 | 17 + card: 4 | 21 + max length: 4 | 25 + bits per: 4 | 29 + max mvs: 4 | 33 + total entries: 4 | 37 + min value: 4 | 41 + max value: 4 | 45 + minmax invalid: 1 | 46 + number indexes: 4 | 50 + + Index Size: + Forward Index: + ID ("forward_index"): 4 + 26 | 80 + Size: 8 | 88 + + Total Size: 88 + */ + + MapIndexHeader.HeaderSizeComputer writer = new MapIndexHeader.HeaderSizeComputer(); + dk.writeColumnMetadata(writer, 0); + Assert.assertEquals(writer.size(), 88); + } + + @Test + public void testDenseKeyMetadataSize() throws IOException { + ColumnMetadata meta = createIntKeyMetadata(new DimensionFieldSpec("test", FieldSpec.DataType.INT, true)); + DenseMapHeader.DenseKeyMetadata dk = new DenseMapHeader.DenseKeyMetadata( + "test", + meta, + Map.of(StandardIndexes.forward(), 100L), + 5 + ); + + /* + Key ("test"): 4 + 8 + Doc Id offset: 4 + Number of Indices : 4 + Index ID ("forward_index"): 4 + 26 + Index Offset: 8 + Col Metadata: 88 + + Total: 12 + 4 + 4 + 30 + 8 + 88 = 146 + */ + + MapIndexHeader.HeaderSizeComputer writer = new MapIndexHeader.HeaderSizeComputer(); + dk.write(writer, 0); + Assert.assertEquals(writer.size(), 146); + } + + @Test + public void testDenseMapMetadataSize() throws IOException { + ColumnMetadata meta = createIntKeyMetadata(new DimensionFieldSpec("test", FieldSpec.DataType.INT, true)); + DenseMapHeader.DenseKeyMetadata dk = new DenseMapHeader.DenseKeyMetadata( + "test", + meta, + Map.of(StandardIndexes.forward(), 100L), + 5 + ); + ColumnMetadata meta2 = createIntKeyMetadata(new DimensionFieldSpec("test2", FieldSpec.DataType.INT, true)); + DenseMapHeader.DenseKeyMetadata dk2 = new DenseMapHeader.DenseKeyMetadata( + "test2", + meta2, + Map.of(StandardIndexes.forward(), 100L), + 500 + ); + + MapIndexHeader.HeaderSizeComputer writer = new MapIndexHeader.HeaderSizeComputer(); + dk.write(writer, 0); + Assert.assertEquals(writer.size(), 146); + + MapIndexHeader.HeaderSizeComputer writer2 = new MapIndexHeader.HeaderSizeComputer(); + dk2.write(writer2, 0); + Assert.assertEquals(writer2.size(), 148); + + DenseMapHeader mapMD = new DenseMapHeader(); + mapMD.addKey("test", meta, List.of(StandardIndexes.forward()), 5); + mapMD.addKey("test2", meta2, List.of(StandardIndexes.forward()), 500); + + /* + Size: + + Index type ("dense_map_index"): 4 + 30 + Num Keys: 4 + + Key ('test'): 146 + Key ('test2'): 148 + + total = 146 + 148 + 34 + 4 = 332 + */ + + MapIndexHeader.HeaderSizeComputer writer3 = new MapIndexHeader.HeaderSizeComputer(); + mapMD.write(writer3, 0); + Assert.assertEquals(writer3.size(), 332); + } + + @Test + public void testWriterMapIndexHeader() throws IOException { + ColumnMetadata meta = createIntKeyMetadata(new DimensionFieldSpec("test", FieldSpec.DataType.INT, true)); + DenseMapHeader.DenseKeyMetadata dk = new DenseMapHeader.DenseKeyMetadata( + "test", + meta, + Map.of(StandardIndexes.forward(), 100L), + 5 + ); + ColumnMetadata meta2 = createIntKeyMetadata(new DimensionFieldSpec("test2", FieldSpec.DataType.INT, true)); + DenseMapHeader.DenseKeyMetadata dk2 = new DenseMapHeader.DenseKeyMetadata( + "test2", + meta2, + Map.of(StandardIndexes.forward(), 100L), + 500 + ); + + + DenseMapHeader mapMD = new DenseMapHeader(); + mapMD.addKey("test", meta, List.of(StandardIndexes.forward()), 5); + mapMD.addKey("test2", meta2, List.of(StandardIndexes.forward()), 500); + + MapIndexHeader header = new MapIndexHeader(); + header.addMapIndex(mapMD); + + /* + Size: + + Version: 4 + number indexes: 4 + Dense Map: 332 + */ + + Assert.assertEquals(header.size(), 340); + } + + @Test + public void testWritingColumnMetadata() { + ColumnMetadata meta = createIntKeyMetadata(new DimensionFieldSpec("test", FieldSpec.DataType.INT, true)); + DenseMapHeader.DenseKeyMetadata dk = new DenseMapHeader.DenseKeyMetadata( + "test", + meta, + Map.of(StandardIndexes.forward(), 100L), + 5 + ); + MapIndexHeader.HeaderSizeComputer writer = new MapIndexHeader.HeaderSizeComputer(); + dk.writeColumnMetadata(writer, 0); + long size = writer.size(); + + PinotDataBuffer buffer = new ByteBufferPinotBufferFactory().allocateDirect(size, ByteOrder.BIG_ENDIAN); + MapIndexHeader.PinotDataBufferWriter bufferWriter = new MapIndexHeader.PinotDataBufferWriter(buffer); + dk.writeColumnMetadata(bufferWriter, 0); + buffer.flush(); + + Pair actualMD = DenseMapHeader.readColumnMetadata( + buffer, 0, "test"); + Assert.assertEquals(actualMD.getLeft(), meta); + } + + @Test + public void testWritingDenseKeyMetadata() throws IOException { + ColumnMetadata meta = createIntKeyMetadata(new DimensionFieldSpec("test", FieldSpec.DataType.INT, true)); + DenseMapHeader.DenseKeyMetadata dk = new DenseMapHeader.DenseKeyMetadata( + "test", + meta, + Map.of(StandardIndexes.forward(), 100L), + 5 + ); + MapIndexHeader.HeaderSizeComputer writer = new MapIndexHeader.HeaderSizeComputer(); + dk.write(writer, 0); + long size = writer.size(); + + PinotDataBuffer buffer = new ByteBufferPinotBufferFactory().allocateDirect(size, ByteOrder.BIG_ENDIAN); + + MapIndexHeader.PinotDataBufferWriter bufferWriter = new MapIndexHeader.PinotDataBufferWriter(buffer); + dk.write(bufferWriter, 0); + buffer.flush(); + + Pair result = + DenseMapHeader.DenseKeyMetadata.read(buffer, 0); + Assert.assertEquals(result.getLeft(), dk); + Assert.assertEquals(result.getLeft().getIndexOffset(StandardIndexes.forward()), 100L); + } + + @Test + public void testUpdatingIndexOffsetDenseKeyMetadata() throws IOException { + ColumnMetadata meta = createIntKeyMetadata(new DimensionFieldSpec("test", FieldSpec.DataType.INT, true)); + DenseMapHeader.DenseKeyMetadata dk = new DenseMapHeader.DenseKeyMetadata( + "test", + meta, + List.of(StandardIndexes.forward()), + 5 + ); + MapIndexHeader.HeaderSizeComputer writer = new MapIndexHeader.HeaderSizeComputer(); + dk.write(writer, 0); + long size = writer.size(); + + PinotDataBuffer buffer = new ByteBufferPinotBufferFactory().allocateDirect(size, ByteOrder.BIG_ENDIAN); + + MapIndexHeader.PinotDataBufferWriter bufferWriter = new MapIndexHeader.PinotDataBufferWriter(buffer); + dk.write(bufferWriter, 0); + buffer.flush(); + + { + Pair result + = DenseMapHeader.DenseKeyMetadata.read(buffer, 0); + Assert.assertEquals(result.getLeft(), dk); + Assert.assertEquals(result.getLeft().getIndexOffset(StandardIndexes.forward()), 0xDEADBEEFDEADBEEFL); + } + + dk.setIndexOffset(bufferWriter, StandardIndexes.forward(), 100L); + buffer.flush(); + + { + Pair result + = DenseMapHeader.DenseKeyMetadata.read(buffer, 0); + Assert.assertEquals(result.getLeft(), dk); + Assert.assertEquals(result.getLeft().getIndexOffset(StandardIndexes.forward()), 100L); + } + } + + @Test + public void testDenseMapMetadataWrite() throws IOException { + ColumnMetadata meta = createIntKeyMetadata(new DimensionFieldSpec("test", FieldSpec.DataType.INT, true)); + ColumnMetadata meta2 = createIntKeyMetadata(new DimensionFieldSpec("test2", FieldSpec.DataType.INT, true)); + + DenseMapHeader mapMD = new DenseMapHeader(); + mapMD.addKey("test", meta, List.of(StandardIndexes.forward()), 5); + mapMD.addKey("test2", meta2, List.of(StandardIndexes.forward(), StandardIndexes.range()), 500); + + MapIndexHeader.HeaderSizeComputer sizer = new MapIndexHeader.HeaderSizeComputer(); + mapMD.write(sizer, 0); + + PinotDataBuffer buffer = new ByteBufferPinotBufferFactory().allocateDirect(sizer.size(), ByteOrder.BIG_ENDIAN); + + MapIndexHeader.PinotDataBufferWriter bufferWriter = new MapIndexHeader.PinotDataBufferWriter(buffer); + mapMD.write(bufferWriter, 0); + buffer.flush(); + + Pair result = + DenseMapHeader.read(buffer, 0); + Assert.assertEquals(result.getLeft(), mapMD); + } + + @Test + public void testMapHeaderWrite() throws IOException { + ColumnMetadata meta = createIntKeyMetadata(new DimensionFieldSpec("test", FieldSpec.DataType.INT, true)); + ColumnMetadata meta2 = createIntKeyMetadata(new DimensionFieldSpec("test2", FieldSpec.DataType.INT, true)); + ColumnMetadata meta3 = createLongKeyMetadata(new DimensionFieldSpec("test3", FieldSpec.DataType.LONG, true)); + + DenseMapHeader mapMD = new DenseMapHeader(); + mapMD.addKey("test", meta, List.of(StandardIndexes.forward()), 5); + mapMD.addKey("test2", meta2, List.of(StandardIndexes.forward(), StandardIndexes.range()), 500); + mapMD.addKey("test3", meta3, List.of(StandardIndexes.forward(), StandardIndexes.range()), 1500); + + MapIndexHeader header = new MapIndexHeader(); + header.addMapIndex(mapMD); + + long size = header.size(); + PinotDataBuffer buffer = new ByteBufferPinotBufferFactory().allocateDirect(size, ByteOrder.BIG_ENDIAN); + + header.write(buffer, 0); + buffer.flush(); + + Pair result = MapIndexHeader.read(buffer, 0); + Assert.assertEquals(result.getLeft(), header); + } + + private ColumnMetadata createIntKeyMetadata(FieldSpec spec) { + HashMap, Long> indexSizeMap = new HashMap<>(); + indexSizeMap.put(StandardIndexes.forward(), 100L); + ColumnMetadataImpl.Builder builder = ColumnMetadataImpl.builder() + .setFieldSpec(spec) + .setCardinality(1) + .setTotalDocs(10) + .setAutoGenerated(false) + .setBitsPerElement(5) + .setMinValue(0) + .setMaxValue(1000) + .setHasDictionary(false) + .setSorted(false) + .setMaxNumberOfMultiValues(0) + .setColumnMaxLength(4) + .setPartitions(null) + .setPartitionFunction(null) + .setMinMaxValueInvalid(false) + .setTotalNumberOfEntries(5); + builder.setIndexSizeMap(indexSizeMap); + + return builder.build(); + } + + private ColumnMetadata createLongKeyMetadata(FieldSpec spec) { + HashMap, Long> indexSizeMap = new HashMap<>(); + indexSizeMap.put(StandardIndexes.forward(), 100L); + ColumnMetadataImpl.Builder builder = ColumnMetadataImpl.builder() + .setFieldSpec(spec) + .setCardinality(1) + .setTotalDocs(10) + .setAutoGenerated(false) + .setBitsPerElement(5) + .setMinValue(0L) + .setMaxValue(1000L) + .setHasDictionary(false) + .setSorted(false) + .setMaxNumberOfMultiValues(0) + .setColumnMaxLength(4) + .setPartitions(null) + .setPartitionFunction(null) + .setMinMaxValueInvalid(false) + .setTotalNumberOfEntries(5); + builder.setIndexSizeMap(indexSizeMap); + + return builder.build(); + } +} diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/map/ImmutableMapIndexReaderTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/map/ImmutableMapIndexReaderTest.java new file mode 100644 index 000000000000..3f1bffdc6e88 --- /dev/null +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/map/ImmutableMapIndexReaderTest.java @@ -0,0 +1,454 @@ +/** + * 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.segment.index.map; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteOrder; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pinot.segment.local.segment.creator.impl.map.MapIndexCreator; +import org.apache.pinot.segment.local.segment.index.readers.map.ImmutableMapIndexReader; +import org.apache.pinot.segment.spi.creator.IndexCreationContext; +import org.apache.pinot.segment.spi.creator.MapColumnStatistics; +import org.apache.pinot.segment.spi.index.StandardIndexes; +import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader; +import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext; +import org.apache.pinot.segment.spi.memory.PinotDataBuffer; +import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.spi.config.table.MapIndexConfig; +import org.apache.pinot.spi.data.DimensionFieldSpec; +import org.apache.pinot.spi.data.FieldSpec; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import static org.apache.pinot.segment.spi.V1Constants.Indexes.MAP_FORWARD_INDEX_FILE_EXTENSION; + + +public class ImmutableMapIndexReaderTest { + private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "MapIndexTest"); + private static final String MAP_COLUMN_NAME = "test_map"; + + @BeforeMethod + public void setUp() + throws IOException { + FileUtils.forceMkdir(INDEX_DIR); + } + + @AfterMethod + public void tearDown() + throws IOException { + FileUtils.deleteDirectory(INDEX_DIR); + } + + @Test + public void testingWritingMultipleChunks() { + List keys = List.of( + new DimensionFieldSpec("foo", FieldSpec.DataType.INT, true), + new DimensionFieldSpec("barry", FieldSpec.DataType.INT, true)); + List> records = createTestData(keys, 2000); + MapIndexConfig config = new MapIndexConfig(); + config.setDenseKeys(keys); + config.setMaxKeys(4); + try { + String mapIndexFilePath = createIndex(config, records); + + File mapIndexFile = new File(mapIndexFilePath); + long size = Files.size(mapIndexFile.toPath()); + try (PinotDataBuffer mapBuffer = PinotDataBuffer.mapFile(mapIndexFile, true, 0, size, ByteOrder.BIG_ENDIAN, + "test")) { + ImmutableMapIndexReader mapReader = new ImmutableMapIndexReader(mapBuffer); + ForwardIndexReader aReader = (ForwardIndexReader) mapReader.getKeyReader("foo", StandardIndexes.forward()); + + ForwardIndexReaderContext context = aReader.createContext(); + int doc0 = aReader.getInt(0, context); + Assert.assertEquals(doc0, 1); + } catch (Exception ex) { + throw ex; + } + } catch (Exception ex) { + Assert.fail("Error Creating Index", ex); + } + } + + @Test + public void testingWithDoubleType() { + List keys = List.of( + new DimensionFieldSpec("foo", FieldSpec.DataType.DOUBLE, true), + new DimensionFieldSpec("barry", FieldSpec.DataType.DOUBLE, true)); + List> records = createTestData(keys, 2000); + MapIndexConfig config = new MapIndexConfig(); + config.setDenseKeys(keys); + config.setMaxKeys(4); + try { + String mapIndexFilePath = createIndex(config, records); + + File mapIndexFile = new File(mapIndexFilePath); + long size = Files.size(mapIndexFile.toPath()); + try (PinotDataBuffer mapBuffer = PinotDataBuffer.mapFile(mapIndexFile, true, 0, size, ByteOrder.BIG_ENDIAN, + "test")) { + ImmutableMapIndexReader mapReader = new ImmutableMapIndexReader(mapBuffer); + + ForwardIndexReader aReader = (ForwardIndexReader) mapReader.getKeyReader("foo", StandardIndexes.forward()); + testForwardIndexReader("foo", aReader, records); + + ForwardIndexReader bReader = (ForwardIndexReader) mapReader.getKeyReader("barry", StandardIndexes.forward()); + testForwardIndexReader("barry", bReader, records); + } catch (Exception ex) { + throw ex; + } + } catch (Exception ex) { + Assert.fail("Error Creating Index", ex); + } + } + + @Test + public void testingWithMixedTypes() { + List keys = List.of( + new DimensionFieldSpec("foo", FieldSpec.DataType.INT, true), + new DimensionFieldSpec("barry", FieldSpec.DataType.LONG, true), + new DimensionFieldSpec("c", FieldSpec.DataType.FLOAT, true), + new DimensionFieldSpec("d", FieldSpec.DataType.DOUBLE, true) + ); + List> records = createTestData(keys, 2000); + MapIndexConfig config = new MapIndexConfig(); + config.setDenseKeys(keys); + config.setMaxKeys(4); + try { + String mapIndexFilePath = createIndex(config, records); + + File mapIndexFile = new File(mapIndexFilePath); + long size = Files.size(mapIndexFile.toPath()); + try (PinotDataBuffer mapBuffer = PinotDataBuffer.mapFile(mapIndexFile, true, 0, size, ByteOrder.BIG_ENDIAN, + "test")) { + ImmutableMapIndexReader mapReader = new ImmutableMapIndexReader(mapBuffer); + + for (FieldSpec key : keys) { + ForwardIndexReader keyReader = (ForwardIndexReader) mapReader.getKeyReader(key.getName(), + StandardIndexes.forward()); + testForwardIndexReader(key.getName(), keyReader, records); + } + } catch (Exception ex) { + throw ex; + } + } catch (Exception ex) { + Assert.fail("Error Creating Index", ex); + } + } + + private List> createTestData(List keys, int numRecords) { + HashMap record = new HashMap<>(); + + for (int i = 0; i < keys.size(); i++) { + record.put(keys.get(i).getName(), generateTestValue(keys.get(i).getDataType().getStoredType())); + } + + ArrayList> records = new ArrayList<>(); + for (int i = 0; i < numRecords; i++) { + records.add(record); + } + + return records; + } + + private Object generateTestValue(FieldSpec.DataType type) { + switch (type) { + case INT: + return 1; + case LONG: + return 2L; + case FLOAT: + return 3.0F; + case DOUBLE: + return 4.5D; + case BOOLEAN: + return true; + case TIMESTAMP: + case STRING: + return "hello"; + case JSON: + case BIG_DECIMAL: + case BYTES: + case STRUCT: + case MAP: + case LIST: + case UNKNOWN: + default: + throw new UnsupportedOperationException(); + } + } + + private void testForwardIndexReader(String key, ForwardIndexReader reader, + List> records) { + FieldSpec.DataType type = reader.getStoredType(); + ForwardIndexReaderContext context = reader.createContext(); + for (int docId = 0; docId < records.size(); docId++) { + switch (type) { + case INT: { + int expectedValue = (Integer) records.get(docId).get(key); + int actualValue = reader.getInt(docId, context); + Assert.assertEquals(actualValue, expectedValue, String.format("Mismatch for DocId: %d", docId)); + break; + } + case LONG: { + long expectedValue = (Long) records.get(docId).get(key); + long actualValue = reader.getLong(docId, context); + Assert.assertEquals(actualValue, expectedValue, String.format("Mismatch for DocId: %d", docId)); + break; + } + case FLOAT: { + float expectedValue = (Float) records.get(docId).get(key); + float actualValue = reader.getFloat(docId, context); + Assert.assertEquals(actualValue, expectedValue, String.format("Mismatch for DocId: %d", docId)); + break; + } + case DOUBLE: { + double expectedValue = (Double) records.get(docId).get(key); + double actualValue = reader.getDouble(docId, context); + Assert.assertEquals(actualValue, expectedValue, String.format("Mismatch for DocId: %d", docId)); + break; + } + case BIG_DECIMAL: + case BOOLEAN: + case TIMESTAMP: + case STRING: + case JSON: + case BYTES: + case STRUCT: + case MAP: + case LIST: + case UNKNOWN: + default: + throw new UnsupportedOperationException(); + } + } + } + + /** + * Creates a Dense Map index with the given config and adds the given records + * @param records + * @throws IOException on error + */ + private String createIndex(MapIndexConfig mapIndexConfig, List> records) + throws IOException { + DimensionFieldSpec mapSpec = new DimensionFieldSpec(); + mapSpec.setDataType(FieldSpec.DataType.MAP); + mapSpec.setName(MAP_COLUMN_NAME); + + IndexCreationContext context = + new IndexCreationContext.Common.Builder().withIndexDir(INDEX_DIR).withTotalDocs(records.size()) + .withColumnStatistics(createMapColumnStats(records)) + .withLengthOfLongestEntry(30).sorted(false).onHeap(false).withDictionary(false).withFieldSpec(mapSpec) + .build(); + try (org.apache.pinot.segment.spi.index.creator.MapIndexCreator indexCreator = new MapIndexCreator(context, + MAP_COLUMN_NAME, mapIndexConfig)) { + for (HashMap record : records) { + indexCreator.add(record); + } + indexCreator.seal(); + } + + return String.format("%s/%s%s", INDEX_DIR, MAP_COLUMN_NAME, MAP_FORWARD_INDEX_FILE_EXTENSION); + } + + private MapColumnStatistics createMapColumnStats(List> records) { + TestMapColStatistics stats = new TestMapColStatistics(); + + for (HashMap record : records) { + for (Map.Entry entry : record.entrySet()) { + stats.recordValue(entry.getKey(), (Comparable) entry.getValue()); + } + } + + return stats; + } + + static class TestMapColStatistics implements MapColumnStatistics { + private HashMap _minValueByKey; + private HashMap _maxValueByKey; + private HashMap _shortestLengthByKey; + private HashMap _longestLengthByKey; + + public TestMapColStatistics() { + _minValueByKey = new HashMap<>(); + _maxValueByKey = new HashMap<>(); + _shortestLengthByKey = new HashMap<>(); + _longestLengthByKey = new HashMap<>(); + } + + public void recordValue(String key, Comparable value) { + if (_minValueByKey.containsKey(key)) { + if (value.compareTo(_minValueByKey.get(key)) < 0) { + _minValueByKey.put(key, value); + } + } else { + _minValueByKey.put(key, value); + } + + if (_maxValueByKey.containsKey(key)) { + if (value.compareTo(_maxValueByKey.get(key)) > 0) { + _maxValueByKey.put(key, value); + } + } else { + _maxValueByKey.put(key, value); + } + + // Get the length of the value + if (value instanceof Integer) { + _shortestLengthByKey.put(key, Integer.BYTES); + _longestLengthByKey.put(key, Integer.BYTES); + } else if (value instanceof Long) { + _shortestLengthByKey.put(key, Long.BYTES); + _longestLengthByKey.put(key, Long.BYTES); + } else if (value instanceof Float) { + _shortestLengthByKey.put(key, Float.BYTES); + _longestLengthByKey.put(key, Float.BYTES); + } else if (value instanceof Double) { + _shortestLengthByKey.put(key, Double.BYTES); + _longestLengthByKey.put(key, Double.BYTES); + } else if (value instanceof String) { + int length = ((String) value).length(); + if (_shortestLengthByKey.containsKey(key)) { + if (length < _shortestLengthByKey.get(key)) { + _shortestLengthByKey.put(key, length); + } + } else { + _shortestLengthByKey.put(key, length); + } + + if (_longestLengthByKey.containsKey(key)) { + if (length > _longestLengthByKey.get(key)) { + _longestLengthByKey.put(key, length); + } + } else { + _longestLengthByKey.put(key, length); + } + } + } + + @Override + public Object getMinValue() { + return null; + } + + @Override + public Object getMaxValue() { + return null; + } + + @Override + public Object getUniqueValuesSet() { + return null; + } + + @Override + public int getCardinality() { + return 0; + } + + @Override + public int getLengthOfShortestElement() { + return 0; + } + + @Override + public int getLengthOfLargestElement() { + return 0; + } + + @Override + public boolean isSorted() { + return false; + } + + @Override + public int getTotalNumberOfEntries() { + return 0; + } + + @Override + public int getMaxNumberOfMultiValues() { + return 0; + } + + @Override + public PartitionFunction getPartitionFunction() { + return null; + } + + @Override + public int getNumPartitions() { + return 0; + } + + @Override + public Map getPartitionFunctionConfig() { + return null; + } + + @Override + public Set getPartitions() { + return null; + } + + @Override + public Object getMinValueForKey(String key) { + return null; + } + + @Override + public Object getMaxValueForKey(String key) { + return null; + } + + @Override + public int getLengthOfShortestElementForKey(String key) { + return 0; + } + + @Override + public int getLengthOfLargestElementForKey(String key) { + return 0; + } + + @Override + public Set> getKeys() { + return null; + } + + @Override + public boolean isSortedForKey(String key) { + return false; + } + + @Override + public int getTotalNumberOfEntriesForKey(String key) { + return 0; + } + } +} diff --git a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/map/MapIndexWriterTest.java b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/map/MapIndexWriterTest.java new file mode 100644 index 000000000000..4fb055b988b5 --- /dev/null +++ b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/segment/index/map/MapIndexWriterTest.java @@ -0,0 +1,618 @@ +/** + * 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.segment.index.map; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteOrder; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pinot.segment.local.segment.creator.impl.map.DenseMapHeader; +import org.apache.pinot.segment.local.segment.creator.impl.map.MapIndexCreator; +import org.apache.pinot.segment.local.segment.creator.impl.map.MapIndexHeader; +import org.apache.pinot.segment.local.segment.index.forward.ForwardIndexReaderFactory; +import org.apache.pinot.segment.spi.ColumnMetadata; +import org.apache.pinot.segment.spi.creator.IndexCreationContext; +import org.apache.pinot.segment.spi.creator.MapColumnStatistics; +import org.apache.pinot.segment.spi.index.StandardIndexes; +import org.apache.pinot.segment.spi.index.reader.ForwardIndexReader; +import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext; +import org.apache.pinot.segment.spi.memory.PinotDataBuffer; +import org.apache.pinot.segment.spi.partition.PartitionFunction; +import org.apache.pinot.spi.config.table.MapIndexConfig; +import org.apache.pinot.spi.data.DimensionFieldSpec; +import org.apache.pinot.spi.data.FieldSpec; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import static org.apache.pinot.segment.spi.V1Constants.Indexes.MAP_FORWARD_INDEX_FILE_EXTENSION; + +public class MapIndexWriterTest { + private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "MapIndexTest"); + private static final String MAP_COLUMN_NAME = "test_map"; + + @BeforeMethod + public void setUp() + throws IOException { + FileUtils.forceMkdir(INDEX_DIR); + } + + @AfterMethod + public void tearDown() + throws IOException { + FileUtils.deleteDirectory(INDEX_DIR); + } + + @Test + public void testingWritingMultipleChunks() { + List keys = List.of( + new DimensionFieldSpec("a", FieldSpec.DataType.INT, true), + new DimensionFieldSpec("b", FieldSpec.DataType.INT, true) + ); + List> records = createTestData(keys, 2000); + MapIndexConfig config = new MapIndexConfig(); + config.setDenseKeys(keys); + config.setMaxKeys(4); + try { + String mapIndexFilePath = createIndex(config, records); + + File mapIndexFile = new File(mapIndexFilePath); + long size = Files.size(mapIndexFile.toPath()); + try (PinotDataBuffer keyBuffer = PinotDataBuffer.mapFile(mapIndexFile, true, 0, size, ByteOrder.BIG_ENDIAN, + "test")) { + Pair result = MapIndexHeader.read(keyBuffer, 0); + MapIndexHeader header = result.getLeft(); + + Assert.assertEquals(header.getMapIndex().getKeys().size(), 2); + for (DenseMapHeader.DenseKeyMetadata keyMeta : header.getMapIndex().getKeys()) { + String key = keyMeta.getName(); + if (keyMeta.getColumnMetadata().getColumnName().equals("a")) { + ColumnMetadata aMd = keyMeta.getColumnMetadata(); + Assert.assertEquals(keyMeta.getIndexOffset(StandardIndexes.forward()), 326); + Assert.assertEquals(aMd.getFieldSpec().getDataType(), FieldSpec.DataType.INT); + Assert.assertEquals(aMd.getColumnName(), "a"); + + long offset = keyMeta.getIndexOffset(StandardIndexes.forward()); + long keyIndexSize = keyMeta.getColumnMetadata().getIndexSizeMap().get(StandardIndexes.forward()); + PinotDataBuffer innerBuffer = + PinotDataBuffer.mapFile(mapIndexFile, false, offset, keyIndexSize, ByteOrder.BIG_ENDIAN, "test_a"); + ForwardIndexReader innerFwdReader = + ForwardIndexReaderFactory.createIndexReader(innerBuffer, keyMeta.getColumnMetadata()); + + testForwardIndexReader("a", innerFwdReader, records); + } else if (keyMeta.getColumnMetadata().getColumnName().equals("b")) { + Assert.assertEquals(keyMeta.getIndexOffset(StandardIndexes.forward()), 8390); + } else { + Assert.fail(); + } + } + } catch (Exception ex) { + throw ex; + } + } catch (Exception ex) { + Assert.fail("Error Creating Index", ex); + } + } + + @Test + public void testingWritingDropKeyIfNotInDenseSet() { + List keys = List.of( + new DimensionFieldSpec("a", FieldSpec.DataType.INT, true), + new DimensionFieldSpec("b", FieldSpec.DataType.INT, true) + ); + + // Create a set of data that has two extra keys that are not in the config + List> records = createTestData( + List.of( + new DimensionFieldSpec("a", FieldSpec.DataType.INT, true), + new DimensionFieldSpec("b", FieldSpec.DataType.INT, true), + new DimensionFieldSpec("c", FieldSpec.DataType.INT, true), + new DimensionFieldSpec("d", FieldSpec.DataType.INT, true) + ), + 2000); + + MapIndexConfig config = new MapIndexConfig(); + config.setDynamicallyCreateDenseKeys(false); + config.setDenseKeys(keys); + config.setMaxKeys(4); + + try { + String mapIndexFilePath = createIndex(config, records); + + File mapIndexFile = new File(mapIndexFilePath); + long size = Files.size(mapIndexFile.toPath()); + try (PinotDataBuffer keyBuffer = PinotDataBuffer.mapFile(mapIndexFile, true, 0, size, ByteOrder.BIG_ENDIAN, + "test")) { + Pair result = MapIndexHeader.read(keyBuffer, 0); + MapIndexHeader header = result.getLeft(); + Assert.assertEquals(header.getMapIndex().getKeys().size(), 2); + + Assert.assertNotNull(header.getMapIndex().getKey("a"), "Header does not contain key 'a'"); + Assert.assertNotNull(header.getMapIndex().getKey("b"), "Header does not contain key 'a'"); + Assert.assertNull(header.getMapIndex().getKey("c"), "Header should not contain key 'a'"); + Assert.assertNull(header.getMapIndex().getKey("d"), "Header should not contain key 'a'"); + + for (DenseMapHeader.DenseKeyMetadata keyMeta : header.getMapIndex().getKeys()) { + if (keyMeta.getColumnMetadata().getColumnName().equals("a")) { + ColumnMetadata aMd = keyMeta.getColumnMetadata(); + Assert.assertEquals(keyMeta.getIndexOffset(StandardIndexes.forward()), 326); + Assert.assertEquals(aMd.getFieldSpec().getDataType(), FieldSpec.DataType.INT); + Assert.assertEquals(aMd.getColumnName(), "a"); + + long offset = keyMeta.getIndexOffset(StandardIndexes.forward()); + long keyIndexSize = keyMeta.getColumnMetadata().getIndexSizeMap().get(StandardIndexes.forward()); + PinotDataBuffer innerBuffer = + PinotDataBuffer.mapFile(mapIndexFile, false, offset, keyIndexSize, ByteOrder.BIG_ENDIAN, "test_a"); + ForwardIndexReader innerFwdReader = + ForwardIndexReaderFactory.createIndexReader(innerBuffer, keyMeta.getColumnMetadata()); + + testForwardIndexReader("a", innerFwdReader, records); + } else if (keyMeta.getColumnMetadata().getColumnName().equals("b")) { + Assert.assertEquals(keyMeta.getIndexOffset(StandardIndexes.forward()), 8390); + } else { + Assert.fail(); + } + } + } catch (Exception ex) { + throw ex; + } + } catch (Exception ex) { + Assert.fail("Error Creating Index", ex); + } + } + + @Test + public void testingDynamicallyCreateDenseKey() { + List keys = List.of( + new DimensionFieldSpec("a", FieldSpec.DataType.INT, true), + new DimensionFieldSpec("b", FieldSpec.DataType.INT, true) + ); + + // Create a set of data that has two extra keys that are not in the config + // This will cause the key "c" to be dynamically created at teh start of teh segment + List> records = createTestData( + List.of( + new DimensionFieldSpec("a", FieldSpec.DataType.INT, true), + new DimensionFieldSpec("b", FieldSpec.DataType.INT, true) + ), + 2000); + + // This will test creating a key after the index has already been written to + records.get(15).put("c", 1500); + records.get(16).put("c", 1600); + records.get(17).put("c", 1700); + records.get(15).put("d", 1500); + records.get(16).put("d", 1600); + records.get(17).put("d", 1700); + + MapIndexConfig config = new MapIndexConfig(); + config.setDynamicallyCreateDenseKeys(true); + config.setDenseKeys(keys); + config.setMaxKeys(4); + + try { + String mapIndexFilePath = createIndex(config, records); + + File mapIndexFile = new File(mapIndexFilePath); + long size = Files.size(mapIndexFile.toPath()); + try (PinotDataBuffer keyBuffer = PinotDataBuffer.mapFile(mapIndexFile, true, 0, size, ByteOrder.BIG_ENDIAN, + "test")) { + Pair result = MapIndexHeader.read(keyBuffer, 0); + MapIndexHeader header = result.getLeft(); + Assert.assertEquals(header.getMapIndex().getKeys().size(), 4); + + Assert.assertNotNull(header.getMapIndex().getKey("a"), "Header does not contain key 'a'"); + Assert.assertNotNull(header.getMapIndex().getKey("b"), "Header does not contain key 'b'"); + Assert.assertNotNull(header.getMapIndex().getKey("c"), "Header does not contain key 'c'"); + Assert.assertNotNull(header.getMapIndex().getKey("d"), "Header does not contain key 'd'"); + + for (DenseMapHeader.DenseKeyMetadata keyMeta : header.getMapIndex().getKeys()) { + if (keyMeta.getColumnMetadata().getColumnName().equals("a")) { + ColumnMetadata aMd = keyMeta.getColumnMetadata(); + Assert.assertEquals(aMd.getFieldSpec().getDataType(), FieldSpec.DataType.INT); + Assert.assertEquals(aMd.getColumnName(), "a"); + + long offset = keyMeta.getIndexOffset(StandardIndexes.forward()); + long keyIndexSize = keyMeta.getColumnMetadata().getIndexSizeMap().get(StandardIndexes.forward()); + PinotDataBuffer innerBuffer = + PinotDataBuffer.mapFile(mapIndexFile, false, offset, keyIndexSize, ByteOrder.BIG_ENDIAN, "test_a"); + ForwardIndexReader innerFwdReader = + ForwardIndexReaderFactory.createIndexReader(innerBuffer, keyMeta.getColumnMetadata()); + + testForwardIndexReader("a", innerFwdReader, records); + } else if (keyMeta.getColumnMetadata().getColumnName().equals("b")) { + ColumnMetadata aMd = keyMeta.getColumnMetadata(); + Assert.assertEquals(aMd.getFieldSpec().getDataType(), FieldSpec.DataType.INT); + Assert.assertEquals(aMd.getColumnName(), "b"); + + long offset = keyMeta.getIndexOffset(StandardIndexes.forward()); + long keyIndexSize = keyMeta.getColumnMetadata().getIndexSizeMap().get(StandardIndexes.forward()); + PinotDataBuffer innerBuffer = + PinotDataBuffer.mapFile(mapIndexFile, false, offset, keyIndexSize, ByteOrder.BIG_ENDIAN, "test_b"); + ForwardIndexReader innerFwdReader = + ForwardIndexReaderFactory.createIndexReader(innerBuffer, keyMeta.getColumnMetadata()); + + testForwardIndexReader("b", innerFwdReader, records); + } else if (keyMeta.getColumnMetadata().getColumnName().equals("c")) { + } else if (keyMeta.getColumnMetadata().getColumnName().equals("d")) { + ColumnMetadata aMd = keyMeta.getColumnMetadata(); + Assert.assertEquals(aMd.getFieldSpec().getDataType(), FieldSpec.DataType.INT); + Assert.assertEquals(aMd.getColumnName(), "d"); + + long offset = keyMeta.getIndexOffset(StandardIndexes.forward()); + long keyIndexSize = keyMeta.getColumnMetadata().getIndexSizeMap().get(StandardIndexes.forward()); + PinotDataBuffer innerBuffer = + PinotDataBuffer.mapFile(mapIndexFile, false, offset, keyIndexSize, ByteOrder.BIG_ENDIAN, "test_d"); + ForwardIndexReader innerFwdReader = + ForwardIndexReaderFactory.createIndexReader(innerBuffer, keyMeta.getColumnMetadata()); + + ForwardIndexReaderContext context = innerFwdReader.createContext(); + for (int i = 0; i < 15; i++) { + int val = innerFwdReader.getInt(i, context); + Assert.assertEquals(val, -2147483648); + } + Assert.assertEquals(innerFwdReader.getInt(15, context), 1500); + Assert.assertEquals(innerFwdReader.getInt(16, context), 1600); + Assert.assertEquals(innerFwdReader.getInt(17, context), 1700); + for (int i = 18; i < records.size(); i++) { + int val = innerFwdReader.getInt(i, context); + Assert.assertEquals(val, -2147483648); + } + + Assert.assertEquals(innerFwdReader.getInt(15, context), 1500); + } else { + Assert.fail(); + } + } + } catch (Exception ex) { + throw ex; + } + } catch (Exception ex) { + Assert.fail("Error Creating Index", ex); + } + } + + @Test + public void testingWithStringType() { + // Create test data + List keys = List.of( + new DimensionFieldSpec("a", FieldSpec.DataType.STRING, true) + ); + List> records = createTestData(keys, 2000); + + // Configure map index + MapIndexConfig config = new MapIndexConfig(); + config.setDenseKeys(keys); + config.setMaxKeys(4); + try { + createIndex(config, records); + } catch (Exception ex) { + Assert.fail("Error Creating Index", ex); + } + } + + @Test + public void testingWithDifferentTypes() { + // Create test data + List keys = List.of( + new DimensionFieldSpec("a", FieldSpec.DataType.INT, true), + new DimensionFieldSpec("b", FieldSpec.DataType.LONG, true), + new DimensionFieldSpec("c", FieldSpec.DataType.FLOAT, true), + new DimensionFieldSpec("d", FieldSpec.DataType.DOUBLE, true), + new DimensionFieldSpec("e", FieldSpec.DataType.BOOLEAN, true), + new DimensionFieldSpec("f", FieldSpec.DataType.STRING, true) + ); + List> records = createTestData(keys, 2000); + + // Configure map index + MapIndexConfig config = new MapIndexConfig(); + config.setDenseKeys(keys); + config.setMaxKeys(4); + try { + createIndex(config, records); + } catch (Exception ex) { + Assert.fail("Error Creating Index", ex); + } + } + + @Test + public void typeMismatch() { + // If a key from the input Map Value has a type that differs from the already defined type of the dense key then + // Write the default value to the column. + List keys = List.of( + new DimensionFieldSpec("a", FieldSpec.DataType.STRING, true), + new DimensionFieldSpec("b", FieldSpec.DataType.INT, true) + ); + List> records = createTestData(keys, 2000); + MapIndexConfig config = new MapIndexConfig(); + config.setDenseKeys(keys); + config.setMaxKeys(4); + try { + createIndex(config, records); + } catch (Exception ex) { + Assert.fail("Error Creating Index", ex); + } + } + + private List> createTestData(List keys, int numRecords) { + ArrayList> records = new ArrayList<>(); + for (int i = 0; i < numRecords; i++) { + HashMap record = new HashMap<>(); + + for (int col = 0; col < keys.size(); col++) { + record.put(keys.get(col).getName(), generateTestValue(keys.get(col).getDataType().getStoredType(), i)); + } + + records.add(record); + } + + return records; + } + + private Object generateTestValue(FieldSpec.DataType type, int i) { + switch (type) { + case INT: + return i; + case LONG: + return 2L + (long) i; + case FLOAT: + return 3.0F + (float) i; + case DOUBLE: + return 4.5D + (double) i; + case BOOLEAN: + return i % 2 == 0; + case TIMESTAMP: + case STRING: + return "hello" + i; + case JSON: + case BIG_DECIMAL: + case BYTES: + case STRUCT: + case MAP: + case LIST: + case UNKNOWN: + default: + throw new UnsupportedOperationException(); + } + } + + private void testForwardIndexReader(String key, ForwardIndexReader reader, + List> records) { + ForwardIndexReaderContext context = reader.createContext(); + for (int docId = 0; docId < records.size(); docId++) { + int expectedValue = (Integer) records.get(docId).get(key); + int actualValue = reader.getInt(docId, context); + Assert.assertEquals(actualValue, expectedValue, String.format("Mismatch for DocId: %d", docId)); + } + } + + /** + * Creates a Dense Map index with the given config and adds the given records + * @param records + * @throws IOException on error + */ + private String createIndex(MapIndexConfig mapIndexConfig, List> records) + throws IOException { + MapColumnStatistics stats = createMapColumnStats(records); + DimensionFieldSpec mapSpec = new DimensionFieldSpec(); + mapSpec.setDataType(FieldSpec.DataType.MAP); + mapSpec.setName(MAP_COLUMN_NAME); + IndexCreationContext context = + new IndexCreationContext.Common.Builder().withIndexDir(INDEX_DIR).withTotalDocs(records.size()) + .withLengthOfLongestEntry(30).sorted(false).onHeap(false).withDictionary(false).withFieldSpec(mapSpec) + .withColumnStatistics(stats) + .build(); + try (org.apache.pinot.segment.spi.index.creator.MapIndexCreator indexCreator = new MapIndexCreator(context, + MAP_COLUMN_NAME, mapIndexConfig)) { + for (int i = 0; i < records.size(); i++) { + indexCreator.add(records.get(i)); + } + indexCreator.seal(); + } + + String mapIndexFilePath = + String.format("%s/%s%s", INDEX_DIR, MAP_COLUMN_NAME, MAP_FORWARD_INDEX_FILE_EXTENSION); + return mapIndexFilePath; + } + + private MapColumnStatistics createMapColumnStats(List> records) { + TestMapColStatistics stats = new TestMapColStatistics(); + + for (HashMap record : records) { + for (Map.Entry entry : record.entrySet()) { + stats.recordValue(entry.getKey(), (Comparable) entry.getValue()); + } + } + + return stats; + } + + static class TestMapColStatistics implements MapColumnStatistics { + private HashMap _minValueByKey; + private HashMap _maxValueByKey; + private HashMap _shortestLengthByKey; + private HashMap _longestLengthByKey; + + public TestMapColStatistics() { + _minValueByKey = new HashMap<>(); + _maxValueByKey = new HashMap<>(); + _shortestLengthByKey = new HashMap<>(); + _longestLengthByKey = new HashMap<>(); + } + + public void recordValue(String key, Comparable value) { + if (_minValueByKey.containsKey(key)) { + if (value.compareTo(_minValueByKey.get(key)) < 0) { + _minValueByKey.put(key, value); + } + } else { + _minValueByKey.put(key, value); + } + + if (_maxValueByKey.containsKey(key)) { + if (value.compareTo(_maxValueByKey.get(key)) > 0) { + _maxValueByKey.put(key, value); + } + } else { + _maxValueByKey.put(key, value); + } + + // Get the length of the value + if (value instanceof Integer) { + _shortestLengthByKey.put(key, Integer.BYTES); + _longestLengthByKey.put(key, Integer.BYTES); + } else if (value instanceof Long) { + _shortestLengthByKey.put(key, Long.BYTES); + _longestLengthByKey.put(key, Long.BYTES); + } else if (value instanceof Float) { + _shortestLengthByKey.put(key, Float.BYTES); + _longestLengthByKey.put(key, Float.BYTES); + } else if (value instanceof Double) { + _shortestLengthByKey.put(key, Double.BYTES); + _longestLengthByKey.put(key, Double.BYTES); + } else if (value instanceof String) { + int length = ((String) value).length(); + if (_shortestLengthByKey.containsKey(key)) { + if (length < _shortestLengthByKey.get(key)) { + _shortestLengthByKey.put(key, length); + } + } else { + _shortestLengthByKey.put(key, length); + } + + if (_longestLengthByKey.containsKey(key)) { + if (length > _longestLengthByKey.get(key)) { + _longestLengthByKey.put(key, length); + } + } else { + _longestLengthByKey.put(key, length); + } + } + } + + @Override + public Object getMinValue() { + return null; + } + + @Override + public Object getMaxValue() { + return null; + } + + @Override + public Object getUniqueValuesSet() { + return null; + } + + @Override + public int getCardinality() { + return 0; + } + + @Override + public int getLengthOfShortestElement() { + return 0; + } + + @Override + public int getLengthOfLargestElement() { + return 0; + } + + @Override + public boolean isSorted() { + return false; + } + + @Override + public int getTotalNumberOfEntries() { + return 0; + } + + @Override + public int getMaxNumberOfMultiValues() { + return 0; + } + + @Override + public PartitionFunction getPartitionFunction() { + return null; + } + + @Override + public int getNumPartitions() { + return 0; + } + + @Override + public Map getPartitionFunctionConfig() { + return null; + } + + @Override + public Set getPartitions() { + return null; + } + + @Override + public Object getMinValueForKey(String key) { + return null; + } + + @Override + public Object getMaxValueForKey(String key) { + return null; + } + + @Override + public int getLengthOfShortestElementForKey(String key) { + return 0; + } + + @Override + public int getLengthOfLargestElementForKey(String key) { + return 0; + } + + @Override + public Set> getKeys() { + return null; + } + + @Override + public boolean isSortedForKey(String key) { + return false; + } + + @Override + public int getTotalNumberOfEntriesForKey(String key) { + return 0; + } + } +} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java index 3ecec032add2..e1c7e6b2fd16 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/V1Constants.java @@ -42,6 +42,7 @@ public static class Indexes { public static final String RAW_SV_FORWARD_INDEX_FILE_EXTENSION = ".sv.raw.fwd"; public static final String RAW_MV_FORWARD_INDEX_FILE_EXTENSION = ".mv.raw.fwd"; public static final String UNSORTED_MV_FORWARD_INDEX_FILE_EXTENSION = ".mv.fwd"; + public static final String MAP_FORWARD_INDEX_FILE_EXTENSION = ".map.fwd"; public static final String BITMAP_INVERTED_INDEX_FILE_EXTENSION = ".bitmap.inv"; public static final String BITMAP_RANGE_INDEX_FILE_EXTENSION = ".bitmap.range"; public static final String JSON_INDEX_FILE_EXTENSION = ".json.idx"; diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/IndexCreationContext.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/IndexCreationContext.java index 5d3a1a78ba77..e5cb79d53de2 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/IndexCreationContext.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/IndexCreationContext.java @@ -90,6 +90,11 @@ public interface IndexCreationContext { */ boolean isTextCommitOnClose(); + /** + * Returns an abstraction representing the detailed statistics about a column. + * + * @return + */ ColumnStatistics getColumnStatistics(); /** * This flags whether the index creation is done during realtime segment conversion diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/MapColumnStatistics.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/MapColumnStatistics.java new file mode 100644 index 000000000000..405bbe75dc2a --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/creator/MapColumnStatistics.java @@ -0,0 +1,44 @@ +/** + * 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.spi.creator; + +import java.util.Set; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pinot.spi.data.FieldSpec; + + +/** + * An interface to read the map column statistics from statistics collectors. + */ +public interface MapColumnStatistics extends ColumnStatistics { + + Object getMinValueForKey(String key); + + Object getMaxValueForKey(String key); + + int getLengthOfShortestElementForKey(String key); + + int getLengthOfLargestElementForKey(String key); + + Set> getKeys(); + + boolean isSortedForKey(String key); + + int getTotalNumberOfEntriesForKey(String key); +} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/ForwardIndexConfig.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/ForwardIndexConfig.java index 757ca2368890..628c3cb4a8c7 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/ForwardIndexConfig.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/ForwardIndexConfig.java @@ -31,6 +31,7 @@ import org.apache.pinot.spi.config.table.FieldConfig; import org.apache.pinot.spi.config.table.FieldConfig.CompressionCodec; import org.apache.pinot.spi.config.table.IndexConfig; +import org.apache.pinot.spi.config.table.MapIndexConfig; import org.apache.pinot.spi.utils.DataSizeUtils; @@ -41,7 +42,7 @@ public class ForwardIndexConfig extends IndexConfig { DataSizeUtils.fromBytes(DEFAULT_TARGET_MAX_CHUNK_SIZE_BYTES); public static final int DEFAULT_TARGET_DOCS_PER_CHUNK = 1000; public static final ForwardIndexConfig DISABLED = - new ForwardIndexConfig(true, null, null, null, null, null, null, null); + new ForwardIndexConfig(true, null, null, null, null, null, null, null, null); public static final ForwardIndexConfig DEFAULT = new Builder().build(); @Nullable @@ -57,13 +58,18 @@ public class ForwardIndexConfig extends IndexConfig { @Nullable private final DictIdCompressionType _dictIdCompressionType; + @Nullable + private final MapIndexConfig _mapIndexConfig; + public ForwardIndexConfig(@Nullable Boolean disabled, @Nullable CompressionCodec compressionCodec, @Nullable Boolean deriveNumDocsPerChunk, @Nullable Integer rawIndexWriterVersion, - @Nullable String targetMaxChunkSize, @Nullable Integer targetDocsPerChunk) { + @Nullable String targetMaxChunkSize, @Nullable Integer targetDocsPerChunk, + @Nullable MapIndexConfig mapIndexConfig) { super(disabled); _deriveNumDocsPerChunk = Boolean.TRUE.equals(deriveNumDocsPerChunk); _rawIndexWriterVersion = rawIndexWriterVersion == null ? DEFAULT_RAW_WRITER_VERSION : rawIndexWriterVersion; _compressionCodec = compressionCodec; + _mapIndexConfig = mapIndexConfig; if (targetMaxChunkSize != null && !(_deriveNumDocsPerChunk || _rawIndexWriterVersion == 4)) { throw new IllegalStateException( @@ -119,9 +125,10 @@ public ForwardIndexConfig(@JsonProperty("disabled") @Nullable Boolean disabled, @JsonProperty("deriveNumDocsPerChunk") @Nullable Boolean deriveNumDocsPerChunk, @JsonProperty("rawIndexWriterVersion") @Nullable Integer rawIndexWriterVersion, @JsonProperty("targetMaxChunkSize") @Nullable String targetMaxChunkSizeBytes, - @JsonProperty("targetDocsPerChunk") @Nullable Integer targetDocsPerChunk) { + @JsonProperty("targetDocsPerChunk") @Nullable Integer targetDocsPerChunk, + @JsonProperty("mapIndexConfig") @Nullable MapIndexConfig mapIndexConfig) { this(disabled, getActualCompressionCodec(compressionCodec, chunkCompressionType, dictIdCompressionType), - deriveNumDocsPerChunk, rawIndexWriterVersion, targetMaxChunkSizeBytes, targetDocsPerChunk); + deriveNumDocsPerChunk, rawIndexWriterVersion, targetMaxChunkSizeBytes, targetDocsPerChunk, mapIndexConfig); } public static CompressionCodec getActualCompressionCodec(@Nullable CompressionCodec compressionCodec, @@ -195,6 +202,12 @@ public DictIdCompressionType getDictIdCompressionType() { return _dictIdCompressionType; } + @JsonIgnore + @Nullable + public MapIndexConfig getMapIndexConfig() { + return _mapIndexConfig; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -207,15 +220,18 @@ public boolean equals(Object o) { return false; } ForwardIndexConfig that = (ForwardIndexConfig) o; - return _compressionCodec == that._compressionCodec && _deriveNumDocsPerChunk == that._deriveNumDocsPerChunk - && _rawIndexWriterVersion == that._rawIndexWriterVersion && Objects.equals(_targetMaxChunkSize, - that._targetMaxChunkSize) && _targetDocsPerChunk == that._targetDocsPerChunk; + return _compressionCodec == that._compressionCodec + && _deriveNumDocsPerChunk == that._deriveNumDocsPerChunk + && _rawIndexWriterVersion == that._rawIndexWriterVersion + && Objects.equals(_targetMaxChunkSize, that._targetMaxChunkSize) + && _targetDocsPerChunk == that._targetDocsPerChunk + && Objects.equals(_mapIndexConfig, that._mapIndexConfig); } @Override public int hashCode() { return Objects.hash(super.hashCode(), _compressionCodec, _deriveNumDocsPerChunk, _rawIndexWriterVersion, - _targetMaxChunkSize, _targetDocsPerChunk); + _targetMaxChunkSize, _targetDocsPerChunk, _mapIndexConfig); } public static class Builder { @@ -225,6 +241,7 @@ public static class Builder { private int _rawIndexWriterVersion = DEFAULT_RAW_WRITER_VERSION; private String _targetMaxChunkSize; private int _targetDocsPerChunk = DEFAULT_TARGET_DOCS_PER_CHUNK; + private MapIndexConfig _mapIndexConfig = null; public Builder() { } @@ -235,6 +252,12 @@ public Builder(ForwardIndexConfig other) { _rawIndexWriterVersion = other._rawIndexWriterVersion; _targetMaxChunkSize = other._targetMaxChunkSize; _targetDocsPerChunk = other._targetDocsPerChunk; + _mapIndexConfig = other._mapIndexConfig; + } + + public Builder withMapIndexConfig(MapIndexConfig mapIndexConfig) { + _mapIndexConfig = mapIndexConfig; + return this; } public Builder withCompressionCodec(CompressionCodec compressionCodec) { @@ -322,7 +345,7 @@ public Builder withLegacyProperties(Map properties) { public ForwardIndexConfig build() { return new ForwardIndexConfig(false, _compressionCodec, _deriveNumDocsPerChunk, _rawIndexWriterVersion, - _targetMaxChunkSize, _targetDocsPerChunk); + _targetMaxChunkSize, _targetDocsPerChunk, _mapIndexConfig); } } } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/creator/MapIndexCreator.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/creator/MapIndexCreator.java new file mode 100644 index 000000000000..c66f1af02392 --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/creator/MapIndexCreator.java @@ -0,0 +1,54 @@ +/** + * 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.spi.index.creator; + +import java.io.IOException; +import java.util.Map; + + +/** + * Creates the durable representation of a map index. Metadata about the Map Column can be passed through via + * the IndexCreationContext and the implementation of this Interface can use that to determine the on + * disk representation of the Map. + */ +public interface MapIndexCreator extends ForwardIndexCreator { + int VERSION_1 = 1; + + default void seal() + throws IOException { + } + + /** + * + * @param value The nonnull value of the cell. In case the cell was actually null, a default value is received instead + * @param dict This is ignored as the MapIndexCreator will manage the construction of dictionaries itself. + */ + @Override + default void add(Object value, int dict) { + Map mapValue = (Map) value; + add(mapValue); + } + + @Override + default void add(Object[] values, int[] dictIds) { + throw new UnsupportedOperationException("Array of Maps not supported yet"); + } + + void add(Map mapValue); +} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/metadata/ColumnMetadataImpl.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/metadata/ColumnMetadataImpl.java index c599ae707e58..b2abe274fb16 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/metadata/ColumnMetadataImpl.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/metadata/ColumnMetadataImpl.java @@ -294,6 +294,10 @@ public static ColumnMetadataImpl fromPropertiesConfiguration(String column, Prop builder.setMinValue(BytesUtils.toByteArray(minString)); builder.setMaxValue(BytesUtils.toByteArray(maxString)); break; + case MAP: + builder.setMinValue(minString); + builder.setMaxValue(maxString); + break; default: throw new IllegalStateException("Unsupported data type: " + dataType + " for column: " + column); } diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableMapIndex.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableMapIndex.java new file mode 100644 index 000000000000..ab84416d6e23 --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/mutable/MutableMapIndex.java @@ -0,0 +1,88 @@ +/** + * 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.spi.index.mutable; + +import java.util.Map; +import java.util.Set; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pinot.segment.spi.index.reader.ForwardIndexReaderContext; +import org.apache.pinot.segment.spi.index.reader.MapIndexReader; +import org.apache.pinot.spi.data.FieldSpec; + + +/** + * Implementations of this interface can be used to represent indexes that store dynamically typed map values. + */ +public interface MutableMapIndex extends MapIndexReader, MutableForwardIndex { + Set> getKeys(); + + FieldSpec getKeySpec(String key); + + @Override + default void add(@Nonnull Object value, int dictId, int docId) { + Map mapValue = (Map) value; + add(mapValue, docId); + } + + @Override + default void add(@Nonnull Object[] values, @Nullable int[] dictIds, int docId) { + throw new UnsupportedOperationException("MultiValues are not yet supported for MAP columns"); + } + + /** + * Adds the given single value cell to the index. + * + * Unlike {@link org.apache.pinot.segment.spi.index.IndexCreator#add(Object, int)}, rows can be added in no + * particular order, so the docId is required by this method. + * + * @param value The nonnull value of the cell. In case the cell was actually null, a default value is received instead + * @param docId The document id of the given row. A non-negative value. + */ + void add(Map value, int docId); + + /** + * Adds the given multi value cell to the index. + * + * Unlike {@link org.apache.pinot.segment.spi.index.IndexCreator#add(Object[], int[])}, rows can be added in no + * particular order, so the docId is required by this method. + * + * @param values The nonnull value of the cell. In case the cell was actually null, an empty array is received instead + * @param docIds The document id of the given row. A non-negative value. + */ + void add(Map[] values, int[] docIds); + + /** + * Get the Min Value that the given Key has within the segment that this Reader is bound to. + * + * @param key A Key within the given Map column. + * @return The minimum value that is bound to that key within the Segment that this Reader is bound to. + */ + Comparable getMinValueForKey(String key); + + /** + * Get the Max Value that the given Key has within the segment that this Reader is bound to. + * + * @param key A Key within the given Map column. + * @return The maximum value that is bound to that key within the Segment that this Reader is bound to. + */ + Comparable getMaxValueForKey(String key); +} diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/reader/ForwardIndexReader.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/reader/ForwardIndexReader.java index c1b8cdeaff27..17ebdb4241cf 100644 --- a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/reader/ForwardIndexReader.java +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/reader/ForwardIndexReader.java @@ -20,6 +20,7 @@ import java.math.BigDecimal; import java.util.List; +import java.util.Map; import java.util.Objects; import javax.annotation.Nullable; import org.apache.pinot.segment.spi.compression.ChunkCompressionType; @@ -449,6 +450,12 @@ default byte[] getBytes(int docId, T context) { throw new UnsupportedOperationException(); } + default Map getMap(int docId, T context) { + throw new UnsupportedOperationException("This ForwardIndexReader does not support MAP types. " + + "This indicates that either the column is getting mistyped or the wrong " + + "ForwardIndexReader is being created to read this column."); + } + /** * MULTI-VALUE COLUMN RAW INDEX APIs */ diff --git a/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/reader/MapIndexReader.java b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/reader/MapIndexReader.java new file mode 100644 index 000000000000..672f8d44ec0d --- /dev/null +++ b/pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/index/reader/MapIndexReader.java @@ -0,0 +1,45 @@ +/** + * 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.spi.index.reader; + +import java.util.Map; +import org.apache.pinot.segment.spi.ColumnMetadata; +import org.apache.pinot.segment.spi.index.IndexReader; +import org.apache.pinot.segment.spi.index.IndexType; +import org.apache.pinot.spi.data.FieldSpec.DataType; + + +/** + * Interface for reading from the Mutable Map Index. + * + * @param Type of the ReaderContext + */ +public interface MapIndexReader + extends ForwardIndexReader { + IndexReader getKeyReader(String key, IndexType type); + Map getKeyIndexes(String key); + + /** + * Returns the data type of the values in the forward index. Returns {@link DataType#INT} for dictionary-encoded + * forward index. + */ + DataType getStoredType(String key); + + ColumnMetadata getKeyMetadata(String key); +} diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java index ce5bc79dda7b..1d1ae4274653 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/IndexingConfig.java @@ -41,6 +41,8 @@ public class IndexingConfig extends BaseJsonConfig { @Deprecated private List _jsonIndexColumns; private Map _jsonIndexConfigs; + private List _mapIndexColumns; + private Map _mapIndexConfigs; private List _h3IndexColumns; private List _vectorIndexColumns; private List _sortedColumn; @@ -141,6 +143,22 @@ public void setJsonIndexConfigs(Map jsonIndexConfigs) { _jsonIndexConfigs = jsonIndexConfigs; } + public List getMapIndexColumns() { + return _mapIndexColumns; + } + + public void setMapIndexColumns(List mapIndexColumns) { + _mapIndexColumns = mapIndexColumns; + } + + public void setMapIndexConfigs(Map mapIndexConfigs) { + _mapIndexConfigs = mapIndexConfigs; + } + + public Map getMapIndexConfigs() { + return _mapIndexConfigs; + } + public boolean isAutoGeneratedInvertedIndex() { return _autoGeneratedInvertedIndex; } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/MapIndexConfig.java b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/MapIndexConfig.java new file mode 100644 index 000000000000..64ce659b51dc --- /dev/null +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/MapIndexConfig.java @@ -0,0 +1,122 @@ +/** + * 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.spi.config.table; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.LinkedList; +import java.util.List; +import java.util.Objects; +import javax.annotation.Nullable; +import org.apache.pinot.spi.data.DimensionFieldSpec; + + +/** + * Configs related to the MAP index: + */ +public class MapIndexConfig extends IndexConfig { + public static final MapIndexConfig DISABLED = new MapIndexConfig(true); + + private int _maxKeys = 100; + private List _denseKeys; + private boolean _dynamicallyCreateDenseKeys; + + public MapIndexConfig() { + this(false); + } + + public MapIndexConfig(Boolean disabled) { + super(disabled); + _dynamicallyCreateDenseKeys = true; + _denseKeys = List.of(); + } + + @JsonCreator + public MapIndexConfig(@JsonProperty("disabled") @Nullable Boolean disabled, + @JsonProperty("maxKeys") @Nullable Integer maxKeys, + @JsonProperty("denseKeys") @Nullable List denseKeys, + @JsonProperty("dynamicallyCreateDenseKeys") @Nullable Boolean dynamicallyCreateDenseKeys + ) { + super(disabled); + _maxKeys = maxKeys != null ? maxKeys : 50; + _dynamicallyCreateDenseKeys = dynamicallyCreateDenseKeys != null ? dynamicallyCreateDenseKeys : false; + + if (denseKeys != null) { + _denseKeys = denseKeys; + } else { + _denseKeys = new LinkedList<>(); + } + } + + public int getMaxKeys() { + return _maxKeys; + } + + public void setMaxKeys(int maxKeys) { + _maxKeys = maxKeys; + } + + public List getDenseKeys() { + return _denseKeys; + } + + public void setDenseKeys(List denseKeys) { + _denseKeys = denseKeys; + } + + public void setDynamicallyCreateDenseKeys(boolean dynamicallyCreateDenseKeys) { + _dynamicallyCreateDenseKeys = dynamicallyCreateDenseKeys; + } + + public boolean getDynamicallyCreateDenseKeys() { + return _dynamicallyCreateDenseKeys; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + MapIndexConfig config = (MapIndexConfig) o; + return _maxKeys == config._maxKeys && _dynamicallyCreateDenseKeys == config._dynamicallyCreateDenseKeys + && _denseKeys.equals(config._denseKeys); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), _dynamicallyCreateDenseKeys, _maxKeys, _denseKeys); + } + + public static class KeyConfigEntry { + final String _name; + final String _dataType; + + @JsonCreator + public KeyConfigEntry(@JsonProperty("name") String name, @JsonProperty("dataType") String dataType) { + _name = name; + _dataType = dataType; + } + } +} diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/data/FieldSpec.java b/pinot-spi/src/main/java/org/apache/pinot/spi/data/FieldSpec.java index 5fa586b6b9ff..5806c20970cc 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/data/FieldSpec.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/data/FieldSpec.java @@ -63,6 +63,7 @@ public abstract class FieldSpec implements Comparable, Serializable { public static final String DEFAULT_DIMENSION_NULL_VALUE_OF_JSON = "null"; public static final byte[] DEFAULT_DIMENSION_NULL_VALUE_OF_BYTES = new byte[0]; public static final BigDecimal DEFAULT_DIMENSION_NULL_VALUE_OF_BIG_DECIMAL = BigDecimal.ZERO; + public static final HashMap DEFAULT_DIMENSION_NULL_VALUE_OF_MAP = new HashMap<>(); public static final Integer DEFAULT_METRIC_NULL_VALUE_OF_INT = 0; public static final Long DEFAULT_METRIC_NULL_VALUE_OF_LONG = 0L; @@ -302,6 +303,8 @@ public static Object getDefaultNullValue(FieldType fieldType, DataType dataType, return DEFAULT_DIMENSION_NULL_VALUE_OF_BYTES; case BIG_DECIMAL: return DEFAULT_DIMENSION_NULL_VALUE_OF_BIG_DECIMAL; + case MAP: + return DEFAULT_DIMENSION_NULL_VALUE_OF_MAP; default: throw new IllegalStateException("Unsupported dimension/time data type: " + dataType); } @@ -584,6 +587,8 @@ public Object convert(String value) { return value; case BYTES: return BytesUtils.toBytes(value); + case MAP: + return JsonUtils.stringToObject(value, HashMap.class); default: throw new IllegalStateException(); } diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/data/Schema.java b/pinot-spi/src/main/java/org/apache/pinot/spi/data/Schema.java index 63add349874e..b085df00d297 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/data/Schema.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/data/Schema.java @@ -121,6 +121,7 @@ public static void validate(FieldType fieldType, DataType dataType) { case TIMESTAMP: case STRING: case JSON: + case MAP: case BYTES: break; default: