You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "jugomezv (via GitHub)" <gi...@apache.org> on 2023/02/21 17:41:28 UTC

[GitHub] [pinot] jugomezv opened a new issue, #10316: Add JSON integration tests to LLCRealtimeClusterIntegrationTest

jugomezv opened a new issue, #10316:
URL: https://github.com/apache/pinot/issues/10316

   There seems to be some issues in the code path that ingest JSON data for REALTIME tables. 
   
   If data like the following gets ingested from Kafka:
   "EvaluationResult" : {
       "AssertionEvaluationResult" : {
         "summary" : {
           "passed" : 0,
           "failed" : 0,
           "warned" : 0,
           "overallStatus" : "XXXX"
         },
         "evaluatedAssertions" : [ ]
       }
   }
   
   The nested empty array gets turned into a NULL in pinot DB, expected behavior would be we get an empty array in Pinot:
   
   ![Screenshot 2023-02-21 at 9 40 17 AM](https://user-images.githubusercontent.com/109560870/220419676-aaf1bb24-95d7-4157-97df-b4c2007d2a8e.png)
   


-- 
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: commits-unsubscribe@pinot.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] jugomezv commented on issue #10316: Add JSON integration tests to LLCRealtimeClusterIntegrationTest

Posted by "jugomezv (via GitHub)" <gi...@apache.org>.
jugomezv commented on issue #10316:
URL: https://github.com/apache/pinot/issues/10316#issuecomment-1458447919

   @snleee @navina @Jackie-Jiang can you give us your opinion on this?


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] sahoss commented on issue #10316: Add JSON integration tests to LLCRealtimeClusterIntegrationTest

Posted by "sahoss (via GitHub)" <gi...@apache.org>.
sahoss commented on issue #10316:
URL: https://github.com/apache/pinot/issues/10316#issuecomment-1453573271

   So here is my investigation so far:
   
   ### Reproducible example:
   
   **schema.json**
   
   
   ```
   {
     "schemaName": "sample",
     "dimensionFieldSpecs": [
       {
         "name": "data",
         "dataType": "JSON"
       }
     ],
     "metricFieldSpecs": [],
     "dateTimeFieldSpecs": [
       {
         "name": "timestamp",
         "dataType": "LONG",
         "format": "1:MILLISECONDS:EPOCH",
         "granularity": "1:MILLISECONDS"
       }
     ]
   }
   ```
   
   **tableconfig-realtime.json**
   ```
   {
     "tableName": "sample",
     "tableType": "REALTIME",
     "segmentsConfig": {
       "timeColumnName": "timestamp",
       "timeType": "MILLISECONDS",
       "schemaName": "sample",
       "replicasPerPartition": "1"
     },
     "tenants": {},
     "tableIndexConfig": {
       "loadMode": "MMAP",
       "streamConfigs": {
         "streamType": "kafka",
         "stream.kafka.consumer.type": "lowlevel",
         "stream.kafka.topic.name": "sample",
         "stream.kafka.decoder.class.name": "org.apache.pinot.plugin.stream.kafka.KafkaJSONMessageDecoder",
         "stream.kafka.consumer.factory.class.name": "org.apache.pinot.plugin.stream.kafka20.KafkaConsumerFactory",
         "stream.kafka.broker.list": "localhost:9092",
         "realtime.segment.flush.threshold.time": "3600000",
         "realtime.segment.flush.threshold.rows": "50000",
         "stream.kafka.consumer.prop.auto.offset.reset": "smallest"
       }
     },
     "metadata": {
       "customConfigs": {}
     }
   }
   ```
   
   Input Json record:
   ```
   {"data": {"key1": [], "key2": [1, 2], "key3": "value"}, "timestamp": 1677851924425}
   
   ```
   
   
   Result stored in segment:
   ```
   {
     "resultTable": {
       "dataSchema": {
         "columnNames": [
           "data",
           "timestamp"
         ],
         "columnDataTypes": [
           "JSON",
           "LONG"
         ]
       },
       "rows": [
         [
           "{\"key2\":[1,2],\"key3\":\"value\"}",
           1677851018568
         ]
       ]
     },
     "exceptions": [],
     "numServersQueried": 1,
     "numServersResponded": 1,
     "numSegmentsQueried": 1,
     "numSegmentsProcessed": 1,
     "numSegmentsMatched": 1,
     "numConsumingSegmentsQueried": 1,
     "numConsumingSegmentsProcessed": 1,
     "numConsumingSegmentsMatched": 1,
     "numDocsScanned": 1,
     "numEntriesScannedInFilter": 0,
     "numEntriesScannedPostFilter": 2,
     "numGroupsLimitReached": false,
     "totalDocs": 5,
     "timeUsedMs": 4,
     "offlineThreadCpuTimeNs": 0,
     "realtimeThreadCpuTimeNs": 0,
     "offlineSystemActivitiesCpuTimeNs": 0,
     "realtimeSystemActivitiesCpuTimeNs": 0,
     "offlineResponseSerializationCpuTimeNs": 0,
     "realtimeResponseSerializationCpuTimeNs": 0,
     "offlineTotalCpuTimeNs": 0,
     "realtimeTotalCpuTimeNs": 0,
     "segmentStatistics": [],
     "traceInfo": {},
     "minConsumingFreshnessTimeMs": 1677851970095,
     "numSegmentsPrunedByBroker": 0,
     "numSegmentsPrunedByServer": 0,
     "numSegmentsPrunedInvalid": 0,
     "numSegmentsPrunedByLimit": 0,
     "numSegmentsPrunedByValue": 0,
     "explainPlanNumEmptyFilterSegments": 0,
     "explainPlanNumMatchAllFilterSegments": 0,
     "numRowsResultSet": 1
   }
   ```


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] jugomezv commented on issue #10316: Add JSON integration tests to LLCRealtimeClusterIntegrationTest

