-
Notifications
You must be signed in to change notification settings - Fork 14.4k
Scala3 migration #11350
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Scala3 migration #11350
Changes from all commits
3533fcc
72b438f
2873540
ae519e2
12b45de
731b527
7b42121
0c43f3e
8aa0690
53cb88b
2cf0742
ccacb53
6d8dc74
c0ed254
4331f9c
a2a000c
5b45882
53cd92c
06006f5
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -1036,8 +1036,8 @@ object ConsumerGroupCommand extends Logging { | |
val timeoutMsOpt = parser.accepts("timeout", TimeoutMsDoc) | ||
.withRequiredArg | ||
.describedAs("timeout (ms)") | ||
.ofType(classOf[Long]) | ||
.defaultsTo(5000) | ||
.ofType(classOf[java.lang.Long]) | ||
.defaultsTo(5000L) | ||
Comment on lines
+1039
to
+1040
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This type of changes are due to this bug: scala/scala3#13630 |
||
val commandConfigOpt = parser.accepts("command-config", CommandConfigDoc) | ||
.withRequiredArg | ||
.describedAs("command config property file") | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -238,14 +238,14 @@ object ReassignPartitionsCommand extends Logging { | |
executeAssignment(adminClient, | ||
opts.options.has(opts.additionalOpt), | ||
Utils.readFileAsString(opts.options.valueOf(opts.reassignmentJsonFileOpt)), | ||
opts.options.valueOf(opts.interBrokerThrottleOpt), | ||
opts.options.valueOf(opts.replicaAlterLogDirsThrottleOpt), | ||
opts.options.valueOf(opts.timeoutOpt)) | ||
opts.options.valueOf(opts.interBrokerThrottleOpt).longValue(), | ||
opts.options.valueOf(opts.replicaAlterLogDirsThrottleOpt).longValue(), | ||
opts.options.valueOf(opts.timeoutOpt).longValue()) | ||
Comment on lines
+241
to
+243
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Some times Scala 3 can't automatically widen Ints to Longs |
||
} else if (opts.options.has(opts.cancelOpt)) { | ||
cancelAssignment(adminClient, | ||
Utils.readFileAsString(opts.options.valueOf(opts.reassignmentJsonFileOpt)), | ||
opts.options.has(opts.preserveThrottlesOpt), | ||
opts.options.valueOf(opts.timeoutOpt)) | ||
opts.options.valueOf(opts.timeoutOpt).longValue()) | ||
} else if (opts.options.has(opts.listOpt)) { | ||
listReassignments(adminClient) | ||
} else { | ||
|
@@ -1473,25 +1473,25 @@ object ReassignPartitionsCommand extends Logging { | |
.ofType(classOf[String]) | ||
val disableRackAware = parser.accepts("disable-rack-aware", "Disable rack aware replica assignment") | ||
val interBrokerThrottleOpt = parser.accepts("throttle", "The movement of partitions between brokers will be throttled to this value (bytes/sec). " + | ||
"This option can be included with --execute when a reassignment is started, and it can be altered by resubmitting the current reassignment " + | ||
"along with the --additional flag. The throttle rate should be at least 1 KB/s.") | ||
.withRequiredArg() | ||
.describedAs("throttle") | ||
.ofType(classOf[Long]) | ||
.defaultsTo(-1) | ||
"This option can be included with --execute when a reassignment is started, and it can be altered by resubmitting the current reassignment " + | ||
"along with the --additional flag. The throttle rate should be at least 1 KB/s.") | ||
.withRequiredArg() | ||
.describedAs("throttle") | ||
.ofType(classOf[java.lang.Long]) | ||
.defaultsTo(-1L) | ||
val replicaAlterLogDirsThrottleOpt = parser.accepts("replica-alter-log-dirs-throttle", | ||
"The movement of replicas between log directories on the same broker will be throttled to this value (bytes/sec). " + | ||
"This option can be included with --execute when a reassignment is started, and it can be altered by resubmitting the current reassignment " + | ||
"along with the --additional flag. The throttle rate should be at least 1 KB/s.") | ||
.withRequiredArg() | ||
.describedAs("replicaAlterLogDirsThrottle") | ||
.ofType(classOf[Long]) | ||
.defaultsTo(-1) | ||
"The movement of replicas between log directories on the same broker will be throttled to this value (bytes/sec). " + | ||
"This option can be included with --execute when a reassignment is started, and it can be altered by resubmitting the current reassignment " + | ||
"along with the --additional flag. The throttle rate should be at least 1 KB/s.") | ||
.withRequiredArg() | ||
.describedAs("replicaAlterLogDirsThrottle") | ||
.ofType(classOf[java.lang.Long]) | ||
.defaultsTo(-1L) | ||
val timeoutOpt = parser.accepts("timeout", "The maximum time in ms to wait for log directory replica assignment to begin.") | ||
.withRequiredArg() | ||
.describedAs("timeout") | ||
.ofType(classOf[Long]) | ||
.defaultsTo(10000) | ||
.ofType(classOf[java.lang.Long]) | ||
.defaultsTo(10000L) | ||
val additionalOpt = parser.accepts("additional", "Execute this reassignment in addition to any " + | ||
"other ongoing ones. This option can also be used to change the throttle of an ongoing reassignment.") | ||
val preserveThrottlesOpt = parser.accepts("preserve-throttles", "Do not modify broker or topic throttles.") | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -22,7 +22,7 @@ import java.util.{Collections, Properties} | |
import joptsimple._ | ||
import kafka.common.AdminCommandFailedException | ||
import kafka.log.LogConfig | ||
import kafka.utils._ | ||
import kafka.utils.{immutable => _, _} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Changes like this are due to shadowing between |
||
import org.apache.kafka.clients.CommonClientConfigs | ||
import org.apache.kafka.clients.admin.CreatePartitionsOptions | ||
import org.apache.kafka.clients.admin.CreateTopicsOptions | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -477,7 +477,7 @@ class ZkPartitionStateMachine(config: KafkaConfig, | |
} else { | ||
val (logConfigs, failed) = zkClient.getLogConfigs( | ||
partitionsWithNoLiveInSyncReplicas.iterator.map { case (partition, _) => partition.topic }.toSet, | ||
config.originals() | ||
config.originals | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Changes like this are the parenthesis-less methods that should be called without parenthesis |
||
) | ||
|
||
partitionsWithNoLiveInSyncReplicas.map { case (partition, leaderAndIsr) => | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -21,7 +21,7 @@ import java.io.File | |
import java.nio.file.{Files, NoSuchFileException} | ||
import java.util.concurrent.locks.ReentrantLock | ||
|
||
import LazyIndex._ | ||
|
||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This import was causing a cyclic problem in Scala3. |
||
import kafka.utils.CoreUtils.inLock | ||
import kafka.utils.threadsafe | ||
import org.apache.kafka.common.utils.Utils | ||
|
@@ -46,8 +46,8 @@ import org.apache.kafka.common.utils.Utils | |
* `AbstractIndex` instance. | ||
*/ | ||
@threadsafe | ||
class LazyIndex[T <: AbstractIndex] private (@volatile private var indexWrapper: IndexWrapper, loadIndex: File => T) { | ||
|
||
class LazyIndex[T <: AbstractIndex] private (@volatile private var indexWrapper: LazyIndex.IndexWrapper, loadIndex: File => T) { | ||
import LazyIndex._ | ||
private val lock = new ReentrantLock() | ||
|
||
def file: File = indexWrapper.file | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This has been solved in upcoming Scala versions, Bug fix here: scala/scala3#13572