Jackie-Jiang commented on a change in pull request #6682: URL: https://github.com/apache/incubator-pinot/pull/6682#discussion_r594577905
########## File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/SegmentBuildTimeLeaseExtender.java ########## @@ -46,25 +46,25 @@ // Retransmit lease request 10% before lease expires. private static final int REPEAT_REQUEST_PERIOD_SEC = (EXTRA_TIME_SECONDS * 9 / 10); private static Logger LOGGER = LoggerFactory.getLogger(SegmentBuildTimeLeaseExtender.class); - private static final Map<String, SegmentBuildTimeLeaseExtender> INSTANCE_TO_LEASE_EXTENDER = new HashMap<>(1); + private static final Map<String, SegmentBuildTimeLeaseExtender> TABLE_TO_LEASE_EXTENDER = new HashMap<>(1); private ScheduledExecutorService _executor; private final Map<String, Future> _segmentToFutureMap = new ConcurrentHashMap<>(); private final String _instanceId; private final ServerSegmentCompletionProtocolHandler _protocolHandler; - public static SegmentBuildTimeLeaseExtender getLeaseExtender(final String instanceId) { - return INSTANCE_TO_LEASE_EXTENDER.get(instanceId); + public static SegmentBuildTimeLeaseExtender getLeaseExtender(final String tableNameWithType) { + return TABLE_TO_LEASE_EXTENDER.get(tableNameWithType); } public static synchronized SegmentBuildTimeLeaseExtender create(final String instanceId, ServerMetrics serverMetrics, String tableNameWithType) { - SegmentBuildTimeLeaseExtender leaseExtender = INSTANCE_TO_LEASE_EXTENDER.get(instanceId); + SegmentBuildTimeLeaseExtender leaseExtender = TABLE_TO_LEASE_EXTENDER.get(tableNameWithType); if (leaseExtender != null) { - LOGGER.warn("Instance already exists"); + LOGGER.warn("Lease extender for {} already exists", tableNameWithType); } else { leaseExtender = new SegmentBuildTimeLeaseExtender(instanceId, serverMetrics, tableNameWithType); Review comment: This will create one executor per table. To keep the same overhead of the lease extender, we might want to share the same executor across all tables. ########## File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/SegmentBuildTimeLeaseExtender.java ########## @@ -46,25 +46,25 @@ // Retransmit lease request 10% before lease expires. private static final int REPEAT_REQUEST_PERIOD_SEC = (EXTRA_TIME_SECONDS * 9 / 10); private static Logger LOGGER = LoggerFactory.getLogger(SegmentBuildTimeLeaseExtender.class); - private static final Map<String, SegmentBuildTimeLeaseExtender> INSTANCE_TO_LEASE_EXTENDER = new HashMap<>(1); + private static final Map<String, SegmentBuildTimeLeaseExtender> TABLE_TO_LEASE_EXTENDER = new HashMap<>(1); Review comment: Make this `ConcurrentHashMap` and remove the `synchronized` on line 60. `HashMap` is not thread-safe on line 57 ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org