Skip to content
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

KAFKA-9324: Drop support for Scala 2.11 (KIP-531) #7859

Merged
merged 12 commits into from
Jan 6, 2020

Conversation

ijuma
Copy link
Contributor

@ijuma ijuma commented Dec 22, 2019

  • Adjust build and documentation.
  • Use lambda syntax for SAM types in core, streams-scala and
    connect-runtime modules.
  • Remove runnable and newThread from CoreUtils as lambda
    syntax for SAM types make them unnecessary.
  • Remove stale comment in FunctionsCompatConversions,
    KGroupedStream, KGroupedTable' and KStream` about Scala 2.11,
    the conversions are needed for Scala 2.12 too.
  • Deprecate org.apache.kafka.streams.scala.kstream.Suppressed
    and use org.apache.kafka.streams.kstream.Suppressed instead.
  • Use Admin.create instead of AdminClient.create. Static methods
    in Java interfaces can be invoked since Scala 2.12. I noticed that
    MirrorMaker 2 uses AdminClient.create, but I did not change them
    as Connectors have restrictions on newer client APIs.
  • Improve efficiency in a few Gauge implementations by avoiding
    unnecessary intermediate collections.
  • Remove pointless Option.apply in ZookeeperClient
    SessionState metric.
  • Fix unused import/variable and other compiler warnings.
  • Reduce visibility of some vals/defs.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@gwenshap
Copy link
Contributor

Wow. Thanks for the amazing refactoring. Out of curiosity:

  • The change of SAM types to lambdas was done in Java classes. How is it related to the drop of 2.11 support? Did 2.11 prevent us from using lambda syntax somehow?
  • How did you find all the SAM types to replace? Is there a linting tool?

}
})
newGauge("cleaner-recopy-percent", () => {
val stats = cleaners.map(_.lastStats)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This will still create an intermediate collection, no? Why not use cleaners.iterator.map, turn stats into an iterator... and avoid creating the extra collection completely?

Copy link
Contributor Author

@ijuma ijuma Dec 23, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, good catch. I missed that in the initial change. Will update.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I went to make this change and then I remembered why I didn't do it in the first place. The iterator can only be used once and we need two iterations. We would need to allocate two separator iterators. But cleaners should be a small collection (size 1 by default) so the additional code complexity didn't seem worth it.

@ijuma
Copy link
Contributor Author

ijuma commented Dec 23, 2019

Thanks for taking a look.

I used an IntelliJ inspection to find candidates. It didn't find most of the Gauge usages, so I relied on plain old search to find those. That's how I also came across the Java examples you asked about. You are correct that the Java changes could have been done independently of dropping support for Scala 2.11.

@ijuma
Copy link
Contributor Author

ijuma commented Dec 23, 2019

I noticed that
MirrorMaker 2 uses AdminClient.create, but I did not change them
as Connectors have restrictions on newer client APIs.

@rhauch @ryannedolan Can you please confirm that we should stick with the old AdminClient.create API for MirrorMaker 2?

val alterResourceName = if (resourceName.nonEmpty)
resourceName
else
ConfigEntityName.Default
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This was unused.

@@ -102,7 +102,7 @@ object AclCommand extends Logging {
else
new Properties()
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt))
val adminClient = JAdminClient.create(props)
val adminClient = Admin.create(props)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice.

} else {
invocation.callRealMethod().asInstanceOf[Try[File]]
}
doAnswer { invocation =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: should we use () here? Do we have any non-written rules on lambdas for when () or {} should be used.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you mean doAnswer(invocation => {...? If so, I would say no. The block syntax is more concise and generally recommended for cases like this.

@@ -78,8 +78,6 @@ class KGroupedStream[K, V](val inner: KGroupedStreamJ[K, V]) {
* @see `org.apache.kafka.streams.kstream.KGroupedStream#reduce`
*/
def reduce(reducer: (V, V) => V)(implicit materialized: Materialized[K, V, ByteArrayKeyValueStore]): KTable[K, V] =
// need this explicit asReducer for Scala 2.11 or else the SAM conversion doesn't take place
// works perfectly with Scala 2.12 though
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we remove the asReducer now? Ditto below.

Copy link
Contributor Author

@ijuma ijuma Dec 24, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, we cannot. The comment is wrong as far as I can tell.

*/
// TODO: Deprecate this class if support for Scala 2.11 + Java 1.8 is dropped.
@deprecated(message = "Use org.apache.kafka.streams.kstream.Suppressed", since = "2.5")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we remove this class then?

Copy link
Contributor Author

@ijuma ijuma Dec 24, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, we cannot. The comment is wrong as far as I can tell.

@ijuma
Copy link
Contributor Author

ijuma commented Dec 24, 2019

retest this please.

@ijuma
Copy link
Contributor Author

ijuma commented Dec 25, 2019

JDK 11 and Scala 2.12 passed, JDK 11 and Scala 2.13 had one flaky failure:

org.apache.kafka.clients.consumer.internals.ConsumerCoordinatorTest.testCloseTimeoutCoordinatorUnavailableForCommit[rebalance protocol = EAGER]

@ijuma
Copy link
Contributor Author

ijuma commented Dec 25, 2019

retest this please

newGauge("SessionState", new Gauge[String] {
override def value: String = Option(connectionState.toString).getOrElse("DISCONNECTED")
})
newGauge("SessionState", () => connectionState.toString)
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The Option wrapping doesn't seem to achieve anything here. If connectionState is null, we'd get a NPE. toString should always return a non null value.

@ijuma
Copy link
Contributor Author

ijuma commented Dec 26, 2019

Tests passed for both jobs that are still enabled: JDK 8 and Scala 2.12 and JDK 11 and Scala 2.13.

@guozhangwang (for Streams) and @omkreddy (for Core) please review when you have a chance.

@rhauch
Copy link
Contributor

rhauch commented Dec 27, 2019

@ijuma wrote:

I noticed that MirrorMaker 2 uses AdminClient.create, but I did not change them
as Connectors have restrictions on newer client APIs.

@rhauch @ryannedolan Can you please confirm that we should stick with the old AdminClient.create API for MirrorMaker 2?

Is MirrorMaker 2 designed to be installed into older Kafka installations? If so, then we should probably stick with the old AdminClient API for MirrorMaker 2. If not and MirrorMaker 2 should always be used with whatever Kafka installation that includes it, then it can use the newer API.

Copy link
Contributor

@omkreddy omkreddy left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@ijuma Thanks for the PR. Core related changes looks good to me.

@guozhangwang
Copy link
Contributor

LGTM, thanks @ijuma !

@ijuma ijuma merged commit 6dc6f6a into apache:trunk Jan 6, 2020
@ijuma ijuma deleted the kafka-9324-drop-scala-2.11 branch January 6, 2020 18:51
@ijuma
Copy link
Contributor Author

ijuma commented Jan 6, 2020

Thanks for the reviews, merged to trunk.

omkreddy added a commit to confluentinc/kafka that referenced this pull request Jan 8, 2020
qinghui-xu added a commit to qinghui-xu/kafka that referenced this pull request Nov 17, 2020
- Revert "KAFKA-9324: Drop support for Scala 2.11 (KIP-531) (apache#7859)"
- Fix some issue with scala 2.11 build (eg. scala compile option
  "-target:jvm-1.8")
- Change all java lambda expression to anonymous class

This reverts commit 6dc6f6a.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants