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 2022/09/13 22:58:05 UTC

[GitHub] [pinot] bagipriyank opened a new issue, #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

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

   Use https://docs.pinot.apache.org/users/tutorials/ingest-parquet-files-from-s3-using-spark to create an offline table. I invoked delete all segments api, or delete segment api from swagger with retention 0d or -1d and observed that the segment got deleted from the controller ui and zookeeper but did not get deleted from the server disc. If i used 
   ```
   pushJobSpec:
     copyToDeepStoreForMetadataPush: true
   ```
   
   in the spark ingestion job spec then the segments got deleted from the s3 segment store but still didn't get deleted from the server disc.
   
   I tried deleting the table, and even then the segment(s) did not get deleted from the server disc. I do not have a retention configured in the table config for this table.
   
   Even disabling the table, and then invoking delete segment(s) apis didn't delete segments from the server disc. This is resulting in increased disc usage on daily refresh.


-- 
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] Jackie-Jiang commented on issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

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

   Interesting.. Do you have the server connected to the cluster when you delete the segments?
   When the segments are deleted from the controller, controller will send message to the server to drop the segment, which will remove the segment from the server's disk. It is done asynchronously, so some delay is expected.


-- 
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] bagipriyank commented on issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

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

   i found what caused it, and am able to fix it. i don't know why it causes the issue though.
   we have `dataDir` and `segmentTarDir` but then we also have extra configs
   ```
         pinot.server.instance.dataDir=/var/pinot/server/data/index
         pinot.server.instance.segmentTarDir=/var/pinot/server/data/segment
   ```
   if i remove these two extra server configs then segments are stored at a path with table name in the path  :shrug:


-- 
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] bagipriyank commented on issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

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

   @Jackie-Jiang yes the issue is resolved after removing the 2 extra entires.


-- 
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] bagipriyank closed issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

Posted by GitBox <gi...@apache.org>.
bagipriyank closed issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc
URL: https://github.com/apache/pinot/issues/9393


-- 
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] walterddr commented on issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

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

   how did you change the retention period? did you post that as part of the REST API request to delete the segment or did you change the `controller.deleted.segments.retentionInDays` setting after the segment is deleted?
   
   the expected behavior is that the retention for a segment is set at the moment of segment deletion, once segment is deleted the retention period will not change (wether via changing the controller config or change the table config)


-- 
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] walterddr commented on issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

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

   ah. ok i think this is indeed unexpected behavior. please let us know what your find in the log. thanks!


-- 
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] bagipriyank commented on issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

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

   I think I know what is happening 
   From https://github.com/apache/pinot/blob/c9a6e5207a6cedf325f4abaeac2970077ab5685d/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java#L241
   ```
         // Clean up the segment data on default tier unconditionally.
         File segmentDir = getSegmentDataDirectory(tableNameWithType, segmentName);
         if (segmentDir.exists()) {
           FileUtils.deleteQuietly(segmentDir);
           LOGGER.info("Deleted segment directory {} on default tier", segmentDir);
         }
   ```
   Now from https://github.com/apache/pinot/blob/c9a6e5207a6cedf325f4abaeac2970077ab5685d/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixInstanceDataManager.java#L507
   ```
   /**
      * Assemble the path to segment dir directly, when table mgr object is not
      * created for the given table yet.
      */
     @Override
     public File getSegmentDataDirectory(String tableNameWithType, String segmentName) {
       return new File(new File(_instanceDataManagerConfig.getInstanceDataDir(), tableNameWithType), segmentName);
     }
   ```
   
   in our case this is point to `/var/pinot/server/data/index/students_OFFLINE/students_OFFLINE_1597044264380_1597044264670_0` instead of `/var/pinot/server/data/index/students_OFFLINE_1597044264380_1597044264670_0` which leaves the delete as a no-op.
   
   How can i verify / test 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] Jackie-Jiang commented on issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

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

   This is a known issue, which is tracked here #9102
   
   Is the problem resolved after removing these 2 extra entries?


-- 
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] bagipriyank commented on issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

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

   > Do you have the server connected to the cluster when you delete the segments?
   Yes
   
   I tried waiting but I don't see any files getting deleted even after a day. 


-- 
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] bagipriyank closed issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

Posted by GitBox <gi...@apache.org>.
bagipriyank closed issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc
URL: https://github.com/apache/pinot/issues/9393


