mneedham opened a new issue #7940: URL: https://github.com/apache/pinot/issues/7940
I noticed that when a new segment in a real-time table is created it ignores changes that I make to `tableIndexConfig.streamConfigs.realtime.segment.flush.threshold.rows`. Even if I restart the Pinot Server it doesn't help, only restarting the Pinot Controller will have it pick up the updated threshold for new segments. I think the problem is that when the Controller creates the new entry in ZooKeeper, it uses a cached version of the TableConfig: https://github.com/apache/pinot/blob/master/pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/realtime/PinotLLCRealtimeSegmentManager.java#L520 ``` private void commitSegmentMetadataInternal(String realtimeTableName, CommittingSegmentDescriptor committingSegmentDescriptor) { String committingSegmentName = committingSegmentDescriptor.getSegmentName(); LLCSegmentName committingLLCSegment = new LLCSegmentName(committingSegmentName); int committingSegmentPartitionGroupId = committingLLCSegment.getPartitionGroupId(); LOGGER.info("Committing segment metadata for segment: {}", committingSegmentName); TableConfig tableConfig = getTableConfig(realtimeTableName); ``` ``` public TableConfig getTableConfig(String realtimeTableName) { try { return _tableConfigCache.getTableConfig(realtimeTableName); } catch (ExecutionException e) { _controllerMetrics.addMeteredTableValue(realtimeTableName, ControllerMeter.LLC_ZOOKEEPER_FETCH_FAILURES, 1L); throw new IllegalStateException( "Caught exception while loading table config from property store to cache for table: " + realtimeTableName, e); } } ```` The `PinotLLCRealtimeSegmentManager` is instantiated in the BaseControllerStarter - https://github.com/apache/pinot/blob/master/pinot-controller/src/main/java/org/apache/pinot/controller/BaseControllerStarter.java#L398 and I don't think the table config is refreshed when there's a change, only after the 60 minute expiry limit is reached. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
