Browse Source

KAFKA-4423: Drop support for Java 7 (KIP-118) and update deps (#5046)

* Set --source, --target and --release to 1.8.
* Build Scala 2.12 by default.
* Remove some conditionals in the build file now that Java 8
is the minimum version.
* Bump the version of Jetty, Jersey and Checkstyle (the newer
versions require Java 8).
* Fixed issues uncovered by the new version if Checkstyle.
* A couple of minor updates to handle an incompatible source
change in the new version of Jetty.
* Add dependency to jersey-hk2 to fix failing tests caused
by Jersey upgrade.
* Update release script to use Java 8 and to take into account
that Scala 2.12 is now built by default.
* While we're at it, bump the version of Gradle, Gradle plugins,
ScalaLogging, JMH and apache directory api.
* Minor documentation updates including the readme and upgrade
notes. A number of Streams Java 7 examples can be removed
subsequently.
pull/5058/head
Ismael Juma 7 years ago committed by GitHub
parent
commit
e70a191d30
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 6
      README.md
  2. 46
      build.gradle
  3. 2
      checkstyle/import-control.xml
  4. 26
      checkstyle/suppressions.xml
  5. 2
      clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java
  6. 2
      clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
  7. 2
      clients/src/main/java/org/apache/kafka/common/utils/Shell.java
  8. 4
      clients/src/main/java/org/apache/kafka/common/utils/Utils.java
  9. 4
      clients/src/test/java/org/apache/kafka/common/record/FileLogInputStreamTest.java
  10. 4
      connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java
  11. 2
      connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java
  12. 8
      connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/util/SSLUtilsTest.java
  13. 4
      docs/ops.html
  14. 1
      docs/upgrade.html
  15. 17
      gradle/dependencies.gradle
  16. 15
      release.py
  17. 5
      vagrant/base.sh

6
README.md

@ -4,9 +4,9 @@ See our [web site](http://kafka.apache.org) for details on the project. @@ -4,9 +4,9 @@ See our [web site](http://kafka.apache.org) for details on the project.
You need to have [Gradle](http://www.gradle.org/installation) and [Java](http://www.oracle.com/technetwork/java/javase/downloads/index.html) installed.
Kafka requires Gradle 3.0 or higher.
Kafka requires Gradle 4.5 or higher.
Java 7 should be used for building in order to support both Java 7 and Java 8 at runtime.
Java 8 should be used for building in order to support both Java 8 and Java 10 at runtime.
### First bootstrap and download the wrapper ###
cd kafka_source_dir
@ -85,8 +85,6 @@ You can pass either the major version (eg 2.11) or the full version (eg 2.11.12) @@ -85,8 +85,6 @@ You can pass either the major version (eg 2.11) or the full version (eg 2.11.12)
./gradlew -PscalaVersion=2.11 test
./gradlew -PscalaVersion=2.11 releaseTarGz
Scala 2.12.x requires Java 8.
### Running a task for a specific project ###
This is for `core`, `examples` and `clients`

46
build.gradle

@ -26,9 +26,9 @@ buildscript { @@ -26,9 +26,9 @@ buildscript {
// For Apache Rat plugin to ignore non-Git files
classpath "org.ajoberstar:grgit:1.9.3"
classpath 'com.github.ben-manes:gradle-versions-plugin:0.17.0'
classpath 'org.scoverage:gradle-scoverage:2.1.0'
classpath 'com.github.jengelman.gradle.plugins:shadow:2.0.2'
classpath 'org.owasp:dependency-check-gradle:3.1.1'
classpath 'org.scoverage:gradle-scoverage:2.3.0'
classpath 'com.github.jengelman.gradle.plugins:shadow:2.0.4'
classpath 'org.owasp:dependency-check-gradle:3.1.2'
}
}
@ -67,24 +67,19 @@ allprojects { @@ -67,24 +67,19 @@ allprojects {
}
}
if (JavaVersion.current().isJava8Compatible()) {
tasks.withType(Javadoc) {
// disable the crazy super-strict doclint tool in Java 8
// noinspection SpellCheckingInspection
options.addStringOption('Xdoclint:none', '-quiet')
}
tasks.withType(Javadoc) {
// disable the crazy super-strict doclint tool in Java 8
// noinspection SpellCheckingInspection
options.addStringOption('Xdoclint:none', '-quiet')
}
}
ext {
gradleVersion = "4.5.1"
gradleVersion = "4.7"
minJavaVersion = "8"
buildVersionFileName = "kafka-version.properties"
maxPermSizeArgs = []
if (!JavaVersion.current().isJava8Compatible())
maxPermSizeArgs += '-XX:MaxPermSize=512m'
userMaxForks = project.hasProperty('maxParallelForks') ? maxParallelForks.toInteger() : null
skipSigning = project.hasProperty('skipSigning') && skipSigning.toBoolean()
@ -145,20 +140,17 @@ subprojects { @@ -145,20 +140,17 @@ subprojects {
if (!JavaVersion.current().isJava9Compatible())
apply plugin: 'findbugs'
sourceCompatibility = 1.7
targetCompatibility = 1.7
sourceCompatibility = minJavaVersion
targetCompatibility = minJavaVersion
compileJava {
options.encoding = 'UTF-8'
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"
options.compilerArgs << "-Xlint:deprecation,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"
options.compilerArgs << "--release" << minJavaVersion
}
uploadArchives {
@ -202,7 +194,6 @@ subprojects { @@ -202,7 +194,6 @@ subprojects {
minHeapSize = "256m"
maxHeapSize = "2048m"
jvmArgs = maxPermSizeArgs
testLogging {
events = userTestLoggingEvents ?: testLoggingEvents
@ -217,7 +208,6 @@ subprojects { @@ -217,7 +208,6 @@ subprojects {
minHeapSize = "256m"
maxHeapSize = "2048m"
jvmArgs = maxPermSizeArgs
testLogging {
events = userTestLoggingEvents ?: testLoggingEvents
@ -236,7 +226,6 @@ subprojects { @@ -236,7 +226,6 @@ subprojects {
minHeapSize = "256m"
maxHeapSize = "2048m"
jvmArgs = maxPermSizeArgs
testLogging {
events = userTestLoggingEvents ?: testLoggingEvents
@ -343,7 +332,7 @@ subprojects { @@ -343,7 +332,7 @@ subprojects {
"-Xlint:private-shadow",
"-Xlint:stars-align",
"-Xlint:type-parameter-shadow",
"-Xlint:unsound-match",
"-Xlint:unsound-match"
]
if (versions.baseScala != '2.11') {
@ -355,15 +344,14 @@ subprojects { @@ -355,15 +344,14 @@ subprojects {
configure(scalaCompileOptions.forkOptions) {
memoryMaximumSize = '1g'
jvmArgs = ['-Xss2m'] + maxPermSizeArgs
jvmArgs = ['-Xss2m']
}
}
checkstyle {
configFile = new File(rootDir, "checkstyle/checkstyle.xml")
configProperties = [importControlFile: "$rootDir/checkstyle/import-control.xml"]
// version 7.x requires Java 8
toolVersion = '6.19'
toolVersion = '8.10'
}
test.dependsOn('checkstyleMain', 'checkstyleTest')
@ -885,6 +873,7 @@ project(':tools') { @@ -885,6 +873,7 @@ project(':tools') {
compile libs.jacksonJaxrsJsonProvider
compile libs.jerseyContainerServlet
compile libs.jerseyHk2
compile libs.jaxbApi // Jersey dependency that was available in the JDK before Java 9
compile libs.activation // Jersey dependency that was available in the JDK before Java 9
compile libs.jettyServer
@ -1335,6 +1324,7 @@ project(':connect:runtime') { @@ -1335,6 +1324,7 @@ project(':connect:runtime') {
compile libs.jacksonJaxrsJsonProvider
compile libs.jerseyContainerServlet
compile libs.jerseyHk2
compile libs.jaxbApi // Jersey dependency that was available in the JDK before Java 9
compile libs.activation // Jersey dependency that was available in the JDK before Java 9
compile libs.jettyServer

2
checkstyle/import-control.xml

@ -207,6 +207,8 @@ @@ -207,6 +207,8 @@
<allow pkg="org.apache.kafka.clients"/>
<allow pkg="org.apache.kafka.clients.producer" exact-match="true"/>
<allow pkg="org.apache.kafka.clients.consumer" exact-match="true"/>
<!-- Temporary until EosTestDriver migrates to the Java AdminClient -->
<allow pkg="kafka.admin" exact-match="true"/>
<allow pkg="org.apache.kafka.streams"/>

26
checkstyle/suppressions.xml

@ -57,7 +57,7 @@ @@ -57,7 +57,7 @@
files="AbstractRequest.java|KerberosLogin.java|WorkerSinkTaskTest.java|TransactionManagerTest.java"/>
<suppress checks="NPathComplexity"
files="(BufferPool|MetricName|Node|ConfigDef|SslTransportLayer|MetadataResponse|KerberosLogin|Selector|Sender|Serdes|Agent|Values|PluginUtils|MiniTrogdorCluster).java"/>
files="(BufferPool|Fetcher|MetricName|Node|ConfigDef|RecordBatch|SslFactory|SslTransportLayer|MetadataResponse|KerberosLogin|Selector|Sender|Serdes|TokenInformation|Agent|Values|PluginUtils|MiniTrogdorCluster).java"/>
<!-- clients tests -->
<suppress checks="ClassDataAbstractionCoupling"
@ -72,6 +72,9 @@ @@ -72,6 +72,9 @@
<suppress checks="JavaNCSS"
files="RequestResponseTest.java"/>
<suppress checks="NPathComplexity"
files="MemoryRecordsTest.java"/>
<!-- Connect -->
<suppress checks="ClassFanOutComplexity"
files="DistributedHerder(|Test).java"/>
@ -111,17 +114,8 @@ @@ -111,17 +114,8 @@
files="Values.java"/>
<suppress checks="NPathComplexity"
files="ConnectRecord.java"/>
<suppress checks="NPathComplexity"
files="ConnectSchema.java"/>
<suppress checks="NPathComplexity"
files="FileStreamSourceTask.java"/>
<suppress checks="NPathComplexity"
files="JsonConverter.java"/>
<suppress checks="NPathComplexity"
files="DistributedHerder.java"/>
<suppress checks="NPathComplexity"
files="ConnectHeaders.java"/>
files="(AbstractStatus|ConnectHeaders|ConnectRecord|ConnectSchema|DistributedHerder|FileStreamSourceTask|JsonConverter|KafkaConfigBackingStore).java"/>
<suppress checks="MethodLength"
files="Values.java"/>
@ -155,11 +149,7 @@ @@ -155,11 +149,7 @@
files="StreamsPartitionAssignor.java"/>
<suppress checks="NPathComplexity"
files="ProcessorStateManager.java"/>
<suppress checks="NPathComplexity"
files="StreamsPartitionAssignor.java"/>
<suppress checks="NPathComplexity"
files="StreamThread.java"/>
files="(ProcessorStateManager|InternalTopologyBuilder|StreamsPartitionAssignor|StreamThread).java"/>
<!-- Streams tests -->
<suppress checks="ClassFanOutComplexity"
@ -189,7 +179,7 @@ @@ -189,7 +179,7 @@
files="SmokeTestDriver.java"/>
<suppress checks="NPathComplexity"
files="KStreamKStreamJoinTest.java|SmokeTestDriver.java"/>
files="EosTestDriver|KStreamKStreamJoinTest.java|SmokeTestDriver.java"/>
<suppress checks="NPathComplexity"
files="KStreamKStreamLeftJoinTest.java"/>

2
clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java

@ -96,7 +96,7 @@ public class Metrics implements Closeable { @@ -96,7 +96,7 @@ public class Metrics implements Closeable {
* Expiration of Sensors is disabled.
*/
public Metrics(MetricConfig defaultConfig, Time time) {
this(defaultConfig, new ArrayList<MetricsReporter>(0), time);
this(defaultConfig, new ArrayList<MetricsReporter>(0), time);
}

2
clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java

@ -584,7 +584,7 @@ public class MemoryRecords extends AbstractRecords { @@ -584,7 +584,7 @@ public class MemoryRecords extends AbstractRecords {
public static MemoryRecords withRecords(byte magic, long initialOffset, CompressionType compressionType,
TimestampType timestampType, long producerId, short producerEpoch,
int baseSequence, int partitionLeaderEpoch, boolean isTransactional,
SimpleRecord ... records) {
SimpleRecord... records) {
if (records.length == 0)
return MemoryRecords.EMPTY;
int sizeEstimate = AbstractRecords.estimateSizeInBytes(magic, compressionType, Arrays.asList(records));

2
clients/src/main/java/org/apache/kafka/common/utils/Shell.java

@ -251,7 +251,7 @@ abstract public class Shell { @@ -251,7 +251,7 @@ abstract public class Shell {
* @param cmd shell command to execute.
* @return the output of the executed command.
*/
public static String execCommand(String ... cmd) throws IOException {
public static String execCommand(String... cmd) throws IOException {
return execCommand(cmd, -1);
}

4
clients/src/main/java/org/apache/kafka/common/utils/Utils.java

@ -151,7 +151,7 @@ public final class Utils { @@ -151,7 +151,7 @@ public final class Utils {
* @param rest The remaining values to compare
* @return The minimum of all passed values
*/
public static long min(long first, long ... rest) {
public static long min(long first, long... rest) {
long min = first;
for (long r : rest) {
if (r < min)
@ -166,7 +166,7 @@ public final class Utils { @@ -166,7 +166,7 @@ public final class Utils {
* @param rest The remaining values to compare
* @return The maximum of all passed values
*/
public static long max(long first, long ... rest) {
public static long max(long first, long... rest) {
long max = first;
for (long r : rest) {
if (r > max)

4
clients/src/test/java/org/apache/kafka/common/record/FileLogInputStreamTest.java

@ -221,7 +221,7 @@ public class FileLogInputStreamTest { @@ -221,7 +221,7 @@ public class FileLogInputStreamTest {
}
private void assertProducerData(RecordBatch batch, long producerId, short producerEpoch, int baseSequence,
boolean isTransactional, SimpleRecord ... records) {
boolean isTransactional, SimpleRecord... records) {
assertEquals(producerId, batch.producerId());
assertEquals(producerEpoch, batch.producerEpoch());
assertEquals(baseSequence, batch.baseSequence());
@ -237,7 +237,7 @@ public class FileLogInputStreamTest { @@ -237,7 +237,7 @@ public class FileLogInputStreamTest {
assertFalse(batch.isTransactional());
}
private void assertGenericRecordBatchData(RecordBatch batch, long baseOffset, long maxTimestamp, SimpleRecord ... records) {
private void assertGenericRecordBatchData(RecordBatch batch, long baseOffset, long maxTimestamp, SimpleRecord... records) {
assertEquals(magic, batch.magic());
assertEquals(compression, batch.compressionType());

4
connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java

@ -367,7 +367,7 @@ public class ValuesTest { @@ -367,7 +367,7 @@ public class ValuesTest {
assertConsumable(parser, expectedTokens);
}
protected void assertConsumable(Parser parser, String ... expectedTokens) {
protected void assertConsumable(Parser parser, String... expectedTokens) {
for (String expectedToken : expectedTokens) {
if (!expectedToken.trim().isEmpty()) {
int position = parser.mark();
@ -459,4 +459,4 @@ public class ValuesTest { @@ -459,4 +459,4 @@ public class ValuesTest {
}
}
}
}

2
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java

@ -114,7 +114,7 @@ public class SSLUtils { @@ -114,7 +114,7 @@ public class SSLUtils {
if (sslCipherSuites != null)
ssl.setIncludeCipherSuites(sslCipherSuites.toArray(new String[sslCipherSuites.size()]));
ssl.setSslKeyManagerFactoryAlgorithm((String) getOrDefault(sslConfigValues, SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, SslConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM));
ssl.setKeyManagerFactoryAlgorithm((String) getOrDefault(sslConfigValues, SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, SslConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM));
String sslSecureRandomImpl = (String) sslConfigValues.get(SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG);
if (sslSecureRandomImpl != null)

8
connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/util/SSLUtilsTest.java

@ -80,8 +80,8 @@ public class SSLUtilsTest { @@ -80,8 +80,8 @@ public class SSLUtilsTest {
DistributedConfig config = new DistributedConfig(configMap);
SslContextFactory ssl = SSLUtils.createSslContextFactory(config);
Assert.assertEquals("/path/to/keystore", ssl.getKeyStorePath());
Assert.assertEquals("/path/to/truststore", ssl.getTrustStore());
Assert.assertEquals("file:///path/to/keystore", ssl.getKeyStorePath());
Assert.assertEquals("file:///path/to/truststore", ssl.getTrustStorePath());
Assert.assertEquals("SunJSSE", ssl.getProvider());
Assert.assertArrayEquals(new String[] {"SSL_RSA_WITH_RC4_128_SHA", "SSL_RSA_WITH_RC4_128_MD5"}, ssl.getIncludeCipherSuites());
Assert.assertEquals("SHA1PRNG", ssl.getSecureRandomAlgorithm());
@ -90,7 +90,7 @@ public class SSLUtilsTest { @@ -90,7 +90,7 @@ public class SSLUtilsTest {
Assert.assertEquals("JKS", ssl.getTrustStoreType());
Assert.assertEquals("TLS", ssl.getProtocol());
Assert.assertArrayEquals(new String[] {"TLSv1.2", "TLSv1.1", "TLSv1"}, ssl.getIncludeProtocols());
Assert.assertEquals("SunX509", ssl.getSslKeyManagerFactoryAlgorithm());
Assert.assertEquals("SunX509", ssl.getKeyManagerFactoryAlgorithm());
Assert.assertEquals("PKIX", ssl.getTrustManagerFactoryAlgorithm());
}
@ -118,7 +118,7 @@ public class SSLUtilsTest { @@ -118,7 +118,7 @@ public class SSLUtilsTest {
Assert.assertEquals(SslConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE, ssl.getTrustStoreType());
Assert.assertEquals(SslConfigs.DEFAULT_SSL_PROTOCOL, ssl.getProtocol());
Assert.assertArrayEquals(Arrays.asList(SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS.split("\\s*,\\s*")).toArray(), ssl.getIncludeProtocols());
Assert.assertEquals(SslConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM, ssl.getSslKeyManagerFactoryAlgorithm());
Assert.assertEquals(SslConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM, ssl.getKeyManagerFactoryAlgorithm());
Assert.assertEquals(SslConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM, ssl.getTrustManagerFactoryAlgorithm());
Assert.assertFalse(ssl.getNeedClientAuth());
Assert.assertFalse(ssl.getWantClientAuth());

4
docs/ops.html

@ -639,9 +639,7 @@ @@ -639,9 +639,7 @@
From a security perspective, we recommend you use the latest released version of JDK 1.8 as older freely available versions have disclosed security vulnerabilities.
LinkedIn is currently running JDK 1.8 u5 (looking to upgrade to a newer version) with the G1 collector. If you decide to use the G1 collector (the current default) and you are still on JDK 1.7, make sure you are on u51 or newer. LinkedIn tried out u21 in testing, but they had a number of problems with the GC implementation in that version.
LinkedIn's tuning looks like this:
LinkedIn is currently running JDK 1.8 u5 (looking to upgrade to a newer version) with the G1 collector. LinkedIn's tuning looks like this:
<pre class="brush: text;">
-Xmx6g -Xms6g -XX:MetaspaceSize=96m -XX:+UseG1GC
-XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:G1HeapRegionSize=16M

1
docs/upgrade.html

@ -66,6 +66,7 @@ @@ -66,6 +66,7 @@
<h5><a id="upgrade_200_notable" href="#upgrade_200_notable">Notable changes in 2.0.0</a></h5>
<ul>
<li><a href="https://cwiki.apache.org/confluence/x/oYtjB">KIP-186</a> increases the default offset retention time from 1 day to 7 days. This makes it less likely to "lose" offsets in an application that commits infrequently. It also increases the active set of offsets and therefore can increase memory usage on the broker. Note that the console consumer currently enables offset commit by default and can be the source of a large number of offsets which this change will now preserve for 7 days instead of 1. You can preserve the existing behavior by setting the broker config <code>offsets.retention.minutes</code> to 1440.</li>
<li>Support for Java 7 has been dropped, Java 8 is now the minimum version required.</li>
<li><a href="https://issues.apache.org/jira/browse/KAFKA-5674">KAFKA-5674</a> extends the lower interval of <code>max.connections.per.ip minimum</code> to zero and therefore allows IP-based filtering of inbound connections.</li>
<li><a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-272%3A+Add+API+version+tag+to+broker%27s+RequestsPerSec+metric">KIP-272</a>
added API version tag to the metric <code>kafka.network:type=RequestMetrics,name=RequestsPerSec,request={Produce|FetchConsumer|FetchFollower|...}</code>.

17
gradle/dependencies.gradle

@ -22,9 +22,9 @@ ext { @@ -22,9 +22,9 @@ ext {
libs = [:]
// Enabled by default when commands like `testAll` are invoked
defaultScalaVersions = [ '2.11' ]
// Available if -PscalaVersion is used, this is necessary because Scala 2.12 requires Java 8 while Kafka is usually
// built with Java 7
defaultScalaVersions = [ '2.11', '2.12' ]
// Available if -PscalaVersion is used. This is useful when we want to support a Scala version that has
// a higher minimum Java requirement than Kafka. This was previously the case for Scala 2.12 and Java 7.
availableScalaVersions = [ '2.11', '2.12' ]
}
@ -48,17 +48,17 @@ versions["baseScala"] = versions.scala.substring(0, versions.scala.lastIndexOf(" @@ -48,17 +48,17 @@ versions["baseScala"] = versions.scala.substring(0, versions.scala.lastIndexOf("
versions += [
activation: "1.1.1",
apacheda: "1.0.0",
apacheda: "1.0.1",
apacheds: "2.0.0-M24",
argparse4j: "0.7.0",
bcpkix: "1.59",
easymock: "3.6",
jackson: "2.9.5",
jetty: "9.2.24.v20180105",
jersey: "2.25.1",
jmh: "1.20",
jetty: "9.4.10.v20180503",
jersey: "2.27",
jmh: "1.21",
log4j: "1.2.17",
scalaLogging: "3.8.0",
scalaLogging: "3.9.0",
jaxb: "2.3.0",
jfreechart: "1.0.0",
jopt: "5.0.4",
@ -106,6 +106,7 @@ libs += [ @@ -106,6 +106,7 @@ libs += [
jettyServlet: "org.eclipse.jetty:jetty-servlet:$versions.jetty",
jettyServlets: "org.eclipse.jetty:jetty-servlets:$versions.jetty",
jerseyContainerServlet: "org.glassfish.jersey.containers:jersey-container-servlet:$versions.jersey",
jerseyHk2: "org.glassfish.jersey.inject:jersey-hk2:$versions.jersey",
jfreechart: "1.0.0",
jmhCore: "org.openjdk.jmh:jmh-core:$versions.jmh",
jmhCoreBenchmarks: "org.openjdk.jmh:jmh-core-benchmarks:$versions.jmh",

15
release.py

@ -272,7 +272,7 @@ prefs = load_prefs() @@ -272,7 +272,7 @@ prefs = load_prefs()
if not user_ok("""Requirements:
1. Updated docs to reference the new release version where appropriate.
2. JDK7 and JDK8 compilers and libraries
2. JDK8 compilers and libraries
3. Your Apache ID, already configured with SSH keys on id.apache.org and SSH keys available in this shell session
4. All issues in the target release resolved with valid resolutions (if not, this script will report the problematic JIRAs)
5. A GPG key used for signing the release. This key should have been added to public Apache servers and the KEYS file on the Kafka site
@ -363,7 +363,6 @@ if not rc: @@ -363,7 +363,6 @@ if not rc:
# Prereq checks
apache_id = get_pref(prefs, 'apache_id', lambda: raw_input("Enter your apache username: "))
jdk7_env = get_jdk(prefs, 7)
jdk8_env = get_jdk(prefs, 8)
@ -448,10 +447,8 @@ params = { 'release_version': release_version, @@ -448,10 +447,8 @@ params = { 'release_version': release_version,
}
cmd("Creating source archive", "git archive --format tar.gz --prefix kafka-%(release_version)s-src/ -o %(artifacts_dir)s/kafka-%(release_version)s-src.tgz %(rc_tag)s" % params)
cmd("Building artifacts", "gradle", cwd=kafka_dir, env=jdk7_env)
cmd("Building artifacts", "./gradlew clean releaseTarGzAll aggregatedJavadoc", cwd=kafka_dir, env=jdk7_env)
# we need extra cmd to build 2.12 with jdk8 specifically
cmd("Building artifacts for Scala 2.12", "./gradlew releaseTarGz -PscalaVersion=2.12", cwd=kafka_dir, env=jdk8_env)
cmd("Building artifacts", "gradle", cwd=kafka_dir, env=jdk8_env)
cmd("Building artifacts", "./gradlew clean releaseTarGzAll aggregatedJavadoc", cwd=kafka_dir, env=jdk8_env)
cmd("Copying artifacts", "cp %s/core/build/distributions/* %s" % (kafka_dir, artifacts_dir), shell=True)
cmd("Copying artifacts", "cp -R %s/build/docs/javadoc %s" % (kafka_dir, artifacts_dir))
@ -497,9 +494,9 @@ with open(os.path.expanduser("~/.gradle/gradle.properties")) as f: @@ -497,9 +494,9 @@ with open(os.path.expanduser("~/.gradle/gradle.properties")) as f:
contents = f.read()
if not user_ok("Going to build and upload mvn artifacts based on these settings:\n" + contents + '\nOK (y/n)?: '):
fail("Retry again later")
cmd("Building and uploading archives", "./gradlew uploadArchivesAll", cwd=kafka_dir, env=jdk7_env)
cmd("Building and uploading archives", "./gradlew uploadArchivesAll", cwd=kafka_dir, env=jdk8_env)
cmd("Building and uploading archives", "./gradlew uploadCoreArchives_2_12 -PscalaVersion=2.12", cwd=kafka_dir, env=jdk8_env)
cmd("Building and uploading archives", "mvn deploy -Pgpg-signing", cwd=streams_quickstart_dir, env=jdk7_env)
cmd("Building and uploading archives", "mvn deploy -Pgpg-signing", cwd=streams_quickstart_dir, env=jdk8_env)
release_notification_props = { 'release_version': release_version,
'rc': rc,
@ -593,7 +590,7 @@ http://kafka.apache.org/%(docs_version)s/documentation.html @@ -593,7 +590,7 @@ http://kafka.apache.org/%(docs_version)s/documentation.html
http://kafka.apache.org/%(docs_version)s/protocol.html
* Successful Jenkins builds for the %(dev_branch)s branch:
Unit/integration tests: https://builds.apache.org/job/kafka-%(dev_branch)s-jdk7/<BUILD NUMBER>/
Unit/integration tests: https://builds.apache.org/job/kafka-%(dev_branch)s-jdk8/<BUILD NUMBER>/
System tests: https://jenkins.confluent.io/job/system-test-kafka-%(dev_branch)s/<BUILD_NUMBER>/
/**************************************

5
vagrant/base.sh

@ -100,8 +100,9 @@ popd @@ -100,8 +100,9 @@ popd
popd
# Test multiple Kafka versions
# we want to use the latest Scala version per Kafka version
# however, we cannot pull in Scala 2.12 builds atm, because Scala 2.12 requires Java 8, but we use Java 7 to run the system tests
# We want to use the latest Scala version per Kafka version
# Previously we could not pull in Scala 2.12 builds, because Scala 2.12 requires Java 8 and we were running the system
# tests with Java 7. We have since switched to Java 8, so 2.0.0 and later use Scala 2.12.
get_kafka 0.8.2.2 2.11
chmod a+rw /opt/kafka-0.8.2.2
get_kafka 0.9.0.1 2.11

Loading…
Cancel
Save