KAFKA-14607: Move Scheduler/KafkaScheduler to server-common#13092
KAFKA-14607: Move Scheduler/KafkaScheduler to server-common#13092ijuma merged 8 commits intoapache:trunkfrom
Conversation
| delay = delay, unit = unit) | ||
| kafkaScheduler.scheduleOnce("auto-leader-rebalance-task", | ||
| () => eventManager.put(AutoPreferredReplicaLeaderElection), | ||
| unit.toMillis(delay)) |
There was a problem hiding this comment.
This is the only case where we were calling schedule with a unit that is not `ms.
| // stop token expiry check scheduler | ||
| if (tokenCleanScheduler.isStarted) | ||
| tokenCleanScheduler.shutdown() | ||
| tokenCleanScheduler.shutdown() |
There was a problem hiding this comment.
It's safe to call shutdown even if isStarted was not called.
| shuttingDown.set(true) | ||
| if (scheduler.isStarted) | ||
| scheduler.shutdown() | ||
| scheduler.shutdown() |
There was a problem hiding this comment.
It's redundant to check if the scheduler has started before calling shutdown.
| nextDelayMs) | ||
| } catch { | ||
| case e: Throwable => | ||
| if (scheduler.isStarted) { |
There was a problem hiding this comment.
Need to double check if this change makes sense.
There was a problem hiding this comment.
@junrao What's your thought regarding this? This check seems a bit odd. If the scheduler is not started, we return NoOpScheduledFutureTask.
There was a problem hiding this comment.
Thanks for catching this. #11351 changed the scheduler to return NoOpScheduledFutureTask if the scheduler is not started. But we forgot to change the code here accordingly.
| initialOfflineDirs: Seq[String], | ||
| configRepository: ConfigRepository, | ||
| kafkaScheduler: KafkaScheduler, | ||
| kafkaScheduler: Scheduler, |
There was a problem hiding this comment.
Using the interface instead of the concrete class.
| def dataPlaneRequestHandlerPool: KafkaRequestHandlerPool | ||
| def dataPlaneRequestProcessor: KafkaApis | ||
| def kafkaScheduler: KafkaScheduler | ||
| def kafkaScheduler: Scheduler |
There was a problem hiding this comment.
We should use the interface instead of the concrete class.
|
|
||
| <Match> | ||
| <!-- Uncallable anonymous methods are left behind after inlining by scalac 2.12, fixed in 2.13 --> | ||
| <Source name="LogConfig.scala"/> |
There was a problem hiding this comment.
This is no longer needed since LogConfig.scala no longer exists.
| return scheduleOnce(name, task, 0L); | ||
| } | ||
|
|
||
| default ScheduledFuture<?> scheduleOnce(String name, Runnable task, long delayMs) { |
There was a problem hiding this comment.
In Java we have to use multiple methods (potentially with overloads) instead of default arguments. For clarity, we use scheduleOnce for the case where the task is executed only once (periodMs < 0).
7cbf398 to
f0a18cd
Compare
| testImplementation project(':clients').sourceSets.test.output | ||
| testImplementation project(':core') | ||
| testImplementation project(':core').sourceSets.test.output | ||
| testImplementation project(':server-common').sourceSets.test.output |
There was a problem hiding this comment.
The streams integration harness has a transitive runtime dependency to Scheduler/KafkaScheduler and dependencies to test jars are not transitive.
| expiredSnapshots: mutable.TreeMap[OffsetAndEpoch, Option[FileRawSnapshotReader]], | ||
| logging: Logging | ||
| ): () => Unit = () => { | ||
| ): Unit = { |
There was a problem hiding this comment.
This made the code less clear, it's better to have the clarity that a lambda is being passed.
| final long period; | ||
| final Time time; | ||
|
|
||
| private final AtomicLong nextExecution; |
There was a problem hiding this comment.
Made this an atomic instead of the inconsistent synchronization that existed before.
| /** | ||
| * If this task is periodic, reschedule it and return true. Otherwise, do nothing and return false. | ||
| */ | ||
| public boolean rescheduleIfPeriodic() { |
There was a problem hiding this comment.
Extracted this logic into its own method.
|
All builds passed. @junrao this is ready for review when you have cycles. |
server-common/src/main/java/org/apache/kafka/server/util/KafkaScheduler.java
Show resolved
Hide resolved
| nextDelayMs) | ||
| } catch { | ||
| case e: Throwable => | ||
| if (scheduler.isStarted) { |
There was a problem hiding this comment.
Thanks for catching this. #11351 changed the scheduler to return NoOpScheduledFutureTask if the scheduler is not started. But we forgot to change the code here accordingly.
|
JDK 17 build passed and the other two had unrelated failures:
|
* apache-github/trunk: KAFKA-14601: Improve exception handling in KafkaEventQueue apache#13089 KAFKA-14367; Add `OffsetCommit` to the new `GroupCoordinator` interface (apache#12886) KAFKA-14530: Check state updater more often (apache#13017) KAFKA-14304 Use boolean for ZK migrating brokers in RPC/record (apache#13103) KAFKA-14003 Kafka Streams JUnit4 to JUnit5 migration part 2 (apache#12301) KAFKA-14607: Move Scheduler/KafkaScheduler to server-common (apache#13092) KAFKA-14367; Add `OffsetFetch` to the new `GroupCoordinator` interface (apache#12870) KAFKA-14557; Lock metadata log dir (apache#13058) MINOR: Implement toString method for TopicAssignment and PartitionAssignment (apache#13101) KAFKA-12558: Do not prematurely mutate internal partition state in Mirror Maker 2 (apache#11818) KAFKA-14540: Fix DataOutputStreamWritable#writeByteBuffer (apache#13032) KAFKA-14600: Reduce flakiness in ProducerIdExpirationTest (apache#13087) KAFKA-14279: Add 3.3.x streams system tests (apache#13077) MINOR: bump streams quickstart pom versions and add to list in gradle.properties (apache#13064) MINOR: Update KRaft cluster upgrade documentation for 3.4 (apache#13063) KAFKA-14493: Introduce Zk to KRaft migration state machine STUBs in KRaft controller. (apache#12998) KAFKA-14570: Fix parenthesis in verifyFullFetchResponsePartitions output (apache#13072) MINOR: Remove public mutable fields from ProducerAppendInfo (apache#13091)
…master * apache-github/trunk: (23 commits) MINOR: Include the inner exception stack trace when re-throwing an exception (apache#12229) MINOR: Fix docs to state that sendfile implemented in `TransferableRecords` instead of `MessageSet` (apache#13109) Update ProducerConfig.java (apache#13115) KAFKA-14618; Fix off by one error in snapshot id (apache#13108) KAFKA-13709 (follow-up): Avoid mention of 'exactly-once delivery' or 'delivery guarantees' in Connect (apache#13106) KAFKA-14367; Add `TxnOffsetCommit` to the new `GroupCoordinator` interface (apache#12901) KAFKA-14568: Move FetchDataInfo and related to storage module (apache#13085) KAFKA-14612: Make sure to write a new topics ConfigRecords to metadata log iff the topic is created (apache#13104) KAFKA-14601: Improve exception handling in KafkaEventQueue apache#13089 KAFKA-14367; Add `OffsetCommit` to the new `GroupCoordinator` interface (apache#12886) KAFKA-14530: Check state updater more often (apache#13017) KAFKA-14304 Use boolean for ZK migrating brokers in RPC/record (apache#13103) KAFKA-14003 Kafka Streams JUnit4 to JUnit5 migration part 2 (apache#12301) KAFKA-14607: Move Scheduler/KafkaScheduler to server-common (apache#13092) KAFKA-14367; Add `OffsetFetch` to the new `GroupCoordinator` interface (apache#12870) KAFKA-14557; Lock metadata log dir (apache#13058) MINOR: Implement toString method for TopicAssignment and PartitionAssignment (apache#13101) KAFKA-12558: Do not prematurely mutate internal partition state in Mirror Maker 2 (apache#11818) KAFKA-14540: Fix DataOutputStreamWritable#writeByteBuffer (apache#13032) KAFKA-14600: Reduce flakiness in ProducerIdExpirationTest (apache#13087) ...
…3092) There were some concurrency inconsistencies in `KafkaScheduler` flagged by spotBugs that had to be fixed, summary of changes below: * Executor is `volatile` * We always synchronize and check `isStarted` as the first thing within the critical section when a mutating operation is performed. * We don't synchronize (but ensure the executor is not null in a safe way) in read-only operations that operate on the executor. With regards to `MockScheduler/MockTask`: * Set the type of `nextExecution` to `AtomicLong` and replaced inconsistent synchronization * Extracted logic into `MockTask.rescheduleIfPeriodic` Tweaked the `Scheduler` interface a bit: * Removed `unit` parameter since we always used `ms` except one invocation * Introduced a couple of `scheduleOnce` overloads to replace the usage of default arguments in Scala * Pulled up `resizeThreadPool` to the interface and removed `isStarted` from the interface. Other cleanups: * Removed spotBugs exclusion affecting `kafka.log.LogConfig`, which no longer exists. For broader context, see: * KAFKA-14470: Move log layer to storage module Reviewers: Jun Rao <junrao@gmail.com>
There were some concurrency inconsistencies in
KafkaSchedulerflagged by spotBugsthat had to be fixed, summary of changes below:
volatileisStartedas the first thing within the criticalsection when a mutating operation is performed.
operations that operate on the executor.
With regards to
MockScheduler/MockTask:nextExecutiontoAtomicLongand replaced inconsistent synchronizationMockTask.rescheduleIfPeriodicTweaked the
Schedulerinterface a bit:unitparameter since we always usedmsexcept one invocationscheduleOnceoverloads to replace the usage of defaultarguments in Scala
resizeThreadPoolto the interface and removedisStartedfrom theinterface.
Other cleanups:
kafka.log.LogConfig, which no longer exists.For broader context, see:
Committer Checklist (excluded from commit message)