You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/09/21 21:09:52 UTC

[GitHub] [kafka] jlprat opened a new pull request #11350: Scala3 migration

jlprat opened a new pull request #11350:
URL: https://github.com/apache/kafka/pull/11350


   This PR lays the groundwork for the Scala3 migration of the code base. Gradle's Scala3 plugin is merged but not released yet (see [PR](https://github.com/gradle/gradle/pull/18001)).
   During the migration I encountered 2 different bugs in Scala3, they are filed [here](https://github.com/lampepfl/dotty/issues/13549) and [here](https://github.com/lampepfl/dotty/issues/13572).
   
   # Types of changes done in this PR:
   * Shadowing variables or packages are now errors instead of Scala doing guess work. A fair amount of the changes done in this PR was disambiguation. These include:
     * Parameter shadowing
     * Shadowing among imports
   * Parameterless methods called with parameters is not possible any more in Scala3
   * `SockerServer` extra classes were split into different files as Scala3 compiler was failing to find them when referenced in other classes in this same file.
   * `foreach` java collections method was not directly usable under Scala3, so I transformed it to the Scala collection's one (this might be another bug in Scala3, I need to investigate)
   * Scala3 fails to determine the right overload method when SAM is involved if types don't match exactly (a.k.a. bug https://github.com/lampepfl/dotty/issues/13549)
   * Scala3 companion object of a trait doesn't have the static forwarder methods, needed to reference the companion object "old style" (a.k.a. bug https://github.com/lampepfl/dotty/issues/13572)
   * Extra manual typing was needed occasionally as Scala3 is probably stricter than Scala 2.13
   
   # Compiling with Scala3
   In order to test this locally one can run the following:
   
   `./gradlew wrapper --gradle-distribution-url=https://services.gradle.org/distributions-snapshots/gradle-7.3-20210906222431+0000-bin.zip`
   And then the usual
   `./gradlew compileTestScala -PscalaVersion=3.0`
   
   # Notes
   Jackson is using "2.13.0-rc2" version as it's the one that contains Scala3 improvements, it's not really needed to successfully compile though.
   
   Extra information, Scala3 is compiling in "Migration Mode", meaning it outputs some warnings about deprecated and dropped features. See [Migration Mode](https://docs.scala-lang.org/scala3/guides/migration/tooling-migration-mode.html) for further info.
   All these warnings can be automatically fixed by the Scala compiler itself.
   
   # Current Problems
   Spotbugs is currently detecting 30 problems with Scala3, it works fine when compiling with Scala 2.13. This currently blocks the execution of core and streams tests. By excluding `spotbugs` tests can be run and some tests are still failing, I need to find out why is this. Tests run successfully in Scala 2.13. To exclude `spotbugs` run the following:
   `./gradlew test -x spotbugsMain -PscalaVersion=3.0`
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on pull request #11350: Scala3 migration

Posted by GitBox <gi...@apache.org>.
jlprat commented on pull request #11350:
URL: https://github.com/apache/kafka/pull/11350#issuecomment-924613583


   One of tus failures was https://issues.apache.org/jira/browse/KAFKA-8785
   I need to check the other one


-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on pull request #11350: Scala3 migration

Posted by GitBox <gi...@apache.org>.
jlprat commented on pull request #11350:
URL: https://github.com/apache/kafka/pull/11350#issuecomment-924387703


   Ping @ijuma I was able to have some progress with migrating to Scala3. I still need to check why some test failures occur only on Scala3. Let me know what you think about the changes.


-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on a change in pull request #11350: Scala3 migration

Posted by GitBox <gi...@apache.org>.
jlprat commented on a change in pull request #11350:
URL: https://github.com/apache/kafka/pull/11350#discussion_r713423728



##########
File path: core/src/main/scala/kafka/controller/PartitionStateMachine.scala
##########
@@ -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

Review comment:
       Changes like this are the parenthesis-less methods that should be called without parenthesis

##########
File path: core/src/test/java/kafka/testkit/KafkaClusterTestKit.java
##########
@@ -234,7 +234,7 @@ public KafkaClusterTestKit build() throws Exception {
                         Option.apply(threadNamePrefix),
                         JavaConverters.asScalaBuffer(Collections.<String>emptyList()).toSeq(),
                         connectFutureManager.future,
-                        Server.SUPPORTED_FEATURES()
+                        Server$.MODULE$.SUPPORTED_FEATURES()

Review comment:
       Changes like this are related to https://github.com/lampepfl/dotty/issues/13572

##########
File path: core/src/main/scala/kafka/admin/TopicCommand.scala
##########
@@ -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 => _, _}

Review comment:
       Changes like this are due to shadowing between `kafka.utils.immutable` and the `immutable` package in `scala.collections`.

##########
File path: core/src/main/scala/kafka/tools/ConsoleConsumer.scala
##########
@@ -27,7 +27,7 @@ import com.typesafe.scalalogging.LazyLogging
 import joptsimple._
 import kafka.utils.Implicits._
 import kafka.utils.{Exit, _}
-import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerRecord, KafkaConsumer}
+import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig => ClientConsumerConfig, ConsumerRecord, KafkaConsumer}