-- 
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] bagipriyank commented on issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

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

   https://github.com/apache/pinot/blob/b2da31005b9e958a022150cfaac228a951daf0b3/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/DefaultHelixStarterServerConfig.java#L40 logs the config map
   ```
   2022-11-17 13:53:41 | External config key: pinot.server.instance.segmenttardir, value: /var/pinot/server/data/segment,/var/pinot/server/data/segment
   2022-11-17 13:53:41 | External config key: pinot.server.swagger.use.https, value: true
   2022-11-17 13:53:41 | External config key: pinot.server.instance.realtime.alloc.offheap, value: true
   2022-11-17 13:53:41 | External config key: pinot.server.adminapi.port, value: 8097
   2022-11-17 13:53:41 | External config key: pinot.zk.server, value: zookeeper-offline.de-nrt-pinot.svc.cluster.local:2181
   2022-11-17 13:53:41 | External config key: pinot.server.instance.datadir, value: /var/pinot/server/data/index,/var/pinot/server/data/index
   ```
   so the value is now a comma separated string


-- 
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] bagipriyank commented on issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

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

   @walterddr looks like you are correct. i see two entries in the pinot server configmap
   ```
   apiVersion: v1
   data:
     pinot-server.conf: |-
       pinot.server.netty.port=8098
       pinot.server.adminapi.port=8097
       pinot.server.instance.dataDir=/var/pinot/server/data/index
       pinot.server.instance.segmentTarDir=/var/pinot/server/data/segment
       pinot.set.instance.id.to.hostname=true
       pinot.server.instance.realtime.alloc.offheap=true
       pinot.server.instance.currentDataTableVersion=2
       pinot.server.instance.dataDir=/var/pinot/server/data/index
       pinot.server.instance.segmentTarDir=/var/pinot/server/data/segment
       pinot.server.storage.factory.class.s3=org.apache.pinot.plugin.filesystem.S3PinotFS
       pinot.server.storage.factory.s3.region=us-east-1
       pinot.server.segment.fetcher.protocols=file,http,s3
       pinot.controller.storage.factory.s3.disableAcl=false
       pinot.server.segment.fetcher.s3.class=org.apache.pinot.common.utils.fetcher.PinotFSSegmentFetcher
       pinot.server.segment.fetcher.s3.retry.count=10
       pinot.server.instance.segment.store.uri=s3://<redacted>/controller-data
       pinot.server.instance.max.parallel.refresh.threads=10
       pinot.zk.server=zookeeper-offline.de-nrt-pinot.svc.cluster.local:2181
       pinot.server.swagger.use.https=true
   kind: ConfigMap
   metadata:
     annotations:
       meta.helm.sh/release-name: pinot-offline
       meta.helm.sh/release-namespace: de-nrt-pinot
     creationTimestamp: "2022-11-17T06:35:21Z"
     labels:
       app.kubernetes.io/managed-by: Helm
     name: pinot-offline-server-config
     namespace: de-nrt-pinot
     resourceVersion: "282315454"
     uid: 5edaf2d3-eda0-48ba-afba-4a006020a543
   ```


-- 
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] bagipriyank commented on issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

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

   > how did you change the retention period? 
   
   @walterddr i didn't change / add the retention period on the table. basically there is no retention config for the table. i used retention of 0d / -1d as a parameter on the delete segments api [here](https://github.com/apache/pinot/blob/master/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentRestletResource.java#L822)


