diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index fc5ee1bd7558b..d10352f7ffb6e 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -140,7 +140,7 @@ jobs: # --scan: Publish the build scan. This will only work on PRs from apache/kafka and trunk # --no-scan: For public fork PRs, we won't attempt to publish the scan run: | - ./gradlew --build-cache --info $SCAN_ARG check releaseTarGz -x test + ./gradlew check rewriteDryRun --build-cache --info $SCAN_ARG releaseTarGz -x test - name: Archive check reports if: always() uses: actions/upload-artifact@v4 diff --git a/README.md b/README.md index 4c2fc4e1f160b..9b3807db3f656 100644 --- a/README.md +++ b/README.md @@ -201,7 +201,17 @@ For experiments (or regression testing purposes) add `-PcheckstyleVersion=X.y.z` #### Spotless #### The import order is a part of static check. please call `spotlessApply` to optimize the imports of Java codes before filing pull request. - ./gradlew spotlessApply +`./gradlew spotlessApply` + +#### Rewrite +The build system incorporates [Moderne](https://moderne.io/) rewrite capabilities for automated code transformations. + +- **Convention** (e.g., JUnit's naming rules) +- **Refactor** safely (e.g., rename methods, migrate APIs) +- **Modernize** (e.g., Java 8 → Java 17 features) +- **Patterns** (e.g., replace `Vector` with `ArrayList`) + +`./gradlew rewriteRun` #### Spotbugs #### Spotbugs uses static analysis to look for bugs in the code. diff --git a/build.gradle b/build.gradle index 5f9e81789cc2b..7f2936eaa677e 100644 --- a/build.gradle +++ b/build.gradle @@ -1,18 +1,22 @@ // Licensed to the Apache Software Foundation (ASF) under one or more -// contributor license agreements. See the NOTICE file distributed with -// this work for additional information regarding copyright ownership. -// The ASF licenses this file to You under the Apache License, Version 2.0 -// (the "License"); you may not use this file except in compliance with -// the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +import com.github.spotbugs.snom.SpotBugsTask import org.ajoberstar.grgit.Grgit import java.nio.charset.StandardCharsets @@ -29,19 +33,26 @@ buildscript { } plugins { - id 'com.github.ben-manes.versions' version '0.48.0' - id 'idea' - id 'jacoco' - id 'java-library' - id 'org.owasp.dependencycheck' version '8.2.1' - id 'org.nosphere.apache.rat' version "0.8.1" + // active + id "com.github.ben-manes.versions" version "0.48.0" + id "idea" id "io.swagger.core.v3.swagger-gradle-plugin" version "${swaggerVersion}" - - id "com.github.spotbugs" version '6.0.25' apply false - id 'org.scoverage' version '8.0.3' apply false - id 'com.gradleup.shadow' version '8.3.6' apply false - id 'com.diffplug.spotless' version "6.25.0" -} + id "jacoco" + id "java-library" + id "org.nosphere.apache.rat" version "0.8.1" + id "org.owasp.dependencycheck" version "8.2.1" + // passive + id "com.diffplug.spotless" version "7.2.1" apply false + id "com.github.spotbugs" version "6.0.25" apply false + id "com.gradleup.shadow" version "8.3.6" apply false + id "org.openrewrite.rewrite" version "7.12.1" apply false + id "org.scoverage" version "8.0.3" apply false +} + +apply from: "$rootDir/gradle/.qa/checkstyle.gradle" +apply from: "$rootDir/gradle/.qa/rewrite.gradle" +apply from: "$rootDir/gradle/.qa/spotbugs.gradle" +apply from: "$rootDir/gradle/.qa/spotless.gradle" ext { gradleVersion = versions.gradle @@ -157,75 +168,11 @@ ext { libs.log4j2Api, libs.log4j2Core ] - -} - -allprojects { - - repositories { - mavenCentral() - } - dependencyUpdates { - revision="release" - resolutionStrategy { - componentSelection { rules -> - rules.all { ComponentSelection selection -> - boolean rejected = ['snap', 'alpha', 'beta', 'rc', 'cr', 'm'].any { qualifier -> - selection.candidate.version ==~ /(?i).*[.-]${qualifier}[.\d-]*/ - } - if (rejected) { - selection.reject('Release candidate') - } - } - } - } - } - - configurations.all { - // zinc is the Scala incremental compiler, it has a configuration for its own dependencies - // that are unrelated to the project dependencies, we should not change them - if (name != "zinc") { - resolutionStrategy { - force( - // be explicit about the javassist dependency version instead of relying on the transitive version - libs.javassist, - // ensure we have a single version in the classpath despite transitive dependencies - libs.scalaLibrary, - libs.scalaReflect, - // Workaround before `commons-validator` has new release. See KAFKA-19359. - libs.commonsBeanutils, - libs.jacksonAnnotations, - libs.commonsLang - ) - } - } - } - task printAllDependencies(type: DependencyReportTask) {} - - tasks.withType(Javadoc) { - options.charSet = 'UTF-8' - options.docEncoding = 'UTF-8' - options.encoding = 'UTF-8' - options.memberLevel = JavadocMemberLevel.PUBLIC // Document only public members/API - // Turn off doclint for now, see https://blog.joda.org/2014/02/turning-off-doclint-in-jdk-8-javadoc.html for rationale - options.addStringOption('Xdoclint:none', '-quiet') - // Javadoc warnings should fail the build in JDK 15+ https://bugs.openjdk.org/browse/JDK-8200363 - options.addBooleanOption('Werror', JavaVersion.current().isCompatibleWith(JavaVersion.VERSION_15)) - options.links "https://docs.oracle.com/en/java/javase/${JavaVersion.current().majorVersion}/docs/api/" - } - - tasks.withType(Checkstyle) { - minHeapSize = "200m" - maxHeapSize = "1g" - } - - clean { - delete "${projectDir}/src/generated" - delete "${projectDir}/src/generated-test" - } } +apply from: "$rootDir/gradle/allprojects.gradle" + def determineCommitId() { def takeFromHash = 16 if (project.hasProperty('commitId')) { @@ -254,7 +201,6 @@ static def projectToJUnitXmlPath(project) { return projectNames.join("/") } - apply from: file('wrapper.gradle') if (repo != null) { @@ -302,535 +248,7 @@ if (repo != null) { println("Starting build with version $version (commit id ${commitId == null ? "null" : commitId.take(8)}) using Gradle $gradleVersion, Java ${JavaVersion.current()} and Scala ${versions.scala}") println("Build properties: ignoreFailures=$userIgnoreFailures, maxParallelForks=$maxTestForks, maxScalacThreads=$maxScalacThreads, maxTestRetries=$userMaxTestRetries") -subprojects { - - // enable running :dependencies task recursively on all subprojects - // eg: ./gradlew allDeps - task allDeps(type: DependencyReportTask) {} - // enable running :dependencyInsight task recursively on all subprojects - // eg: ./gradlew allDepInsight --configuration runtime --dependency com.fasterxml.jackson.core:jackson-databind - task allDepInsight(type: DependencyInsightReportTask) {showingAllVariants = false} doLast {} - - apply plugin: 'java-library' - apply plugin: 'checkstyle' - apply plugin: "com.github.spotbugs" - - // We use the shadow plugin for the jmh-benchmarks module and the `-all` jar can get pretty large, so - // don't publish it - def shouldPublish = !project.name.equals('jmh-benchmarks') - def shouldPublishWithShadow = (['clients'].contains(project.name)) - - if (shouldPublish) { - apply plugin: 'maven-publish' - apply plugin: 'signing' - - // Add aliases for the task names used by the maven plugin for backwards compatibility - // The maven plugin was replaced by the maven-publish plugin in Gradle 7.0 - tasks.register('install').configure { dependsOn(publishToMavenLocal) } - tasks.register('uploadArchives').configure { dependsOn(publish) } - } - - // apply the eclipse plugin only to subprojects that hold code. 'connect' is just a folder. - if (!project.name.equals('connect')) { - apply plugin: 'eclipse' - fineTuneEclipseClasspathFile(eclipse, project) - } - - java { - consistentResolution { - // resolve the compileClasspath and then "inject" the result of resolution as strict constraints into the runtimeClasspath - useCompileClasspathVersions() - } - } - - tasks.withType(JavaCompile) { - configureJavaCompiler(name, options, project.path) - } - - if (shouldPublish) { - - publishing { - repositories { - // To test locally, invoke gradlew with `-PmavenUrl=file:///some/local/path` - maven { - url = mavenUrl - credentials { - username = mavenUsername - password = mavenPassword - } - } - } - publications { - mavenJava(MavenPublication) { - if (!shouldPublishWithShadow) { - from components.java - } else { - apply plugin: 'com.gradleup.shadow' - from components.shadow - - // Fix for avoiding inclusion of runtime dependencies marked as 'shadow' in MANIFEST Class-Path. - // https://github.com/GradleUp/shadow/issues/324 - pom.withXml { xml -> - def dependenciesNode = xml.asNode().get('dependencies') ?: xml.asNode().appendNode('dependencies') - project.configurations.shadowed.allDependencies.each { - def dependencyNode = dependenciesNode.appendNode('dependency') - dependencyNode.appendNode('groupId', it.group) - dependencyNode.appendNode('artifactId', it.name) - dependencyNode.appendNode('version', it.version) - dependencyNode.appendNode('scope', 'runtime') - } - } - } - - afterEvaluate { - ["srcJar", "javadocJar", "scaladocJar", "testJar", "testSrcJar"].forEach { taskName -> - def task = tasks.findByName(taskName) - if (task != null) - artifact task - } - - artifactId = base.archivesName.get() - pom { - name = 'Apache Kafka' - url = 'https://kafka.apache.org' - licenses { - license { - name = 'The Apache License, Version 2.0' - url = 'http://www.apache.org/licenses/LICENSE-2.0.txt' - distribution = 'repo' - } - } - } - } - } - } - } - - if (shouldSign) { - signing { - sign publishing.publications.mavenJava - } - } - } - - def testLoggingEvents = ["passed", "skipped", "failed"] - def testShowStandardStreams = false - def testExceptionFormat = 'full' - // Gradle built-in logging only supports sending test output to stdout, which generates a lot - // of noise, especially for passing tests. We really only want output for failed tests. This - // hooks into the output and logs it (so we don't have to buffer it all in memory) and only - // saves the output for failing tests. Directory and filenames are such that you can, e.g., - // create a Jenkins rule to collect failed test output. - def logTestStdout = { - def testId = { TestDescriptor descriptor -> - "${descriptor.className}.${descriptor.name}".toString() - } - - def logFiles = new HashMap() - def logStreams = new HashMap() - beforeTest { TestDescriptor td -> - def tid = testId(td) - // truncate the file name if it's too long - def logFile = new File( - "${projectDir}/build/reports/testOutput/${tid.substring(0, Math.min(tid.size(),240))}.test.stdout" - ) - logFile.parentFile.mkdirs() - logFiles.put(tid, logFile) - logStreams.put(tid, new FileOutputStream(logFile)) - } - onOutput { TestDescriptor td, TestOutputEvent toe -> - def tid = testId(td) - // Some output can happen outside the context of a specific test (e.g. at the class level) - // and beforeTest/afterTest seems to not be invoked for these cases (and similarly, there's - // a TestDescriptor hierarchy that includes the thread executing the test, Gradle tasks, - // etc). We see some of these in practice and it seems like something buggy in the Gradle - // test runner since we see it *before* any tests and it is frequently not related to any - // code in the test (best guess is that it is tail output from last test). We won't have - // an output file for these, so simply ignore them. If they become critical for debugging, - // they can be seen with showStandardStreams. - if (td.name == td.className || td.className == null) { - // silently ignore output unrelated to specific test methods - return - } else if (logStreams.get(tid) == null) { - println "WARNING: unexpectedly got output for a test [${tid}]" + - " that we didn't previously see in the beforeTest hook." + - " Message for debugging: [" + toe.message + "]." - return - } - try { - logStreams.get(tid).write(toe.message.getBytes(StandardCharsets.UTF_8)) - } catch (Exception e) { - println "ERROR: Failed to write output for test ${tid}" - e.printStackTrace() - } - } - afterTest { TestDescriptor td, TestResult tr -> - def tid = testId(td) - try { - logStreams.get(tid).close() - if (tr.resultType != TestResult.ResultType.FAILURE) { - logFiles.get(tid).delete() - } else { - def file = logFiles.get(tid) - println "${tid} failed, log available in ${file}" - } - } catch (Exception e) { - println "ERROR: Failed to close stdout file for ${tid}" - e.printStackTrace() - } finally { - logFiles.remove(tid) - logStreams.remove(tid) - } - } - } - - // The suites are for running sets of tests in IDEs. - // Gradle will run each test class, so we exclude the suites to avoid redundantly running the tests twice. - def testsToExclude = ['**/*Suite.class'] - - - // This task will copy JUnit XML files out of the sub-project's build directory and into - // a top-level build/junit-xml directory. This is necessary to avoid reporting on tests which - // were not run, but instead were restored via FROM-CACHE. See KAFKA-17479 for more details. - def copyTestXml = tasks.register('copyTestXml') { - onlyIf("Environment GITHUB_ACTIONS is set") { isGithubActions } - onlyIf("Project '${project.name}:test' has sources") { ! test.state.noSource } - onlyIf("Task '${project.name}:test' did work") { test.state.didWork } - - ext { - output = project.findProperty("kafka.test.xml.output.dir") - } - - // Never cache this task - outputs.cacheIf { false } - outputs.upToDateWhen { false } - - doLast { - def moduleDirPath = projectToJUnitXmlPath(project) - def dest = rootProject.layout.buildDirectory.dir("junit-xml/${moduleDirPath}/${output}").get().asFile - println "Copy JUnit XML for ${project.name} to $dest" - ant.copy(todir: "$dest") { - ant.fileset(dir: "${test.reports.junitXml.entryPoint}") { - ant.include(name: "**/*.xml") - } - } - } - } - - test { - maxParallelForks = maxTestForks - ignoreFailures = userIgnoreFailures - - maxHeapSize = "3g" - jvmArgs = defaultJvmArgs - - // KAFKA-17433 Used by deflake.yml github action to repeat individual tests - systemProperty("kafka.cluster.test.repeat", project.findProperty("kafka.cluster.test.repeat")) - systemProperty("kafka.test.catalog.file", project.findProperty("kafka.test.catalog.file")) - systemProperty("kafka.test.run.new", project.findProperty("kafka.test.run.new")) - systemProperty("kafka.test.run.flaky", project.findProperty("kafka.test.run.flaky")) - systemProperty("kafka.test.verbose", project.findProperty("kafka.test.verbose")) - - testLogging { - events = userTestLoggingEvents ?: testLoggingEvents - showStandardStreams = userShowStandardStreams ?: testShowStandardStreams - exceptionFormat = testExceptionFormat - displayGranularity = 0 - } - logTestStdout.rehydrate(delegate, owner, this)() - - exclude testsToExclude - - useJUnitPlatform { - includeEngines 'junit-jupiter' - } - - develocity { - testRetry { - maxRetries = userMaxTestRetries - maxFailures = userMaxTestRetryFailures - } - } - - finalizedBy("copyTestXml") - } - - task integrationTest(type: Test, dependsOn: compileJava) { - maxParallelForks = maxTestForks - ignoreFailures = userIgnoreFailures - - // Increase heap size for integration tests - maxHeapSize = "2560m" - jvmArgs = defaultJvmArgs - - - testLogging { - events = userTestLoggingEvents ?: testLoggingEvents - showStandardStreams = userShowStandardStreams ?: testShowStandardStreams - exceptionFormat = testExceptionFormat - displayGranularity = 0 - } - logTestStdout.rehydrate(delegate, owner, this)() - - exclude testsToExclude - - useJUnitPlatform { - includeTags "integration" - includeEngines 'junit-jupiter' - } - - develocity { - testRetry { - maxRetries = userMaxTestRetries - maxFailures = userMaxTestRetryFailures - } - } - } - - task unitTest(type: Test, dependsOn: compileJava) { - maxParallelForks = maxTestForks - ignoreFailures = userIgnoreFailures - - maxHeapSize = defaultMaxHeapSize - jvmArgs = defaultJvmArgs - - testLogging { - events = userTestLoggingEvents ?: testLoggingEvents - showStandardStreams = userShowStandardStreams ?: testShowStandardStreams - exceptionFormat = testExceptionFormat - displayGranularity = 0 - } - logTestStdout.rehydrate(delegate, owner, this)() - - exclude testsToExclude - - useJUnitPlatform { - excludeTags "integration" - includeEngines 'junit-jupiter' - } - - develocity { - testRetry { - maxRetries = userMaxTestRetries - maxFailures = userMaxTestRetryFailures - } - } - } - - // remove test output from all test types - tasks.withType(Test).all { t -> - cleanTest { - delete t.reports.junitXml.outputLocation - delete t.reports.html.outputLocation - } - } - - jar { - from "$rootDir/LICENSE" - from "$rootDir/NOTICE" - } - - task srcJar(type: Jar) { - archiveClassifier = 'sources' - from "$rootDir/LICENSE" - from "$rootDir/NOTICE" - from sourceSets.main.allSource - } - - task javadocJar(type: Jar, dependsOn: javadoc) { - archiveClassifier = 'javadoc' - from "$rootDir/LICENSE" - from "$rootDir/NOTICE" - from javadoc.destinationDir - } - - task docsJar(dependsOn: javadocJar) - - check.dependsOn('javadoc') - - task systemTestLibs(dependsOn: jar) - - if (!sourceSets.test.allSource.isEmpty()) { - task testJar(type: Jar) { - archiveClassifier = 'test' - from "$rootDir/LICENSE" - from "$rootDir/NOTICE" - from sourceSets.test.output - // The junit-platform.properties file is used for configuring and customizing the behavior of the JUnit platform. - // It should only apply to Kafka's own JUnit tests, and should not exist in the test JAR. - // If we include it in the test JAR, it could lead to conflicts with user configurations. - exclude 'junit-platform.properties' - } - - task testSrcJar(type: Jar, dependsOn: testJar) { - archiveClassifier = 'test-sources' - from "$rootDir/LICENSE" - from "$rootDir/NOTICE" - from sourceSets.test.allSource - } - - } - - plugins.withType(ScalaPlugin) { - - scala { - zincVersion = versions.zinc - } - - task scaladocJar(type:Jar, dependsOn: scaladoc) { - archiveClassifier = 'scaladoc' - from "$rootDir/LICENSE" - from "$rootDir/NOTICE" - from scaladoc.destinationDir - } - - //documentation task should also trigger building scala doc jar - docsJar.dependsOn scaladocJar - - } - - tasks.withType(ScalaCompile) { - def releaseVersion = modulesNeedingJava11.any { project.path == it } ? minClientJavaVersion : minNonClientJavaVersion - scalaCompileOptions.keepAliveMode = userKeepAliveMode - - scalaCompileOptions.additionalParameters = [ - "-deprecation:false", - "-unchecked", - "-encoding", "utf8", - "-Xlog-reflective-calls", - "-feature", - "-language:postfixOps", - "-language:implicitConversions", - "-language:existentials", - "-Ybackend-parallelism", maxScalacThreads.toString(), - "-Xlint:constant", - "-Xlint:delayedinit-select", - "-Xlint:doc-detached", - "-Xlint:missing-interpolator", - "-Xlint:nullary-unit", - "-Xlint:option-implicit", - "-Xlint:package-object-classes", - "-Xlint:poly-implicit-overload", - "-Xlint:private-shadow", - "-Xlint:stars-align", - "-Xlint:type-parameter-shadow", - "-Xlint:unused" - ] - - // See README.md for details on this option and the meaning of each value - if (userScalaOptimizerMode.equals("method")) - scalaCompileOptions.additionalParameters += ["-opt:l:method"] - else if (userScalaOptimizerMode.startsWith("inline-")) { - List inlineFrom = ["-opt-inline-from:org.apache.kafka.**"] - if (project.name.equals('core')) - inlineFrom.add("-opt-inline-from:kafka.**") - if (userScalaOptimizerMode.equals("inline-scala")) - inlineFrom.add("-opt-inline-from:scala.**") - - scalaCompileOptions.additionalParameters += ["-opt:l:inline"] - scalaCompileOptions.additionalParameters += inlineFrom - } - - scalaCompileOptions.additionalParameters += ["-opt-warnings", "-Xlint:strict-unsealed-patmat"] - // Scala 2.13.2 introduces compiler warnings suppression, which is a pre-requisite for -Xfatal-warnings - scalaCompileOptions.additionalParameters += ["-Xfatal-warnings"] - scalaCompileOptions.additionalParameters += ["--release", String.valueOf(releaseVersion)] - - // Gradle does not support the `release` configuration when performing joint Java-Scala compilation. - // For more details, refer to https://github.com/gradle/gradle/issues/13762. - // As a result, we need to explicitly configure the Scala compiler with this setting. - options.compilerArgs += ["--release", String.valueOf(releaseVersion)] - - configureJavaCompiler(name, options, project.path) - - configure(scalaCompileOptions.forkOptions) { - memoryMaximumSize = defaultMaxHeapSize - jvmArgs = defaultJvmArgs - } - } - - checkstyle { - configDirectory = rootProject.layout.projectDirectory.dir("checkstyle") - configProperties = checkstyleConfigProperties("import-control.xml") - toolVersion = versions.checkstyle - } - - configure(checkstyleMain) { - group = 'Verification' - description = 'Run checkstyle on all main Java sources' - } - - configure(checkstyleTest) { - group = 'Verification' - description = 'Run checkstyle on all test Java sources' - } - - test.dependsOn('checkstyleMain', 'checkstyleTest') - - spotbugs { - toolVersion = versions.spotbugs - excludeFilter = file("$rootDir/gradle/spotbugs-exclude.xml") - ignoreFailures = false - } - test.dependsOn('spotbugsMain') - - tasks.withType(com.github.spotbugs.snom.SpotBugsTask).configureEach { - reports.configure { - // Continue supporting `xmlFindBugsReport` for compatibility - xml.enabled(project.hasProperty('xmlSpotBugsReport') || project.hasProperty('xmlFindBugsReport')) - html.enabled(!project.hasProperty('xmlSpotBugsReport') && !project.hasProperty('xmlFindBugsReport')) - } - maxHeapSize = defaultMaxHeapSize - jvmArgs = defaultJvmArgs - } - - // Ignore core since its a scala project - if (it.path != ':core') { - if (userEnableTestCoverage) { - apply plugin: "jacoco" - - jacoco { - toolVersion = versions.jacoco - } - - jacocoTestReport { - dependsOn tasks.test - sourceSets sourceSets.main - reports { - html.required = true - xml.required = true - csv.required = false - } - } - - } - } - - if (userEnableTestCoverage) { - def coverageGen = it.path == ':core' ? 'reportTestScoverage' : 'jacocoTestReport' - tasks.register('reportCoverage').configure { dependsOn(coverageGen) } - } - - dependencyCheck { - suppressionFile = "$rootDir/gradle/resources/dependencycheck-suppressions.xml" - skipProjects = [ ":jmh-benchmarks", ":trogdor" ] - skipConfigurations = [ "zinc" ] - } - apply plugin: 'com.diffplug.spotless' - spotless { - java { - targetExclude('**/generated/**/*.java','**/generated-test/**/*.java') - importOrder('kafka', 'org.apache.kafka', 'com', 'net', 'org', 'java', 'javax', '', '\\#') - removeUnusedImports() - } - } -} - -gradle.taskGraph.whenReady { taskGraph -> - taskGraph.getAllTasks().findAll { it.name.contains('spotbugsScoverage') || it.name.contains('spotbugsTest') }.each { task -> - task.enabled = false - } -} +apply from: "$rootDir/gradle/subprojects.gradle" def fineTuneEclipseClasspathFile(eclipse, project) { eclipse.classpath.file { @@ -863,7 +281,7 @@ def fineTuneEclipseClasspathFile(eclipse, project) { } } -def checkstyleConfigProperties(configFileName) { +static def checkstyleConfigProperties(configFileName) { [importControlFile: "$configFileName"] } @@ -2812,14 +2230,6 @@ project(':streams:streams-scala') { dependsOn 'copyDependantLibs' } - apply plugin: 'com.diffplug.spotless' - spotless { - scala { - target '**/*.scala' - scalafmt("$versions.scalafmt").configFile('../../checkstyle/.scalafmt.conf').scalaMajorVersion(versions.baseScala) - licenseHeaderFile '../../checkstyle/java.header', 'package' - } - } } project(':streams:integration-tests') { diff --git a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java index 6cfee84178021..ad58ad58d6a67 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/oauthbearer/internals/secured/OAuthBearerTest.java @@ -46,6 +46,7 @@ import java.net.URL; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; +import java.nio.file.Files; import java.nio.file.StandardOpenOption; import java.security.KeyPair; import java.security.KeyPairGenerator; @@ -236,7 +237,7 @@ protected JwtConsumer jwtConsumer(PublicKey publicKey) { } protected File generatePrivateKey(PrivateKey privateKey) throws IOException { - File file = File.createTempFile("private-", ".key"); + File file = Files.createTempFile("private-", ".key").toFile(); byte[] bytes = Base64.getEncoder().encode(privateKey.getEncoded()); try (FileChannel channel = FileChannel.open(file.toPath(), EnumSet.of(StandardOpenOption.WRITE))) { diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java index 6be0b4a377266..130e0184edd78 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java @@ -872,7 +872,7 @@ public void shouldConvertTimeValues() { // ISO8601 strings - accept a string matching pattern "HH:mm:ss.SSS'Z'" java.util.Date t3 = Values.convertToTime(Time.SCHEMA, localTime.format(DateTimeFormatter.ofPattern(Values.ISO_8601_TIME_FORMAT_PATTERN))); - LocalTime time3 = LocalDateTime.ofInstant(Instant.ofEpochMilli(t3.getTime()), ZoneId.systemDefault()).toLocalTime(); + LocalTime time3 = LocalTime.ofInstant(Instant.ofEpochMilli(t3.getTime()), ZoneId.systemDefault()); assertEquals(localTimeTruncated, time3); // Millis as string diff --git a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceTaskTest.java b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceTaskTest.java index c8118faf1589e..a96a8ec7dd9fa 100644 --- a/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceTaskTest.java +++ b/connect/file/src/test/java/org/apache/kafka/connect/file/FileStreamSourceTaskTest.java @@ -53,7 +53,7 @@ public class FileStreamSourceTaskTest { @BeforeEach public void setup() throws IOException { - tempFile = File.createTempFile("file-stream-source-task-test", null); + tempFile = Files.createTempFile("file-stream-source-task-test", null).toFile(); config = new HashMap<>(); config.put(FileStreamSourceConnector.FILE_CONFIG, tempFile.getAbsolutePath()); config.put(FileStreamSourceConnector.TOPIC_CONFIG, TOPIC); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java index 204fcc283bd94..100832028ca54 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java @@ -81,7 +81,7 @@ public void setup() { when(converter.toConnectData(anyString(), any(byte[].class))).thenReturn(new SchemaAndValue(null, Arrays.asList("connector", Collections.singletonMap("partitionKey", "dummy")))); store = new FileOffsetBackingStore(converter); - tempFile = assertDoesNotThrow(() -> File.createTempFile("fileoffsetbackingstore", null)); + tempFile = assertDoesNotThrow(() -> Files.createTempFile("fileoffsetbackingstore", null).toFile()); Map props = new HashMap<>(); props.put(StandaloneConfig.OFFSET_STORAGE_FILE_FILENAME_CONFIG, tempFile.getAbsolutePath()); props.put(StandaloneConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); diff --git a/gradle/.qa/checkstyle.gradle b/gradle/.qa/checkstyle.gradle new file mode 100644 index 0000000000000..fff781f79c713 --- /dev/null +++ b/gradle/.qa/checkstyle.gradle @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +project.apply plugin: "checkstyle" + +checkstyle { + configDirectory = rootProject.layout.projectDirectory.dir("checkstyle") + configProperties = [importControlFile: "import-control.xml"] + toolVersion = versions.checkstyle +} + +configure(checkstyleMain) { + group = 'Verification' + description = 'Run checkstyle on all main Java sources' +} + +configure(checkstyleTest) { + group = 'Verification' + description = 'Run checkstyle on all test Java sources' +} + +test.dependsOn(checkstyleMain, checkstyleTest) diff --git a/gradle/.qa/rewrite.gradle b/gradle/.qa/rewrite.gradle new file mode 100644 index 0000000000000..0a4b581d11ac1 --- /dev/null +++ b/gradle/.qa/rewrite.gradle @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +project.apply plugin: "org.openrewrite.rewrite" + +rewrite { + activeRecipe( + "tech.picnic.errorprone.refasterrules.BugCheckerRulesRecipes", + "tech.picnic.errorprone.refasterrules.FileRulesRecipes", + "tech.picnic.errorprone.refasterrules.MicrometerRulesRecipes", + "tech.picnic.errorprone.refasterrules.SuggestedFixRulesRecipes", + "tech.picnic.errorprone.refasterrules.TimeRulesRecipes", + //"org.apache.kafka.openrewrite.java.AddLicenseHeader", https://github.com/openrewrite/rewrite/issues/5762 + //"org.openrewrite.java.RemoveUnusedImports", + //"org.openrewrite.java.migrate.UpgradeToJava17", + //"org.openrewrite.java.recipes.JavaRecipeBestPractices", + //"org.openrewrite.java.recipes.RecipeNullabilityBestPractices", + //"org.openrewrite.java.recipes.RecipeTestingBestPractices", + //"org.openrewrite.staticanalysis.CodeCleanup", + //"org.openrewrite.staticanalysis.CommonStaticAnalysis", + //"org.openrewrite.staticanalysis.EqualsAvoidsNull", + //"org.openrewrite.staticanalysis.JavaApiBestPractices", + //"org.openrewrite.staticanalysis.LowercasePackage", + //"org.openrewrite.staticanalysis.MissingOverrideAnnotation", + //"org.openrewrite.staticanalysis.ModifierOrder", + //"org.openrewrite.staticanalysis.NoFinalizer", + //"org.openrewrite.staticanalysis.RemoveUnusedLocalVariables", + //"org.openrewrite.staticanalysis.RemoveUnusedPrivateFields", + //"org.openrewrite.staticanalysis.RemoveUnusedPrivateMethods", + //"org.openrewrite.staticanalysis.UnnecessaryThrows", + //"tech.picnic.errorprone.refasterrules.BigDecimalRulesRecipes", + //"tech.picnic.errorprone.refasterrules.CharSequenceRulesRecipes", + //"tech.picnic.errorprone.refasterrules.ClassRulesRecipes", + //"tech.picnic.errorprone.refasterrules.CollectionRulesRecipes", + //"tech.picnic.errorprone.refasterrules.ComparatorRulesRecipes", + //"tech.picnic.errorprone.refasterrules.EqualityRulesRecipes", + //"tech.picnic.errorprone.refasterrules.JUnitRulesRecipes", + //"tech.picnic.errorprone.refasterrules.NullRulesRecipes", + //"tech.picnic.errorprone.refasterrules.OptionalRulesRecipes", + //"tech.picnic.errorprone.refasterrules.PatternRulesRecipes", + //"tech.picnic.errorprone.refasterrules.PreconditionsRulesRecipes", + //"tech.picnic.errorprone.refasterrules.PrimitiveRulesRecipes", + //"tech.picnic.errorprone.refasterrules.StreamRulesRecipes", + //"tech.picnic.errorprone.refasterrules.StringRulesRecipes", + ) + exportDatatables = true + exclusions.add("**.GarbageCollectedMemoryPoolTest.java") + exclusions.add("**.MetadataVersionTest.java") + failOnDryRunResults = true +} + +dependencies { + //rewrite("org.openrewrite.recipe:rewrite-migrate-java") + //rewrite("org.openrewrite.recipe:rewrite-static-analysis") + rewrite("org.openrewrite.recipe:rewrite-third-party") + rewrite(platform("org.openrewrite.recipe:rewrite-recipe-bom:latest.release")) +} \ No newline at end of file diff --git a/gradle/spotbugs-exclude.xml b/gradle/.qa/spotbugs-exclude.xml similarity index 100% rename from gradle/spotbugs-exclude.xml rename to gradle/.qa/spotbugs-exclude.xml diff --git a/gradle/.qa/spotbugs.gradle b/gradle/.qa/spotbugs.gradle new file mode 100644 index 0000000000000..c5158e9de6c78 --- /dev/null +++ b/gradle/.qa/spotbugs.gradle @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +project.apply plugin: "com.github.spotbugs" + +spotbugs { + toolVersion = versions.spotbugs + excludeFilter = file("$rootDir/gradle/.qa/spotbugs-exclude.xml") + ignoreFailures = false +} + +test.dependsOn(spotbugsMain) + +gradle.taskGraph.whenReady { taskGraph -> + taskGraph.getAllTasks().findAll { it.name.contains('spotbugsScoverage') || it.name.contains('spotbugsTest') }.each { task -> + task.enabled = false + } +} \ No newline at end of file diff --git a/gradle/.qa/spotless.gradle b/gradle/.qa/spotless.gradle new file mode 100644 index 0000000000000..57c226828d759 --- /dev/null +++ b/gradle/.qa/spotless.gradle @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +project.apply plugin: "com.diffplug.spotless" + +spotless { + java { + targetExclude('**/generated/**/*.java', '**/generated-test/**/*.java') + importOrder('kafka', 'org.apache.kafka', 'com', 'net', 'org', 'java', 'javax', '', '\\#') + licenseHeaderFile("$rootDir/checkstyle/java.header", 'package') + } + scala { + scalafmt("$versions.scalafmt").configFile("$rootDir/checkstyle/.scalafmt.conf").scalaMajorVersion(versions.baseScala) + licenseHeaderFile("$rootDir/checkstyle/java.header", 'package') + } +} + +test.dependsOn(spotlessCheck) \ No newline at end of file diff --git a/metadata/src/main/java/org/apache/kafka/image/node/ProvenanceNode.java b/metadata/src/main/java/org/apache/kafka/image/node/ProvenanceNode.java index 6687be8d42dda..c68cf571e5f44 100644 --- a/metadata/src/main/java/org/apache/kafka/image/node/ProvenanceNode.java +++ b/metadata/src/main/java/org/apache/kafka/image/node/ProvenanceNode.java @@ -21,7 +21,7 @@ import org.apache.kafka.image.node.printer.MetadataNodePrinter; import java.time.Instant; -import java.time.ZoneId; +import java.time.ZoneOffset; import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; @@ -49,7 +49,7 @@ public boolean isDirectory() { @Override public void print(MetadataNodePrinter printer) { ZonedDateTime zonedDateTime = - Instant.ofEpochMilli(provenance.lastContainedLogTimeMs()).atZone(ZoneId.of("UTC")); + Instant.ofEpochMilli(provenance.lastContainedLogTimeMs()).atZone(ZoneOffset.UTC); printer.output("offset " + provenance.lastContainedOffset() + ", epoch " + provenance.lastContainedEpoch() + ", time " + DateTimeFormatter.ISO_ZONED_DATE_TIME.format(zonedDateTime)); diff --git a/settings.gradle b/settings.gradle index 906ea1f8f7d28..f75a6dac3973e 100644 --- a/settings.gradle +++ b/settings.gradle @@ -1,17 +1,19 @@ -// Licensed to the Apache Software Foundation (ASF) under one or more -// contributor license agreements. See the NOTICE file distributed with -// this work for additional information regarding copyright ownership. -// The ASF licenses this file to You under the Apache License, Version 2.0 -// (the "License"); you may not use this file except in compliance with -// the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ plugins { id 'com.gradle.develocity' version '3.19' diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/IQv2StoreIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/IQv2StoreIntegrationTest.java index 963183fd66598..0a0a6f46087d5 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/IQv2StoreIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/IQv2StoreIntegrationTest.java @@ -1469,8 +1469,8 @@ private void shouldHandleSessionKeyDSLQueries() { // Should fail to execute this query on a SessionStore. final WindowRangeQuery query = WindowRangeQuery.withWindowStartRange( - Instant.ofEpochMilli(0L), - Instant.ofEpochMilli(0L) + Instant.EPOCH, + Instant.EPOCH ); final StateQueryRequest, T>> request = @@ -1542,8 +1542,8 @@ private void shouldHandleSessionKeyPAPIQueries() { // Should fail to execute this query on a SessionStore. final WindowRangeQuery query = WindowRangeQuery.withWindowStartRange( - Instant.ofEpochMilli(0L), - Instant.ofEpochMilli(0L) + Instant.EPOCH, + Instant.EPOCH ); final StateQueryRequest, T>> request = diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java index 4406f292205f8..70539590f6af6 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/ProcessingExceptionHandlerIntegrationTest.java @@ -102,7 +102,7 @@ public void shouldFailWhenProcessingExceptionOccursIfExceptionHandlerReturnsFail final Properties properties = new Properties(); properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, FailProcessingExceptionHandlerMockTest.class); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.EPOCH)) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); final StreamsException exception = assertThrows(StreamsException.class, @@ -150,7 +150,7 @@ public void shouldFailWhenProcessingExceptionOccursFromFlushingCacheIfExceptionH final Properties properties = new Properties(); properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, LogAndFailProcessingExceptionHandler.class); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.EPOCH)) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); final StreamsException exception = assertThrows(StreamsException.class, @@ -198,7 +198,7 @@ public void shouldContinueWhenProcessingExceptionOccursIfExceptionHandlerReturns final Properties properties = new Properties(); properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ContinueProcessingExceptionHandlerMockTest.class); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.EPOCH)) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); inputTopic.pipeKeyValueList(events, TIMESTAMP, Duration.ZERO); @@ -242,7 +242,7 @@ public void shouldContinueWhenProcessingExceptionOccursFromFlushingCacheIfExcept final Properties properties = new Properties(); properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, LogAndContinueProcessingExceptionHandler.class); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.EPOCH)) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); inputTopic.pipeKeyValueList(events, TIMESTAMP, Duration.ZERO); @@ -279,7 +279,7 @@ public void shouldStopOnFailedProcessorWhenProcessingExceptionOccursInFailProces final Properties properties = new Properties(); properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, FailProcessingExceptionHandlerMockTest.class); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.EPOCH)) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); isExecuted.set(false); inputTopic.pipeInput(event.key, event.value, TIMESTAMP); @@ -315,7 +315,7 @@ public void shouldStopOnFailedProcessorWhenProcessingExceptionOccursInContinuePr final Properties properties = new Properties(); properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ContinueProcessingExceptionHandlerMockTest.class); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.EPOCH)) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); isExecuted.set(false); inputTopic.pipeInput(event.key, event.value, TIMESTAMP); @@ -348,7 +348,7 @@ public void shouldStopProcessingWhenProcessingExceptionHandlerReturnsNull() { final Properties properties = new Properties(); properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ContinueProcessingExceptionHandlerMockTest.class); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.EPOCH)) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); isExecuted.set(false); inputTopic.pipeInput(event.key, event.value, TIMESTAMP); @@ -384,7 +384,7 @@ public void shouldStopProcessingWhenFatalUserExceptionProcessingExceptionHandler final Properties properties = new Properties(); properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, ContinueProcessingExceptionHandlerMockTest.class); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.ofEpochMilli(0L))) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), properties, Instant.EPOCH)) { final TestInputTopic inputTopic = driver.createInputTopic("TOPIC_NAME", new StringSerializer(), new StringSerializer()); isExecuted.set(false); inputTopic.pipeInput(event.key, event.value, TIMESTAMP); @@ -506,7 +506,7 @@ public void shouldVerifySourceRawKeyAndSourceRawValuePresentOrNotInErrorHandlerC properties.put(StreamsConfig.PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG, AssertSourceRawRecordProcessingExceptionHandlerMockTest.class); - try (final TopologyTestDriver driver = new TopologyTestDriver(topology, properties, Instant.ofEpochMilli(0L))) { + try (final TopologyTestDriver driver = new TopologyTestDriver(topology, properties, Instant.EPOCH)) { for (final ProducerRecord event : events) { final TestInputTopic inputTopic = driver.createInputTopic(event.topic(), new StringSerializer(), new StringSerializer()); diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java index 2eb35e826a0a1..a1b3c6dda3116 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java @@ -80,6 +80,7 @@ import java.io.PrintStream; import java.io.StringReader; import java.time.Duration; +import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -1237,7 +1238,7 @@ private void waitUntilAtLeastNumRecordProcessed(final String topic, private Set> fetch(final ReadOnlyWindowStore store, final String key) { try (final WindowStoreIterator fetch = - store.fetch(key, ofEpochMilli(0), ofEpochMilli(System.currentTimeMillis()))) { + store.fetch(key, Instant.EPOCH, ofEpochMilli(System.currentTimeMillis()))) { if (fetch.hasNext()) { final KeyValue next = fetch.next(); return Collections.singleton(KeyValue.pair(key, next.value)); diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/TimeWindowedKStreamIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/TimeWindowedKStreamIntegrationTest.java index 53746cd7fd640..e0a9dca62d760 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/TimeWindowedKStreamIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/TimeWindowedKStreamIntegrationTest.java @@ -61,6 +61,7 @@ import java.io.IOException; import java.time.Duration; +import java.time.Instant; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -421,7 +422,7 @@ public void shouldThrowUnlimitedWindows(final StrategyType type) { final TimeWindowedKStream windowedStream = builder.stream(streamOneInput, Consumed.with(Serdes.String(), Serdes.String())) .groupByKey() .windowedBy( - UnlimitedWindows.of().startOn(ofEpochMilli(0)) + UnlimitedWindows.of().startOn(Instant.EPOCH) ); final boolean emitFinal = type == StrategyType.ON_WINDOW_CLOSE; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java index 6ad61851ddae3..b5e970add9faa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/assignment/assignors/StickyTaskAssignor.java @@ -83,7 +83,7 @@ public TaskAssignment assign(final ApplicationState applicationState) { // We set the followup deadline for only one of the clients. final ProcessId clientId = finalAssignments.entrySet().iterator().next().getKey(); final KafkaStreamsAssignment previousAssignment = finalAssignments.get(clientId); - finalAssignments.put(clientId, previousAssignment.withFollowupRebalance(Instant.ofEpochMilli(0))); + finalAssignments.put(clientId, previousAssignment.withFollowupRebalance(Instant.EPOCH)); } return new TaskAssignment(finalAssignments.values()); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java index 6afb4d1531d44..0d9c78079fd3b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredVersionedKeyValueStore.java @@ -250,7 +250,7 @@ private QueryResult runMultiVersionedKeyQuery(final Query query, final final Instant fromTime = typedKeyQuery.fromTime().orElse(Instant.ofEpochMilli(Long.MIN_VALUE)); final Instant toTime = typedKeyQuery.toTime().orElse(Instant.ofEpochMilli(Long.MAX_VALUE)); - if (fromTime.compareTo(toTime) > 0) { + if (fromTime.isAfter(toTime)) { throw new IllegalArgumentException("The `fromTime` timestamp must be smaller than the `toTime` timestamp."); } MultiVersionedKeyQuery rawKeyQuery = MultiVersionedKeyQuery.withKey(keyBytes(typedKeyQuery.key())); diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java index b839df53167e2..e3e96133c9575 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java @@ -348,7 +348,7 @@ public void shouldProcessingFromSinkTopic() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = - driver.createInputTopic("topic-source", new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic("topic-source", new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic.pipeInput("A", "aa"); } @@ -370,7 +370,7 @@ public void shouldProcessViaRepartitionTopic() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = - driver.createInputTopic("topic-source", new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic("topic-source", new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic.pipeInput("A", "aa"); } @@ -392,9 +392,9 @@ public void shouldMergeStreams() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic1.pipeInput("A", "aa"); inputTopic2.pipeInput("B", "bb"); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/MaterializedTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/MaterializedTest.java index 93726825fa882..e0e6314b28939 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/MaterializedTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/MaterializedTest.java @@ -26,7 +26,6 @@ import org.junit.jupiter.api.Test; import java.time.Duration; -import java.time.temporal.ChronoUnit; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -85,7 +84,7 @@ public void shouldThrowNullPointerIfSessionBytesStoreSupplierIsNull() { @Test public void shouldThrowIllegalArgumentExceptionIfRetentionIsNegative() { final IllegalArgumentException e = assertThrows(IllegalArgumentException.class, - () -> Materialized.as("valid-name").withRetention(Duration.of(-1, ChronoUnit.DAYS))); + () -> Materialized.as("valid-name").withRetention(Duration.ofDays(-1))); assertEquals(e.getMessage(), "Retention must not be negative."); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java index 523fa05f2ab47..f64f605cc5316 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java @@ -69,7 +69,7 @@ public void testFlatMap() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = - driver.createInputTopic(topicName, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0), Duration.ZERO); + driver.createInputTopic(topicName, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); for (final int expectedKey : expectedKeys) { inputTopic.pipeInput(expectedKey, "V" + expectedKey); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java index 115e2b94359a3..e3ebcb74b11fe 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java @@ -62,7 +62,7 @@ public void testFlatMapValues() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = - driver.createInputTopic(topicName, new IntegerSerializer(), new IntegerSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topicName, new IntegerSerializer(), new IntegerSerializer(), Instant.EPOCH, Duration.ZERO); for (final int expectedKey : expectedKeys) { // passing the timestamp to inputTopic.create to disambiguate the call inputTopic.pipeInput(expectedKey, expectedKey, 0L); @@ -99,7 +99,7 @@ public void testFlatMapValuesWithKeys() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = - driver.createInputTopic(topicName, new IntegerSerializer(), new IntegerSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topicName, new IntegerSerializer(), new IntegerSerializer(), Instant.EPOCH, Duration.ZERO); for (final int expectedKey : expectedKeys) { // passing the timestamp to inputTopic.create to disambiguate the call inputTopic.pipeInput(expectedKey, expectedKey, 0L); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java index 6085bdacf68f8..5427eaddc3ed7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java @@ -106,7 +106,7 @@ private void init(final Optional versionedStoreHistoryRetentionMs) { processor = supplier.theCapturedProcessor(); // auto-advance stream timestamps by default, but not global table timestamps - inputStreamTopic = driver.createInputTopic(streamTopic, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ofMillis(1L)); + inputStreamTopic = driver.createInputTopic(streamTopic, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ofMillis(1L)); inputTableTopic = driver.createInputTopic(globalTableTopic, new StringSerializer(), new StringSerializer()); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java index 6f49917a012f9..92e92459e1d5a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java @@ -106,8 +106,8 @@ private void init(final Optional versionedStoreHistoryRetentionMs) { processor = supplier.theCapturedProcessor(); // auto-advance timestamps by default - inputStreamTopic = driver.createInputTopic(streamTopic, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ofMillis(1L)); - inputTableTopic = driver.createInputTopic(globalTableTopic, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ofMillis(1L)); + inputStreamTopic = driver.createInputTopic(streamTopic, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ofMillis(1L)); + inputTableTopic = driver.createInputTopic(globalTableTopic, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ofMillis(1L)); } @AfterEach diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java index 1794a98bd319f..a0b4ce99d2888 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -1234,7 +1234,7 @@ public void shouldSendDataThroughRepartitionTopicUsingRepartitioned() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = - driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic.pipeInput("a", "b"); } assertThat(processorSupplier.theCapturedProcessor().processed(), equalTo(Collections.singletonList(new KeyValueTimestamp<>("a", "b", 0)))); @@ -1250,7 +1250,7 @@ public void shouldSendDataToTopicUsingProduced() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = - driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic.pipeInput("e", "f"); } assertThat(processorSupplier.theCapturedProcessor().processed(), equalTo(Collections.singletonList(new KeyValueTimestamp<>("e", "f", 0)))); @@ -1268,7 +1268,7 @@ public void shouldSendDataToDynamicTopics() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = - driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic.pipeInput("a", "v1"); inputTopic.pipeInput("a", "v2"); inputTopic.pipeInput("b", "v1"); @@ -1370,9 +1370,9 @@ public void shouldMergeTwoStreams() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic1.pipeInput("A", "aa"); inputTopic2.pipeInput("B", "bb"); inputTopic2.pipeInput("C", "cc"); @@ -1402,13 +1402,13 @@ public void shouldMergeMultipleStreams() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic3 = - driver.createInputTopic(topic3, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic3, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic4 = - driver.createInputTopic(topic4, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic4, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic1.pipeInput("A", "aa", 1L); inputTopic2.pipeInput("B", "bb", 9L); @@ -1439,15 +1439,15 @@ public void shouldProcessFromSourceThatMatchPattern() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic3 = - driver.createInputTopic("topic-3", new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic("topic-3", new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic4 = - driver.createInputTopic("topic-4", new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic("topic-4", new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic5 = - driver.createInputTopic("topic-5", new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic("topic-5", new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic6 = - driver.createInputTopic("topic-6", new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic("topic-6", new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic7 = - driver.createInputTopic("topic-7", new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic("topic-7", new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic3.pipeInput("A", "aa", 1L); inputTopic4.pipeInput("B", "bb", 5L); @@ -1477,15 +1477,15 @@ public void shouldProcessFromSourcesThatMatchMultiplePattern() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic3 = - driver.createInputTopic("topic-3", new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic("topic-3", new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic4 = - driver.createInputTopic("topic-4", new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic("topic-4", new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopicA = - driver.createInputTopic("topic-A", new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic("topic-A", new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopicZ = - driver.createInputTopic("topic-Z", new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic("topic-Z", new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic = - driver.createInputTopic(topic3, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic3, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic3.pipeInput("A", "aa", 1L); inputTopic4.pipeInput("B", "bb", 5L); @@ -1801,8 +1801,8 @@ public void shouldNotMaterializedKTableFromKStream() { final List> outputExpectRecords = new ArrayList<>(); outputExpectRecords.add(new TestRecord<>("A", "01", Instant.ofEpochMilli(5L))); outputExpectRecords.add(new TestRecord<>("B", "02", Instant.ofEpochMilli(100L))); - outputExpectRecords.add(new TestRecord<>("C", "03", Instant.ofEpochMilli(0L))); - outputExpectRecords.add(new TestRecord<>("D", "04", Instant.ofEpochMilli(0L))); + outputExpectRecords.add(new TestRecord<>("C", "03", Instant.EPOCH)); + outputExpectRecords.add(new TestRecord<>("D", "04", Instant.EPOCH)); outputExpectRecords.add(new TestRecord<>("A", "05", Instant.ofEpochMilli(10L))); outputExpectRecords.add(new TestRecord<>("A", "06", Instant.ofEpochMilli(8L))); @@ -2032,8 +2032,8 @@ public void process(final FixedKeyRecord record) { final List> outputExpectRecords = new ArrayList<>(); outputExpectRecords.add(new TestRecord<>("A", 1, Instant.ofEpochMilli(5L))); outputExpectRecords.add(new TestRecord<>("B", 2, Instant.ofEpochMilli(100L))); - outputExpectRecords.add(new TestRecord<>("C", 3, Instant.ofEpochMilli(0L))); - outputExpectRecords.add(new TestRecord<>("D", 4, Instant.ofEpochMilli(0L))); + outputExpectRecords.add(new TestRecord<>("C", 3, Instant.EPOCH)); + outputExpectRecords.add(new TestRecord<>("D", 4, Instant.EPOCH)); outputExpectRecords.add(new TestRecord<>("A", 5, Instant.ofEpochMilli(10L))); outputExpectRecords.add(new TestRecord<>("A", 6, Instant.ofEpochMilli(8L))); @@ -2141,8 +2141,8 @@ public void shouldSupportKeyChangeKTableFromKStream() { final List> outputExpectRecords = new ArrayList<>(); outputExpectRecords.add(new TestRecord<>(0, "01", Instant.ofEpochMilli(5L))); outputExpectRecords.add(new TestRecord<>(1, "02", Instant.ofEpochMilli(100L))); - outputExpectRecords.add(new TestRecord<>(2, "03", Instant.ofEpochMilli(0L))); - outputExpectRecords.add(new TestRecord<>(3, "04", Instant.ofEpochMilli(0L))); + outputExpectRecords.add(new TestRecord<>(2, "03", Instant.EPOCH)); + outputExpectRecords.add(new TestRecord<>(3, "04", Instant.EPOCH)); outputExpectRecords.add(new TestRecord<>(0, "05", Instant.ofEpochMilli(10L))); outputExpectRecords.add(new TestRecord<>(0, "06", Instant.ofEpochMilli(8L))); @@ -2475,7 +2475,7 @@ public void shouldSupportGroupByCountWithKStreamToKTable() { try ( final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = - driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestOutputTopic outputTopic = driver.createOutputTopic(output, new StringDeserializer(), new LongDeserializer()); @@ -2540,7 +2540,7 @@ public void shouldSupportTriggerMaterializedWithKTableFromKStream() { try ( final TopologyTestDriver driver = new TopologyTestDriver(topology, props)) { final TestInputTopic inputTopic = - driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestOutputTopic outputTopic = driver.createOutputTopic(output, new StringDeserializer(), new IntegerDeserializer()); final KeyValueStore store = driver.getKeyValueStore(storeName); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java index b7fb8669697f6..826c699b32c9b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java @@ -541,9 +541,9 @@ private void runJoin(final StreamJoined streamJoined, try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopicLeft = - driver.createInputTopic("left", new StringSerializer(), new IntegerSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic("left", new StringSerializer(), new IntegerSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopicRight = - driver.createInputTopic("right", new StringSerializer(), new IntegerSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic("right", new StringSerializer(), new IntegerSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); inputTopicLeft.pipeInput("A", 1, 1L); @@ -587,9 +587,9 @@ public void testJoin() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); // push two items to the primary stream; the other window is empty @@ -709,9 +709,9 @@ public void testOuterJoin() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); // push two items to the primary stream; the other window is empty; this should not produce items yet @@ -832,9 +832,9 @@ public void testWindowing() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); long time = 0L; @@ -1397,9 +1397,9 @@ public void testAsymmetricWindowingAfter() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); long time = 1000L; @@ -1664,9 +1664,9 @@ public void testAsymmetricWindowingBefore() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); long time = 1000L; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java index 74ebaa0e844ab..3ad786ab33898 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java @@ -79,9 +79,9 @@ public void testLeftJoinWithSpuriousResultFixDisabledOldApi() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(PROPS), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); // Only 2 window stores should be available @@ -137,9 +137,9 @@ public void testLeftJoinDuplicatesWithSpuriousResultFixDisabledOldApi() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(PROPS), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); // Only 2 window stores should be available @@ -179,9 +179,9 @@ public void testLeftJoinDuplicates() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); // verifies non-joined duplicates are emitted when window has closed @@ -235,9 +235,9 @@ public void testLeftExpiredNonJoinedRecordsAreEmittedByTheLeftProcessor() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); final long windowStart = 0L; @@ -290,9 +290,9 @@ public void testLeftExpiredNonJoinedRecordsAreEmittedByTheRightProcessor() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); final long windowStart = 0L; @@ -345,9 +345,9 @@ public void testRightNonJoinedRecordsAreNeverEmittedByTheLeftProcessor() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); final long windowStart = 0L; @@ -397,9 +397,9 @@ public void testRightNonJoinedRecordsAreNeverEmittedByTheRightProcessor() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); final long windowStart = 0L; @@ -459,9 +459,9 @@ public void testLeftJoinedRecordsWithZeroAfterAreEmitted() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); processor.init(null); @@ -660,9 +660,9 @@ public void runLeftJoin(final StreamJoined streamJoined try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); // 2 window stores + 1 shared window store should be available @@ -765,9 +765,9 @@ public void testOrdering() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); // push two items to the primary stream; the other window is empty; this should not produce any item yet @@ -823,9 +823,9 @@ public void testGracePeriod() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); // push two items to the primary stream; the other window is empty; this should not produce items because window has not closed @@ -895,9 +895,9 @@ public void testWindowing() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); final long time = 0L; @@ -950,9 +950,9 @@ public void shouldNotEmitLeftJoinResultForAsymmetricWindow() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); long time = 0L; @@ -998,9 +998,9 @@ private void testUpperWindowBound(final int[] expectedKeys, long time; final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); // push four items with larger and increasing timestamp (out of window) to the other stream; this should produce no items // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java index 5fc4e0a418b80..01c93e1177ca8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java @@ -88,9 +88,9 @@ public void testOuterJoinDuplicatesWithFixDisabledOldApi() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(PROPS), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); // Only 2 window stores should be available @@ -132,9 +132,9 @@ public void testOuterJoinDuplicates() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); // verifies non-joined duplicates are emitted when window has closed @@ -198,9 +198,9 @@ public void testLeftExpiredNonJoinedRecordsAreEmittedByTheLeftProcessor() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); final long windowStart = 0L; @@ -253,9 +253,9 @@ public void testLeftExpiredNonJoinedRecordsAreEmittedByTheRightProcessor() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); final long windowStart = 0L; @@ -308,9 +308,9 @@ public void testRightExpiredNonJoinedRecordsAreEmittedByTheLeftProcessor() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); final long windowStart = 0L; @@ -363,9 +363,9 @@ public void testRightExpiredNonJoinedRecordsAreEmittedByTheRightProcessor() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); final long windowStart = 0L; @@ -418,9 +418,9 @@ public void testOrdering() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); // push two items to the primary stream; the other window is empty; this should not produce any item yet @@ -474,9 +474,9 @@ public void testGracePeriod() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); // push one item to the primary stream; and one item in other stream; this should not produce items because there are no joins @@ -540,9 +540,9 @@ public void testEmitAllNonJoinedResultsForAsymmetricWindow() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); // push one item to the primary stream; this should not produce any items because there are no matching keys @@ -668,9 +668,9 @@ public void runOuterJoin(final StreamJoined streamJoined, try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); // 2 window stores + 1 shared window store should be available @@ -781,9 +781,9 @@ public void testWindowing() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); final long time = 0L; @@ -837,9 +837,9 @@ public void testShouldNotEmitLeftJoinResultForAsymmetricBeforeWindow() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); long time = 0L; @@ -924,9 +924,9 @@ public void testShouldNotEmitLeftJoinResultForAsymmetricAfterWindow() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); long time = 0L; @@ -1015,9 +1015,9 @@ public void testShouldForwardCurrentHeaders() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor processor = supplier.theCapturedProcessor(); inputTopic1.pipeInput(new TestRecord<>( @@ -1088,9 +1088,9 @@ private void testUpperWindowBound(final int[] expectedKeys, long time; final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, new IntegerSerializer(), new LongSerializer(), Instant.EPOCH, Duration.ZERO); // push four items with larger and increasing timestamp (out of window) to the other stream; this should produced 2 expired non-joined records // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java index 19a8b5e5205cb..bbc45e26f3237 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java @@ -86,8 +86,8 @@ public void setUp() { stream.join(table, MockValueJoiner.TOSTRING_JOINER).process(supplier); final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.String()); driver = new TopologyTestDriver(builder.build(), props); - inputStreamTopic = driver.createInputTopic(streamTopic, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - inputTableTopic = driver.createInputTopic(tableTopic, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + inputStreamTopic = driver.createInputTopic(streamTopic, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); + inputTableTopic = driver.createInputTopic(tableTopic, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); processor = supplier.theCapturedProcessor(); } @@ -145,8 +145,8 @@ private void makeJoin(final Duration grace) { ).process(supplier); final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.String()); driver = new TopologyTestDriver(builder.build(), props); - inputStreamTopic = driver.createInputTopic(streamTopic, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - inputTableTopic = driver.createInputTopic("tableTopic2", new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + inputStreamTopic = driver.createInputTopic(streamTopic, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); + inputTableTopic = driver.createInputTopic("tableTopic2", new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); processor = supplier.theCapturedProcessor(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java index f78432b95e724..1c0827dfcc326 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java @@ -78,8 +78,8 @@ public void setUp() { final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.String()); driver = new TopologyTestDriver(builder.build(), props); - inputStreamTopic = driver.createInputTopic(streamTopic, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); - inputTableTopic = driver.createInputTopic(tableTopic, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + inputStreamTopic = driver.createInputTopic(streamTopic, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); + inputTableTopic = driver.createInputTopic(tableTopic, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); processor = supplier.theCapturedProcessor(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java index 41b17c9de9593..3b95a996e280e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java @@ -57,7 +57,7 @@ public void testMap() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { for (final int expectedKey : expectedKeys) { final TestInputTopic inputTopic = - driver.createInputTopic(topicName, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topicName, new IntegerSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic.pipeInput(expectedKey, "V" + expectedKey, 10L - expectedKey); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java index 5ae92a681cd1e..2506007359ca5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java @@ -63,7 +63,7 @@ public void testSelectKey() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = - driver.createInputTopic(topicName, new StringSerializer(), new IntegerSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topicName, new StringSerializer(), new IntegerSerializer(), Instant.EPOCH, Duration.ZERO); for (final int expectedValue : expectedValues) { inputTopic.pipeInput(expectedValue); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java index 2ad5d36893b8f..1f468018d9c55 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java @@ -92,9 +92,9 @@ public void testAggBasic() { try ( final TopologyTestDriver driver = new TopologyTestDriver( - builder.build(), CONFIG, Instant.ofEpochMilli(0L))) { + builder.build(), CONFIG, Instant.EPOCH)) { final TestInputTopic inputTopic = - driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic.pipeInput("A", "1", 10L); inputTopic.pipeInput("B", "2", 15L); @@ -149,9 +149,9 @@ public void testAggRepartition() { try ( final TopologyTestDriver driver = new TopologyTestDriver( - builder.build(), CONFIG, Instant.ofEpochMilli(0L))) { + builder.build(), CONFIG, Instant.EPOCH)) { final TestInputTopic inputTopic = - driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic.pipeInput("A", "1", 10L); inputTopic.pipeInput("A", (String) null, 15L); @@ -208,9 +208,9 @@ public void testAggOfVersionedStore() { try ( final TopologyTestDriver driver = new TopologyTestDriver( - builder.build(), CONFIG, Instant.ofEpochMilli(0L))) { + builder.build(), CONFIG, Instant.EPOCH)) { final TestInputTopic inputTopic = - driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic.pipeInput("A", "1", 10L); inputTopic.pipeInput("A", (String) null, 15L); @@ -237,9 +237,9 @@ private static void testCountHelper(final StreamsBuilder builder, final MockApiProcessorSupplier supplier) { try ( final TopologyTestDriver driver = new TopologyTestDriver( - builder.build(), CONFIG, Instant.ofEpochMilli(0L))) { + builder.build(), CONFIG, Instant.EPOCH)) { final TestInputTopic inputTopic = - driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic.pipeInput("A", "green", 10L); inputTopic.pipeInput("B", "green", 9L); @@ -306,9 +306,9 @@ public void testCountOfVersionedStore() { try ( final TopologyTestDriver driver = new TopologyTestDriver( - builder.build(), CONFIG, Instant.ofEpochMilli(0L))) { + builder.build(), CONFIG, Instant.EPOCH)) { final TestInputTopic inputTopic = - driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic.pipeInput("A", "green", 10L); inputTopic.pipeInput("B", "green", 9L); @@ -353,9 +353,9 @@ public void testRemoveOldBeforeAddNew() { try ( final TopologyTestDriver driver = new TopologyTestDriver( - builder.build(), CONFIG, Instant.ofEpochMilli(0L))) { + builder.build(), CONFIG, Instant.EPOCH)) { final TestInputTopic inputTopic = - driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor proc = supplier.theCapturedProcessor(); @@ -390,9 +390,9 @@ private void testUpgradeFromConfig(final Properties config, final List inputTopic = - driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(input, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final TestOutputTopic outputTopic = driver.createOutputTopic(output, new StringDeserializer(), new LongDeserializer()); @@ -472,9 +472,9 @@ private void testKeyWithNoEquals( .toStream() .to(output); - try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), CONFIG, Instant.ofEpochMilli(0L))) { + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), CONFIG, Instant.EPOCH)) { final TestInputTopic inputTopic = - driver.createInputTopic(input, noEqualsImplSerde.serializer(), noEqualsImplSerde.serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(input, noEqualsImplSerde.serializer(), noEqualsImplSerde.serializer(), Instant.EPOCH, Duration.ZERO); final TestOutputTopic outputTopic = driver.createOutputTopic(output, noEqualsImplSerde.deserializer(), new LongDeserializer()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java index 278b18da8b52e..a82e8f8709f4d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java @@ -78,7 +78,7 @@ private void doTestKTable(final StreamsBuilder builder, try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = - driver.createInputTopic(topic, new StringSerializer(), new IntegerSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic, new StringSerializer(), new IntegerSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic.pipeInput("A", 1, 10L); inputTopic.pipeInput("B", 2, 5L); inputTopic.pipeInput("C", 3, 8L); @@ -151,7 +151,7 @@ private void doTestValueGetter(final StreamsBuilder builder, try (final TopologyTestDriverWrapper driver = new TopologyTestDriverWrapper(topology, props)) { final TestInputTopic inputTopic = - driver.createInputTopic(topic1, new StringSerializer(), new IntegerSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new StringSerializer(), new IntegerSerializer(), Instant.EPOCH, Duration.ZERO); final KTableValueGetter getter2 = getterSupplier2.get(); final KTableValueGetter getter3 = getterSupplier3.get(); @@ -238,7 +238,7 @@ private void doTestNotSendingOldValue(final StreamsBuilder builder, try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic = - driver.createInputTopic(topic1, new StringSerializer(), new IntegerSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new StringSerializer(), new IntegerSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic.pipeInput("A", 1, 5L); inputTopic.pipeInput("B", 1, 10L); @@ -329,7 +329,7 @@ private void doTestSendingOldValue(final StreamsBuilder builder, try (final TopologyTestDriver driver = new TopologyTestDriver(topology, props)) { final TestInputTopic inputTopic = - driver.createInputTopic(topic1, new StringSerializer(), new IntegerSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new StringSerializer(), new IntegerSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic.pipeInput("A", 1, 5L); inputTopic.pipeInput("B", 1, 10L); @@ -447,7 +447,7 @@ private void doTestSkipNullOnMaterialization(final StreamsBuilder builder, try (final TopologyTestDriver driver = new TopologyTestDriver(topology, props)) { final TestInputTopic stringinputTopic = - driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); stringinputTopic.pipeInput("A", "reject", 5L); stringinputTopic.pipeInput("B", "reject", 10L); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java index 9d1e7adec95e8..40d5ad74e9086 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java @@ -157,9 +157,9 @@ public void testSendingOldValues() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor proc = supplier.theCapturedProcessor(); assertTrue(((KTableImpl) table1).sendingOldValueEnabled()); @@ -279,9 +279,9 @@ private void doTestNotSendingOldValues(final StreamsBuilder builder, try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor proc = supplier.theCapturedProcessor(); assertFalse(((KTableImpl) table1).sendingOldValueEnabled()); @@ -376,9 +376,9 @@ private void doTestJoin(final StreamsBuilder builder, final int[] expectedKeys) try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final TestOutputTopic outputTopic = driver.createOutputTopic(output, Serdes.Integer().deserializer(), Serdes.String().deserializer()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java index 87a38cc7a448d..02afeb53949d7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java @@ -95,9 +95,9 @@ public void testJoin() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final TestOutputTopic outputTopic = driver.createOutputTopic(output, Serdes.Integer().deserializer(), Serdes.String().deserializer()); @@ -216,9 +216,9 @@ public void testNotSendingOldValue() { try (final TopologyTestDriver driver = new TopologyTestDriver(topology, props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor proc = supplier.theCapturedProcessor(); assertTrue(((KTableImpl) table1).sendingOldValueEnabled()); @@ -338,9 +338,9 @@ public void testSendingOldValue() { try (final TopologyTestDriver driver = new TopologyTestDriverWrapper(topology, props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor proc = supplier.theCapturedProcessor(); assertTrue(((KTableImpl) table1).sendingOldValueEnabled()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java index 080cdbd82d142..de0735a9d21b8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java @@ -81,9 +81,9 @@ public void testJoin() { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final TestOutputTopic outputTopic = driver.createOutputTopic(output, Serdes.Integer().deserializer(), Serdes.String().deserializer()); // push two items to the primary stream. the other table is empty @@ -201,9 +201,9 @@ public void testNotSendingOldValue() { try (final TopologyTestDriver driver = new TopologyTestDriver(topology, props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor proc = supplier.theCapturedProcessor(); assertTrue(((KTableImpl) table1).sendingOldValueEnabled()); @@ -312,9 +312,9 @@ public void testSendingOldValue() { try (final TopologyTestDriver driver = new TopologyTestDriver(topology, props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final TestInputTopic inputTopic2 = - driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic2, Serdes.Integer().serializer(), Serdes.String().serializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor proc = supplier.theCapturedProcessor(); assertTrue(((KTableImpl) table1).sendingOldValueEnabled()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java index ffc95436495db..4167761ceeaa0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java @@ -59,7 +59,7 @@ private void doTestKTable(final StreamsBuilder builder, final MockApiProcessorSupplier supplier) { try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); inputTopic1.pipeInput("A", "1", 5L); inputTopic1.pipeInput("B", "2", 25L); inputTopic1.pipeInput("C", "3", 20L); @@ -119,7 +119,7 @@ private void doTestValueGetter(final StreamsBuilder builder, try (final TopologyTestDriverWrapper driver = new TopologyTestDriverWrapper(builder.build(), props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final KTableValueGetter getter2 = getterSupplier2.get(); final KTableValueGetter getter3 = getterSupplier3.get(); @@ -215,7 +215,7 @@ public void testNotSendingOldValue() { try (final TopologyTestDriver driver = new TopologyTestDriver(topology, props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor proc = supplier.theCapturedProcessor(); assertFalse(table1.sendingOldValueEnabled()); @@ -290,7 +290,7 @@ private void testSendingOldValues( try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { final TestInputTopic inputTopic1 = - driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + driver.createInputTopic(topic1, new StringSerializer(), new StringSerializer(), Instant.EPOCH, Duration.ZERO); final MockApiProcessor proc = supplier.theCapturedProcessor(); inputTopic1.pipeInput("A", "01", 5L); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java index 5436182749101..78431ba22247b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java @@ -145,7 +145,7 @@ public void kTableShouldLogAndMeterOnSkippedRecords() { topic, new StringSerializer(), new StringSerializer(), - Instant.ofEpochMilli(0L), + Instant.EPOCH, Duration.ZERO ); inputTopic.pipeInput(null, "value"); @@ -174,7 +174,7 @@ public void kTableShouldLogOnOutOfOrder() { topic, new StringSerializer(), new StringSerializer(), - Instant.ofEpochMilli(0L), + Instant.EPOCH, Duration.ZERO ); inputTopic.pipeInput("key", "value", 10L); @@ -211,7 +211,7 @@ public void testValueGetter() { topic1, new StringSerializer(), new StringSerializer(), - Instant.ofEpochMilli(0L), + Instant.EPOCH, Duration.ZERO ); final KTableValueGetter getter1 = getterSupplier1.get(); @@ -265,7 +265,7 @@ public void testNotSendingOldValue() { topic1, new StringSerializer(), new StringSerializer(), - Instant.ofEpochMilli(0L), + Instant.EPOCH, Duration.ZERO ); final MockApiProcessor proc1 = supplier.theCapturedProcessor(); @@ -320,7 +320,7 @@ public void testSendingOldValue() { topic1, new StringSerializer(), new StringSerializer(), - Instant.ofEpochMilli(0L), + Instant.EPOCH, Duration.ZERO ); final MockApiProcessor proc1 = supplier.theCapturedProcessor(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java index e9c52c33831e8..71936bfa42231 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java @@ -45,6 +45,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.time.Instant; import java.util.Arrays; import java.util.List; import java.util.Properties; @@ -208,7 +209,7 @@ public void shouldMaterializeCount() { { final WindowStore windowStore = driver.getWindowStore("count-store"); final List, Long>> data = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", Instant.EPOCH, ofEpochMilli(1000L))); assertThat(data, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), 1L), @@ -223,7 +224,7 @@ public void shouldMaterializeCount() { final WindowStore> windowStore = driver.getTimestampedWindowStore("count-store"); final List, ValueAndTimestamp>> data = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", Instant.EPOCH, ofEpochMilli(1000L))); assertThat(data, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), ValueAndTimestamp.make(1L, 100L)), KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 150)), ValueAndTimestamp.make(2L, 150L)), @@ -248,7 +249,7 @@ public void shouldMaterializeReduced() { { final WindowStore windowStore = driver.getWindowStore("reduced"); final List, String>> data = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", Instant.EPOCH, ofEpochMilli(1000L))); assertThat(data, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), "1"), KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 150)), "1+2"), @@ -262,7 +263,7 @@ public void shouldMaterializeReduced() { final WindowStore> windowStore = driver.getTimestampedWindowStore("reduced"); final List, ValueAndTimestamp>> data = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", Instant.EPOCH, ofEpochMilli(1000L))); assertThat(data, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), ValueAndTimestamp.make("1", 100L)), KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 150)), ValueAndTimestamp.make("1+2", 150L)), @@ -289,7 +290,7 @@ public void shouldMaterializeAggregated() { { final WindowStore windowStore = driver.getWindowStore("aggregated"); final List, String>> data = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", Instant.EPOCH, ofEpochMilli(1000L))); assertThat(data, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), "0+1"), KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 150)), "0+1+2"), @@ -303,7 +304,7 @@ public void shouldMaterializeAggregated() { final WindowStore> windowStore = driver.getTimestampedWindowStore("aggregated"); final List, ValueAndTimestamp>> data = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", Instant.EPOCH, ofEpochMilli(1000L))); assertThat(data, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 100)), ValueAndTimestamp.make("0+1", 100L)), KeyValue.pair(new Windowed<>("1", new TimeWindow(50, 150)), ValueAndTimestamp.make("0+1+2", 150L)), @@ -410,7 +411,7 @@ public void shouldDropWindowsOutsideOfRetention() { { final WindowStore windowStore = driver.getWindowStore("aggregated"); final List, String>> data = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "1", ofEpochMilli(0), ofEpochMilli(10000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "1", Instant.EPOCH, ofEpochMilli(10000L))); assertThat(data, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("1", new TimeWindow(900, 1000)), "0+4"), KeyValue.pair(new Windowed<>("1", new TimeWindow(1900, 2000)), "0+5")))); @@ -419,7 +420,7 @@ public void shouldDropWindowsOutsideOfRetention() { final WindowStore> windowStore = driver.getTimestampedWindowStore("aggregated"); final List, ValueAndTimestamp>> data = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "1", ofEpochMilli(0), ofEpochMilli(2000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "1", Instant.EPOCH, ofEpochMilli(2000L))); assertThat(data, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("1", new TimeWindow(900, 1000)), ValueAndTimestamp.make("0+4", 1000L)), KeyValue.pair(new Windowed<>("1", new TimeWindow(1900, 2000)), ValueAndTimestamp.make("0+5", 2000L))))); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java index c526b09f9c64a..b0c1b5f7bcf79 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java @@ -48,6 +48,7 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import java.time.Instant; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -239,7 +240,7 @@ public void shouldMaterializeCount(final StrategyType inputType, final boolean i { final WindowStore windowStore = driver.getWindowStore("count-store"); final List, Long>> data = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", Instant.EPOCH, ofEpochMilli(1000L))); if (withCache) { // with cache returns all records (expired from underneath as well) as part of @@ -266,7 +267,7 @@ public void shouldMaterializeCount(final StrategyType inputType, final boolean i final WindowStore> windowStore = driver.getTimestampedWindowStore("count-store"); final List, ValueAndTimestamp>> data = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", Instant.EPOCH, ofEpochMilli(1000L))); // the same values and logic described above applies here as well. if (withCache) { @@ -305,7 +306,7 @@ public void shouldMaterializeReduced(final StrategyType inputType, final boolean { final WindowStore windowStore = driver.getWindowStore("reduced"); final List, String>> data = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", Instant.EPOCH, ofEpochMilli(1000L))); if (withCache) { // with cache returns all records (expired from underneath as well) as part of @@ -325,7 +326,7 @@ public void shouldMaterializeReduced(final StrategyType inputType, final boolean { final WindowStore> windowStore = driver.getTimestampedWindowStore("reduced"); final List, ValueAndTimestamp>> data = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", Instant.EPOCH, ofEpochMilli(1000L))); // same logic/data as explained above. if (withCache) { @@ -358,7 +359,7 @@ public void shouldMaterializeAggregated(final StrategyType inputType, final bool { final WindowStore windowStore = driver.getWindowStore("aggregated"); final List, String>> data = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", Instant.EPOCH, ofEpochMilli(1000L))); if (withCache) { // with cache returns all records (expired from underneath as well) as part of @@ -379,7 +380,7 @@ public void shouldMaterializeAggregated(final StrategyType inputType, final bool { final WindowStore> windowStore = driver.getTimestampedWindowStore("aggregated"); final List, ValueAndTimestamp>> data = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", ofEpochMilli(0), ofEpochMilli(1000L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("1", "2", Instant.EPOCH, ofEpochMilli(1000L))); if (withCache) { assertThat(data, equalTo(asList( KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), ValueAndTimestamp.make("0+1+2", 15L)), diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java index a578e5b25f23e..0e6aeb5d91d75 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java @@ -280,7 +280,7 @@ public void shouldThrowIfMultipleSourceNodeOfSameSubtopologySubscribedToSameTopi public void testDrivingSimpleTopology() { final int partition = 10; driver = new TopologyTestDriver(createSimpleTopology(partition), props); - final TestInputTopic inputTopic = driver.createInputTopic(INPUT_TOPIC_1, STRING_SERIALIZER, STRING_SERIALIZER, Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic = driver.createInputTopic(INPUT_TOPIC_1, STRING_SERIALIZER, STRING_SERIALIZER, Instant.EPOCH, Duration.ZERO); final TestOutputTopic outputTopic1 = driver.createOutputTopic(OUTPUT_TOPIC_1, new StringDeserializer(), new StringDeserializer()); @@ -304,7 +304,7 @@ public void testDrivingSimpleTopology() { @Test public void testDrivingSimpleTopologyWithDroppingPartitioner() { driver = new TopologyTestDriver(createSimpleTopologyWithDroppingPartitioner(), props); - final TestInputTopic inputTopic = driver.createInputTopic(INPUT_TOPIC_1, STRING_SERIALIZER, STRING_SERIALIZER, Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic = driver.createInputTopic(INPUT_TOPIC_1, STRING_SERIALIZER, STRING_SERIALIZER, Instant.EPOCH, Duration.ZERO); final TestOutputTopic outputTopic1 = driver.createOutputTopic(OUTPUT_TOPIC_1, new StringDeserializer(), new StringDeserializer()); @@ -808,7 +808,7 @@ public void testPrefixScanLruMapWithCachingWithLogging() { public void testDrivingSimpleMultiSourceTopology() { final int partition = 10; driver = new TopologyTestDriver(createSimpleMultiSourceTopology(partition), props); - final TestInputTopic inputTopic = driver.createInputTopic(INPUT_TOPIC_1, STRING_SERIALIZER, STRING_SERIALIZER, Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic = driver.createInputTopic(INPUT_TOPIC_1, STRING_SERIALIZER, STRING_SERIALIZER, Instant.EPOCH, Duration.ZERO); final TestOutputTopic outputTopic1 = driver.createOutputTopic(OUTPUT_TOPIC_1, new StringDeserializer(), new StringDeserializer()); final TestOutputTopic outputTopic2 = @@ -818,7 +818,7 @@ public void testDrivingSimpleMultiSourceTopology() { assertNextOutputRecord(outputTopic1.readRecord(), "key1", "value1"); assertTrue(outputTopic2.isEmpty()); - final TestInputTopic inputTopic2 = driver.createInputTopic(INPUT_TOPIC_2, STRING_SERIALIZER, STRING_SERIALIZER, Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = driver.createInputTopic(INPUT_TOPIC_2, STRING_SERIALIZER, STRING_SERIALIZER, Instant.EPOCH, Duration.ZERO); inputTopic2.pipeInput("key2", "value2"); assertNextOutputRecord(outputTopic2.readRecord(), "key2", "value2"); assertTrue(outputTopic2.isEmpty()); @@ -827,7 +827,7 @@ public void testDrivingSimpleMultiSourceTopology() { @Test public void testDrivingForwardToSourceTopology() { driver = new TopologyTestDriver(createForwardToSourceTopology(), props); - final TestInputTopic inputTopic = driver.createInputTopic(INPUT_TOPIC_1, STRING_SERIALIZER, STRING_SERIALIZER, Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic = driver.createInputTopic(INPUT_TOPIC_1, STRING_SERIALIZER, STRING_SERIALIZER, Instant.EPOCH, Duration.ZERO); inputTopic.pipeInput("key1", "value1"); inputTopic.pipeInput("key2", "value2"); inputTopic.pipeInput("key3", "value3"); @@ -841,7 +841,7 @@ public void testDrivingForwardToSourceTopology() { @Test public void testDrivingInternalRepartitioningTopology() { driver = new TopologyTestDriver(createInternalRepartitioningTopology(), props); - final TestInputTopic inputTopic = driver.createInputTopic(INPUT_TOPIC_1, STRING_SERIALIZER, STRING_SERIALIZER, Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic = driver.createInputTopic(INPUT_TOPIC_1, STRING_SERIALIZER, STRING_SERIALIZER, Instant.EPOCH, Duration.ZERO); inputTopic.pipeInput("key1", "value1"); inputTopic.pipeInput("key2", "value2"); inputTopic.pipeInput("key3", "value3"); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBWindowStoreTest.java index 66a896598f131..8cdbb2a2fdfe3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBWindowStoreTest.java @@ -32,6 +32,7 @@ import org.junit.jupiter.api.Test; import java.io.File; +import java.time.Instant; import java.util.ArrayList; import java.util.List; import java.util.Set; @@ -479,7 +480,7 @@ public void testSegmentMaintenance() { int fetchedCount; - try (final WindowStoreIterator iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(SEGMENT_INTERVAL * 4))) { + try (final WindowStoreIterator iter = windowStore.fetch(0, Instant.EPOCH, ofEpochMilli(SEGMENT_INTERVAL * 4))) { fetchedCount = 0; while (iter.hasNext()) { iter.next(); @@ -495,7 +496,7 @@ public void testSegmentMaintenance() { windowStore.put(0, "v", SEGMENT_INTERVAL * 3); - try (final WindowStoreIterator iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(SEGMENT_INTERVAL * 4))) { + try (final WindowStoreIterator iter = windowStore.fetch(0, Instant.EPOCH, ofEpochMilli(SEGMENT_INTERVAL * 4))) { fetchedCount = 0; while (iter.hasNext()) { iter.next(); @@ -567,7 +568,7 @@ public void testInitialLoading() { assertEquals(expected, actual); - try (final WindowStoreIterator iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(1000000L))) { + try (final WindowStoreIterator iter = windowStore.fetch(0, Instant.EPOCH, ofEpochMilli(1000000L))) { while (iter.hasNext()) { iter.next(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java index 1486cca8c7e0d..b0eb3cab360a0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java @@ -46,6 +46,7 @@ import org.junit.jupiter.api.Test; import java.io.File; +import java.time.Instant; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -854,12 +855,12 @@ public void shouldFetchAndIterateOverExactKeys() { final Set expected = Set.of("0001", "0003", "0005"); assertThat( - valuesToSetAndCloseIterator(windowStore.fetch("a", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), + valuesToSetAndCloseIterator(windowStore.fetch("a", Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))), equalTo(expected) ); Set, String>> set = - toSet(windowStore.fetch("a", "a", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); + toSet(windowStore.fetch("a", "a", Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))); assertThat( set, equalTo(Set.of( @@ -869,7 +870,7 @@ public void shouldFetchAndIterateOverExactKeys() { )) ); - set = toSet(windowStore.fetch("aa", "aa", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); + set = toSet(windowStore.fetch("aa", "aa", Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))); assertThat( set, equalTo(Set.of( @@ -936,17 +937,17 @@ public void shouldFetchAndIterateOverExactBinaryKeys() { final Set expectedKey1 = Set.of("1", "4", "7"); assertThat( - valuesToSetAndCloseIterator(windowStore.fetch(key1, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), + valuesToSetAndCloseIterator(windowStore.fetch(key1, Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))), equalTo(expectedKey1) ); final Set expectedKey2 = Set.of("2", "5", "8"); assertThat( - valuesToSetAndCloseIterator(windowStore.fetch(key2, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), + valuesToSetAndCloseIterator(windowStore.fetch(key2, Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))), equalTo(expectedKey2) ); final Set expectedKey3 = Set.of("3", "6", "9"); assertThat( - valuesToSetAndCloseIterator(windowStore.fetch(key3, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), + valuesToSetAndCloseIterator(windowStore.fetch(key3, Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))), equalTo(expectedKey3) ); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java index 4548da5bd1134..88c215397e12e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java @@ -207,7 +207,7 @@ public void process(final Record record) { streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10 * 1000L); - final Instant initialWallClockTime = Instant.ofEpochMilli(0L); + final Instant initialWallClockTime = Instant.EPOCH; final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), streamsConfiguration, initialWallClockTime); final TestInputTopic inputTopic = driver.createInputTopic(TOPIC, @@ -529,7 +529,7 @@ public void shouldFetchAllWithinTimestampRange() { } try (final KeyValueIterator, byte[]> iterator = - cachingStore.fetchAll(ofEpochMilli(0), ofEpochMilli(7))) { + cachingStore.fetchAll(Instant.EPOCH, ofEpochMilli(7))) { for (int i = 0; i < array.length; i++) { final String str = array[i]; verifyWindowedKeyValue( @@ -573,7 +573,7 @@ public void shouldFetchAllBackwardWithinTimestampRange() { } try (final KeyValueIterator, byte[]> iterator = - cachingStore.backwardFetchAll(ofEpochMilli(0), ofEpochMilli(7))) { + cachingStore.backwardFetchAll(Instant.EPOCH, ofEpochMilli(7))) { for (int i = array.length - 1; i >= 0; i--) { final String str = array[i]; verifyWindowedKeyValue( @@ -818,14 +818,14 @@ public void shouldClearNamespaceCacheOnClose() { @Test public void shouldThrowIfTryingToFetchFromClosedCachingStore() { cachingStore.close(); - assertThrows(InvalidStateStoreException.class, () -> cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(10))); + assertThrows(InvalidStateStoreException.class, () -> cachingStore.fetch(bytesKey("a"), Instant.EPOCH, ofEpochMilli(10))); } @SuppressWarnings("resource") @Test public void shouldThrowIfTryingToFetchRangeFromClosedCachingStore() { cachingStore.close(); - assertThrows(InvalidStateStoreException.class, () -> cachingStore.fetch(bytesKey("a"), bytesKey("b"), ofEpochMilli(0), ofEpochMilli(10))); + assertThrows(InvalidStateStoreException.class, () -> cachingStore.fetch(bytesKey("a"), bytesKey("b"), Instant.EPOCH, ofEpochMilli(10))); } @Test @@ -848,7 +848,7 @@ public void shouldFetchAndIterateOverExactKeys() { KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005")) ); final List> actual = - toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); + toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))); verifyKeyValueList(expected, actual); } @@ -866,7 +866,7 @@ public void shouldBackwardFetchAndIterateOverExactKeys() { KeyValue.pair(0L, bytesValue("0001")) ); final List> actual = - toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); + toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))); verifyKeyValueList(expected, actual); } @@ -884,14 +884,14 @@ public void shouldFetchAndIterateOverKeyRange() { windowedPair("a", "0003", 1), windowedPair("a", "0005", SEGMENT_INTERVAL) ), - toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) + toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("a"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); verifyKeyValueList( asList( windowedPair("aa", "0002", 0), windowedPair("aa", "0004", 1)), - toListAndCloseIterator(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) + toListAndCloseIterator(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); verifyKeyValueList( @@ -902,7 +902,7 @@ public void shouldFetchAndIterateOverKeyRange() { windowedPair("aa", "0004", 1), windowedPair("a", "0005", SEGMENT_INTERVAL) ), - toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) + toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); } @@ -920,14 +920,14 @@ public void shouldFetchAndIterateOverKeyBackwardRange() { windowedPair("a", "0003", 1), windowedPair("a", "0001", 0) ), - toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) + toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("a"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); verifyKeyValueList( asList( windowedPair("aa", "0004", 1), windowedPair("aa", "0002", 0)), - toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) + toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); verifyKeyValueList( @@ -938,7 +938,7 @@ public void shouldFetchAndIterateOverKeyBackwardRange() { windowedPair("a", "0003", 1), windowedPair("a", "0001", 0) ), - toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) + toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); } @@ -967,9 +967,9 @@ public void shouldReturnSameResultsForSingleKeyFetchAndEqualKeyRangeBackwardFetc cachingStore.put(bytesKey("aaa"), bytesValue("0004"), 3); try (final WindowStoreIterator singleKeyIterator = - cachingStore.backwardFetch(bytesKey("aa"), Instant.ofEpochMilli(0L), Instant.ofEpochMilli(5L)); + cachingStore.backwardFetch(bytesKey("aa"), Instant.EPOCH, Instant.ofEpochMilli(5L)); final KeyValueIterator, byte[]> keyRangeIterator = - cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), Instant.ofEpochMilli(0L), Instant.ofEpochMilli(5L))) { + cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), Instant.EPOCH, Instant.ofEpochMilli(5L))) { assertEquals(stringFrom(singleKeyIterator.next().value), stringFrom(keyRangeIterator.next().value)); assertEquals(stringFrom(singleKeyIterator.next().value), stringFrom(keyRangeIterator.next().value)); @@ -1023,7 +1023,7 @@ public void shouldNotThrowInvalidBackwardRangeExceptionWithNegativeFromKey() { try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(CachingWindowStore.class); final KeyValueIterator, byte[]> iterator = - cachingStore.backwardFetch(keyFrom, keyTo, Instant.ofEpochMilli(0L), Instant.ofEpochMilli(10L))) { + cachingStore.backwardFetch(keyFrom, keyTo, Instant.EPOCH, Instant.ofEpochMilli(10L))) { assertFalse(iterator.hasNext()); final List messages = appender.getMessages(); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStoreTest.java index 1c1b713ce21a0..01cd4f274194c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStoreTest.java @@ -36,6 +36,8 @@ import org.mockito.junit.jupiter.MockitoSettings; import org.mockito.quality.Strictness; +import java.time.Instant; + import static java.time.Instant.ofEpochMilli; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; @@ -102,7 +104,7 @@ public void shouldLogPutsWithPosition() { @SuppressWarnings({"resource", "unused"}) @Test public void shouldDelegateToUnderlyingStoreWhenFetching() { - try (final WindowStoreIterator unused = store.fetch(bytesKey, ofEpochMilli(0), ofEpochMilli(10))) { + try (final WindowStoreIterator unused = store.fetch(bytesKey, Instant.EPOCH, ofEpochMilli(10))) { verify(inner).fetch(bytesKey, 0, 10); } } @@ -110,7 +112,7 @@ public void shouldDelegateToUnderlyingStoreWhenFetching() { @SuppressWarnings({"resource", "unused"}) @Test public void shouldDelegateToUnderlyingStoreWhenFetchingRange() { - try (final KeyValueIterator, byte[]> unused = store.fetch(bytesKey, bytesKey, ofEpochMilli(0), ofEpochMilli(1))) { + try (final KeyValueIterator, byte[]> unused = store.fetch(bytesKey, bytesKey, Instant.EPOCH, ofEpochMilli(1))) { verify(inner).fetch(bytesKey, bytesKey, 0, 1); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java index e80a2325a2a54..ed781c2c251d0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java @@ -36,6 +36,8 @@ import org.mockito.junit.jupiter.MockitoSettings; import org.mockito.quality.Strictness; +import java.time.Instant; + import static java.time.Instant.ofEpochMilli; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; @@ -101,7 +103,7 @@ public void shouldLogPutsWithPosition() { @SuppressWarnings({"resource", "unused"}) @Test public void shouldDelegateToUnderlyingStoreWhenFetching() { - try (final WindowStoreIterator unused = store.fetch(bytesKey, ofEpochMilli(0), ofEpochMilli(10))) { + try (final WindowStoreIterator unused = store.fetch(bytesKey, Instant.EPOCH, ofEpochMilli(10))) { verify(inner).fetch(bytesKey, 0, 10); } } @@ -109,7 +111,7 @@ public void shouldDelegateToUnderlyingStoreWhenFetching() { @SuppressWarnings({"resource", "unused"}) @Test public void shouldDelegateToUnderlyingStoreWhenBackwardFetching() { - try (final WindowStoreIterator unused = store.backwardFetch(bytesKey, ofEpochMilli(0), ofEpochMilli(10))) { + try (final WindowStoreIterator unused = store.backwardFetch(bytesKey, Instant.EPOCH, ofEpochMilli(10))) { verify(inner).backwardFetch(bytesKey, 0, 10); } } @@ -117,7 +119,7 @@ public void shouldDelegateToUnderlyingStoreWhenBackwardFetching() { @SuppressWarnings({"resource", "unused"}) @Test public void shouldDelegateToUnderlyingStoreWhenFetchingRange() { - try (final KeyValueIterator, byte[]> unused = store.fetch(bytesKey, bytesKey, ofEpochMilli(0), ofEpochMilli(1))) { + try (final KeyValueIterator, byte[]> unused = store.fetch(bytesKey, bytesKey, Instant.EPOCH, ofEpochMilli(1))) { verify(inner).fetch(bytesKey, bytesKey, 0, 1); } } @@ -125,7 +127,7 @@ public void shouldDelegateToUnderlyingStoreWhenFetchingRange() { @SuppressWarnings({"resource", "unused"}) @Test public void shouldDelegateToUnderlyingStoreWhenBackwardFetchingRange() { - try (final KeyValueIterator, byte[]> unused = store.backwardFetch(bytesKey, bytesKey, ofEpochMilli(0), ofEpochMilli(1))) { + try (final KeyValueIterator, byte[]> unused = store.backwardFetch(bytesKey, bytesKey, Instant.EPOCH, ofEpochMilli(1))) { verify(inner).backwardFetch(bytesKey, bytesKey, 0, 1); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java index 2d22e6e15a745..34baf5fa6825c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStoreTest.java @@ -33,6 +33,7 @@ import org.mockito.junit.jupiter.MockitoSettings; import org.mockito.quality.Strictness; +import java.time.Instant; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -90,7 +91,7 @@ public void shouldFetchValuesFromWindowStore() { assertEquals( asList(new KeyValue<>(0L, "my-value"), new KeyValue<>(10L, "my-later-value")), - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L))) + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("my-key", Instant.EPOCH, ofEpochMilli(25L))) ); } @@ -102,14 +103,14 @@ public void shouldBackwardFetchValuesFromWindowStore() { assertEquals( asList(new KeyValue<>(10L, "my-later-value"), new KeyValue<>(0L, "my-value")), - StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L))) + StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("my-key", Instant.EPOCH, ofEpochMilli(25L))) ); } @Test public void shouldReturnEmptyIteratorIfNoData() { try (final WindowStoreIterator iterator = - windowStore.fetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L))) { + windowStore.fetch("my-key", Instant.EPOCH, ofEpochMilli(25L))) { assertFalse(iterator.hasNext()); } } @@ -117,7 +118,7 @@ public void shouldReturnEmptyIteratorIfNoData() { @Test public void shouldReturnBackwardEmptyIteratorIfNoData() { try (final WindowStoreIterator iterator = - windowStore.backwardFetch("my-key", ofEpochMilli(0L), ofEpochMilli(25L))) { + windowStore.backwardFetch("my-key", Instant.EPOCH, ofEpochMilli(25L))) { assertFalse(iterator.hasNext()); } } @@ -132,7 +133,7 @@ public void shouldFindValueForKeyWhenMultiStores() { secondUnderlying.put("key-two", "value-two", 10L); final List> keyOneResults = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("key-one", ofEpochMilli(0L), ofEpochMilli(1L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("key-one", Instant.EPOCH, ofEpochMilli(1L))); final List> keyTwoResults = StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("key-two", ofEpochMilli(10L), ofEpochMilli(11L))); @@ -150,7 +151,7 @@ public void shouldFindValueForKeyWhenMultiStoresBackwards() { secondUnderlying.put("key-two", "value-two", 10L); final List> keyOneResults = - StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("key-one", ofEpochMilli(0L), ofEpochMilli(1L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("key-one", Instant.EPOCH, ofEpochMilli(1L))); final List> keyTwoResults = StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("key-two", ofEpochMilli(10L), ofEpochMilli(11L))); @@ -164,7 +165,7 @@ public void shouldNotGetValuesFromOtherStores() { underlyingWindowStore.put("some-key", "my-value", 1L); final List> results = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("some-key", ofEpochMilli(0L), ofEpochMilli(2L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("some-key", Instant.EPOCH, ofEpochMilli(2L))); assertEquals(Collections.singletonList(new KeyValue<>(1L, "my-value")), results); } @@ -174,7 +175,7 @@ public void shouldNotGetValuesBackwardFromOtherStores() { underlyingWindowStore.put("some-key", "my-value", 1L); final List> results = - StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("some-key", ofEpochMilli(0L), ofEpochMilli(2L))); + StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("some-key", Instant.EPOCH, ofEpochMilli(2L))); assertEquals(Collections.singletonList(new KeyValue<>(1L, "my-value")), results); } @@ -348,7 +349,7 @@ public void shouldFetchKeyRangeAcrossStores() { underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); final List, String>> results = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("a", "b", ofEpochMilli(0), ofEpochMilli(10))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("a", "b", Instant.EPOCH, ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); @@ -362,7 +363,7 @@ public void shouldFetchKeyRangeAcrossStoresWithNullKeyTo() { secondUnderlying.put("b", "b", 10L); secondUnderlying.put("c", "c", 10L); final List, String>> results = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("b", null, ofEpochMilli(0), ofEpochMilli(10))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch("b", null, Instant.EPOCH, ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"), KeyValue.pair(new Windowed<>("c", new TimeWindow(10, 10 + WINDOW_SIZE)), "c")))); @@ -376,7 +377,7 @@ public void shouldFetchKeyRangeAcrossStoresWithNullKeyFrom() { secondUnderlying.put("b", "b", 10L); secondUnderlying.put("c", "c", 10L); final List, String>> results = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch(null, "b", ofEpochMilli(0), ofEpochMilli(10))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch(null, "b", Instant.EPOCH, ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); @@ -390,7 +391,7 @@ public void shouldFetchKeyRangeAcrossStoresWithNullKeyFromKeyTo() { secondUnderlying.put("b", "b", 10L); secondUnderlying.put("c", "c", 10L); final List, String>> results = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetch(null, null, ofEpochMilli(0), ofEpochMilli(10))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetch(null, null, Instant.EPOCH, ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b"), @@ -405,7 +406,7 @@ public void shouldBackwardFetchKeyRangeAcrossStoresWithNullKeyTo() { secondUnderlying.put("b", "b", 10L); secondUnderlying.put("c", "c", 10L); final List, String>> results = - StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("b", null, ofEpochMilli(0), ofEpochMilli(10))); + StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("b", null, Instant.EPOCH, ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("c", new TimeWindow(10, 10 + WINDOW_SIZE)), "c"), KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); @@ -419,7 +420,7 @@ public void shouldBackwardFetchKeyRangeAcrossStoresWithNullKeyFrom() { secondUnderlying.put("b", "b", 10L); secondUnderlying.put("c", "c", 10L); final List, String>> results = - StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch(null, "b", ofEpochMilli(0), ofEpochMilli(10))); + StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch(null, "b", Instant.EPOCH, ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b") @@ -434,7 +435,7 @@ public void shouldBackwardFetchKeyRangeAcrossStoresWithNullKeyFromKeyTo() { secondUnderlying.put("b", "b", 10L); secondUnderlying.put("c", "c", 10L); final List, String>> results = - StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch(null, null, ofEpochMilli(0), ofEpochMilli(10))); + StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch(null, null, Instant.EPOCH, ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("c", new TimeWindow(10, 10 + WINDOW_SIZE)), "c"), @@ -448,7 +449,7 @@ public void shouldBackwardFetchKeyRangeAcrossStores() { underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); final List, String>> results = - StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("a", "b", ofEpochMilli(0), ofEpochMilli(10))); + StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetch("a", "b", Instant.EPOCH, ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); @@ -500,7 +501,7 @@ public void shouldFetchAllAcrossStores() { underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); final List, String>> results = - StreamsTestUtils.toListAndCloseIterator(windowStore.fetchAll(ofEpochMilli(0), ofEpochMilli(10))); + StreamsTestUtils.toListAndCloseIterator(windowStore.fetchAll(Instant.EPOCH, ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); @@ -514,7 +515,7 @@ public void shouldBackwardFetchAllAcrossStores() { underlyingWindowStore.put("a", "a", 0L); secondUnderlying.put("b", "b", 10L); final List, String>> results = - StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetchAll(ofEpochMilli(0), ofEpochMilli(10))); + StreamsTestUtils.toListAndCloseIterator(windowStore.backwardFetchAll(Instant.EPOCH, ofEpochMilli(10))); assertThat(results, equalTo(Arrays.asList( KeyValue.pair(new Windowed<>("a", new TimeWindow(0, WINDOW_SIZE)), "a"), KeyValue.pair(new Windowed<>("b", new TimeWindow(10, 10 + WINDOW_SIZE)), "b")))); @@ -522,6 +523,6 @@ public void shouldBackwardFetchAllAcrossStores() { @Test public void shouldThrowNPEIfKeyIsNull() { - assertThrows(NullPointerException.class, () -> windowStore.fetch(null, ofEpochMilli(0), ofEpochMilli(0))); + assertThrows(NullPointerException.class, () -> windowStore.fetch(null, Instant.EPOCH, Instant.EPOCH)); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingPersistentWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingPersistentWindowStoreTest.java index ffa509d518871..1f4bc6bd8147f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingPersistentWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedCachingPersistentWindowStoreTest.java @@ -223,7 +223,7 @@ public void process(final Record record) { streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10 * 1000L); - final Instant initialWallClockTime = Instant.ofEpochMilli(0L); + final Instant initialWallClockTime = Instant.EPOCH; final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), streamsConfiguration, initialWallClockTime); final TestInputTopic inputTopic = driver.createInputTopic(TOPIC, @@ -573,7 +573,7 @@ public void shouldFetchAllWithinTimestampRange(final boolean hasIndex) { } try (final KeyValueIterator, byte[]> iterator = - cachingStore.fetchAll(ofEpochMilli(0), ofEpochMilli(7))) { + cachingStore.fetchAll(Instant.EPOCH, ofEpochMilli(7))) { for (int i = 0; i < array.length; i++) { final String str = array[i]; verifyWindowedKeyValue( @@ -619,7 +619,7 @@ public void shouldFetchAllBackwardWithinTimestampRange(final boolean hasIndex) { } try (final KeyValueIterator, byte[]> iterator = - cachingStore.backwardFetchAll(ofEpochMilli(0), ofEpochMilli(7))) { + cachingStore.backwardFetchAll(Instant.EPOCH, ofEpochMilli(7))) { for (int i = array.length - 1; i >= 0; i--) { final String str = array[i]; verifyWindowedKeyValue( @@ -895,7 +895,7 @@ public void shouldClearNamespaceCacheOnClose(final boolean hasIndex) { public void shouldThrowIfTryingToFetchFromClosedCachingStore(final boolean hasIndex) { setUp(hasIndex); cachingStore.close(); - assertThrows(InvalidStateStoreException.class, () -> cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(10))); + assertThrows(InvalidStateStoreException.class, () -> cachingStore.fetch(bytesKey("a"), Instant.EPOCH, ofEpochMilli(10))); } @SuppressWarnings("resource") @@ -904,7 +904,7 @@ public void shouldThrowIfTryingToFetchFromClosedCachingStore(final boolean hasIn public void shouldThrowIfTryingToFetchRangeFromClosedCachingStore(final boolean hasIndex) { setUp(hasIndex); cachingStore.close(); - assertThrows(InvalidStateStoreException.class, () -> cachingStore.fetch(bytesKey("a"), bytesKey("b"), ofEpochMilli(0), ofEpochMilli(10))); + assertThrows(InvalidStateStoreException.class, () -> cachingStore.fetch(bytesKey("a"), bytesKey("b"), Instant.EPOCH, ofEpochMilli(10))); } @ParameterizedTest @@ -952,7 +952,7 @@ public void shouldSkipNonExistBaseKeyInCache(final boolean hasIndex) { windowedPair("a", "0001", 1), windowedPair("aa", "0002", 0) ), - toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), + toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); } else { @@ -961,7 +961,7 @@ public void shouldSkipNonExistBaseKeyInCache(final boolean hasIndex) { windowedPair("aa", "0002", 0), windowedPair("a", "0001", 1) ), - toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), + toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); } @@ -983,7 +983,7 @@ public void shouldFetchAndIterateOverExactKeys(final boolean hasIndex) { KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005")) ); final List> actual = - toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); + toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))); verifyKeyValueList(expected, actual); } @@ -1003,7 +1003,7 @@ public void shouldBackwardFetchAndIterateOverExactKeys(final boolean hasIndex) { KeyValue.pair(0L, bytesValue("0001")) ); final List> actual = - toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); + toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))); verifyKeyValueList(expected, actual); } @@ -1023,14 +1023,14 @@ public void shouldFetchAndIterateOverKeyRange(final boolean hasIndex) { windowedPair("a", "0003", 1), windowedPair("a", "0005", SEGMENT_INTERVAL) ), - toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) + toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("a"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); verifyKeyValueList( asList( windowedPair("aa", "0002", 0), windowedPair("aa", "0004", 1)), - toListAndCloseIterator(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) + toListAndCloseIterator(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); if (hasIndex) { @@ -1042,7 +1042,7 @@ public void shouldFetchAndIterateOverKeyRange(final boolean hasIndex) { windowedPair("aa", "0004", 1), windowedPair("a", "0005", SEGMENT_INTERVAL) ), - toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), + toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); } else { @@ -1054,7 +1054,7 @@ public void shouldFetchAndIterateOverKeyRange(final boolean hasIndex) { windowedPair("aa", "0004", 1), windowedPair("a", "0005", SEGMENT_INTERVAL) ), - toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), + toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); } @@ -1076,14 +1076,14 @@ public void shouldFetchAndIterateOverKeyBackwardRange(final boolean hasIndex) { windowedPair("a", "0003", 1), windowedPair("a", "0001", 0) ), - toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) + toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("a"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); verifyKeyValueList( asList( windowedPair("aa", "0004", 1), windowedPair("aa", "0002", 0)), - toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) + toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); if (!hasIndex) { @@ -1096,7 +1096,7 @@ public void shouldFetchAndIterateOverKeyBackwardRange(final boolean hasIndex) { windowedPair("aa", "0002", 0), windowedPair("a", "0001", 0) ), - toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), + toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); } else { @@ -1109,7 +1109,7 @@ public void shouldFetchAndIterateOverKeyBackwardRange(final boolean hasIndex) { windowedPair("a", "0003", 1), windowedPair("a", "0001", 0) ), - toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), + toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); } @@ -1144,9 +1144,9 @@ public void shouldReturnSameResultsForSingleKeyFetchAndEqualKeyRangeBackwardFetc cachingStore.put(bytesKey("aaa"), bytesValue("0004"), 3); try (final WindowStoreIterator singleKeyIterator = - cachingStore.backwardFetch(bytesKey("aa"), Instant.ofEpochMilli(0L), Instant.ofEpochMilli(5L)); + cachingStore.backwardFetch(bytesKey("aa"), Instant.EPOCH, Instant.ofEpochMilli(5L)); final KeyValueIterator, byte[]> keyRangeIterator = - cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), Instant.ofEpochMilli(0L), Instant.ofEpochMilli(5L))) { + cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), Instant.EPOCH, Instant.ofEpochMilli(5L))) { assertEquals(stringFrom(singleKeyIterator.next().value), stringFrom(keyRangeIterator.next().value)); assertEquals(stringFrom(singleKeyIterator.next().value), stringFrom(keyRangeIterator.next().value)); @@ -1210,7 +1210,7 @@ public void shouldNotThrowInvalidBackwardRangeExceptionWithNegativeFromKey(final try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(TimeOrderedCachingWindowStore.class); final KeyValueIterator, byte[]> iterator = - cachingStore.backwardFetch(keyFrom, keyTo, Instant.ofEpochMilli(0L), Instant.ofEpochMilli(10L))) { + cachingStore.backwardFetch(keyFrom, keyTo, Instant.EPOCH, Instant.ofEpochMilli(10L))) { assertFalse(iterator.hasNext()); final List messages = appender.getMessages(); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreTest.java index 9d0db9bae0fbb..4abaa1de9d7aa 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreTest.java @@ -229,7 +229,7 @@ public void process(final Record record) { streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10 * 1000L); - final Instant initialWallClockTime = Instant.ofEpochMilli(0L); + final Instant initialWallClockTime = Instant.EPOCH; final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), streamsConfiguration, initialWallClockTime); final TestInputTopic inputTopic = driver.createInputTopic(TOPIC, @@ -579,7 +579,7 @@ public void shouldFetchAllWithinTimestampRange(final boolean hasIndex) { } try (final KeyValueIterator, byte[]> iterator = - cachingStore.fetchAll(ofEpochMilli(0), ofEpochMilli(7))) { + cachingStore.fetchAll(Instant.EPOCH, ofEpochMilli(7))) { for (int i = 0; i < array.length; i++) { final String str = array[i]; verifyWindowedKeyValue( @@ -625,7 +625,7 @@ public void shouldFetchAllBackwardWithinTimestampRange(final boolean hasIndex) { } try (final KeyValueIterator, byte[]> iterator = - cachingStore.backwardFetchAll(ofEpochMilli(0), ofEpochMilli(7))) { + cachingStore.backwardFetchAll(Instant.EPOCH, ofEpochMilli(7))) { for (int i = array.length - 1; i >= 0; i--) { final String str = array[i]; verifyWindowedKeyValue( @@ -901,7 +901,7 @@ public void shouldClearNamespaceCacheOnClose(final boolean hasIndex) { public void shouldThrowIfTryingToFetchFromClosedCachingStore(final boolean hasIndex) { setUp(hasIndex); cachingStore.close(); - assertThrows(InvalidStateStoreException.class, () -> cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(10))); + assertThrows(InvalidStateStoreException.class, () -> cachingStore.fetch(bytesKey("a"), Instant.EPOCH, ofEpochMilli(10))); } @SuppressWarnings("resource") @@ -910,7 +910,7 @@ public void shouldThrowIfTryingToFetchFromClosedCachingStore(final boolean hasIn public void shouldThrowIfTryingToFetchRangeFromClosedCachingStore(final boolean hasIndex) { setUp(hasIndex); cachingStore.close(); - assertThrows(InvalidStateStoreException.class, () -> cachingStore.fetch(bytesKey("a"), bytesKey("b"), ofEpochMilli(0), ofEpochMilli(10))); + assertThrows(InvalidStateStoreException.class, () -> cachingStore.fetch(bytesKey("a"), bytesKey("b"), Instant.EPOCH, ofEpochMilli(10))); } @ParameterizedTest @@ -958,7 +958,7 @@ public void shouldSkipNonExistBaseKeyInCache(final boolean hasIndex) { windowedPair("a", "0001", 1), windowedPair("aa", "0002", 0) ), - toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), + toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); } else { @@ -967,7 +967,7 @@ public void shouldSkipNonExistBaseKeyInCache(final boolean hasIndex) { windowedPair("aa", "0002", 0), windowedPair("a", "0001", 1) ), - toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), + toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); } @@ -989,7 +989,7 @@ public void shouldFetchAndIterateOverExactKeys(final boolean hasIndex) { KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005")) ); final List> actual = - toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); + toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))); verifyKeyValueList(expected, actual); } @@ -1009,7 +1009,7 @@ public void shouldBackwardFetchAndIterateOverExactKeys(final boolean hasIndex) { KeyValue.pair(0L, bytesValue("0001")) ); final List> actual = - toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); + toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))); verifyKeyValueList(expected, actual); } @@ -1029,14 +1029,14 @@ public void shouldFetchAndIterateOverKeyRange(final boolean hasIndex) { windowedPair("a", "0003", 1), windowedPair("a", "0005", SEGMENT_INTERVAL) ), - toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) + toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("a"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); verifyKeyValueList( asList( windowedPair("aa", "0002", 0), windowedPair("aa", "0004", 1)), - toListAndCloseIterator(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) + toListAndCloseIterator(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); if (hasIndex) { @@ -1048,7 +1048,7 @@ public void shouldFetchAndIterateOverKeyRange(final boolean hasIndex) { windowedPair("aa", "0004", 1), windowedPair("a", "0005", SEGMENT_INTERVAL) ), - toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), + toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); } else { @@ -1060,7 +1060,7 @@ public void shouldFetchAndIterateOverKeyRange(final boolean hasIndex) { windowedPair("aa", "0004", 1), windowedPair("a", "0005", SEGMENT_INTERVAL) ), - toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), + toListAndCloseIterator(cachingStore.fetch(bytesKey("a"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); } @@ -1082,14 +1082,14 @@ public void shouldFetchAndIterateOverKeyBackwardRange(final boolean hasIndex) { windowedPair("a", "0003", 1), windowedPair("a", "0001", 0) ), - toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("a"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) + toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("a"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); verifyKeyValueList( asList( windowedPair("aa", "0004", 1), windowedPair("aa", "0002", 0)), - toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) + toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); if (!hasIndex) { @@ -1102,7 +1102,7 @@ public void shouldFetchAndIterateOverKeyBackwardRange(final boolean hasIndex) { windowedPair("aa", "0002", 0), windowedPair("a", "0001", 0) ), - toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), + toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); } else { @@ -1115,7 +1115,7 @@ public void shouldFetchAndIterateOverKeyBackwardRange(final boolean hasIndex) { windowedPair("a", "0003", 1), windowedPair("a", "0001", 0) ), - toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), ofEpochMilli(0), + toListAndCloseIterator(cachingStore.backwardFetch(bytesKey("a"), bytesKey("aa"), Instant.EPOCH, ofEpochMilli(Long.MAX_VALUE))) ); } @@ -1150,9 +1150,9 @@ public void shouldReturnSameResultsForSingleKeyFetchAndEqualKeyRangeBackwardFetc cachingStore.put(bytesKey("aaa"), bytesValue("0004"), 3); try (final WindowStoreIterator singleKeyIterator = - cachingStore.backwardFetch(bytesKey("aa"), Instant.ofEpochMilli(0L), Instant.ofEpochMilli(5L)); + cachingStore.backwardFetch(bytesKey("aa"), Instant.EPOCH, Instant.ofEpochMilli(5L)); final KeyValueIterator, byte[]> keyRangeIterator = - cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), Instant.ofEpochMilli(0L), Instant.ofEpochMilli(5L))) { + cachingStore.backwardFetch(bytesKey("aa"), bytesKey("aa"), Instant.EPOCH, Instant.ofEpochMilli(5L))) { assertEquals(stringFrom(singleKeyIterator.next().value), stringFrom(keyRangeIterator.next().value)); assertEquals(stringFrom(singleKeyIterator.next().value), stringFrom(keyRangeIterator.next().value)); @@ -1216,7 +1216,7 @@ public void shouldNotThrowInvalidBackwardRangeExceptionWithNegativeFromKey(final try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(TimeOrderedCachingWindowStore.class); final KeyValueIterator, byte[]> iterator = - cachingStore.backwardFetch(keyFrom, keyTo, Instant.ofEpochMilli(0L), Instant.ofEpochMilli(10L))) { + cachingStore.backwardFetch(keyFrom, keyTo, Instant.EPOCH, Instant.ofEpochMilli(10L))) { assertFalse(iterator.hasNext()); final List messages = appender.getMessages(); diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java index 2dc0089990e69..2453ae721fda8 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java @@ -839,7 +839,7 @@ public void shouldPunctuateOnWallClockTime() { final MockPunctuator mockPunctuator = new MockPunctuator(); testDriver = new TopologyTestDriver( setupSingleProcessorTopology(10L, PunctuationType.WALL_CLOCK_TIME, mockPunctuator), - config, Instant.ofEpochMilli(0L)); + config, Instant.EPOCH); final List expectedPunctuations = new LinkedList<>(); diff --git a/tools/src/test/java/org/apache/kafka/tools/ProducerPerformanceTest.java b/tools/src/test/java/org/apache/kafka/tools/ProducerPerformanceTest.java index 0ec9c73316457..f2ba1233dd06f 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ProducerPerformanceTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ProducerPerformanceTest.java @@ -72,7 +72,7 @@ public class ProducerPerformanceTest { ProducerPerformance producerPerformanceSpy; private File createTempFile(String contents) throws IOException { - File file = File.createTempFile("ProducerPerformanceTest", ".tmp"); + File file = Files.createTempFile("ProducerPerformanceTest", ".tmp").toFile(); file.deleteOnExit(); Files.write(file.toPath(), contents.getBytes()); return file;