You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@pinot.apache.org by Pinot Slack Email Digest <sn...@apache.org> on 2021/06/16 02:00:18 UTC

Apache Pinot Daily Email Digest (2021-06-15)

### _#general_

  
 **@patidar.rahul8392:** Hi Everyone, Is there any way to change the default
schema name. I.e. I have some tables in Pinot , and have a connection b/w
presto and pinot, at presto side when I am doing show schemas so it's showing
schema name as default and in this default schema my pinot tables are
available.How can I change this default schema name.  
 **@vaibhav.mital:** @vaibhav.mital has joined the channel  
 **@pedro.cls93:** Hello, Has anyone succesfully configured Pinot to work in
Trino in a Kubernetes environment? Following their , they mention that `The
Pinot broker and server must be accessible via DNS as Pinot returns hostnames
and not IP addresses.` , does this mean the actual pods or the services? Can
someone share what their configurations look like? I've tried the trino slack
unsuccessfully...  
**@mayanks:** @elon.azoulay ^^  
**@pedro.cls93:** FYI, I've been looking at the pinot connector source code,
this are my preliminary findings. When querying trino for table data in pinot
I get the following message: `Query 20210615_110517_00024_m2a8t failed: No
valid brokers found for hitexecutionview` , where `hitexecutionview` is the
name of my table but *lowercased*. In the connector source code this message
occurs when the connector is the brokers for a given table. This search is
done by a cache which is populated using the REST `tables/<table
name>/instances` GET endpoint from the Table API. So far I see 2 distinct
points of concern: 1.) Table name being lowercased means that the api is
unable to find the table. 2.) The REST call is `http`. In my pinot setup, I
can't use the swagger api with the `http` scheme, only `https`  
**@elon.azoulay:** Is this a mixed case table?  
**@pedro.cls93:** Real-time  
**@elon.azoulay:** I mean, in pinot is the table mixed case? i.e.
hitExecutionView or something similar?  
**@mayanks:** I think so  
**@mayanks:** Does that matter @elon.azoulay?  
**@elon.azoulay:** Yes, for the connector. We have an unmerged pull request
that handles this. Currently trino lowercases all sql identifiers. We do
handle mixed case columns, but not tables yet.  
**@pedro.cls93:** Yes it is Elon  
**@pedro.cls93:** I am now trying a lowercase table  
**@pedro.cls93:** Do columns fields also have to be lowercase?  
**@elon.azoulay:** Nope, that should be handled  
**@pedro.cls93:** Alright :+1:  
**@mayanks:** THanks @elon.azoulay.  
**@mayanks:** Good candidate to be added into FAQ? <#C023BNDT0N8|pinot-
docsrus> :slightly_smiling_face:  
**@elon.azoulay:** Sure, lmk if you have any more issues.  
**@pedro.cls93:** So futher updates, after lowercasing the table when I try to
query the table the requests time-out. ```trino:default> select * from
hitexecutionview limit 1; Query 20210615_164649_00006_ag23g, FAILED, 1 node
Splits: 49 total, 0 done (0.00%) 1:00 [0 rows, 0B] [0 rows/s, 0B/s] Query
20210615_164649_00006_ag23g failed: null value in entry: Server_pinot-
server-1.pinot-server-headless.dc-pinot.svc.cluster.local_8098=null``` In the
pinot sql UI this query takes 5ms  
**@elon.azoulay:** Is `pinot.set.instance.id.to.hostname` set to true in the
server config?  
**@pedro.cls93:** In pinot or trino?  
**@elon.azoulay:** Good point:) In pinot  
**@pedro.cls93:** Edit, actually it is, yes  
**@elon.azoulay:** Try setting that in pinot, and just to be safe (if you can)
restart trino before retrying the select.  
**@elon.azoulay:** oh  
**@elon.azoulay:** Can you try ```select * from "select * from
hitexecutionview limit 1"```  
**@elon.azoulay:** in trino?  
**@pedro.cls93:** ```trino:default> select * from "select * from
hitexecutionview limit 1" -> ; Query 20210615_165418_00007_ag23g, FAILED, 1
node Splits: 17 total, 0 done (0.00%) 0.83 [0 rows, 0B] [0 rows/s, 0B/s] Query
20210615_165418_00007_ag23g failed: Internal error```  
**@elon.azoulay:** Maybe try restarting trino (just in case it's something
with cached information from the other table)?  
**@elon.azoulay:** Also, enable `--debug` in the trino cli, so we can see the
stack trace  
**@pedro.cls93:** restart the trino cli or the pods?  
**@elon.azoulay:** the trino pods  
**@pedro.cls93:** ```$ kubectl -n dc-trino exec -it trino-
coordinator-7ddccf54fc-cbq5m -- /bin/bash [trino@trino-coordinator-7ddccf54fc-
cbq5m /]$ trino trino> exit [trino@trino-coordinator-7ddccf54fc-cbq5m /]$
trino --debug --catalog pinot --schema default trino:default> select * from
hitexecutionview limit 1; Query 20210615_170048_00002_ieyau, FAILED, 1 node
Splits: 97 total, 0 done (0.00%) CPU Time: 0.0s total, 0 rows/s, 0B/s, 100%
active Per Node: 0.0 parallelism, 0 rows/s, 0B/s Parallelism: 0.0 Peak Memory:
0B 1:01 [0 rows, 0B] [0 rows/s, 0B/s] Query 20210615_170048_00002_ieyau
failed: null value in entry: Server_pinot-server-0.pinot-server-headless.dc-
pinot.svc.cluster.local_8098=null java.lang.NullPointerException: null value
in entry: Server_pinot-server-0.pinot-server-headless.dc-
pinot.svc.cluster.local_8098=null at
com.google.common.collect.CollectPreconditions.checkEntryNotNull(CollectPreconditions.java:32)
at
com.google.common.collect.SingletonImmutableBiMap.<init>(SingletonImmutableBiMap.java:42)
at com.google.common.collect.ImmutableBiMap.of(ImmutableBiMap.java:72) at
com.google.common.collect.ImmutableMap.of(ImmutableMap.java:119) at
com.google.common.collect.ImmutableMap.copyOf(ImmutableMap.java:454) at
com.google.common.collect.ImmutableMap.copyOf(ImmutableMap.java:433) at
io.trino.plugin.pinot.PinotSegmentPageSource.queryPinot(PinotSegmentPageSource.java:221)
at
io.trino.plugin.pinot.PinotSegmentPageSource.fetchPinotData(PinotSegmentPageSource.java:182)
at
io.trino.plugin.pinot.PinotSegmentPageSource.getNextPage(PinotSegmentPageSource.java:150)
at io.trino.operator.TableScanOperator.getOutput(TableScanOperator.java:301)
at io.trino.operator.Driver.processInternal(Driver.java:387) at
io.trino.operator.Driver.lambda$processFor$9(Driver.java:291) at
io.trino.operator.Driver.tryWithLock(Driver.java:683) at
io.trino.operator.Driver.processFor(Driver.java:284) at
io.trino.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:1075)
at
io.trino.execution.executor.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:163)
at
io.trino.execution.executor.TaskExecutor$TaskRunner.run(TaskExecutor.java:484)
at io.trino.$gen.Trino_358____20210615_165930_2.run(Unknown Source) at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at java.base/java.lang.Thread.run(Thread.java:829)```  
**@elon.azoulay:** hmmm - is pinot running in kube but trino running locally?  
**@pedro.cls93:** No, both are running in k8s in separate namespaces  
**@elon.azoulay:** But same k8s cluster?  
**@pedro.cls93:** yes  
**@elon.azoulay:** hmm really strange - we use pinot in k8s (version 0.6.0)
(and latest trino connector). What version of pinot are you using?  
**@elon.azoulay:** I might try to repro locally...  
**@elon.azoulay:** Can you run `desc hitexecutionview` from trino cli?  
**@pedro.cls93:** Pinot 0.7.1  
**@pedro.cls93:** ```trino:default> desc hitexecutionview -> ; Column | Type |
Extra | Comment \----------------------------+---------+-------+---------
jobstatus | bigint | | gender | varchar | | rightclickcount | bigint | |
internalcrowdmemberid | bigint | | externalcrowdmemberid | varchar | |
internalhitexecutionid | bigint | | accuracy | varchar | | inputforuicontrols
| varchar | | audiolength | bigint | | hitstatus | bigint | | internaljobid |
bigint | | internaljobmemberid | bigint | | priceperunit | double | | result |
varchar | | dateofbirthms | bigint | | baseinternalhitid | bigint | |
dateofevent | bigint | | beforeinputdelay | bigint | | jobtypecode | varchar |
| externalhitexecutionid | varchar | | cancelreason | bigint | | keyscount |
bigint | | externaljobmemberid | varchar | | isjobmembervendor | varchar | |
activetime | bigint | | isrta | varchar | | fluency | varchar | | dateofbirth
| varchar | | languagecode | varchar | | leftclickcount | bigint | |
baseinternalhitexecutionid | bigint | | source-intelligible | varchar | |
externalprojectid | varchar | | externaljobid | varchar | | totaltasktime |
bigint | | translation | varchar | | isqualification | varchar | | skipreason
| varchar | | messagetimestamp | varchar | | externalhitid | varchar | |
hitexecutionstatus | bigint | | (41 rows) Query 20210615_171235_00003_ieyau,
FINISHED, 2 nodes  Splits: 19 total, 19 done (100.00%) CPU Time: 0.0s total,
1.32K rows/s, 106KB/s, 55% active Per Node: 0.0 parallelism, 64 rows/s,
5.18KB/s Parallelism: 0.1 Peak Memory: 0B 0.32 [41 rows, 3.3KB] [128 rows/s,
10.4KB/s]```  
**@elon.azoulay:** Nice, so that works.  
**@elon.azoulay:** Trying to see if there's any changes between 0.6.0 and
0.7.1 that would affect the routing table.  
**@pedro.cls93:** metadata queries all seem to work, its just data that does
not  
**@elon.azoulay:** Yep, looks like it tried to run the query and got back a
null response  
**@elon.azoulay:** We are still poc'ing the upgrade, still using pinot 0.6.0  
**@pedro.cls93:** how does the connector work when running the query?  
**@pedro.cls93:** Does it serialize the query and send it through a REST
endpoint?  
**@elon.azoulay:** Yep, I just read from the beginning - your pinot cluster is
only accessible via https?  
**@elon.azoulay:** That query uses the PinotQueryClient  
**@pedro.cls93:** Yes, http does not work, can not tell you why though.
@mayanks was the one that told me to use https schema in the swagger client  
**@elon.azoulay:** Trying to see if this is due to changes in 0.7.1 - would it
be possible to spin up a test pinot 0.6.0 cluster and try it?  
**@pedro.cls93:** I think so, let me try...  
**@pedro.cls93:** Unfortunately I'm not able to @elon.azoulay. I have a hard
dependency on the following image:
pinot:0.8.0-SNAPSHOT-f15225f9c-20210523-jdk8 because of some fixes I need.  
**@pedro.cls93:** Is there any way I can help the upgrade of the connector to
a more recent pinot version?  
**@elon.azoulay:** We're working on that too, should have a pr ready (there
are many pr's in flight right now). If I use the pinot 0.7.1 image should that
be compatible with the snapshot you mentioned above?  
**@elon.azoulay:** Want to try this locally to debug it.  
**@pedro.cls93:** Can't confirm but I'm hopeful that yes. The image I'm using
is from 22nd of May 2021 (), 0.7.1 was released on April 15th, that's ~37 days
of potential changes.  
**@elon.azoulay:** Ok, we were working on updating the pinot connector to use
0.7.1 (latest stable) but 0.8.0 may be coming out very soon, first trying
0.7.1 and will see what happens:)  
**@elon.azoulay:** Will update shortly...  
**@pedro.cls93:** Thank you very very much for all the assistence Elon, if
there is anything I can do to help please let me know.  
**@elon.azoulay:** Sure:)  
 **@azri:** @azri has joined the channel  
 **@pedro.cls93:** Is there an ETA for a newer pinot release than 0.7.1? The