-- 
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] bagipriyank commented on issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

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

   This is what i see in the logs
   ```
   2022-11-15 16:59:43 | 77 END:INVOKE CallbackHandler 0, /pinot-offline-staging/INSTANCES/Server_pinot-offline-server-0.pinot-offline-server-headless.de-nrt-pinot.svc.cluster.local_8098/MESSAGES listener: org.apache.helix.messaging.handling.HelixTaskExecutor@2ef8cae type: CALLBACK Took: 2ms
   2022-11-15 16:59:43 | No Messages to process
   2022-11-15 16:59:43 | CallbackHandler0, Subscribing to path: /pinot-offline--staging/INSTANCES/Server_pinot-offline-server-0.pinot-offline-server-headless.de-nrt-pinot.svc.cluster.local_8098/MESSAGES took: 1
   2022-11-15 16:59:43 | CallbackHandler 0 subscribing changes listener to path: /pinot-offline-staging/INSTANCES/Server_pinot-offline-server-0.pinot-offline-server-headless.de-nrt-pinot.svc.cluster.local_8098/MESSAGES, callback type: CALLBACK, event types: [NodeChildrenChanged], listener: org.apache.helix.messaging.handling.HelixTaskExecutor@2ef8cae, watchChild: false
   2022-11-15 16:59:43 | 77 START: CallbackHandler 0, INVOKE /pinot-offline-staging/INSTANCES/Server_pinot-offline-server-0.pinot-offline-server-headless.de-nrt-pinot.svc.cluster.local_8098/MESSAGES listener: org.apache.helix.messaging.handling.HelixTaskExecutor@2ef8cae type: CALLBACK
     |   | 2022-11-15 16:59:43 | Message: c279d79e-f19e-43ad-ae96-4ab0071e2fbe (parent: null) handling task for students_OFFLINE:students_OFFLINE_1597044264380_1597044264670_0 completed at: 1668560383006, results: true. FrameworkTime: 4 ms; HandlerTime: 6 ms.
   2022-11-15 16:59:43 | message finished: c279d79e-f19e-43ad-ae96-4ab0071e2fbe, took 9
   2022-11-15 16:59:43 | MBean CLMParticipantReport:Cluster=pinot-offline-staging,Transition=OFFLINE--DROPPED has been registered.
   2022-11-15 16:59:43 | Delete message c279d79e-f19e-43ad-ae96-4ab0071e2fbe from zk!
   2022-11-15 16:59:43 | Message c279d79e-f19e-43ad-ae96-4ab0071e2fbe completed.
   2022-11-15 16:59:43 | Removed /pinot-offline-staging/INSTANCES/Server_pinot-offline-server-0.pinot-offline-server-headless.de-nrt-pinot.svc.cluster.local_8098/CURRENTSTATES/30000028dd80026/students_OFFLINE
   2022-11-15 16:59:43 | Merging with delta list, recordId = students_OFFLINE other:students_OFFLINE
   2022-11-15 16:59:42 | SegmentOnlineOfflineStateModel.onBecomeDroppedFromOffline() : ZnRecord=c279d79e-f19e-43ad-ae96-4ab0071e2fbe, {CREATE_TIMESTAMP=1668560382982, ClusterEventName=MessageChange, EXECUTE_START_TIMESTAMP=1668560382997, EXE_SESSION_ID=30000028dd80026, FROM_STATE=OFFLINE, MSG_ID=c279d79e-f19e-43ad-ae96-4ab0071e2fbe, MSG_STATE=read, MSG_TYPE=STATE_TRANSITION, PARTITION_NAME=students_OFFLINE_1597044264380_1597044264670_0, READ_TIMESTAMP=1668560382991, RESOURCE_NAME=students_OFFLINE, RESOURCE_TAG=students_OFFLINE, RETRY_COUNT=3, SRC_NAME=pinot-offline-controller-0.pinot-offline-controller-headless.de-nrt-pinot.svc.cluster.local_9000, SRC_SESSION_ID=30000028dd8001b, STATE_MODEL_DEF=SegmentOnlineOfflineStateModel, STATE_MODEL_FACTORY_NAME=DEFAULT, TGT_NAME=Server_pinot-offline-server-0.pinot-offline-server-headless.de-nrt-pinot.svc.cluster.local_8098, TGT_SESSION_ID=30000028dd80026, TO_STATE=DROPPED}{}{}, Stat=Stat {_version=0, _creationTime=1668560382983, _modifiedTime=1
 668560382983, _ephemeralOwner=0}
   ```
   
   I don't see any entries in map fields for the table in external view or ideal state. i don't see any segments for the table in property store either. It has been more than 12 hours since I invoked delete segments api call via swagger but I still see the segments on the disc on the server
   ```
   root@pinot-offline-server-0:/opt/pinot# cd /var/pinot/server/data/index
   root@pinot-offline-server-0:/var/pinot/server/data/index# ls -lh
   total 8.0K
   drwxr-sr-x 3 root 1337 4.0K Nov 16 00:14 students_OFFLINE_1597044264380_1597044264670_0
   drwxr-sr-x 2 root 1337 4.0K Nov 16 00:14 tmp
   root@pinot-offline-server-0:/var/pinot/server/data/index#
   ```
   
   This is the table config 
   ```
   {
     "OFFLINE": {
       "tableName": "students_OFFLINE",
       "tableType": "OFFLINE",
       "segmentsConfig": {
         "timeType": "MILLISECONDS",
         "schemaName": "students",
         "retentionTimeUnit": "MINUTES",
         "retentionTimeValue": "30",
         "replication": "1",
         "timeColumnName": "timestampInEpoch",
         "minimizeDataMovement": false
       },
       "tenants": {
         "broker": "DefaultTenant",
         "server": "DefaultTenant"
       },
       "tableIndexConfig": {
         "invertedIndexColumns": [],
         "rangeIndexVersion": 2,
         "autoGeneratedInvertedIndex": false,
         "createInvertedIndexDuringSegmentGeneration": false,
         "loadMode": "MMAP",
         "enableDefaultStarTree": false,
         "aggregateMetrics": false,
         "enableDynamicStarTreeCreation": false,
         "nullHandlingEnabled": false,
         "optimizeDictionaryForMetrics": false,
         "noDictionarySizeRatioThreshold": 0
       },
       "metadata": {},
       "isDimTable": false
     }
   }
   ```


