andres-torti opened a new issue, #11968:
URL: https://github.com/apache/pinot/issues/11968
Using Apache Pinot 1.0.0 and I'm having some trouble when creating an
inverted index on a multi-value column. This is my table config:
```
{
"tableName": "devices",
"tableType": "REALTIME",
"upsertConfig": {
"mode": "FULL",
"comparisonColumn": "timestamp",
"enableSnapshot": true,
"enablePreload": true
},
"tenants": {},
"segmentsConfig": {
"timeColumnName": "timestamp",
"timeType": "SECONDS",
"retentionTimeUnit": "DAYS",
"retentionTimeValue": "90",
"replication": "1"
},
"tableIndexConfig": {
"loadMode": "MMAP",
"invertedIndexColumns": [
"segments"
],
"streamConfigs": {
"streamType": "kafka",
"stream.kafka.topic.name": "events-realtime",
"stream.kafka.decoder.class.name":
"org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder",
"stream.kafka.consumer.type": "simple",
"stream.kafka.consumer.factory.class.name":
"org.apache.pinot.plugin.stream.kafka20.KafkaConsumerFactory",
"stream.kafka.broker.list": "{brokers}",
"security.protocol": "SSL",
"realtime.segment.flush.threshold.time": "3600000",
"realtime.segment.flush.threshold.size": "20000"
}
},
"routing": {
"instanceSelectorType": "strictReplicaGroup"
},
"fieldConfigList": [],
"metadata": {
"customConfigs": {}
}
}
```
And this is my schema:
```
{
"metricFieldSpecs": [],
"primaryKeyColumns": ["device_id"],
"dimensionFieldSpecs": [
{
"name": "country",
"dataType": "STRING"
},
{
"name": "device_id",
"dataType": "STRING"
},
{
"name": "device_type",
"dataType": "STRING"
},
{
"name": "segments",
"dataType": "INT",
"singleValueField": false
},
{
"name": "options",
"dataType": "INT",
"singleValueField": false
},
{
"name": "relation_id",
"dataType": "STRING"
},
{
"name": "client",
"dataType": "INT"
}
],
"dateTimeFieldSpecs": [
{
"name": "timestamp",
"dataType": "LONG",
"format": "1:SECONDS:EPOCH",
"granularity": "1:DAYS"
}
],
"schemaName": "devices"
}
```
As soon as Pinot starts consuming events from Kafka I get these errors:
```
pinot-server | 2023/11/04 23:41:17.587 ERROR
[LLRealtimeSegmentDataManager_devices__5__1__20231104T2341Z]
[devices__5__1__20231104T2341Z] Could not build segment
pinot-server | java.lang.RuntimeException: Error occurred while
reading row during indexing
pinot-server | at
org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl.build(SegmentIndexCreationDriverImpl.java:232)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.segment.local.realtime.converter.RealtimeSegmentConverter.build(RealtimeSegmentConverter.java:121)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.core.data.manager.realtime.LLRealtimeSegmentDataManager.buildSegmentInternal(LLRealtimeSegmentDataManager.java:935)
[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.core.data.manager.realtime.LLRealtimeSegmentDataManager.buildSegmentForCommit(LLRealtimeSegmentDataManager.java:842)
[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.core.data.manager.realtime.LLRealtimeSegmentDataManager$PartitionConsumer.run(LLRealtimeSegmentDataManager.java:754)
[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at java.lang.Thread.run(Thread.java:829) [?:?]
pinot-server | Caused by: java.lang.IndexOutOfBoundsException
pinot-server | at java.nio.Buffer.checkIndex(Buffer.java:693) ~[?:?]
pinot-server | at
java.nio.DirectByteBuffer.getInt(DirectByteBuffer.java:758) ~[?:?]
pinot-server | at
org.apache.pinot.segment.spi.memory.PinotByteBuffer.getInt(PinotByteBuffer.java:137)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.segment.local.io.reader.impl.FixedByteSingleValueMultiColReader.getInt(FixedByteSingleValueMultiColReader.java:105)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.segment.local.realtime.impl.forward.FixedByteMVMutableForwardIndex.getDictIdMV(FixedByteMVMutableForwardIndex.java:250)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.segment.spi.index.mutable.MutableForwardIndex.getDictIdMV(MutableForwardIndex.java:225)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.segment.local.segment.readers.PinotSegmentColumnReader.getValue(PinotSegmentColumnReader.java:98)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.segment.local.segment.readers.PinotSegmentRecordReader.getRecord(PinotSegmentRecordReader.java:227)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.segment.local.segment.readers.PinotSegmentRecordReader.next(PinotSegmentRecordReader.java:210)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl.build(SegmentIndexCreationDriverImpl.java:225)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | ... 5 more
pinot-server | 2023/11/04 23:41:17.589 ERROR
[LLRealtimeSegmentDataManager_devices__5__1__20231104T2341Z]
[devices__5__1__20231104T2341Z] Could not build segment for
devices__5__1__20231104T2341Z
pinot-server | 2023/11/04 23:41:17.635 ERROR
[LLRealtimeSegmentDataManager_devices__2__1__20231104T2341Z]
[devices__2__1__20231104T2341Z] Could not build segment
pinot-server | java.lang.RuntimeException: Error occurred while
reading row during indexing
pinot-server | at
org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl.build(SegmentIndexCreationDriverImpl.java:232)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.segment.local.realtime.converter.RealtimeSegmentConverter.build(RealtimeSegmentConverter.java:121)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.core.data.manager.realtime.LLRealtimeSegmentDataManager.buildSegmentInternal(LLRealtimeSegmentDataManager.java:935)
[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.core.data.manager.realtime.LLRealtimeSegmentDataManager.buildSegmentForCommit(LLRealtimeSegmentDataManager.java:842)
[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.core.data.manager.realtime.LLRealtimeSegmentDataManager$PartitionConsumer.run(LLRealtimeSegmentDataManager.java:754)
[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at java.lang.Thread.run(Thread.java:829) [?:?]
pinot-server | Caused by: java.lang.IndexOutOfBoundsException
pinot-server | at java.nio.Buffer.checkIndex(Buffer.java:693) ~[?:?]
pinot-server | at
java.nio.DirectByteBuffer.getInt(DirectByteBuffer.java:758) ~[?:?]
pinot-server | at
org.apache.pinot.segment.spi.memory.PinotByteBuffer.getInt(PinotByteBuffer.java:137)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.segment.local.io.reader.impl.FixedByteSingleValueMultiColReader.getInt(FixedByteSingleValueMultiColReader.java:105)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.segment.local.realtime.impl.forward.FixedByteMVMutableForwardIndex.getDictIdMV(FixedByteMVMutableForwardIndex.java:250)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.segment.spi.index.mutable.MutableForwardIndex.getDictIdMV(MutableForwardIndex.java:225)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.segment.local.segment.readers.PinotSegmentColumnReader.getValue(PinotSegmentColumnReader.java:98)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.segment.local.segment.readers.PinotSegmentRecordReader.getRecord(PinotSegmentRecordReader.java:227)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.segment.local.segment.readers.PinotSegmentRecordReader.next(PinotSegmentRecordReader.java:210)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | at
org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl.build(SegmentIndexCreationDriverImpl.java:225)
~[pinot-all-1.0.0-jar-with-dependencies.jar:1.0.0-b6bdf6c9686b286a149d2d1aea4a385ee98f3e79]
pinot-server | ... 5 more
```
The data is ingested into the table anyways, but when running a query like
`select count(*) from devices where segments = 560` I get this error:
```
Error Code: 200
QueryExecutionError:
java.lang.IndexOutOfBoundsException
at java.base/java.nio.Buffer.checkIndex(Buffer.java:693)
at java.base/java.nio.DirectByteBuffer.getInt(DirectByteBuffer.java:758)
at
org.apache.pinot.segment.spi.memory.PinotByteBuffer.getInt(PinotByteBuffer.java:137)
at
org.apache.pinot.segment.local.io.reader.impl.FixedByteSingleValueMultiColReader.getInt(FixedByteSingleValueMultiColReader.java:105)
```
If I change the table flush size from:
`"realtime.segment.flush.threshold.size": "20000"`
To:
`"realtime.segment.flush.threshold.size": "200000"`
The errors are gone and everything works as expected. This some sample data
in case it's useful:
[sample_data.csv.zip](https://github.com/apache/pinot/files/13291350/sample_data.csv.zip)
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]