Review comment:
       This is done to avoid shadowing

##########
File path: core/src/main/scala/kafka/network/SocketServer.scala
##########
@@ -467,1251 +451,3 @@ object SocketServer {
 
   val ListenerReconfigurableConfigs = Set(KafkaConfig.MaxConnectionsProp, KafkaConfig.MaxConnectionCreationRateProp)
 }
-

Review comment:
       This is what I mentioned about splitting classes present in `SocketServer` into their own file

##########
File path: core/src/main/scala/kafka/raft/KafkaMetadataLog.scala
##########
@@ -42,11 +42,11 @@ final class KafkaMetadataLog private (
   // Access to this object needs to be synchronized because it is used by the snapshotting thread to notify the
   // polling thread when snapshots are created. This object is also used to store any opened snapshot reader.
   snapshots: mutable.TreeMap[OffsetAndEpoch, Option[FileRawSnapshotReader]],
-  topicPartition: TopicPartition,
+  topicPartitionArg: TopicPartition,

Review comment:
       Changes like this are to avoid the shadowing between the parameter and the method

##########
File path: core/src/main/scala/kafka/log/LazyIndex.scala
##########
@@ -21,7 +21,7 @@ import java.io.File
 import java.nio.file.{Files, NoSuchFileException}
 import java.util.concurrent.locks.ReentrantLock
 
-import LazyIndex._
+

Review comment:
       This import was causing a cyclic problem in Scala3.

##########
File path: core/src/main/scala/kafka/utils/json/DecodeJson.scala
##########
@@ -85,13 +85,13 @@ object DecodeJson {
     else decodeJson.decodeEither(node).map(Some(_))
   }
 
-  implicit def decodeSeq[E, S[+T] <: Seq[E]](implicit decodeJson: DecodeJson[E], factory: Factory[E, S[E]]): DecodeJson[S[E]] = (node: JsonNode) => {
+  implicit def decodeSeq[E, S[E] <: Seq[E]](implicit decodeJson: DecodeJson[E], factory: Factory[E, S[E]]): DecodeJson[S[E]] = (node: JsonNode) => {

Review comment:
       Scala3 compiler is more strict with type definitions and the previous one wasn't really being satisfied

##########
File path: core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
##########
@@ -2244,7 +2244,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
   def testDescribeClusterClusterAuthorizedOperationsWithoutDescribeCluster(): Unit = {
     removeAllClientAcls()
 
-    for (version <- ApiKeys.DESCRIBE_CLUSTER.oldestVersion to ApiKeys.DESCRIBE_CLUSTER.latestVersion) {
+    for (version <- ApiKeys.DESCRIBE_CLUSTER.oldestVersion.toInt to ApiKeys.DESCRIBE_CLUSTER.latestVersion.toInt) {

Review comment:
       The `to` method is not present in `Short` type and Scala3 doesn't widen the type automatically

##########
File path: core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
##########
@@ -481,8 +481,10 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
       // try a newCount which would be a decrease
       alterResult = client.createPartitions(Map(topic1 ->
         NewPartitions.increaseTo(1)).asJava, option)
-      
-      var e = assertThrows(classOf[ExecutionException], () => alterResult.values.get(topic1).get,
+      var e = assertThrows(classOf[ExecutionException], () => {
+        alterResult.values.get(topic1).get
+        ()
+      },

Review comment:
       This change is because of https://github.com/lampepfl/dotty/issues/13549




-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on a change in pull request #11350: Scala3 migration

Posted by GitBox <gi...@apache.org>.
jlprat commented on a change in pull request #11350:
URL: https://github.com/apache/kafka/pull/11350#discussion_r724963365



##########
File path: core/src/main/java/kafka/server/builders/LogManagerBuilder.java
##########
@@ -45,7 +46,7 @@
     private long flushStartOffsetCheckpointMs = 10000L;
     private long retentionCheckMs = 1000L;
     private int maxPidExpirationMs = 60000;
-    private ApiVersion interBrokerProtocolVersion = ApiVersion.latestVersion();
+    private ApiVersion interBrokerProtocolVersion = ApiVersion$.MODULE$.latestVersion();

Review comment:
       This has been solved in upcoming Scala versions, Bug fix here: https://github.com/lampepfl/dotty/issues/13572

##########
File path: core/src/test/scala/unit/kafka/server/FinalizedFeatureChangeListenerTest.scala
##########
@@ -239,10 +239,11 @@ class FinalizedFeatureChangeListenerTest extends ZooKeeperTestHarness {
 
     val exitLatch = new CountDownLatch(1)
     Exit.setExitProcedure((_, _) => exitLatch.countDown())
+    val feature1: SupportedVersionRange = brokerFeatures.supportedFeatures.get("feature_1")

Review comment:
       This change is needed because Scala 3 seems to prioritize the highest type in the hierarchy when inferring and the one inferred had package protected visibility. Setting the right type manually solves the problem.

##########
File path: core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala
##########
@@ -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)

Review comment:
       This type of changes are due to this bug: https://github.com/lampepfl/dotty/issues/13630

##########
File path: gradle/spotbugs-exclude.xml
##########
@@ -289,6 +297,63 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read
         <Bug pattern="RV_RETURN_VALUE_IGNORED,RV_RETURN_VALUE_IGNORED_BAD_PRACTICE"/>
     </Match>
 
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.server"/>
+        <Source name="AbstractFetcherThread.scala"/>
+        <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE"/>
+    </Match>
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.log"/>
+        <Source name="LogLoader.scala"/>
+        <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE"/>
+    </Match>
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.tools"/>
+        <Source name="ConsoleConsumer.scala"/>
+        <Bug pattern="DB_DUPLICATE_BRANCHES"/>
+    </Match>

Review comment:
       This seem to me like a false positive when compiling in Scala 3
   

##########
File path: gradle/spotbugs-exclude.xml
##########
@@ -289,6 +297,63 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read
         <Bug pattern="RV_RETURN_VALUE_IGNORED,RV_RETURN_VALUE_IGNORED_BAD_PRACTICE"/>
     </Match>
 
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.server"/>
+        <Source name="AbstractFetcherThread.scala"/>
+        <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE"/>
+    </Match>
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.log"/>
+        <Source name="LogLoader.scala"/>
+        <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE"/>
+    </Match>

Review comment:
       These 2 changes seem to me like a false positive when compiling in Scala 3

##########
File path: core/src/test/scala/unit/kafka/server/IsrExpirationTest.scala
##########
@@ -59,7 +59,7 @@ class IsrExpirationTest {
   @BeforeEach
   def setUp(): Unit = {
     val logManager: LogManager = EasyMock.createMock(classOf[LogManager])
-    EasyMock.expect(logManager.liveLogDirs).andReturn(Array.empty[File]).anyTimes()
+    EasyMock.expect(logManager.liveLogDirs).andReturn(Seq.empty[File]).anyTimes()

Review comment:
       Some of the implicit conversions in the past are not present any more. So we need to set the types more precisely.

##########
File path: core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala
##########
@@ -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())

Review comment:
       Some times Scala 3 can't automatically widen Ints to Longs

##########
File path: gradle/spotbugs-exclude.xml
##########
@@ -90,9 +90,17 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read
             <Bug pattern="MS_CANNOT_BE_FINAL"/>
             <Bug pattern="IC_INIT_CIRCULARITY"/>
             <Bug pattern="SE_NO_SUITABLE_CONSTRUCTOR"/>
+            <!-- This happens in Scala 3 when concatenating an Array and a String -->
+            <Bug pattern="DMI_INVOKING_TOSTRING_ON_ARRAY"/>
         </Or>
     </Match>
 
+    <!-- false positive in Scala 3. Syntactic fields are created in some cases with the suffix `$lz1` and do not conform with starting with lower case -->
+    <Match>
+        <Field name="~.*\$lzy1"/>
+        <Bug pattern="NM_FIELD_NAMING_CONVENTION"/>
+    </Match>

Review comment:
       This exception is due to the new naming convention for lazy fields in Scala 3. This is another false positive

##########
File path: gradle/spotbugs-exclude.xml
##########
@@ -289,6 +297,63 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read
         <Bug pattern="RV_RETURN_VALUE_IGNORED,RV_RETURN_VALUE_IGNORED_BAD_PRACTICE"/>
     </Match>
 
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.server"/>
+        <Source name="AbstractFetcherThread.scala"/>
+        <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE"/>
+    </Match>
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.log"/>
+        <Source name="LogLoader.scala"/>
+        <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE"/>
+    </Match>
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.tools"/>
+        <Source name="ConsoleConsumer.scala"/>
+        <Bug pattern="DB_DUPLICATE_BRANCHES"/>
+    </Match>
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.coordinator.transaction"/>
+        <Source name="ProducerIdManager.scala"/>
+        <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+    </Match>
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.network"/>
+        <Source name="Acceptor.scala"/>
+        <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+    </Match>
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.utils"/>
+        <Source name="KafkaScheduler.scala"/>
+        <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+    </Match>
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.network"/>
+        <Source name="SocketServer.scala"/>
+        <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+    </Match>

Review comment:
       This might be a bug in bytecode generation in regards of synchronized blocks. Need to reproduce this and send a bug report.

##########
File path: gradle/spotbugs-exclude.xml
##########
@@ -289,6 +297,63 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read
         <Bug pattern="RV_RETURN_VALUE_IGNORED,RV_RETURN_VALUE_IGNORED_BAD_PRACTICE"/>
     </Match>
 
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.server"/>
+        <Source name="AbstractFetcherThread.scala"/>
+        <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE"/>
+    </Match>
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.log"/>
+        <Source name="LogLoader.scala"/>
+        <Bug pattern="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE"/>
+    </Match>
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.tools"/>
+        <Source name="ConsoleConsumer.scala"/>
+        <Bug pattern="DB_DUPLICATE_BRANCHES"/>
+    </Match>
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.coordinator.transaction"/>
+        <Source name="ProducerIdManager.scala"/>
+        <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+    </Match>
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.network"/>
+        <Source name="Acceptor.scala"/>
+        <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+    </Match>
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.utils"/>
+        <Source name="KafkaScheduler.scala"/>
+        <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+    </Match>
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.network"/>
+        <Source name="SocketServer.scala"/>
+        <Bug pattern="IS2_INCONSISTENT_SYNC"/>
+    </Match>
+
+    <Match>
+        <!-- This warning only appears during Scala 3 -->
+        <Package name="kafka.network"/>
+        <Source name="ConnectionQuotas.scala"/>
+        <Bug pattern="VO_VOLATILE_INCREMENT"/>
+    </Match>

Review comment:
       This might be a bug in bytecode generation in regards of volatile variables. Need to reproduce this and send a bug report.

##########
File path: core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala
##########
@@ -111,7 +111,7 @@ abstract class AbstractCreateTopicsRequestTest extends BaseRequestTest {
         val replication = if (!topic.assignments().isEmpty)
           topic.assignments().iterator().next().brokerIds().size()
         else
-          topic.replicationFactor
+          topic.replicationFactor.toInt

Review comment:
       Automatic widening and narrowing doesn't happen as often as in Scala 2, o we need to be more explicit about types.

##########
File path: core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
##########
@@ -2374,7 +2374,7 @@ class GroupMetadataManagerTest {
       minOneMessage = EasyMock.eq(true)))
       .andReturn(FetchDataInfo(LogOffsetMetadata(startOffset), mockRecords))
     EasyMock.expect(replicaManager.getLog(groupMetadataTopicPartition)).andStubReturn(Some(logMock))
-    EasyMock.expect(replicaManager.getLogEndOffset(groupMetadataTopicPartition)).andStubReturn(Some(18))
+    EasyMock.expect(replicaManager.getLogEndOffset(groupMetadataTopicPartition)).andStubReturn(Some(18L))

Review comment:
       This is some strange discrepancy in Scala 3 compiler, need to find the cause and report it. Even though the impact on the code base is non-existent

##########
File path: core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala
##########
@@ -37,14 +37,17 @@ class ProducerIdManagerTest {
   val zkClient: KafkaZkClient = EasyMock.createNiceMock(classOf[KafkaZkClient])
 
   // Mutable test implementation that lets us easily set the idStart and error
-  class MockProducerIdManager(val brokerId: Int, var idStart: Long, val idLen: Int, var error: Errors = Errors.NONE)
+  class MockProducerIdManager(val brokerId: Int, val idStart: Long, val idLen: Int)
     extends RPCProducerIdManager(brokerId, () => 1, brokerToController, 100) {
 
+    var startId = idStart
+    var error: Errors = Errors.NONE
+
     override private[transaction] def sendRequest(): Unit = {
       if (error == Errors.NONE) {
         handleAllocateProducerIdsResponse(new AllocateProducerIdsResponse(
-          new AllocateProducerIdsResponseData().setProducerIdStart(idStart).setProducerIdLen(idLen)))
-        idStart += idLen
+          new AllocateProducerIdsResponseData().setProducerIdStart(startId).setProducerIdLen(idLen)))
+        startId += idLen

Review comment:
       These changes are due to this bug that is solved in upcoming Scala versions: https://github.com/lampepfl/dotty/issues/13630




-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on pull request #11350: Scala3 migration

Posted by GitBox <gi...@apache.org>.
jlprat commented on pull request #11350:
URL: https://github.com/apache/kafka/pull/11350#issuecomment-938614404


   Current status of the PR:
   - All tests pass in Scala 3 except one (`KafkaApis#getAllTopicMetadataShouldNotCreateTopicOrReturnUnknownTopicPartition`) due to this bug with no workaround: https://github.com/lampepfl/dotty/issues/13638
   - All tests pass in Scala 2
   - It still uses a snapshot build of Gradle
   
   In this PR there are several types of changes:
   a) Needed changes because of changes in syntax or compiler being currently more strict
   b) Changes where a workaround is present but they are fixed in upcoming Scala versions
   c) Changes in the build
   
   I propose perform the Scala 3 migration in several steps:
   - Firstly apply all changes that would fall into category *a)*
   - Once a new Scala version that incorporates the mentioned fixes is released, revisit point changes in point *b)*
   - Lastly, once https://github.com/lampepfl/dotty/issues/13638 is resolved or a workaround is found and Gradle with Scala 3 support is released we could tackle the final step and incorporate changes mentioned in point *c)*
   
   
   What are your thoughts? @ijuma 
   


-- 
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: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org