-- 
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] bagipriyank commented on issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

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

   I tried reproducing it locally by running Quickstart from with in IDE and I am not able to reproduce the issue.
   ```
   C02TW0TMHTDG:~ pbagrecha$ cd /var/folders/l9/69q0947d2yb9g5wl6hsxbbnmd4s6w1/T/1668642652045/quickstart/
   C02TW0TMHTDG:quickstart pbagrecha$ ls -lh
   total 0
   drwxr-xr-x  10 pbagrecha  2069044465   320B Nov 16 15:52 PinotControllerDir0
   drwxr-xr-x   9 pbagrecha  2069044465   288B Nov 16 15:52 PinotServerDataDir0
   drwxr-xr-x   2 pbagrecha  2069044465    64B Nov 16 15:51 PinotServerSegmentDir0
   drwxr-xr-x   3 pbagrecha  2069044465    96B Nov 16 15:50 PinotZkDir
   C02TW0TMHTDG:quickstart pbagrecha$
   C02TW0TMHTDG:quickstart pbagrecha$
   C02TW0TMHTDG:quickstart pbagrecha$ ls -lh PinotServerDataDir0/
   total 0
   drwxr-xr-x  34 pbagrecha  2069044465   1.1K Nov 16 15:51 airlineStats_OFFLINE
   drwxr-xr-x   4 pbagrecha  2069044465   128B Nov 16 15:51 baseballStats_OFFLINE
   drwxr-xr-x   4 pbagrecha  2069044465   128B Nov 16 15:52 billing_OFFLINE
   drwxr-xr-x   4 pbagrecha  2069044465   128B Nov 16 15:52 dimBaseballTeams_OFFLINE
   drwxr-xr-x   4 pbagrecha  2069044465   128B Nov 16 15:52 githubComplexTypeEvents_OFFLINE
   drwxr-xr-x   4 pbagrecha  2069044465   128B Nov 16 15:52 githubEvents_OFFLINE
   drwxr-xr-x   4 pbagrecha  2069044465   128B Nov 16 15:52 starbucksStores_OFFLINE
   C02TW0TMHTDG:quickstart pbagrecha$
   C02TW0TMHTDG:quickstart pbagrecha$
   C02TW0TMHTDG:quickstart pbagrecha$ ls -lh PinotServerDataDir0/airlineStats_OFFLINE/
   total 0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16071_16071_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16072_16072_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16073_16073_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16074_16074_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16075_16075_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16076_16076_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16077_16077_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16078_16078_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16079_16079_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16080_16080_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16081_16081_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16082_16082_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16083_16083_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16084_16084_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16085_16085_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16086_16086_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16087_16087_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16088_16088_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16089_16089_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16090_16090_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16091_16091_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16092_16092_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16093_16093_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16094_16094_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16095_16095_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16096_16096_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16097_16097_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16098_16098_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16099_16099_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16100_16100_0
   drwxr-xr-x  3 pbagrecha  2069044465    96B Nov 16 15:51 airlineStats_OFFLINE_16101_16101_0
   drwxr-xr-x  2 pbagrecha  2069044465    64B Nov 16 15:51 tmp
   C02TW0TMHTDG:quickstart pbagrecha$ ls -lh PinotServerSegmentDir0/
   total 0
   C02TW0TMHTDG:quickstart pbagrecha$
   ```
   segment directory does have the table name in the path
   i am guessing that we have something mis-configured or maybe we completely missed configuring something.


