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/10/08 12:35:18 UTC

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

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