documentation already refers to new data types:  that do not seems to be
supported in pinot 0.7.1.  
 **@jai.patel856:** I’ve got some questions about the DateTimeFieldSpecs
defined here:  1\. Are there any benefits (either way) to configuring a LONG
column as a dateTimeFieldSpec if it’s being used in GROUP BY clauses of a
query? 2\. If I have a epoch time that I’ve divided into 15 minutes segments…
it seems like the right way to configure format would be to use
15:MINUTES:EPOCH. But what does granularity do? The first sample where format
is 1:MILLISECONDS:EPOCH but granularity is 15:MINUTES doesn’t make sense to
me.  
 **@karinwolok1:** Hey all! :tada: We have a great meetup next week and would
love for you to join! :smiley: TALK 1: Pinot vs Elasticsearch, a Tale of Two
PoCs by @ken TALK 2: Forward index reader performance improvement by
@jackie.jxt  

### _#random_

  
 **@vaibhav.mital:** @vaibhav.mital has joined the channel  
 **@azri:** @azri has joined the channel  

###  _#fraud_

  
 **@juraj.komericki:** @juraj.komericki has joined the channel  

###  _#troubleshooting_

  
 **@khushbu.agarwal:** Hi, I have followed this documentation to query pinot
using Java client:  I have scheduled the query to run after every 24 hours.
The first run goes fine. However the subsequent iteration fails with the below
exception. 2021-06-15 05:23:09.880 INFO 11 --- [ter.local:2181)]
org.apache.zookeeper.ClientCnxn : Unable to read additional data from server
sessionid 0x0, likely server has closed socket, closing socket connection and
attempting reconnect 2021-06-15 05:23:09.910 INFO 11 --- [ter.local:2181)]
org.apache.zookeeper.ClientCnxn : Opening socket connection to server
pinot2-zookeeper.data2.svc.cluster.local/10.100.71.32:2181. Will not attempt
to authenticate using SASL (unknown error) 2021-06-15 05:23:09.910 INFO 11 ---
[ter.local:2181)] org.apache.zookeeper.ClientCnxn : Socket connection
established to pinot2-zookeeper.data2.svc.cluster.local/10.100.71.32:2181,
initiating session 2021-06-15 05:23:09.911 INFO 11 --- [ter.local:2181)]
org.apache.zookeeper.ClientCnxn : Unable to read additional data from server
sessionid 0x0, likely server has closed socket, closing socket connection and
attempting reconnect 2021-06-15 05:23:09.941 INFO 11 --- [ter.local:2181)]
org.apache.zookeeper.ClientCnxn : Opening socket connection to server
pinot2-zookeeper.data2.svc.cluster.local/10.100.71.32:2181. Will not attempt
to authenticate using SASL (unknown error) 2021-06-15 05:23:09.941 INFO 11 ---
[ter.local:2181)] org.apache.zookeeper.ClientCnxn : Socket connection
established to pinot2-zookeeper.data2.svc.cluster.local/10.100.71.32:2181,
initiating session 2021-06-15 05:23:09.941 INFO 11 --- [ter.local:2181)]
org.apache.zookeeper.ClientCnxn : Unable to read additional data from server
sessionid 0x0, likely server has closed socket, closing socket connection and
attempting reconnect 2021-06-15 05:23:10.279 INFO 11 --- [ter.local:2181)]
org.apache.zookeeper.ClientCnxn : Opening socket connection to server
pinot2-zookeeper.data2.svc.cluster.local/10.100.71.32:2181. Will not attempt
to authenticate using SASL (unknown error) Any help here?  
**@g.kishore:** its probably GCing  
**@g.kishore:** whats the query? is it pulling a lot of data out of Pinot?  
**@khushbu.agarwal:** There is no data currently  
**@khushbu.agarwal:** Hardly couple of rows  
**@g.kishore:** running in k8s?  
**@khushbu.agarwal:** Yes  
**@g.kishore:** can you check the pod status and vm resources  
**@g.kishore:** most likely pods are crashing  
**@khushbu.agarwal:** No pod isn't crashing  
**@khushbu.agarwal:** Its running fine  
**@khushbu.agarwal:** There is connection leakage from the report service,
which queries pinot. Steps taken to figure this out: 1\. Checked pinot
zookeeper pod logs on k8s 2\. Post that used the following command to get all
the connection details for all clients to this server: echo stat | nc
<zk_host> <zk port> echo cons | nc <zk_host> <zk port> All the connection was
identified to be from a single ip which belonged to the report service.  
**@g.kishore:** :+1:  
**@khushbu.agarwal:** @g.kishore thank you for the quick help  
 **@vaibhav.mital:** @vaibhav.mital has joined the channel  
 **@azri:** @azri has joined the channel  
 **@azri:** Hi expert, I just add a geospatial index on existing realtime