-- 
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] bagipriyank commented on issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

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

   this is our helm chart config for server
   ```
   server:
     name: server
     replicaCount: 15
     updateStrategy:
       type: RollingUpdate
     podManagementPolicy: Parallel
     podSecurityContext: {}
     securityContext: {}
     probes:
       endpoint: "/health"
       livenessEnabled: false
       readinessEnabled: false
     nodeSelector:
       node_type: normal
     tolerations: []
     affinity: {}
     podAnnotations:
       "prometheus.io/scrape": "true"
       "prometheus.io/port": "8008"
     envFrom: []
     extraEnv: []
     log4j2ConfFile: /opt/pinot/conf/log4j2.xml
     pluginsDir: /opt/pinot/plugins
     service:
       annotations:
         "prometheus.io/scrape": "true"
         "prometheus.io/port": "8008"
       clusterIP: ""
       externalIPs: []
       loadBalancerIP: ""
       loadBalancerSourceRanges: []
       type: ClusterIP
       nettyPort: 8098
       nettyPortName: netty
       adminPort: 8097
       adminExposePort: 80
       adminPortName: admin
       nodePort: ""
       protocol: TCP
     resources:
       limits:
         cpu: 8
         memory: 64G
       requests:
         cpu: 8
         memory: 64G
     dataDir: /var/pinot/server/data/index
     segmentTarDir: /var/pinot/server/data/segment
     persistence:
       enabled: true
       accessMode: ReadWriteOnce
       size: 512G
       mountPath: /var/pinot/server/data
       storageClass: gp2
       replicaCount: 1
     jvmOpts: "-Xms8G -Xmx8G -XX:MaxDirectMemorySize=55G -Xlog:gc*:file=/opt/pinot/gc-pinot-server.log -javaagent:/opt/pinot/etc/jmx_prometheus_javaagent/jmx_prometheus_javaagent.jar=8008:/opt/pinot/etc/jmx_prometheus_javaagent/configs/pinot.yml"
     extra:
       configs: |-
         pinot.set.instance.id.to.hostname=true
         pinot.server.instance.realtime.alloc.offheap=true
         pinot.server.instance.currentDataTableVersion=2
         pinot.server.instance.dataDir=/var/pinot/server/data/index
         pinot.server.instance.segmentTarDir=/var/pinot/server/data/segment
         pinot.server.storage.factory.class.s3=org.apache.pinot.plugin.filesystem.S3PinotFS
         pinot.server.storage.factory.s3.region=us-east-1
         pinot.server.segment.fetcher.protocols=file,http,s3
         pinot.controller.storage.factory.s3.disableAcl=false
         pinot.server.segment.fetcher.s3.class=org.apache.pinot.common.utils.fetcher.PinotFSSegmentFetcher
         pinot.server.segment.fetcher.s3.retry.count=10
         pinot.server.instance.segment.store.uri=s3://<redacted>/controller-data
         pinot.server.instance.max.parallel.refresh.threads=10
         pinot.zk.server=zookeeper-offline.de-nrt-pinot.svc.cluster.local:2181
         pinot.server.swagger.use.https=true
   
   ```


-- 
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] walterddr commented on issue #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

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

   looks like the `pinot.server.instance.dataDir` extra config is conflicting with the definition in service's key `dataDir`. 
   
   according to [kubernetes/helm/pinot/templates/server/configmap.yaml](https://github.com/apache/pinot/blob/287534262fd9ccd8507fd06efaec2dd14a4d29cf/kubernetes/helm/pinot/templates/server/configmap.yaml) you can see it appends both:
   ```
       pinot.server.instance.dataDir={{ .Values.server.dataDir }}
   {{ .Values.server.extra.configs | indent 4 }}
   ```
   e.g. you will have 2 `pinot.server.instance.dataDir` key and I dont even know how that's possible in the first place. 
   
   thank you for provide the information. 
   - could you help dump the ConfigMap from your k8s cluster direclty, it would probably help understand what exactly did helm resolve this into
   - based on the result we can see either it is a config key resolution issue; or a default value resolution issue and then it should be easy to fix
   


-- 
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 #9393: delete segment(s) api for an offline table doesn't delete segments from server disc

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

   This is unexpected. After deleting the segments from the controller, can you check if you see the following INFO log on the servers:
   ```
   SegmentOnlineOfflineStateModel.onBecomeDroppedFromOffline()...
   ```
   And also check if there is any ERROR log on the server side


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