You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2021/11/12 17:01:49 UTC

[GitHub] [pinot] dianaarnos opened a new issue #7758: Problems to use transform functions and ingest dateTime columns with RFC3339 format

dianaarnos opened a new issue #7758:
URL: https://github.com/apache/pinot/issues/7758


   Hi there,
   
   ### What am I trying to do? 
   (in case I'm taking the wrong approach here and someone can point me to a better solution)
   
   I need to select all docs by month and year.
   
   Currently I'm ingesting data from a kafka topic into Pinot and here's an example for a message value:
   ```json
   {
     "some_uuid": "b17e63ac-0766-4d54-a23d-6929e15d13b8",
     "product_name": "my product",
     "condition": true,
     "operation": "create",
     "operationDate": "2021-05-20T12:55:54.000+0000"
   }
   ```
   
   As you can see, the `operationDate` field follows the RFC3339 format.
   
   ### What is happening
   First, I tried the following schema definition for `dateTimeFieldSpecs`:
   ```json
   "dateTimeFieldSpecs": [
       {
         "name": "operationDate",
         "dataType": "STRING",
         "format": "1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd'T'HH:mm:ss.SSS'Z'",
         "granularity": "1:MILLISECONDS"
       }
     ]
   ```
   
   Then from the Controller UI, I tried running the following query,:
   ```sql
   select DATETIMECONVERT(operationDate, "1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-ddTHH:mm:ss.SSSZ", "1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM", "30:DAYS"), count(*) from my_table
   group by DATETIMECONVERT(operationDate, "1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-ddTHH:mm:ss.SSSZ", "1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM", "30:DAYS")
   ```
   And I got zero results, but the table has 20 docs that were successfully ingested. 
   
   I know that for months and years I should use the `DATETRUNC` function, so I tried the following query:
   ```sql
   select DATETRUNC('month', operationDate, 'MILLISECONDS'), count(*) from my_table
   group by DATETRUNC('month', operationDate, 'MILLISECONDS')
   ```
   And I get the following error on the UI:
   ```json
   [
     {
       "message": "QueryExecutionError:\nProcessingException(errorCode:450, message:InternalError:\njava.lang.NullPointerException\n\tat org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator.mergeResults(GroupByOrderByCombineOperator.java:198)\n\tat org.apache.pinot.core.operator.combine.BaseCombineOperator.getNextBlock(BaseCombineOperator.java:117)\n\tat org.apache.pinot.core.operator.combine.BaseCombineOperator.getNextBlock(BaseCombineOperator.java:49)\n\tat org.apache.pinot.core.operator.BaseOperator.nextBlock(BaseOperator.java:49)\n\tat org.apache.pinot.core.operator.InstanceResponseOperator.getNextBlock(InstanceResponseOperator.java:40)\n\tat org.apache.pinot.core.operator.InstanceResponseOperator.getNextBlock(InstanceResponseOperator.java:28)\n\tat org.apache.pinot.core.operator.BaseOperator.nextBlock(BaseOperator.java:49)\n\tat org.apache.pinot.core.plan.GlobalPlanImplV0.execute(GlobalPlanImplV0.java:48)\n\tat org.apache.pinot.core.query.executor.ServerQueryExec
 utorV1Impl.processQuery(ServerQueryExecutorV1Impl.java:296)\n\tat org.apache.pinot.core.query.executor.ServerQueryExecutorV1Impl.processQuery(ServerQueryExecutorV1Impl.java:216)\n\tat org.apache.pinot.core.query.executor.QueryExecutor.processQuery(QueryExecutor.java:60)\n\tat org.apache.pinot.core.query.scheduler.QueryScheduler.processQueryAndSerialize(QueryScheduler.java:155)\n\tat org.apache.pinot.core.query.scheduler.QueryScheduler.lambda$createQueryFutureTask$0(QueryScheduler.java:139)",
       "errorCode": 200
     }
   ]
   ```
   And the following error inside pinot server:
   ```
   pinot-server_1        | 2021/11/12 16:46:08.659 ERROR [GroupByOrderByCombineOperator] [pqw-6] Caught exception while processing and combining group-by order-by for index: 1, operator: org.apache.pinot.core.operator.query.AggregationGroupByOrderByOperator, queryContext: QueryContext{_tableName='my_table_REALTIME', _selectExpressions=[datetrunc('month',operationDate,'MILLISECONDS'), count(*)], _aliasList=[null, null], _filter=null, _groupByExpressions=[datetrunc('month',operationDate,'MILLISECONDS')], _havingFilter=null, _orderByExpressions=null, _limit=10, _offset=0, _queryOptions={responseFormat=sql, groupByMode=sql, timeoutMs=9997}, _debugOptions=null, _brokerRequest=BrokerRequest(querySource:QuerySource(tableName:my_table_REALTIME), pinotQuery:PinotQuery(dataSource:DataSource(tableName:my_table_REALTIME), selectList:[Expression(type:FUNCTION, functionCall:Function(operator:DATETRUNC, operands:[Expression(type:LITERAL, literal:<Literal stringValue:month>), Expression(type:IDENTIF
 IER, identifier:Identifier(name:operationDate)), Expression(type:LITERAL, literal:<Literal stringValue:MILLISECONDS>)])), Expression(type:FUNCTION, functionCall:Function(operator:COUNT, operands:[Expression(type:IDENTIFIER, identifier:Identifier(name:*))]))], groupByList:[Expression(type:FUNCTION, functionCall:Function(operator:DATETRUNC, operands:[Expression(type:LITERAL, literal:<Literal stringValue:month>), Expression(type:IDENTIFIER, identifier:Identifier(name:operationDate)), Expression(type:LITERAL, literal:<Literal stringValue:MILLISECONDS>)]))], queryOptions:{responseFormat=sql, groupByMode=sql, timeoutMs=9997}))}
   pinot-server_1        | java.lang.NumberFormatException: For input string: "2021-05-20T12:55:54.000+0000"
   pinot-server_1        | 	at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65) ~[?:?]
   pinot-server_1        | 	at java.lang.Long.parseLong(Long.java:692) ~[?:?]
   pinot-server_1        | 	at java.lang.Long.parseLong(Long.java:817) ~[?:?]
   pinot-server_1        | 	at org.apache.pinot.segment.local.realtime.impl.dictionary.StringOnHeapMutableDictionary.getLongValue(StringOnHeapMutableDictionary.java:134) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.segment.spi.index.reader.Dictionary.readLongValues(Dictionary.java:158) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.common.DataFetcher$ColumnValueReader.readLongValues(DataFetcher.java:325) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.common.DataFetcher.fetchLongValues(DataFetcher.java:112) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.common.DataBlockCache.getLongValuesForSVColumn(DataBlockCache.java:137) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.operator.docvalsets.ProjectionBlockValSet.getLongValuesSV(ProjectionBlockValSet.java:79) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.operator.transform.function.IdentifierTransformFunction.transformToLongValuesSV(IdentifierTransformFunction.java:84) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.operator.transform.function.DateTruncTransformFunction.transformToLongValuesSV(DateTruncTransformFunction.java:134) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.operator.docvalsets.TransformBlockValSet.getLongValuesSV(TransformBlockValSet.java:75) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.query.aggregation.groupby.NoDictionarySingleColumnGroupKeyGenerator.generateKeysForBlock(NoDictionarySingleColumnGroupKeyGenerator.java:82) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.query.aggregation.groupby.DefaultGroupByExecutor.process(DefaultGroupByExecutor.java:120) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.operator.query.AggregationGroupByOrderByOperator.getNextBlock(AggregationGroupByOrderByOperator.java:120) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.operator.query.AggregationGroupByOrderByOperator.getNextBlock(AggregationGroupByOrderByOperator.java:47) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.operator.BaseOperator.nextBlock(BaseOperator.java:49) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.operator.combine.GroupByOrderByCombineOperator.processSegments(GroupByOrderByCombineOperator.java:106) [pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.operator.combine.BaseCombineOperator$1.runJob(BaseCombineOperator.java:105) [pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.util.trace.TraceRunnable.run(TraceRunnable.java:40) [pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) [?:?]
   pinot-server_1        | 	at java.util.concurrent.FutureTask.run(FutureTask.java:264) [?:?]
   pinot-server_1        | 	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) [?:?]
   pinot-server_1        | 	at shaded.com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:111) [pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at shaded.com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:58) [pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at shaded.com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:75) [pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) [?:?]
   pinot-server_1        | 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) [?:?]
   pinot-server_1        | 	at java.lang.Thread.run(Thread.java:829) [?:?]
   ```
   
   I tried a couple of other ways to use `DATETRUNC` but had the same problem.
   
   So I tried another config for the schema, using the `timestamp`data type, as the following:
   ```json
   "dateTimeFieldSpecs": [
       {
         "name": "operationDate",
         "dataType": "TIMESTAMP",
         "format": "1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd'T'HH:mm:ss.SSS'Z'",
         "granularity": "1:MILLISECONDS"
       }
     ]
   ```
   (obs.: the `TIMESTAMP` dataType is not listed in [the official docs](https://docs.pinot.apache.org/configuration-reference/schema#datetimefieldspec), I had to read the code and release details do find that it was existed)
   
   Then I got the following error inside pinot server:
   ```
   pinot-server_1        | 2021/11/12 16:15:58.933 ERROR [LLRealtimeSegmentDataManager_my_table__3__0__20211112T1615Z] [my_table__3__0__20211112T1615Z] Caught exception while transforming the record: {
   pinot-server_1        |   "fieldToValueMap" : {
   pinot-server_1        |     "some_uuid" : "fb13e0f6-a724-4876-9d4d-902ca94c2051",
   pinot-server_1        |     "operationDate" : "2020-07-20T12:55:54.000+0000",
   pinot-server_1        |     "condition" : "true",
   pinot-server_1        |     "product_name" : "My Product",
   pinot-server_1        |     "operation" : "delete"
   pinot-server_1        |   },
   pinot-server_1        |   "nullValueFields" : [ ]
   pinot-server_1        | }
   pinot-server_1        | java.lang.RuntimeException: Caught exception while transforming data type for column: operationDate
   pinot-server_1        | 	at org.apache.pinot.segment.local.recordtransformer.DataTypeTransformer.transform(DataTypeTransformer.java:120) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.segment.local.recordtransformer.CompositeTransformer.transform(CompositeTransformer.java:82) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.data.manager.realtime.LLRealtimeSegmentDataManager.processStreamEvents(LLRealtimeSegmentDataManager.java:510) [pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.data.manager.realtime.LLRealtimeSegmentDataManager.consumeLoop(LLRealtimeSegmentDataManager.java:417) [pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.core.data.manager.realtime.LLRealtimeSegmentDataManager$PartitionConsumer.run(LLRealtimeSegmentDataManager.java:560) [pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at java.lang.Thread.run(Thread.java:829) [?:?]
   pinot-server_1        | Caused by: java.lang.IllegalArgumentException: Invalid timestamp: '2020-07-20T12:55:54.000+0000'
   pinot-server_1        | 	at org.apache.pinot.spi.utils.TimestampUtils.toTimestamp(TimestampUtils.java:41) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.common.utils.PinotDataType$10.toTimestamp(PinotDataType.java:524) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.common.utils.PinotDataType$9.convert(PinotDataType.java:485) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.common.utils.PinotDataType$9.convert(PinotDataType.java:442) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	at org.apache.pinot.segment.local.recordtransformer.DataTypeTransformer.transform(DataTypeTransformer.java:114) ~[pinot-all-0.8.0-jar-with-dependencies.jar:0.8.0-c4ceff06d21fc1c1b88469a8dbae742a4b609808]
   pinot-server_1        | 	... 5 more
   ```
   
   I noticed 2 open issues related to the RFC3339 date format, but they didn't help me:
   https://github.com/apache/pinot/issues/7195
   https://github.com/apache/pinot/issues/7276
   
   What am I missing here?


-- 
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] dianaarnos commented on issue #7758: Problems to use transform functions and ingest dateTime columns with RFC3339 format

Posted by GitBox <gi...@apache.org>.
dianaarnos commented on issue #7758:
URL: https://github.com/apache/pinot/issues/7758#issuecomment-968662407


   Thanks for the quick answer, @Jackie-Jiang !
   I did the changes to the schema and `dateTimeConvert()` and now I get another error:
   ```json
   [
     {
       "message": "QueryExecutionError:\norg.joda.time.IllegalFieldValueException: Value 0 for dayOfMonth must be in the range [1,31]\n\tat org.joda.time.field.FieldUtils.verifyValueBounds(FieldUtils.java:259)\n\tat org.joda.time.field.PreciseDurationDateTimeField.set(PreciseDurationDateTimeField.java:79)\n\tat org.joda.time.DateTime.withDayOfMonth(DateTime.java:1782)\n\tat org.apache.pinot.core.operator.transform.transformer.datetime.BaseDateTimeTransformer.lambda$new$4(BaseDateTimeTransformer.java:83)\n\tat org.apache.pinot.core.operator.transform.transformer.datetime.BaseDateTimeTransformer.transformMillisToSDF(BaseDateTimeTransformer.java:104)\n\tat org.apache.pinot.core.operator.transform.transformer.datetime.SDFToSDFTransformer.transform(SDFToSDFTransformer.java:41)\n\tat org.apache.pinot.core.operator.transform.function.DateTimeConversionTransformFunction.transformToStringValuesSV(DateTimeConversionTransformFunction.java:164)\n\tat org.apache.pinot.core.operator.docvalsets.Tra
 nsformBlockValSet.getStringValuesSV(TransformBlockValSet.java:90)\n\tat org.apache.pinot.core.common.RowBasedBlockValueFetcher.createFetcher(RowBasedBlockValueFetcher.java:64)\n\tat org.apache.pinot.core.common.RowBasedBlockValueFetcher.<init>(RowBasedBlockValueFetcher.java:32)\n\tat org.apache.pinot.core.operator.query.SelectionOnlyOperator.getNextBlock(SelectionOnlyOperator.java:82)\n\tat org.apache.pinot.core.operator.query.SelectionOnlyOperator.getNextBlock(SelectionOnlyOperator.java:38)\n\tat org.apache.pinot.core.operator.BaseOperator.nextBlock(BaseOperator.java:49)\n\tat org.apache.pinot.core.operator.combine.BaseCombineOperator.processSegments(BaseCombineOperator.java:150)",
       "errorCode": 200
     }
   ]
   ```
   The date on the message is exact the same one on my original post example: `"operationDate": "2021-05-20T12:55:54.000+0000"`
   
   I will try the transform. Worst case scenario I'll change my producer's write the message already as millis since epoch and avoid all of 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] dianaarnos closed issue #7758: Problems to use transform functions and ingest dateTime columns with RFC3339 format

Posted by GitBox <gi...@apache.org>.
dianaarnos closed issue #7758:
URL: https://github.com/apache/pinot/issues/7758


   


-- 
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] Jackie-Jiang commented on issue #7758: Problems to use transform functions and ingest dateTime columns with RFC3339 format

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on issue #7758:
URL: https://github.com/apache/pinot/issues/7758#issuecomment-967703411


   The value you provided are of format `yyyy-MM-dd'T'HH:mm:ss.SSSZ`. Notice there is no single quotes around `Z`.
   
   In the `dateTimeFieldSpecs`, the format should be `1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd'T'HH:mm:ss.SSSZ`
   
   In the `dateTimeConvert` function, it should be `DATETIMECONVERT(operationDate, '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd''T''HH:mm:ss.SSSZ', '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM', '30:DAYS')`. Note that the literal should be single quoted, and single quote needs to be escaped with double single quotes per the sql syntax.
   
   I'd suggest doing ingestion transform to convert the time into millis since epoch and store it as `TIMESTAMP`. Read more about ingestion transform here: https://docs.pinot.apache.org/developers/advanced/ingestion-level-transformations


-- 
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] dianaarnos commented on issue #7758: Problems to use transform functions and ingest dateTime columns with RFC3339 format

Posted by GitBox <gi...@apache.org>.
dianaarnos commented on issue #7758:
URL: https://github.com/apache/pinot/issues/7758#issuecomment-968915918


   I made `dateTimeConvert()` run successfully by changing bucketing from `30:DAYS` to `15:DAYS` 
   ```sql
   DATETIMECONVERT(operationDate, '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM-dd''T''HH:mm:ss.SSSZ', '1:MILLISECONDS:SIMPLE_DATE_FORMAT:yyyy-MM', '15:DAYS')
   ```
   
   Other than that, I noticed a couple of messages had a malformed RFC3339. After fixing those everything worked fine.
   
   In case anyone had to deal with a similar problem:
   Original RFC3339 format looks like the following: `yyyy-MM-ddTHH:mm:ss.SSSZ`.
   So a correct dateTime would be something like: `2021-05-20T12:55:54.000+00:00`
   
   Some of my messages had values like: ``2021-05-20T12:55:54.000+0000` -> there is a colon missing on `+0000`. It should be `+00:00` instead.
   
   I'm closing this and thanks for the help :) 
   


-- 
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