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-4501: Java 9 compilation and runtime fixes #3647

Closed
wants to merge 9 commits into from
38 changes: 25 additions & 13 deletions build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ ext {

maxPermSizeArgs = []
if (!JavaVersion.current().isJava8Compatible())
maxPermSizeArgs = ['-XX:MaxPermSize=512m']
maxPermSizeArgs += '-XX:MaxPermSize=512m'

userMaxForks = project.hasProperty('maxParallelForks') ? maxParallelForks.toInteger() : null

Expand Down Expand Up @@ -137,14 +137,24 @@ subprojects {
apply plugin: 'maven'
apply plugin: 'signing'
apply plugin: 'checkstyle'
apply plugin: 'findbugs'

if (!JavaVersion.current().isJava9Compatible())
apply plugin: 'findbugs'

sourceCompatibility = 1.7
targetCompatibility = 1.7

compileJava {
options.encoding = 'UTF-8'
// Add unchecked once we drop support for Java 7 as @SuppressWarnings("unchecked") is too buggy in Java 7
options.compilerArgs << "-Xlint:deprecation"
// -Xlint:unchecked is too buggy in Java 7, so we only enable for Java 8 or higher
if (JavaVersion.current().isJava8Compatible())
options.compilerArgs << "-Xlint:unchecked"
// --release is the recommended way to select the target release, but it's only supported in Java 9 so we also
// set --source and --target via `sourceCompatibility` and `targetCompatibility`. If/when Gradle supports `--release`
// natively (https://github.com/gradle/gradle/issues/2510), we should switch to that.
if (JavaVersion.current().isJava9Compatible())
options.compilerArgs << "--release" << "7"
}

uploadArchives {
Expand Down Expand Up @@ -349,17 +359,19 @@ subprojects {
}
test.dependsOn('checkstyleMain', 'checkstyleTest')

findbugs {
toolVersion = "3.0.1"
excludeFilter = file("$rootDir/gradle/findbugs-exclude.xml")
ignoreFailures = false
}
test.dependsOn('findbugsMain')
if (!JavaVersion.current().isJava9Compatible()) {
findbugs {
toolVersion = "3.0.1"
excludeFilter = file("$rootDir/gradle/findbugs-exclude.xml")
ignoreFailures = false
}
test.dependsOn('findbugsMain')

tasks.withType(FindBugs) {
reports {
xml.enabled (project.hasProperty('xmlFindBugsReport'))
html.enabled (!project.hasProperty('xmlFindBugsReport'))
tasks.withType(FindBugs) {
reports {
xml.enabled(project.hasProperty('xmlFindBugsReport'))
html.enabled(!project.hasProperty('xmlFindBugsReport'))
}
}
}

Expand Down
2 changes: 0 additions & 2 deletions checkstyle/import-control.xml
Original file line number Diff line number Diff line change
Expand Up @@ -98,7 +98,6 @@
</subpackage>
<subpackage name="scram">
<allow pkg="javax.crypto" />
<allow pkg="javax.xml.bind" />
<allow pkg="org.apache.kafka.common.errors" />
</subpackage>
</subpackage>
Expand Down Expand Up @@ -247,7 +246,6 @@
<subpackage name="test">
<allow pkg="org.apache.kafka" />
<allow pkg="org.bouncycastle" />
<allow pkg="javax.xml.bind" />
</subpackage>

<subpackage name="connect">
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,10 @@ public class PlainSaslServerProvider extends Provider {

private static final long serialVersionUID = 1L;

@SuppressWarnings("deprecation")
protected PlainSaslServerProvider() {
super("Simple SASL/PLAIN Server Provider", 1.0, "Simple SASL/PLAIN Server Provider for Kafka");
super.put("SaslServerFactory." + PlainSaslServer.PLAIN_MECHANISM, PlainSaslServerFactory.class.getName());
put("SaslServerFactory." + PlainSaslServer.PLAIN_MECHANISM, PlainSaslServerFactory.class.getName());
}

public static void initialize() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,8 @@
import java.util.Collection;
import java.util.Properties;

import javax.xml.bind.DatatypeConverter;

import org.apache.kafka.common.security.authenticator.CredentialCache;
import org.apache.kafka.common.utils.Base64;

/**
* SCRAM Credential persistence utility functions. Implements format conversion used
Expand All @@ -41,11 +40,11 @@ public class ScramCredentialUtils {
public static String credentialToString(ScramCredential credential) {
return String.format("%s=%s,%s=%s,%s=%s,%s=%d",
SALT,
DatatypeConverter.printBase64Binary(credential.salt()),
Base64.encoder().encodeToString(credential.salt()),
STORED_KEY,
DatatypeConverter.printBase64Binary(credential.storedKey()),
Base64.encoder().encodeToString(credential.storedKey()),
SERVER_KEY,
DatatypeConverter.printBase64Binary(credential.serverKey()),
Base64.encoder().encodeToString(credential.serverKey()),
ITERATIONS,
credential.iterations());
}
Expand All @@ -56,9 +55,9 @@ public static ScramCredential credentialFromString(String str) {
!props.containsKey(SERVER_KEY) || !props.containsKey(ITERATIONS)) {
throw new IllegalArgumentException("Credentials not valid: " + str);
}
byte[] salt = DatatypeConverter.parseBase64Binary(props.getProperty(SALT));
byte[] storedKey = DatatypeConverter.parseBase64Binary(props.getProperty(STORED_KEY));
byte[] serverKey = DatatypeConverter.parseBase64Binary(props.getProperty(SERVER_KEY));
byte[] salt = Base64.decoder().decode(props.getProperty(SALT));
byte[] storedKey = Base64.decoder().decode(props.getProperty(STORED_KEY));
byte[] serverKey = Base64.decoder().decode(props.getProperty(SERVER_KEY));
int iterations = Integer.parseInt(props.getProperty(ITERATIONS));
return new ScramCredential(salt, storedKey, serverKey, iterations);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,12 +16,13 @@
*/
package org.apache.kafka.common.security.scram;

import org.apache.kafka.common.utils.Base64;

import java.nio.charset.StandardCharsets;
import java.util.regex.Matcher;
import java.util.regex.Pattern;

import javax.security.sasl.SaslException;
import javax.xml.bind.DatatypeConverter;

/**
* SCRAM request/response message creation and parsing based on
Expand Down Expand Up @@ -140,7 +141,7 @@ public ServerFirstMessage(byte[] messageBytes) throws SaslException {
}
this.nonce = matcher.group("nonce");
String salt = matcher.group("salt");
this.salt = DatatypeConverter.parseBase64Binary(salt);
this.salt = Base64.decoder().decode(salt);
}
public ServerFirstMessage(String clientNonce, String serverNonce, byte[] salt, int iterations) {
this.nonce = clientNonce + serverNonce;
Expand All @@ -157,7 +158,7 @@ public int iterations() {
return iterations;
}
String toMessage() {
return String.format("r=%s,s=%s,i=%d", nonce, DatatypeConverter.printBase64Binary(salt), iterations);
return String.format("r=%s,s=%s,i=%d", nonce, Base64.encoder().encodeToString(salt), iterations);
}
}
/**
Expand All @@ -184,9 +185,9 @@ public ClientFinalMessage(byte[] messageBytes) throws SaslException {
if (!matcher.matches())
throw new SaslException("Invalid SCRAM client final message format: " + message);

this.channelBinding = DatatypeConverter.parseBase64Binary(matcher.group("channel"));
this.channelBinding = Base64.decoder().decode(matcher.group("channel"));
this.nonce = matcher.group("nonce");
this.proof = DatatypeConverter.parseBase64Binary(matcher.group("proof"));
this.proof = Base64.decoder().decode(matcher.group("proof"));
}
public ClientFinalMessage(byte[] channelBinding, String nonce) {
this.channelBinding = channelBinding;
Expand All @@ -206,13 +207,13 @@ public void proof(byte[] proof) {
}
public String clientFinalMessageWithoutProof() {
return String.format("c=%s,r=%s",
DatatypeConverter.printBase64Binary(channelBinding),
Base64.encoder().encodeToString(channelBinding),
nonce);
}
String toMessage() {
return String.format("%s,p=%s",
clientFinalMessageWithoutProof(),
DatatypeConverter.printBase64Binary(proof));
Base64.encoder().encodeToString(proof));
}
}
/**
Expand Down Expand Up @@ -243,7 +244,7 @@ public ServerFinalMessage(byte[] messageBytes) throws SaslException {
// ignore
}
if (error == null) {
this.serverSignature = DatatypeConverter.parseBase64Binary(matcher.group("signature"));
this.serverSignature = Base64.decoder().decode(matcher.group("signature"));
this.error = null;
} else {
this.serverSignature = null;
Expand All @@ -264,7 +265,7 @@ String toMessage() {
if (error != null)
return "e=" + error;
else
return "v=" + DatatypeConverter.printBase64Binary(serverSignature);
return "v=" + Base64.encoder().encodeToString(serverSignature);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,10 +25,11 @@ public class ScramSaslClientProvider extends Provider {

private static final long serialVersionUID = 1L;

@SuppressWarnings("deprecation")
protected ScramSaslClientProvider() {
super("SASL/SCRAM Client Provider", 1.0, "SASL/SCRAM Client Provider for Kafka");
for (ScramMechanism mechanism : ScramMechanism.values())
super.put("SaslClientFactory." + mechanism.mechanismName(), ScramSaslClientFactory.class.getName());
put("SaslClientFactory." + mechanism.mechanismName(), ScramSaslClientFactory.class.getName());

Choose a reason for hiding this comment

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

Why was this needed?

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 super qualifier didn't achieve anything.

}

public static void initialize() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,10 +25,11 @@ public class ScramSaslServerProvider extends Provider {

private static final long serialVersionUID = 1L;

@SuppressWarnings("deprecation")
protected ScramSaslServerProvider() {
super("SASL/SCRAM Server Provider", 1.0, "SASL/SCRAM Server Provider for Kafka");
for (ScramMechanism mechanism : ScramMechanism.values())
super.put("SaslServerFactory." + mechanism.mechanismName(), ScramSaslServerFactory.class.getName());
put("SaslServerFactory." + mechanism.mechanismName(), ScramSaslServerFactory.class.getName());
}

public static void initialize() {
Expand Down
Loading