table, the new data ingested generate the index correctly.. but how do I apply
it for previous segment as well?  
**@g.kishore:** You can invoke reload segments controller api  
**@azri:** I did, but seems doesn't have any effect..  
 **@surendra:** Hi, We have below query working in 0.6.0 and failing 0.7.1
with exception `2021/06/10 16:53:02.084 WARN [BaseBrokerRequestHandler]
[jersey-server-managed-async-executor-100] Caught exception while updating
Column names in Query 13040: Illegal AstNode type for TransformExpressionTree:
org.apache.pinot.pql.parsers.pql2.ast.StarExpressionAstNode` and if we update
`else null` in case clause to `else 'null'` it's working in .7.1 , any clues ?
```select emp_id , id , max(timestampMs) as last_updated, count(DISTINCT case
when emp_type = 'admin' then user_count else null end) as admin_counts from
emp_events where org_id = 987 AND timestampMs >= 1620892156939 AND timestampMs
< 1621630122162 group by emp_id, id limit 10```  
**@jackie.jxt:** Pinot does not support `null` in query as of now. In the old
version it might somehow be overridden to string `'null'` but that is not the
correct semantic of the query though  
**@jackie.jxt:** This issue is revealed probably because we moved to the
CalciteSqlParser, which parse `null` as real `null` instead of string `'null'`  
**@surendra:** is there any workaround for this ?  
**@jackie.jxt:** Add single quotes to `null` to make it a string `'null'`  
**@surendra:** Yeah, We did that for now.  
**@jackie.jxt:** Actually I think it should be faster if we split the query
into 2 instead of using case to do the `emp_type` filtering  
**@surendra:** If possible can you share an example ?  
**@jackie.jxt:** Q1 ```select emp_id , id , max(timestampMs) as last_updated,
from emp_events where org_id = 987 AND timestampMs >= 1620892156939 AND
timestampMs < 1621630122162 group by emp_id, id limit 10``` Q2 ```select
emp_id , id , count(DISTINCT user_count) as admin_counts from emp_events where
org_id = 987 AND emp_type = 'admin' AND timestampMs >= 1620892156939 AND
timestampMs < 1621630122162 group by emp_id, id limit 10```  
 **@jmeyer:** Hello ^^ Not really an issue, just checking if this is "normal