Posted by "jugomezv (via GitHub)" <gi...@apache.org>.
jugomezv commented on issue #10316:
URL: https://github.com/apache/pinot/issues/10316#issuecomment-1455506395

   Just from peeking at the code seems you found the source of the problem, but will spend some more time. Now the question  is whether we can distinguish the situation from when the user actually provides null in the input...


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] jugomezv commented on issue #10316: Add JSON integration tests to LLCRealtimeClusterIntegrationTest

Posted by "jugomezv (via GitHub)" <gi...@apache.org>.
jugomezv commented on issue #10316:
URL: https://github.com/apache/pinot/issues/10316#issuecomment-1455504983

   Thanks a lot @sahoss let me take some time to go over this tomorrow and get back to you.


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] sahoss commented on issue #10316: Add JSON integration tests to LLCRealtimeClusterIntegrationTest

Posted by "sahoss (via GitHub)" <gi...@apache.org>.
sahoss commented on issue #10316:
URL: https://github.com/apache/pinot/issues/10316#issuecomment-1453584646

   would be great if someone looks over the analysis.
   would this be something needed to be fixed? would it break someones flow where they depend on this current behavior? 


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] jugomezv commented on issue #10316: Add JSON integration tests to LLCRealtimeClusterIntegrationTest

Posted by "jugomezv (via GitHub)" <gi...@apache.org>.
jugomezv commented on issue #10316:
URL: https://github.com/apache/pinot/issues/10316#issuecomment-1458349898

   I have a question about the execution path above: does the JSON extractor returns anything in line 61? from.get(fieldname="data"), if so what? or do we just get null. If we get null, then we never call covert right?


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] sahoss commented on issue #10316: Add JSON integration tests to LLCRealtimeClusterIntegrationTest

Posted by "sahoss (via GitHub)" <gi...@apache.org>.
sahoss commented on issue #10316:
URL: https://github.com/apache/pinot/issues/10316#issuecomment-1451849432

   I would like to work on this. let me try to first reproduce this issue.


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [pinot] sahoss commented on issue #10316: Add JSON integration tests to LLCRealtimeClusterIntegrationTest

Posted by "sahoss (via GitHub)" <gi...@apache.org>.
sahoss commented on issue #10316:
URL: https://github.com/apache/pinot/issues/10316#issuecomment-1453583245

   **Potential source of bug**
   It seems error is in BaseRecordExtractor. When processing the json node:  `{"key1": [], "key2": [1, 2], "key3": "value"`, the empty key is present.
   ![image](https://user-images.githubusercontent.com/126167264/222739777-df4f032a-06a3-4de2-82ca-5469ed6a251e.png)
   
   It recurses to process node `"key1": []`  and returns null from convert() method. This is because convertMultiValue() checks for empty collection and returns null.
   ![image](https://user-images.githubusercontent.com/126167264/222739924-20df8556-70d6-4adf-81a4-df554ddefbf2.png)
   
   ![image](https://user-images.githubusercontent.com/126167264/222740171-56223b23-66dd-44d0-8e54-26d30f2e5930.png)
   
   Code pointers:
   
   https://github.com/apache/pinot/blob/master/pinot-spi/src/main/java/org/apache/pinot/spi/data/readers/BaseRecordExtractor.java#L109 


-- 
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: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org