behavior" `select DISTINCT(kpi) from kpis` takes ~6ms (with 100M docs, &
`numDocsScanned: 100000`) - this query returns 45 strings only But doing
`select DISTINCT(kpi) from kpis ORDER BY kpi` takes >300ms (50 times slower) -
It scans every documents (`numDocsScanned: 101250000`) I guess the `ORDER BY`
breaks some optimizations down But from the outside it seems like pretty
surprising behavior (sorting 45 strings "should not take this long" is what I
mean) Anyway, not here to complain, just wanted to point it out in case it
would be considered as something worth investigating  
**@mayanks:** Is the cardinality of kpi only 45 and both return just 45
values?  
**@mayanks:** If so, it does seem like some room for optimization. Mind filing
an issue?  
**@jmeyer:** Yes, only 45 different 'kpis' exist in all 100M docs  
**@jmeyer:** @mayanks Okay ! I'll file this one and another sometime this week
:smile:  
**@mayanks:** How many segments do you have?  
**@jmeyer:** 90  
**@mayanks:** `numDocsScanned: 100000` seems to suggest early bailout  
**@jmeyer:** `select COUNT(DISTINCT(kpi)) from kpis` -> 45 `select
COUNT(DISTINCT(kpi)) from kpis ORDER BY kpi` -> 45  
**@jmeyer:** Ah, got something interesting  
**@jmeyer:** With the `COUNT`, queries are equally as fast  
**@mayanks:** count without predicate is just answered via metadata (unless
hybrid table, where implicit time predicate added for offline/realtime
queries)  
**@mayanks:** are you seeing the second query to be consistently slower?  
**@jmeyer:** Oh, thanks for the info, learning everyday
:slightly_smiling_face:  
**@jmeyer:** In case you want to have a look now, here are query logs with
trace  
**@jmeyer:** With `ORDER BY`  
**@jmeyer:** Without `ORDER BY`  
**@jmeyer:** > are you seeing the second query to be consistently slower? Yes,
consistently in the 300-350ms range While the other one is in the 7-13ms range  
**@g.kishore:** It’s a simple optimization.. we use dictionary to solve the
query if there is no predicate..  
**@g.kishore:** But looks like we also check for no order by for that
optimization to kick in  
**@g.kishore:** @mayanks should we a simple fix to enhance the optimizer  
**@mayanks:** Yeah I was about to look at the code on whether we do dictionary
based for distinct or not.  
**@mayanks:** If not then something else might be going on  
**@g.kishore:** We do  
**@g.kishore:** It’s just that if there is order by then we fallback to full
scan  
**@mayanks:** Distinct count we do, distinct was added much later, so not sure  
**@mayanks:** ``` public static boolean
isFitForDictionaryBasedComputation(String functionName) { //@formatter:off
return functionName.equalsIgnoreCase(AggregationFunctionType.MIN.name()) ||
functionName.equalsIgnoreCase(AggregationFunctionType.MAX.name()) ||
functionName.equalsIgnoreCase(AggregationFunctionType.MINMAXRANGE.name()) ||
functionName.equalsIgnoreCase(AggregationFunctionType.DISTINCTCOUNT.name()) ||
functionName.equalsIgnoreCase(AggregationFunctionType.SEGMENTPARTITIONEDDISTINCTCOUNT.name());```  
**@mayanks:** Apparently not for `distinct`  
**@mayanks:** Which is what I was suspecting to begin with, there's something
else going on  
**@g.kishore:** Interesting.. definitely worth looking into.. let’s continue
on GitHub issue.  
**@g.kishore:** @jmeyer can you please an issue  
**@jmeyer:** @g.kishore @mayanks Sure, will do soon Happy to see I brought up
an interesting topic :slightly_smiling_face:  
**@jmeyer:** ->  
**@jai.patel856:** I’m trying to setup a dateTimeFieldSpec with a dateTrunc
and I’m getting an error: ``` { "name": "created_at_1_week_seconds",
"dataType": "LONG", "defaultNullValue": 0, "transformFunction":
"dateTrunc('week', created_at, 'MILLISECONDS')", "format": "1:SECONDS:EPOCH",
"granularity": "1:SECONDS" },``` Error: ```Exception in getting arguments for
transform function 'dateTrunc('week', created_at, MILLISECONDS)' for column
'created_at_1_week_seconds'" }``` This works: ``` { "name":
"created_at_1_day_seconds", "dataType": "LONG", "defaultNullValue": 0,
"transformFunction":
"toEpochSecondsRounded(fromEpochDaysBucket(toEpochDaysBucket(created_at, 1),
1), 1)", "format": "1:SECONDS:EPOCH", "granularity": "1:SECONDS" },```
Something about the dateTrunc function doesn’t seem to be compatible. Looking
through the docs and the code it looks like dateTrunc may not be listed as a
valid transform function in all use cases. The main issue is I want a
toEpochDaysBucket(7) that lines up with the calendar week (Sunday or Monday)
instead of the epoch week (Thursday). Any ideas?  
**@fx19880617:** I think the issue is that dateTrunc implements internal
`BaseTransformFunction` function not `ScalarFunction` function which can be
used in the `transformFunction` field.  
**@jai.patel856:** I see. Is that an alternate query anyone can think of?
Would there be a groovy syntax that would work?  
**@fx19880617:** hmmm  
**@fx19880617:** I saw some function like: ``` /** * The sql compatible
date_trunc function for epoch time. * @param unit truncate to unit
(millisecond, second, minute, hour, day, week, month, quarter, year) * @param
timeValue value to truncate * @param inputTimeUnitStr TimeUnit of value,
expressed in Java's joda TimeUnit * @return truncated timeValue in same
TimeUnit as the input */ @ScalarFunction public static long dateTrunc(String
unit, long timeValue, String inputTimeUnitStr) { return dateTrunc(unit,
timeValue, inputTimeUnitStr, TimeZoneKey.UTC_KEY.getId(), inputTimeUnitStr);
}```  
**@fx19880617:** doe this `dateTrunc(week, created_at, MILLISECONDS)` work?  
**@fx19880617:** without single quote?  
**@fx19880617:** do you have the stacktrace  
**@jai.patel856:** I’ll try that. How do I retrieve the stack trace?  
**@fx19880617:** check pinot server log  
**@fx19880617:** if you are on k8s you may need to enter into the container
and see pinot-server.log file  
**@fx19880617:** mostly the `kubectl logs pinot-server-0` should tell  
 **@surendra:** What are the different options available to delete
tables/segments ? Using a API it's taking lot of time as we have > 20k
segments.  
**@mayanks:** Are you using deep-store? If so, which one?  
**@mayanks:** I have seen it take time because by default deleted segments are
copied to DeletedSegments folder (more as precaution for production env, where
there can be accidental deletes). Perhaps we can provide an option to bypass
that. Could you please file an issue for that?  
**@mayanks:** If this is just in a test/staging env, if you manually delete
the segments from controller, it will be faster. But this is just a hack, and
not to be done in prod env.  
**@surendra:** Thanks for the info. We are using s3 as deep store , I will
open a ticket for that.  

###  _#pinot-dev_

  
 **@vaibhav.mital:** @vaibhav.mital has joined the channel  

###  _#getting-started_

  
 **@agnihotrisuryansh55:** @agnihotrisuryansh55 has joined the channel  

###  _#pinot-docsrus_

  
 **@jackie.jxt:** @jackie.jxt has joined the channel  
 **@pedro.cls93:** @pedro.cls93 has joined the channel  
\--------------------------------------------------------------------- To
unsubscribe, e-mail: dev-unsubscribe@pinot.apache.org For additional commands,
e-mail: dev-help@pinot.apache.org