From 12f310d50e7f5b1c18c4f61a119a6cd830da3bc0 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 12 Nov 2018 22:18:59 -0800 Subject: [PATCH] KAFKA-7612: Fix javac warnings and enable warnings as errors (#5900) - Use Xlint:all with 3 exclusions (filed KAFKA-7613 to remove the exclusions) - Use the same javac options when compiling tests (seems accidental that we didn't do this before) - Replaced several deprecated method calls with non-deprecated ones: - `KafkaConsumer.poll(long)` and `KafkaConsumer.close(long)` - `Class.newInstance` and `new Integer/Long` (deprecated since Java 9) - `scala.Console` (deprecated in Scala 2.11) - `PartitionData` taking a timestamp (one of them seemingly a bug) - `JsonMappingException` single parameter constructor - Fix unnecessary usage of raw types in several places. - Add @SuppressWarnings for deprecations, unchecked and switch fallthrough in several places. - Scala clean-ups (var -> val, ETA expansion warnings, avoid reflective calls) - Use lambdas to simplify code in a few places - Add @SafeVarargs, fix varargs usage and remove unnecessary `Utils.mkList` method Reviewers: Matthias J. Sax , Manikumar Reddy , Randall Hauch , Bill Bejeck , Stanislav Kozlovski --- build.gradle | 9 +- .../kafka/clients/FetchSessionHandler.java | 8 +- .../kafka/clients/admin/ConfigEntry.java | 1 + .../kafka/clients/consumer/KafkaConsumer.java | 2 +- .../kafka/clients/consumer/MockConsumer.java | 1 + .../clients/consumer/internals/Fetcher.java | 13 +- .../common/network/SslTransportLayer.java | 1 + .../record/LazyDownConversionRecords.java | 8 +- .../record/LazyDownConversionRecordsSend.java | 6 +- .../authenticator/CredentialCache.java | 3 +- .../security/authenticator/LoginManager.java | 1 + .../SaslClientAuthenticator.java | 1 + .../SaslClientCallbackHandler.java | 1 + .../SaslServerAuthenticator.java | 1 + .../org/apache/kafka/common/utils/Crc32.java | 1 + .../kafka/common/utils/PureJavaCrc32C.java | 1 + .../org/apache/kafka/common/utils/Utils.java | 17 +- .../clients/FetchSessionHandlerTest.java | 1 + .../clients/admin/KafkaAdminClientTest.java | 1 + .../clients/consumer/KafkaConsumerTest.java | 24 +- .../internals/ConsumerCoordinatorTest.java | 3 +- .../consumer/internals/FetcherTest.java | 6 +- .../clients/producer/KafkaProducerTest.java | 7 +- .../apache/kafka/common/KafkaFutureTest.java | 32 +- .../kafka/common/config/ConfigDefTest.java | 2 +- .../kafka/common/metrics/MetricsTest.java | 1 + .../common/network/ChannelBuildersTest.java | 3 +- .../kafka/common/record/FileRecordsTest.java | 4 +- .../common/requests/RequestResponseTest.java | 1 + .../serialization/SerializationTest.java | 7 +- .../connect/util/ConnectorUtilsTest.java | 4 +- .../kafka/connect/json/JsonConverterTest.java | 2 +- .../connect/converters/NumberConverter.java | 1 + .../connect/runtime/ConnectorConfig.java | 10 +- .../apache/kafka/connect/runtime/Worker.java | 1 + .../connect/runtime/WorkerConnector.java | 1 + .../errors/RetryWithToleranceOperator.java | 1 + .../isolation/DelegatingClassLoader.java | 10 +- .../connect/runtime/isolation/Plugins.java | 11 +- .../runtime/rest/ConnectRestConfigurable.java | 14 +- .../connect/runtime/rest/RestServer.java | 5 +- .../connect/runtime/rest/util/SSLUtils.java | 1 + .../connect/storage/OffsetStorageWriter.java | 1 + .../connect/runtime/AbstractHerderTest.java | 2 +- .../connect/runtime/ConnectMetricsTest.java | 1 + .../connect/runtime/MockConnectMetrics.java | 1 + .../SourceTaskOffsetCommitterTest.java | 9 +- .../runtime/WorkerSinkTaskThreadedTest.java | 12 +- .../kafka/connect/runtime/WorkerTest.java | 7 +- .../distributed/DistributedHerderTest.java | 2 + .../runtime/isolation/PluginsTest.java | 2 + .../connect/runtime/rest/RestServerTest.java | 7 +- .../ConnectorPluginsResourceTest.java | 9 +- .../runtime/rest/util/SSLUtilsTest.java | 1 + .../storage/FileOffsetBackingStoreTest.java | 1 + .../storage/KafkaConfigBackingStoreTest.java | 20 +- .../storage/KafkaOffsetBackingStoreTest.java | 32 +- .../kafka/connect/util/KafkaBasedLogTest.java | 1 + .../connect/transforms/ReplaceField.java | 3 +- .../kafka/connect/transforms/FlattenTest.java | 3 + .../connect/transforms/MaskFieldTest.java | 1 + .../main/scala/kafka/admin/AclCommand.scala | 3 +- .../main/scala/kafka/admin/AdminClient.scala | 2 +- .../main/scala/kafka/admin/AdminUtils.scala | 9 +- .../main/scala/kafka/admin/TopicCommand.scala | 3 +- .../kafka/controller/ControllerContext.scala | 2 +- .../kafka/controller/KafkaController.scala | 2 +- .../group/GroupMetadataManager.scala | 4 +- .../transaction/TransactionCoordinator.scala | 2 +- .../transaction/TransactionStateManager.scala | 4 +- core/src/main/scala/kafka/log/Log.scala | 2 +- .../src/main/scala/kafka/log/LogCleaner.scala | 2 +- .../scala/kafka/log/LogCleanerManager.scala | 3 - .../kafka/server/AbstractFetcherManager.scala | 1 - .../kafka/server/AbstractFetcherThread.scala | 4 +- .../main/scala/kafka/server/KafkaApis.scala | 2 +- .../scala/kafka/tools/ConsoleConsumer.scala | 8 +- .../scala/kafka/tools/ConsoleProducer.scala | 2 +- .../main/scala/kafka/utils/Mx4jLoader.scala | 4 +- .../scala/kafka/utils/json/DecodeJson.scala | 6 +- .../scala/kafka/utils/json/JsonObject.scala | 2 +- .../scala/kafka/utils/json/JsonValue.scala | 4 +- .../api/AdminClientIntegrationTest.scala | 23 +- .../kafka/api/AuthorizerIntegrationTest.scala | 20 +- .../kafka/api/BaseConsumerTest.scala | 1 - .../kafka/api/BaseProducerSendTest.scala | 17 +- ...gationTokenEndToEndAuthorizationTest.scala | 5 +- .../kafka/api/EndToEndAuthorizationTest.scala | 23 +- .../kafka/api/EndToEndClusterIdTest.scala | 15 +- .../kafka/api/LegacyAdminClientTest.scala | 2 + .../kafka/api/LogAppendTimeTest.scala | 10 +- .../kafka/api/PlaintextConsumerTest.scala | 4 +- .../kafka/api/PlaintextProducerSendTest.scala | 2 +- .../DynamicBrokerReconfigurationTest.scala | 16 +- ...nersWithSameSecurityProtocolBaseTest.scala | 9 +- .../common/InterBrokerSendThreadTest.scala | 11 +- .../unit/kafka/admin/TestAdminUtils.scala | 1 + .../TransactionMarkerChannelManagerTest.scala | 15 +- ...onMarkerRequestCompletionHandlerTest.scala | 7 +- .../unit/kafka/server/FetchSessionTest.scala | 24 +- .../unit/kafka/server/KafkaApisTest.scala | 16 +- .../unit/kafka/server/LogDirFailureTest.scala | 13 +- .../unit/kafka/server/LogOffsetTest.scala | 4 + .../unit/kafka/server/RequestQuotaTest.scala | 2 +- ...venReplicationProtocolAcceptanceTest.scala | 32 +- .../kafka/tools/ConsoleProducerTest.scala | 2 +- .../scala/unit/kafka/utils/TestUtils.scala | 4 +- .../scala/unit/kafka/utils/ZkUtilsTest.scala | 1 + .../unit/kafka/zk/AdminZkClientTest.scala | 2 +- .../kafka/zookeeper/ZooKeeperClientTest.scala | 6 +- .../jmh/producer/ProducerRecordBenchmark.java | 2 +- .../examples/pageview/PageViewTypedDemo.java | 4 +- .../pageview/PageViewUntypedDemo.java | 4 +- .../kstream/internals/KStreamImpl.java | 5 +- .../streams/kstream/internals/KTableImpl.java | 9 +- .../kstream/internals/SerializedInternal.java | 5 +- .../internals/graph/KTableKTableJoinNode.java | 37 +-- .../graph/OptimizableRepartitionNode.java | 2 +- .../internals/graph/StreamSinkNode.java | 1 + .../internals/graph/TableProcessorNode.java | 4 +- .../internals/graph/TableSourceNode.java | 3 +- .../internals/InternalTopologyBuilder.java | 4 +- .../kafka/streams/state/WindowStore.java | 3 + .../state/internals/CachingWindowStore.java | 5 +- .../ChangeLoggingWindowBytesStore.java | 5 +- .../CompositeReadOnlyWindowStore.java | 1 + .../state/internals/MeteredWindowStore.java | 3 + .../state/internals/RocksDBWindowStore.java | 3 + .../kafka/streams/KafkaStreamsTest.java | 7 +- .../kafka/streams/StreamsBuilderTest.java | 9 +- .../apache/kafka/streams/TopologyTest.java | 2 +- .../FineGrainedAutoResetIntegrationTest.java | 13 +- ...StreamAggregationDedupIntegrationTest.java | 8 +- .../KStreamAggregationIntegrationTest.java | 12 +- .../integration/RestoreIntegrationTest.java | 3 +- .../utils/EmbeddedKafkaCluster.java | 9 +- .../integration/utils/KafkaEmbedded.java | 8 +- .../streams/kstream/JoinWindowsTest.java | 2 +- .../kstream/RepartitionTopicNamingTest.java | 1 + .../streams/kstream/SessionWindowsTest.java | 1 + .../streams/kstream/TimeWindowsTest.java | 1 + .../streams/kstream/UnlimitedWindowsTest.java | 1 + .../kstream/internals/AbstractStreamTest.java | 11 +- .../internals/InternalStreamsBuilderTest.java | 6 +- .../internals/KGroupedStreamImplTest.java | 5 +- .../internals/KGroupedTableImplTest.java | 11 +- .../kstream/internals/KStreamImplTest.java | 24 +- .../internals/KStreamTransformTest.java | 4 +- .../internals/KStreamTransformValuesTest.java | 1 + .../internals/KStreamWindowAggregateTest.java | 7 +- .../internals/KStreamWindowReduceTest.java | 6 +- .../internals/KTableAggregateTest.java | 24 +- .../kstream/internals/KTableFilterTest.java | 1 + .../kstream/internals/KTableImplTest.java | 17 +- .../internals/KTableKTableInnerJoinTest.java | 7 +- .../internals/KTableKTableLeftJoinTest.java | 11 +- .../internals/KTableKTableOuterJoinTest.java | 7 +- .../internals/KTableKTableRightJoinTest.java | 1 + .../internals/KTableMapValuesTest.java | 5 +- .../kstream/internals/KTableSourceTest.java | 7 +- .../internals/KTableTransformValuesTest.java | 10 +- .../SessionWindowedKStreamImplTest.java | 4 +- .../internals/SuppressScenarioTest.java | 3 + .../TimeWindowedKStreamImplTest.java | 4 +- .../kafka/streams/perf/YahooBenchmark.java | 5 +- ...UsePreviousTimeOnInvalidTimestampTest.java | 2 +- .../AbstractProcessorContextTest.java | 3 + .../internals/GlobalStateManagerImplTest.java | 6 +- .../internals/GlobalStateTaskTest.java | 5 +- .../InternalTopologyBuilderTest.java | 10 +- .../internals/RecordCollectorTest.java | 8 +- .../processor/internals/SinkNodeTest.java | 19 +- .../processor/internals/SourceNodeTest.java | 10 +- .../processor/internals/StandbyTaskTest.java | 35 ++- .../internals/StoreChangelogReaderTest.java | 5 +- .../processor/internals/StreamTaskTest.java | 36 +-- .../processor/internals/StreamThreadTest.java | 6 +- .../internals/StreamsMetadataStateTest.java | 2 +- .../StreamsPartitionAssignorTest.java | 55 ++-- .../processor/internals/TaskManagerTest.java | 16 +- .../kafka/streams/state/StateSerdesTest.java | 1 + .../internals/CachingWindowStoreTest.java | 11 +- .../ChangeLoggingKeyValueBytesStoreTest.java | 2 +- .../ChangeLoggingWindowBytesStoreTest.java | 2 +- .../internals/FilteredCacheIteratorTest.java | 6 +- .../internals/ReadOnlyWindowStoreStub.java | 5 +- .../internals/RocksDBWindowStoreTest.java | 276 +++++++++--------- .../state/internals/ThreadCacheTest.java | 4 +- .../tests/BrokerCompatibilityTest.java | 4 +- .../kafka/streams/tests/SmokeTestClient.java | 6 +- .../test/InternalMockProcessorContext.java | 5 +- .../kafka/test/NoOpProcessorContext.java | 5 +- .../streams/scala/kstream/KGroupedTable.scala | 2 +- .../streams/MockProcessorContextTest.java | 2 +- .../kafka/streams/TopologyTestDriverTest.java | 2 +- .../kafka/tools/ClientCompatibilityTest.java | 3 +- .../trogdor/common/JsonSerializationTest.java | 1 + 197 files changed, 800 insertions(+), 814 deletions(-) diff --git a/build.gradle b/build.gradle index bba3d839671..4d514dfcf97 100644 --- a/build.gradle +++ b/build.gradle @@ -155,9 +155,14 @@ subprojects { sourceCompatibility = minJavaVersion targetCompatibility = minJavaVersion - compileJava { + tasks.withType(JavaCompile) { options.encoding = 'UTF-8' - options.compilerArgs << "-Xlint:deprecation,unchecked" + options.compilerArgs << "-Xlint:all" + // temporary exclusions until all the warnings are fixed + options.compilerArgs << "-Xlint:-rawtypes" + options.compilerArgs << "-Xlint:-serial" + options.compilerArgs << "-Xlint:-try" + options.compilerArgs << "-Werror" // --release is the recommended way to select the target release, but it's only supported in Java 9 so we also // set --source and --target via `sourceCompatibility` and `targetCompatibility`. If/when Gradle supports `--release` // natively (https://github.com/gradle/gradle/issues/2510), we should switch to that. diff --git a/clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java b/clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java index 16990ac79c5..30ae65f2309 100644 --- a/clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java @@ -296,7 +296,7 @@ public class FetchSessionHandler { * @param response The response. * @return True if the full fetch response partitions are valid. */ - private String verifyFullFetchResponsePartitions(FetchResponse response) { + private String verifyFullFetchResponsePartitions(FetchResponse response) { StringBuilder bld = new StringBuilder(); Set omitted = findMissing(response.responseData().keySet(), sessionPartitions.keySet()); @@ -321,7 +321,7 @@ public class FetchSessionHandler { * @param response The response. * @return True if the incremental fetch response partitions are valid. */ - private String verifyIncrementalFetchResponsePartitions(FetchResponse response) { + private String verifyIncrementalFetchResponsePartitions(FetchResponse response) { Set extra = findMissing(response.responseData().keySet(), sessionPartitions.keySet()); if (!extra.isEmpty()) { @@ -340,7 +340,7 @@ public class FetchSessionHandler { * @param response The FetchResponse. * @return The string to log. */ - private String responseDataToLogString(FetchResponse response) { + private String responseDataToLogString(FetchResponse response) { if (!log.isTraceEnabled()) { int implied = sessionPartitions.size() - response.responseData().size(); if (implied > 0) { @@ -376,7 +376,7 @@ public class FetchSessionHandler { * @return True if the response is well-formed; false if it can't be processed * because of missing or unexpected partitions. */ - public boolean handleResponse(FetchResponse response) { + public boolean handleResponse(FetchResponse response) { if (response.error() != Errors.NONE) { log.info("Node {} was unable to process the fetch request with {}: {}.", node, nextMetadata, response.error()); diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java b/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java index e8da646e73f..9976108acb7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java @@ -58,6 +58,7 @@ public class ConfigEntry { * @param isReadOnly whether the config is read-only and cannot be updated * @deprecated since 1.1.0. This constructor will be removed in a future release. */ + @Deprecated public ConfigEntry(String name, String value, boolean isDefault, boolean isSensitive, boolean isReadOnly) { this(name, value, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 065e6631df4..3a756721fd8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -1024,7 +1024,7 @@ public class KafkaConsumer implements Consumer { public void unsubscribe() { acquireAndEnsureOpen(); try { - fetcher.clearBufferedDataForUnassignedPartitions(Collections.EMPTY_SET); + fetcher.clearBufferedDataForUnassignedPartitions(Collections.emptySet()); this.subscriptions.unsubscribe(); this.coordinator.maybeLeaveGroup(); this.metadata.needMetadataForAllTopics(false); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index 9eee6da9d55..f1dcb3279d3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -408,6 +408,7 @@ public class MockConsumer implements Consumer { close(KafkaConsumer.DEFAULT_CLOSE_TIMEOUT_MS, TimeUnit.MILLISECONDS); } + @SuppressWarnings("deprecation") @Override public synchronized void close(long timeout, TimeUnit unit) { ensureNotClosed(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 93abc4056f8..265fc99721d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -222,6 +222,7 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { @Override public void onSuccess(ClientResponse resp) { synchronized (Fetcher.this) { + @SuppressWarnings("unchecked") FetchResponse response = (FetchResponse) resp.responseBody(); FetchSessionHandler handler = sessionHandler(fetchTarget.id()); if (handler == null) { @@ -239,7 +240,7 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { for (Map.Entry> entry : response.responseData().entrySet()) { TopicPartition partition = entry.getKey(); long fetchOffset = data.sessionPartitions().get(partition).fetchOffset; - FetchResponse.PartitionData fetchData = entry.getValue(); + FetchResponse.PartitionData fetchData = entry.getValue(); log.debug("Fetch {} at offset {} for partition {} returned fetch data {}", isolationLevel, fetchOffset, partition, fetchData); @@ -1280,18 +1281,12 @@ public class Fetcher implements SubscriptionState.Listener, Closeable { return batch.isTransactional() && abortedProducerIds.contains(batch.producerId()); } - private PriorityQueue abortedTransactions(FetchResponse.PartitionData partition) { + private PriorityQueue abortedTransactions(FetchResponse.PartitionData partition) { if (partition.abortedTransactions == null || partition.abortedTransactions.isEmpty()) return null; PriorityQueue abortedTransactions = new PriorityQueue<>( - partition.abortedTransactions.size(), - new Comparator() { - @Override - public int compare(FetchResponse.AbortedTransaction o1, FetchResponse.AbortedTransaction o2) { - return Long.compare(o1.firstOffset, o2.firstOffset); - } - } + partition.abortedTransactions.size(), Comparator.comparingLong(o -> o.firstOffset) ); abortedTransactions.addAll(partition.abortedTransactions); return abortedTransactions; diff --git a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java index a6696f79f66..9410fddef50 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java @@ -289,6 +289,7 @@ public class SslTransportLayer implements TransportLayer { } } + @SuppressWarnings("fallthrough") private void doHandshake() throws IOException { boolean read = key.isReadable(); boolean write = key.isWritable(); diff --git a/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecords.java b/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecords.java index 4e844736bbd..217870d5faa 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecords.java @@ -59,7 +59,7 @@ public class LazyDownConversionRecords implements BaseRecords { // need to make sure that we are able to accommodate one full batch of down-converted messages. The way we achieve // this is by having sizeInBytes method factor in the size of the first down-converted batch and return at least // its size. - java.util.Iterator it = iterator(0); + java.util.Iterator> it = iterator(0); if (it.hasNext()) { firstConvertedBatch = it.next(); sizeInBytes = Math.max(records.sizeInBytes(), firstConvertedBatch.records().sizeInBytes()); @@ -106,7 +106,7 @@ public class LazyDownConversionRecords implements BaseRecords { return result; } - public java.util.Iterator iterator(long maximumReadSize) { + public java.util.Iterator> iterator(long maximumReadSize) { // We typically expect only one iterator instance to be created, so null out the first converted batch after // first use to make it available for GC. ConvertedRecords firstBatch = firstConvertedBatch; @@ -119,7 +119,7 @@ public class LazyDownConversionRecords implements BaseRecords { * it as memory-efficient as possible by not having to maintain all down-converted records in-memory. Maintains * a view into batches of down-converted records. */ - private class Iterator extends AbstractIterator { + private class Iterator extends AbstractIterator> { private final AbstractIterator batchIterator; private final long maximumReadSize; private ConvertedRecords firstConvertedBatch; @@ -130,7 +130,7 @@ public class LazyDownConversionRecords implements BaseRecords { * {@link #makeNext()}. This is a soft limit as {@link #makeNext()} will always convert * and return at least one full message batch. */ - private Iterator(Records recordsToDownConvert, long maximumReadSize, ConvertedRecords firstConvertedBatch) { + private Iterator(Records recordsToDownConvert, long maximumReadSize, ConvertedRecords firstConvertedBatch) { this.batchIterator = recordsToDownConvert.batchIterator(); this.maximumReadSize = maximumReadSize; this.firstConvertedBatch = firstConvertedBatch; diff --git a/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecordsSend.java b/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecordsSend.java index ad1f97fa0cb..62551d483fa 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecordsSend.java +++ b/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecordsSend.java @@ -37,7 +37,7 @@ public final class LazyDownConversionRecordsSend extends RecordsSend convertedRecordsIterator; + private Iterator> convertedRecordsIterator; public LazyDownConversionRecordsSend(String destination, LazyDownConversionRecords records) { super(destination, records, records.sizeInBytes()); @@ -74,8 +74,8 @@ public final class LazyDownConversionRecordsSend extends RecordsSend recordsAndStats = convertedRecordsIterator.next(); - convertedRecords = recordsAndStats.records(); + ConvertedRecords recordsAndStats = convertedRecordsIterator.next(); + convertedRecords = (MemoryRecords) recordsAndStats.records(); recordConversionStats.add(recordsAndStats.recordConversionStats()); log.debug("Down-converted records for partition {} with length={}", topicPartition(), convertedRecords.sizeInBytes()); } else { diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/CredentialCache.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/CredentialCache.java index 28923abbfce..ecf3ea9d4a6 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/CredentialCache.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/CredentialCache.java @@ -24,6 +24,7 @@ public class CredentialCache { public Cache createCache(String mechanism, Class credentialClass) { Cache cache = new Cache<>(credentialClass); + @SuppressWarnings("unchecked") Cache oldCache = (Cache) cacheMap.putIfAbsent(mechanism, cache); return oldCache == null ? cache : oldCache; } @@ -64,4 +65,4 @@ public class CredentialCache { return credentialClass; } } -} \ No newline at end of file +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java index 7bb7be7c72d..3d9481c497d 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java @@ -180,6 +180,7 @@ public class LoginManager { String configName, Class defaultClass) { String prefix = jaasContext.type() == JaasContext.Type.SERVER ? ListenerName.saslMechanismPrefix(saslMechanism) : ""; + @SuppressWarnings("unchecked") Class clazz = (Class) configs.get(prefix + configName); if (clazz != null && jaasContext.configurationEntries().size() != 1) { String errorMessage = configName + " cannot be specified with multiple login modules in the JAAS context. " + diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java index 02a42617619..78428eefbad 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java @@ -191,6 +191,7 @@ public class SaslClientAuthenticator implements Authenticator { * The messages are sent and received as size delimited bytes that consists of a 4 byte network-ordered size N * followed by N bytes representing the opaque payload. */ + @SuppressWarnings("fallthrough") public void authenticate() throws IOException { if (netOutBuffer != null && !flushNetOutBufferAndUpdateInterestOps()) return; diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java index 8b830c0c888..e141bb652af 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java @@ -84,6 +84,7 @@ public class SaslClientCallbackHandler implements AuthenticateCallbackHandler { ac.setAuthorizedID(authzId); } else if (callback instanceof ScramExtensionsCallback) { if (ScramMechanism.isScram(mechanism) && subject != null && !subject.getPublicCredentials(Map.class).isEmpty()) { + @SuppressWarnings("unchecked") Map extensions = (Map) subject.getPublicCredentials(Map.class).iterator().next(); ((ScramExtensionsCallback) callback).extensions(extensions); } diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java index 2d2ca2520b1..0583f6656e6 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java @@ -249,6 +249,7 @@ public class SaslServerAuthenticator implements Authenticator { * The messages are sent and received as size delimited bytes that consists of a 4 byte network-ordered size N * followed by N bytes representing the opaque payload. */ + @SuppressWarnings("fallthrough") @Override public void authenticate() throws IOException { if (saslState != SaslState.REAUTH_PROCESS_HANDSHAKE) { diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java b/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java index 9bfc57641b7..777ea2bdfd7 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java @@ -89,6 +89,7 @@ public class Crc32 implements Checksum { crc = 0xffffffff; } + @SuppressWarnings("fallthrough") @Override public void update(byte[] b, int off, int len) { if (off < 0 || len < 0 || off > b.length - len) diff --git a/clients/src/main/java/org/apache/kafka/common/utils/PureJavaCrc32C.java b/clients/src/main/java/org/apache/kafka/common/utils/PureJavaCrc32C.java index 3489dddec71..8abc93dc3b4 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/PureJavaCrc32C.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/PureJavaCrc32C.java @@ -53,6 +53,7 @@ public class PureJavaCrc32C implements Checksum { crc = 0xffffffff; } + @SuppressWarnings("fallthrough") @Override public void update(byte[] b, int off, int len) { int localCrc = crc; diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index f94858ce1f5..9848f8e880f 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -377,6 +377,7 @@ public final class Utils { * @param data byte array to hash * @return 32 bit hash of the given array */ + @SuppressWarnings("fallthrough") public static int murmur2(final byte[] data) { int length = data.length; int seed = 0x9747b28c; @@ -662,18 +663,10 @@ public final class Utils { */ @SafeVarargs public static Set mkSet(T... elems) { - return new HashSet<>(Arrays.asList(elems)); - } - - /* - * Creates a list - * @param elems the elements - * @param the type of element - * @return List - */ - @SafeVarargs - public static List mkList(T... elems) { - return Arrays.asList(elems); + Set result = new HashSet<>((int) (elems.length / 0.75) + 1); + for (T elem : elems) + result.add(elem); + return result; } /** diff --git a/clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java index 4c12fc68bab..ec1b0624169 100644 --- a/clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/FetchSessionHandlerTest.java @@ -130,6 +130,7 @@ public class FetchSessionHandlerTest { } } + @SafeVarargs private static void assertMapsEqual(Map expected, Map... actuals) { for (Map actual : actuals) { diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 1f62d39a963..82c5b1d81da 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -1119,6 +1119,7 @@ public class KafkaAdminClientTest { } } + @SafeVarargs private static void assertCollectionIs(Collection collection, T... elements) { for (T element : elements) { assertTrue("Did not find " + element, collection.contains(element)); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 89fca844262..987bad22219 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -388,7 +388,7 @@ public class KafkaConsumerTest { consumer.updateAssignmentMetadataIfNeeded(time.timer(Long.MAX_VALUE)); assertTrue(heartbeatReceived.get()); - consumer.close(0, TimeUnit.MILLISECONDS); + consumer.close(Duration.ofMillis(0)); } @Test @@ -421,7 +421,7 @@ public class KafkaConsumerTest { consumer.poll(Duration.ZERO); assertTrue(heartbeatReceived.get()); - consumer.close(0, TimeUnit.MILLISECONDS); + consumer.close(Duration.ofMillis(0)); } @Test @@ -446,6 +446,7 @@ public class KafkaConsumerTest { Assert.assertEquals(0, requests.size()); } + @SuppressWarnings("deprecation") @Test public void verifyDeprecatedPollDoesNotTimeOutDuringMetadataUpdate() { final Time time = new MockTime(); @@ -461,7 +462,6 @@ public class KafkaConsumerTest { consumer.subscribe(singleton(topic), getConsumerRebalanceListener(consumer)); prepareRebalance(client, node, assignor, singletonList(tp0), null); - //noinspection deprecation consumer.poll(0L); // The underlying client SHOULD get a fetch request @@ -492,7 +492,7 @@ public class KafkaConsumerTest { ConsumerRecords records = consumer.poll(Duration.ofMillis(1)); assertEquals(5, records.count()); assertEquals(55L, consumer.position(tp0)); - consumer.close(0, TimeUnit.MILLISECONDS); + consumer.close(Duration.ofMillis(0)); } @Test @@ -652,7 +652,7 @@ public class KafkaConsumerTest { offsets.put(tp1, offset2); client.prepareResponseFrom(offsetResponse(offsets, Errors.NONE), coordinator); assertEquals(offset2, consumer.committed(tp1).offset()); - consumer.close(0, TimeUnit.MILLISECONDS); + consumer.close(Duration.ofMillis(0)); } @Test @@ -687,7 +687,7 @@ public class KafkaConsumerTest { consumer.poll(Duration.ZERO); assertTrue(commitReceived.get()); - consumer.close(0, TimeUnit.MILLISECONDS); + consumer.close(Duration.ofMillis(0)); } @Test @@ -716,7 +716,7 @@ public class KafkaConsumerTest { assertEquals(singleton(topic), consumer.subscription()); assertEquals(singleton(tp0), consumer.assignment()); - consumer.close(0, TimeUnit.MILLISECONDS); + consumer.close(Duration.ofMillis(0)); } @Test @@ -753,7 +753,7 @@ public class KafkaConsumerTest { consumer.poll(Duration.ZERO); assertEquals(singleton(otherTopic), consumer.subscription()); - consumer.close(0, TimeUnit.MILLISECONDS); + consumer.close(Duration.ofMillis(0)); } @Test @@ -832,7 +832,7 @@ public class KafkaConsumerTest { // clear interrupted state again since this thread may be reused by JUnit Thread.interrupted(); } - consumer.close(0, TimeUnit.MILLISECONDS); + consumer.close(Duration.ofMillis(0)); } @Test @@ -860,7 +860,7 @@ public class KafkaConsumerTest { ConsumerRecords records = consumer.poll(Duration.ZERO); assertEquals(0, records.count()); - consumer.close(0, TimeUnit.MILLISECONDS); + consumer.close(Duration.ofMillis(0)); } /** @@ -1348,7 +1348,7 @@ public class KafkaConsumerTest { consumer.updateAssignmentMetadataIfNeeded(time.timer(Long.MAX_VALUE)); final ConsumerRecords records = consumer.poll(Duration.ZERO); assertFalse(records.isEmpty()); - consumer.close(0, TimeUnit.MILLISECONDS); + consumer.close(Duration.ofMillis(0)); } private void consumerCloseTest(final long closeTimeoutMs, @@ -1388,7 +1388,7 @@ public class KafkaConsumerTest { public void run() { consumer.commitAsync(); try { - consumer.close(closeTimeoutMs, TimeUnit.MILLISECONDS); + consumer.close(Duration.ofMillis(closeTimeoutMs)); } catch (Exception e) { closeException.set(e); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index b430078f726..1afe0ee2ec5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -42,7 +42,6 @@ import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.internals.Topic; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.FindCoordinatorResponse; import org.apache.kafka.common.requests.HeartbeatResponse; @@ -249,7 +248,7 @@ public class ConsumerCoordinatorTest { final AtomicBoolean asyncCallbackInvoked = new AtomicBoolean(false); Map offsets = singletonMap( new TopicPartition("foo", 0), new OffsetCommitRequest.PartitionData(13L, - RecordBatch.NO_PARTITION_LEADER_EPOCH, "")); + Optional.empty(), "")); consumerClient.send(coordinator.checkAndGetCoordinator(), new OffsetCommitRequest.Builder(groupId, offsets)) .compose(new RequestFutureAdapter() { @Override diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index c7b0b302ede..f4439ce1cb2 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -2575,9 +2575,9 @@ public class FetcherTest { try { Field field = FetchSessionHandler.class.getDeclaredField("sessionPartitions"); field.setAccessible(true); - LinkedHashMap sessionPartitions = - (LinkedHashMap) field.get(handler); - for (Map.Entry entry : sessionPartitions.entrySet()) { + LinkedHashMap sessionPartitions = + (LinkedHashMap) field.get(handler); + for (Map.Entry entry : sessionPartitions.entrySet()) { // If `sessionPartitions` are modified on another thread, Thread.yield will increase the // possibility of ConcurrentModificationException if appropriate synchronization is not used. Thread.yield(); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index b544a654de4..872d3909732 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -37,7 +37,6 @@ import org.apache.kafka.common.network.Selectable; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.serialization.ExtendedSerializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.LogContext; @@ -135,7 +134,7 @@ public class KafkaProducerTest { Properties props = new Properties(); props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); props.put(1, "not string key"); - try (KafkaProducer ff = new KafkaProducer(props, new StringSerializer(), new StringSerializer())) { + try (KafkaProducer ff = new KafkaProducer<>(props, new StringSerializer(), new StringSerializer())) { fail("Constructor should throw exception"); } catch (ConfigException e) { assertTrue("Unexpected exception message: " + e.getMessage(), e.getMessage().contains("not string key")); @@ -460,12 +459,14 @@ public class KafkaProducerTest { assertTrue("Topic should still exist in metadata", metadata.containsTopic(topic)); } + @SuppressWarnings("unchecked") @Test @Deprecated public void testHeadersWithExtendedClasses() { - doTestHeaders(ExtendedSerializer.class); + doTestHeaders(org.apache.kafka.common.serialization.ExtendedSerializer.class); } + @SuppressWarnings("unchecked") @Test public void testHeaders() { doTestHeaders(Serializer.class); diff --git a/clients/src/test/java/org/apache/kafka/common/KafkaFutureTest.java b/clients/src/test/java/org/apache/kafka/common/KafkaFutureTest.java index 6f9efca7c66..37cfd1eefa1 100644 --- a/clients/src/test/java/org/apache/kafka/common/KafkaFutureTest.java +++ b/clients/src/test/java/org/apache/kafka/common/KafkaFutureTest.java @@ -67,7 +67,7 @@ public class KafkaFutureTest { @Test public void testCompletingFutures() throws Exception { final KafkaFutureImpl future = new KafkaFutureImpl<>(); - CompleterThread myThread = new CompleterThread(future, "You must construct additional pylons."); + CompleterThread myThread = new CompleterThread<>(future, "You must construct additional pylons."); assertFalse(future.isDone()); assertFalse(future.isCompletedExceptionally()); assertFalse(future.isCancelled()); @@ -86,39 +86,19 @@ public class KafkaFutureTest { @Test public void testThenApply() throws Exception { KafkaFutureImpl future = new KafkaFutureImpl<>(); - KafkaFuture doubledFuture = future.thenApply(new KafkaFuture.BaseFunction() { - @Override - public Integer apply(Integer integer) { - return 2 * integer; - } - }); + KafkaFuture doubledFuture = future.thenApply(integer -> 2 * integer); assertFalse(doubledFuture.isDone()); - KafkaFuture tripledFuture = future.thenApply(new KafkaFuture.Function() { - @Override - public Integer apply(Integer integer) { - return 3 * integer; - } - }); + KafkaFuture tripledFuture = future.thenApply(integer -> 3 * integer); assertFalse(tripledFuture.isDone()); future.complete(21); assertEquals(Integer.valueOf(21), future.getNow(-1)); assertEquals(Integer.valueOf(42), doubledFuture.getNow(-1)); assertEquals(Integer.valueOf(63), tripledFuture.getNow(-1)); - KafkaFuture quadrupledFuture = future.thenApply(new KafkaFuture.BaseFunction() { - @Override - public Integer apply(Integer integer) { - return 4 * integer; - } - }); + KafkaFuture quadrupledFuture = future.thenApply(integer -> 4 * integer); assertEquals(Integer.valueOf(84), quadrupledFuture.getNow(-1)); KafkaFutureImpl futureFail = new KafkaFutureImpl<>(); - KafkaFuture futureAppliedFail = futureFail.thenApply(new KafkaFuture.BaseFunction() { - @Override - public Integer apply(Integer integer) { - return 2 * integer; - } - }); + KafkaFuture futureAppliedFail = futureFail.thenApply(integer -> 2 * integer); futureFail.completeExceptionally(new RuntimeException()); assertTrue(futureFail.isCompletedExceptionally()); assertTrue(futureAppliedFail.isCompletedExceptionally()); @@ -176,7 +156,7 @@ public class KafkaFutureTest { final int numThreads = 5; final List> futures = new ArrayList<>(); for (int i = 0; i < numThreads; i++) { - futures.add(new KafkaFutureImpl()); + futures.add(new KafkaFutureImpl<>()); } KafkaFuture allFuture = KafkaFuture.allOf(futures.toArray(new KafkaFuture[0])); final List completerThreads = new ArrayList<>(); diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java index affa5dd4436..974d39f934d 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java @@ -378,7 +378,7 @@ public class ConfigDefTest { updateModes.put("my.broker.config", "per-broker"); updateModes.put("my.cluster.config", "cluster-wide"); final String html = configDef.toHtmlTable(updateModes); - Set configsInHtml = new HashSet(); + Set configsInHtml = new HashSet<>(); for (String line : html.split("\n")) { if (line.contains("my.broker.config")) { assertTrue(line.contains("per-broker")); diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java index eb3f7752a6c..e70be2732a5 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java @@ -839,6 +839,7 @@ public class MetricsTest { * This test is to verify the deprecated {@link Metric#value()} method. * @deprecated This will be removed in a future major release. */ + @Deprecated @Test public void testDeprecatedMetricValueMethod() { verifyStats(KafkaMetric::value); diff --git a/clients/src/test/java/org/apache/kafka/common/network/ChannelBuildersTest.java b/clients/src/test/java/org/apache/kafka/common/network/ChannelBuildersTest.java index 630cba1f3e1..55fa9983cc3 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/ChannelBuildersTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/ChannelBuildersTest.java @@ -23,7 +23,6 @@ import org.apache.kafka.common.security.auth.AuthenticationContext; import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.security.auth.KafkaPrincipalBuilder; import org.apache.kafka.common.security.auth.PlaintextAuthenticationContext; -import org.apache.kafka.common.security.auth.PrincipalBuilder; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.junit.Test; @@ -66,7 +65,7 @@ public class ChannelBuildersTest { } @SuppressWarnings("deprecation") - public static class OldPrincipalBuilder implements PrincipalBuilder { + public static class OldPrincipalBuilder implements org.apache.kafka.common.security.auth.PrincipalBuilder { private static boolean configured = false; private static final String PRINCIPAL_NAME = "bob"; diff --git a/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java index 1945bccb613..20ecba1d188 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java @@ -381,7 +381,7 @@ public class FileRecordsTest { // Lazy down-conversion will not return any messages for a partial input batch TopicPartition tp = new TopicPartition("topic-1", 0); LazyDownConversionRecords lazyRecords = new LazyDownConversionRecords(tp, slice, RecordBatch.MAGIC_VALUE_V0, 0, Time.SYSTEM); - Iterator it = lazyRecords.iterator(16 * 1024L); + Iterator> it = lazyRecords.iterator(16 * 1024L); assertTrue("No messages should be returned", !it.hasNext()); } @@ -538,7 +538,7 @@ public class FileRecordsTest { for (long readSize : maximumReadSize) { TopicPartition tp = new TopicPartition("topic-1", 0); LazyDownConversionRecords lazyRecords = new LazyDownConversionRecords(tp, fileRecords, toMagic, firstOffset, Time.SYSTEM); - Iterator it = lazyRecords.iterator(readSize); + Iterator> it = lazyRecords.iterator(readSize); while (it.hasNext()) convertedRecords.add(it.next().records()); verifyConvertedRecords(initialRecords, initialOffsets, convertedRecords, compressionType, toMagic); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index e9b36a0d13f..cdb6bb4068c 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -840,6 +840,7 @@ public class RequestResponseTest { return new MetadataResponse(asList(node), null, MetadataResponse.NO_CONTROLLER_ID, allTopicMetadata); } + @SuppressWarnings("deprecation") private OffsetCommitRequest createOffsetCommitRequest(int version) { Map commitData = new HashMap<>(); commitData.put(new TopicPartition("test", 0), new OffsetCommitRequest.PartitionData(100, diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index 16c35a8e345..f4b15e8ab8e 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -35,7 +35,7 @@ import static org.junit.Assert.assertEquals; public class SerializationTest { final private String topic = "testTopic"; - final private Map, List> testData = new HashMap() { + final private Map, List> testData = new HashMap, List>() { { put(String.class, Arrays.asList("my string")); put(Short.class, Arrays.asList((short) 32767, (short) -32768)); @@ -53,10 +53,11 @@ public class SerializationTest { private class DummyClass { } + @SuppressWarnings("unchecked") @Test public void allSerdesShouldRoundtripInput() { - for (Map.Entry, List> test : testData.entrySet()) { - try (Serde serde = Serdes.serdeFrom(test.getKey())) { + for (Map.Entry, List> test : testData.entrySet()) { + try (Serde serde = Serdes.serdeFrom((Class) test.getKey())) { for (Object value : test.getValue()) { assertEquals("Should get the original " + test.getKey().getSimpleName() + " after serialization and deserialization", value, diff --git a/connect/api/src/test/java/org/apache/kafka/connect/util/ConnectorUtilsTest.java b/connect/api/src/test/java/org/apache/kafka/connect/util/ConnectorUtilsTest.java index ea53084b181..771bba0328b 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/util/ConnectorUtilsTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/util/ConnectorUtilsTest.java @@ -55,8 +55,8 @@ public class ConnectorUtilsTest { Arrays.asList(3), Arrays.asList(4), Arrays.asList(5), - Collections.EMPTY_LIST, - Collections.EMPTY_LIST), grouped); + Collections.emptyList(), + Collections.emptyList()), grouped); } @Test(expected = IllegalArgumentException.class) diff --git a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java index 5bc27db7cd1..1510a37ba00 100644 --- a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java +++ b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java @@ -69,7 +69,7 @@ public class JsonConverterTest { @Before public void setUp() { - converter.configure(Collections.EMPTY_MAP, false); + converter.configure(Collections.emptyMap(), false); } // Schema metadata diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/converters/NumberConverter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/NumberConverter.java index 131a0975bfa..0af4aacbbd6 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/converters/NumberConverter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/converters/NumberConverter.java @@ -86,6 +86,7 @@ abstract class NumberConverter implements Converter, HeaderCon configure(conf); } + @SuppressWarnings("unchecked") protected T cast(Object value) { return (T) value; } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java index 10096a59125..efcc01d2c45 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java @@ -162,6 +162,7 @@ public class ConnectorConfig extends AbstractConfig { .define(VALUE_CONVERTER_CLASS_CONFIG, Type.CLASS, null, Importance.LOW, VALUE_CONVERTER_CLASS_DOC, COMMON_GROUP, ++orderInGroup, Width.SHORT, VALUE_CONVERTER_CLASS_DISPLAY) .define(HEADER_CONVERTER_CLASS_CONFIG, Type.CLASS, HEADER_CONVERTER_CLASS_DEFAULT, Importance.LOW, HEADER_CONVERTER_CLASS_DOC, COMMON_GROUP, ++orderInGroup, Width.SHORT, HEADER_CONVERTER_CLASS_DISPLAY) .define(TRANSFORMS_CONFIG, Type.LIST, Collections.emptyList(), ConfigDef.CompositeValidator.of(new ConfigDef.NonNullValidator(), new ConfigDef.Validator() { + @SuppressWarnings("unchecked") @Override public void ensureValid(String name, Object value) { final List transformAliases = (List) value; @@ -247,14 +248,15 @@ public class ConnectorConfig extends AbstractConfig { final List> transformations = new ArrayList<>(transformAliases.size()); for (String alias : transformAliases) { final String prefix = TRANSFORMS_CONFIG + "." + alias + "."; - final Transformation transformation; try { - transformation = getClass(prefix + "type").asSubclass(Transformation.class).newInstance(); + @SuppressWarnings("unchecked") + final Transformation transformation = getClass(prefix + "type").asSubclass(Transformation.class) + .getDeclaredConstructor().newInstance(); + transformation.configure(originalsWithPrefix(prefix)); + transformations.add(transformation); } catch (Exception e) { throw new ConnectException(e); } - transformation.configure(originalsWithPrefix(prefix)); - transformations.add(transformation); } return transformations; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java index df73a434d31..6e021b90a07 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java @@ -96,6 +96,7 @@ public class Worker { private SourceTaskOffsetCommitter sourceTaskOffsetCommitter; private WorkerConfigTransformer workerConfigTransformer; + @SuppressWarnings("deprecation") public Worker( String workerId, Time time, diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java index 55d4860b2e6..b7fe74f9064 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java @@ -141,6 +141,7 @@ public class WorkerConnector { return state == State.STARTED; } + @SuppressWarnings("fallthrough") private void pause() { try { switch (state) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/RetryWithToleranceOperator.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/RetryWithToleranceOperator.java index eadf276adb3..25135144755 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/RetryWithToleranceOperator.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/RetryWithToleranceOperator.java @@ -190,6 +190,7 @@ public class RetryWithToleranceOperator { } // Visible for testing + @SuppressWarnings("fallthrough") boolean withinToleranceLimits() { switch (errorToleranceType) { case NONE: diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoader.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoader.java index 6104dd4426a..460df39db3d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoader.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoader.java @@ -151,13 +151,8 @@ public class DelegatingClassLoader extends URLClassLoader { final URL[] urls, final ClassLoader parent ) { - return (PluginClassLoader) AccessController.doPrivileged( - new PrivilegedAction() { - @Override - public Object run() { - return new PluginClassLoader(pluginLocation, urls, parent); - } - } + return AccessController.doPrivileged( + (PrivilegedAction) () -> new PluginClassLoader(pluginLocation, urls, parent) ); } @@ -331,6 +326,7 @@ public class DelegatingClassLoader extends URLClassLoader { return result; } + @SuppressWarnings("unchecked") private Collection> getServiceLoaderPluginDesc(Class klass, ClassLoader loader) { ServiceLoader serviceLoader = ServiceLoader.load(klass, loader); Collection> result = new ArrayList<>(); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/Plugins.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/Plugins.java index e766cbd9060..e7cb16db1bc 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/Plugins.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/Plugins.java @@ -62,13 +62,8 @@ public class Plugins { } private static DelegatingClassLoader newDelegatingClassLoader(final List paths) { - return (DelegatingClassLoader) AccessController.doPrivileged( - new PrivilegedAction() { - @Override - public Object run() { - return new DelegatingClassLoader(paths); - } - } + return AccessController.doPrivileged( + (PrivilegedAction) () -> new DelegatingClassLoader(paths) ); } @@ -102,6 +97,7 @@ public class Plugins { ); } + @SuppressWarnings("deprecation") protected static boolean isInternalConverter(String classPropertyName) { return classPropertyName.equals(WorkerConfig.INTERNAL_KEY_CONVERTER_CLASS_CONFIG) || classPropertyName.equals(WorkerConfig.INTERNAL_VALUE_CONVERTER_CLASS_CONFIG); @@ -243,6 +239,7 @@ public class Plugins { } // Determine whether this is a key or value converter based upon the supplied property name ... + @SuppressWarnings("deprecation") final boolean isKeyConverter = WorkerConfig.KEY_CONVERTER_CLASS_CONFIG.equals(classPropertyName) || WorkerConfig.INTERNAL_KEY_CONVERTER_CLASS_CONFIG.equals(classPropertyName); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/ConnectRestConfigurable.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/ConnectRestConfigurable.java index c9c2c3bbd82..0d5cbd6333b 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/ConnectRestConfigurable.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/ConnectRestConfigurable.java @@ -73,7 +73,7 @@ public class ConnectRestConfigurable implements Configurable { } @Override - public ResourceConfig register(Object component, Map contracts) { + public ResourceConfig register(Object component, Map, Integer> contracts) { if (allowedToRegister(component)) { resourceConfig.register(component, contracts); } @@ -81,7 +81,7 @@ public class ConnectRestConfigurable implements Configurable { } @Override - public ResourceConfig register(Object component, Class[] contracts) { + public ResourceConfig register(Object component, Class... contracts) { if (allowedToRegister(component)) { resourceConfig.register(component, contracts); } @@ -89,7 +89,7 @@ public class ConnectRestConfigurable implements Configurable { } @Override - public ResourceConfig register(Class componentClass, Map contracts) { + public ResourceConfig register(Class componentClass, Map, Integer> contracts) { if (allowedToRegister(componentClass)) { resourceConfig.register(componentClass, contracts); } @@ -97,7 +97,7 @@ public class ConnectRestConfigurable implements Configurable { } @Override - public ResourceConfig register(Class componentClass, Class[] contracts) { + public ResourceConfig register(Class componentClass, Class... contracts) { if (allowedToRegister(componentClass)) { resourceConfig.register(componentClass, contracts); } @@ -105,7 +105,7 @@ public class ConnectRestConfigurable implements Configurable { } @Override - public ResourceConfig register(Class componentClass, int priority) { + public ResourceConfig register(Class componentClass, int priority) { if (allowedToRegister(componentClass)) { resourceConfig.register(componentClass, priority); } @@ -113,7 +113,7 @@ public class ConnectRestConfigurable implements Configurable { } @Override - public ResourceConfig register(Class componentClass) { + public ResourceConfig register(Class componentClass) { if (allowedToRegister(componentClass)) { resourceConfig.register(componentClass); } @@ -128,7 +128,7 @@ public class ConnectRestConfigurable implements Configurable { return ALLOWED_TO_REGISTER; } - private boolean allowedToRegister(Class componentClass) { + private boolean allowedToRegister(Class componentClass) { if (resourceConfig.isRegistered(componentClass)) { log.warn("The resource {} is already registered", componentClass); return NOT_ALLOWED_TO_REGISTER; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java index 5a589db8858..15386430bc5 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java @@ -77,7 +77,7 @@ public class RestServer { private final WorkerConfig config; private Server jettyServer; - private List connectRestExtensions = Collections.EMPTY_LIST; + private List connectRestExtensions = Collections.emptyList(); /** * Create a REST server for this herder using the specified configs. @@ -92,6 +92,7 @@ public class RestServer { createConnectors(listeners); } + @SuppressWarnings("deprecation") List parseListeners() { List listeners = config.getList(WorkerConfig.LISTENERS_CONFIG); if (listeners == null || listeners.size() == 0) { @@ -322,4 +323,4 @@ public class RestServer { return base + path; } -} \ No newline at end of file +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java index 980d7c229f0..a50a0b0439c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java @@ -104,6 +104,7 @@ public class SSLUtils { /** * Configures Protocol, Algorithm and Provider related settings in SslContextFactory */ + @SuppressWarnings("unchecked") protected static void configureSslContextFactoryAlgorithms(SslContextFactory ssl, Map sslConfigValues) { List sslEnabledProtocols = (List) getOrDefault(sslConfigValues, SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList(COMMA_WITH_WHITESPACE.split(SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS))); ssl.setIncludeProtocols(sslEnabledProtocols.toArray(new String[sslEnabledProtocols.size()])); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java index 3239b6734cd..c360d4373b0 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageWriter.java @@ -89,6 +89,7 @@ public class OffsetStorageWriter { * @param partition the partition to store an offset for * @param offset the offset */ + @SuppressWarnings("unchecked") public synchronized void offset(Map partition, Map offset) { data.put((Map) partition, (Map) offset); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java index b65612ab095..fc557c641b7 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java @@ -241,7 +241,7 @@ public class AbstractHerderTest { AbstractHerder herder = createConfigValidationHerder(TestSinkConnector.class); replayAll(); - Map config = new HashMap(); + Map config = new HashMap<>(); config.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, TestSinkConnector.class.getName()); config.put(SinkConnectorConfig.TOPICS_CONFIG, "topic1,topic2"); config.put(SinkConnectorConfig.TOPICS_REGEX_CONFIG, "topic.*"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectMetricsTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectMetricsTest.java index 25395f97cc6..fe15c01c0b0 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectMetricsTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectMetricsTest.java @@ -37,6 +37,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertSame; +@SuppressWarnings("deprecation") public class ConnectMetricsTest { private static final Map DEFAULT_WORKER_CONFIG = new HashMap<>(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java index 3f65029a193..64ea526373a 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java @@ -40,6 +40,7 @@ import java.util.Map; * If the same metric is created a second time (e.g., a worker task is re-created), the new metric will replace * the previous metric in the custom reporter. */ +@SuppressWarnings("deprecation") public class MockConnectMetrics extends ConnectMetrics { private static final Map DEFAULT_WORKER_CONFIG = new HashMap<>(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java index c7cb08bfb52..baf0d8e0ebe 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java @@ -48,12 +48,12 @@ import static org.junit.Assert.fail; @RunWith(PowerMockRunner.class) public class SourceTaskOffsetCommitterTest extends ThreadedTest { - private final ConcurrentHashMap committers = new ConcurrentHashMap<>(); + private final ConcurrentHashMap> committers = new ConcurrentHashMap<>(); @Mock private ScheduledExecutorService executor; @Mock private Logger mockLog; - @Mock private ScheduledFuture commitFuture; - @Mock private ScheduledFuture taskFuture; + @Mock private ScheduledFuture commitFuture; + @Mock private ScheduledFuture taskFuture; @Mock private ConnectorTaskId taskId; @Mock private WorkerSourceTask task; @@ -79,6 +79,7 @@ public class SourceTaskOffsetCommitterTest extends ThreadedTest { Whitebox.setInternalState(SourceTaskOffsetCommitter.class, "log", mockLog); } + @SuppressWarnings("unchecked") @Test public void testSchedule() { Capture taskWrapper = EasyMock.newCapture(); @@ -86,7 +87,7 @@ public class SourceTaskOffsetCommitterTest extends ThreadedTest { EasyMock.expect(executor.scheduleWithFixedDelay( EasyMock.capture(taskWrapper), eq(DEFAULT_OFFSET_COMMIT_INTERVAL_MS), eq(DEFAULT_OFFSET_COMMIT_INTERVAL_MS), eq(TimeUnit.MILLISECONDS)) - ).andReturn(commitFuture); + ).andReturn((ScheduledFuture) commitFuture); PowerMock.replayAll(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java index d0089e92b6b..d49c1cd99ff 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java @@ -113,7 +113,7 @@ public class WorkerSinkTaskThreadedTest extends ThreadedTest { @Mock private Converter keyConverter; @Mock private Converter valueConverter; @Mock private HeaderConverter headerConverter; - @Mock private TransformationChain transformationChain; + @Mock private TransformationChain transformationChain; private WorkerSinkTask workerTask; @Mock private KafkaConsumer consumer; private Capture rebalanceListener = EasyMock.newCapture(); @@ -141,7 +141,7 @@ public class WorkerSinkTaskThreadedTest extends ThreadedTest { WorkerSinkTask.class, new String[]{"createConsumer"}, taskId, sinkTask, statusListener, initialState, workerConfig, ClusterConfigState.EMPTY, metrics, keyConverter, valueConverter, headerConverter, - new TransformationChain(Collections.emptyList(), RetryWithToleranceOperatorTest.NOOP_OPERATOR), + new TransformationChain<>(Collections.emptyList(), RetryWithToleranceOperatorTest.NOOP_OPERATOR), pluginLoader, time, RetryWithToleranceOperatorTest.NOOP_OPERATOR); recordsReturned = 0; @@ -578,12 +578,8 @@ public class WorkerSinkTaskThreadedTest extends ThreadedTest { EasyMock.expect(valueConverter.toConnectData(TOPIC, RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)).anyTimes(); final Capture recordCapture = EasyMock.newCapture(); - EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture))).andAnswer(new IAnswer() { - @Override - public SinkRecord answer() { - return recordCapture.getValue(); - } - }).anyTimes(); + EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture))).andAnswer( + (IAnswer) () -> recordCapture.getValue()).anyTimes(); Capture> capturedRecords = EasyMock.newCapture(CaptureType.ALL); sinkTask.put(EasyMock.capture(capturedRecords)); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index 4dd560a63e1..f3cacc469da 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -488,7 +488,7 @@ public class WorkerTest extends ThreadedTest { anyObject(JsonConverter.class), anyObject(JsonConverter.class), anyObject(JsonConverter.class), - EasyMock.eq(new TransformationChain(Collections.emptyList(), NOOP_OPERATOR)), + EasyMock.eq(new TransformationChain<>(Collections.emptyList(), NOOP_OPERATOR)), anyObject(KafkaProducer.class), anyObject(OffsetStorageReader.class), anyObject(OffsetStorageWriter.class), @@ -627,7 +627,7 @@ public class WorkerTest extends ThreadedTest { anyObject(JsonConverter.class), anyObject(JsonConverter.class), anyObject(JsonConverter.class), - EasyMock.eq(new TransformationChain(Collections.emptyList(), NOOP_OPERATOR)), + EasyMock.eq(new TransformationChain<>(Collections.emptyList(), NOOP_OPERATOR)), anyObject(KafkaProducer.class), anyObject(OffsetStorageReader.class), anyObject(OffsetStorageWriter.class), @@ -720,7 +720,7 @@ public class WorkerTest extends ThreadedTest { EasyMock.capture(keyConverter), EasyMock.capture(valueConverter), EasyMock.capture(headerConverter), - EasyMock.eq(new TransformationChain(Collections.emptyList(), NOOP_OPERATOR)), + EasyMock.eq(new TransformationChain<>(Collections.emptyList(), NOOP_OPERATOR)), anyObject(KafkaProducer.class), anyObject(OffsetStorageReader.class), anyObject(OffsetStorageWriter.class), @@ -859,6 +859,7 @@ public class WorkerTest extends ThreadedTest { expectConverters(JsonConverter.class, expectDefaultConverters); } + @SuppressWarnings("deprecation") private void expectConverters(Class converterClass, Boolean expectDefaultConverters) { // As default converters are instantiated when a task starts, they are expected only if the `startTask` method is called if (expectDefaultConverters) { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java index 5f450b1693e..1217ef9a7c5 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java @@ -80,6 +80,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +@SuppressWarnings("deprecation") @RunWith(PowerMockRunner.class) @PrepareForTest({DistributedHerder.class, Plugins.class}) @PowerMockIgnore("javax.management.*") @@ -492,6 +493,7 @@ public class DistributedHerderTest { PowerMock.verifyAll(); } + @SuppressWarnings("unchecked") @Test public void testConnectorNameConflictsWithWorkerGroupId() throws Exception { EasyMock.expect(member.memberId()).andStubReturn("leader"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java index 5c8aa29f338..1100910df5f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java @@ -67,6 +67,7 @@ public class PluginsTest { plugins = new Plugins(pluginProps); } + @SuppressWarnings("deprecation") @Before public void setup() { props = new HashMap<>(pluginProps); @@ -103,6 +104,7 @@ public class PluginsTest { assertEquals("foo2", converter.configs.get("extra.config")); } + @SuppressWarnings("deprecation") @Test public void shouldInstantiateAndConfigureInternalConverters() { instantiateAndConfigureInternalConverter(WorkerConfig.INTERNAL_KEY_CONVERTER_CLASS_CONFIG, ClassLoaderUsage.CURRENT_CLASSLOADER); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java index 2f8704ae4e8..c66ce36d8b0 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java @@ -64,6 +64,7 @@ public class RestServerTest { server.stop(); } + @SuppressWarnings("deprecation") private Map baseWorkerProps() { Map workerProps = new HashMap<>(); workerProps.put(DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG, "status-topic"); @@ -89,6 +90,7 @@ public class RestServerTest { checkCORSRequest("", "http://bar.com", null, null); } + @SuppressWarnings("deprecation") @Test public void testParseListeners() { // Use listeners field @@ -108,6 +110,7 @@ public class RestServerTest { Assert.assertArrayEquals(new String[] {"http://my-hostname:8080"}, server.parseListeners().toArray()); } + @SuppressWarnings("deprecation") @Test public void testAdvertisedUri() { // Advertised URI from listeenrs without protocol @@ -165,10 +168,10 @@ public class RestServerTest { System.setProperty("sun.net.http.allowRestrictedHeaders", "true"); EasyMock.expect(herder.plugins()).andStubReturn(plugins); - EasyMock.expect(plugins.newPlugins(Collections.EMPTY_LIST, + EasyMock.expect(plugins.newPlugins(Collections.emptyList(), workerConfig, ConnectRestExtension.class)) - .andStubReturn(Collections.EMPTY_LIST); + .andStubReturn(Collections.emptyList()); final Capture>> connectorsCallback = EasyMock.newCapture(); herder.connectors(EasyMock.capture(connectorsCallback)); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java index f9b5ad78d32..a3aee6a407d 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java @@ -159,11 +159,14 @@ public class ConnectorPluginsResourceTest { try { for (Class klass : abstractConnectorClasses) { - CONNECTOR_PLUGINS.add( - new MockConnectorPluginDesc((Class) klass, "0.0.0")); + @SuppressWarnings("unchecked") + MockConnectorPluginDesc pluginDesc = new MockConnectorPluginDesc((Class) klass, "0.0.0"); + CONNECTOR_PLUGINS.add(pluginDesc); } for (Class klass : connectorClasses) { - CONNECTOR_PLUGINS.add(new MockConnectorPluginDesc((Class) klass)); + @SuppressWarnings("unchecked") + MockConnectorPluginDesc pluginDesc = new MockConnectorPluginDesc((Class) klass); + CONNECTOR_PLUGINS.add(pluginDesc); } } catch (Exception e) { throw new RuntimeException(e); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/util/SSLUtilsTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/util/SSLUtilsTest.java index b8b7114cf23..63595d65369 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/util/SSLUtilsTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/util/SSLUtilsTest.java @@ -29,6 +29,7 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Map; +@SuppressWarnings("deprecation") public class SSLUtilsTest { private static final Map DEFAULT_CONFIG = new HashMap<>(); static { 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 c6b61b4d204..df955f8813d 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 @@ -47,6 +47,7 @@ public class FileOffsetBackingStoreTest { firstSet.put(null, null); } + @SuppressWarnings("deprecation") @Before public void setup() throws IOException { store = new FileOffsetBackingStore(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java index 8a02c1e68b7..68c447a1aeb 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java @@ -63,7 +63,7 @@ import static org.junit.Assert.assertTrue; @RunWith(PowerMockRunner.class) @PrepareForTest(KafkaConfigBackingStore.class) @PowerMockIgnore("javax.management.*") -@SuppressWarnings("unchecked") +@SuppressWarnings({"unchecked", "deprecation"}) public class KafkaConfigBackingStoreTest { private static final String TOPIC = "connect-configs"; private static final short TOPIC_REPLICATION_FACTOR = 5; @@ -154,7 +154,7 @@ public class KafkaConfigBackingStoreTest { @Test public void testStartStop() throws Exception { expectConfigure(); - expectStart(Collections.EMPTY_LIST, Collections.EMPTY_MAP); + expectStart(Collections.emptyList(), Collections.emptyMap()); expectStop(); PowerMock.replayAll(); @@ -179,7 +179,7 @@ public class KafkaConfigBackingStoreTest { @Test public void testPutConnectorConfig() throws Exception { expectConfigure(); - expectStart(Collections.EMPTY_LIST, Collections.EMPTY_MAP); + expectStart(Collections.emptyList(), Collections.emptyMap()); expectConvertWriteAndRead( CONNECTOR_CONFIG_KEYS.get(0), KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(0), @@ -241,10 +241,10 @@ public class KafkaConfigBackingStoreTest { @Test public void testPutTaskConfigs() throws Exception { expectConfigure(); - expectStart(Collections.EMPTY_LIST, Collections.EMPTY_MAP); + expectStart(Collections.emptyList(), Collections.emptyMap()); // Task configs should read to end, write to the log, read to end, write root, then read to end again - expectReadToEnd(new LinkedHashMap()); + expectReadToEnd(new LinkedHashMap<>()); expectConvertWriteRead( TASK_CONFIG_KEYS.get(0), KafkaConfigBackingStore.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(0), "properties", SAMPLE_CONFIGS.get(0)); @@ -274,7 +274,7 @@ public class KafkaConfigBackingStoreTest { configStorage.start(); // Bootstrap as if we had already added the connector, but no tasks had been added yet - whiteboxAddConnector(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), Collections.EMPTY_LIST); + whiteboxAddConnector(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), Collections.emptyList()); // Null before writing ClusterConfigState configState = configStorage.snapshot(); @@ -305,10 +305,10 @@ public class KafkaConfigBackingStoreTest { @Test public void testPutTaskConfigsZeroTasks() throws Exception { expectConfigure(); - expectStart(Collections.EMPTY_LIST, Collections.EMPTY_MAP); + expectStart(Collections.emptyList(), Collections.emptyMap()); // Task configs should read to end, write to the log, read to end, write root. - expectReadToEnd(new LinkedHashMap()); + expectReadToEnd(new LinkedHashMap<>()); expectConvertWriteRead( COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(0), "tasks", 0); // We have 0 tasks @@ -329,7 +329,7 @@ public class KafkaConfigBackingStoreTest { configStorage.start(); // Bootstrap as if we had already added the connector, but no tasks had been added yet - whiteboxAddConnector(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), Collections.EMPTY_LIST); + whiteboxAddConnector(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), Collections.emptyList()); // Null before writing ClusterConfigState configState = configStorage.snapshot(); @@ -727,7 +727,7 @@ public class KafkaConfigBackingStoreTest { assertEquals(6, configState.offset()); // Should always be next to be read, not last committed assertEquals(Arrays.asList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors())); // Inconsistent data should leave us with no tasks listed for the connector and an entry in the inconsistent list - assertEquals(Collections.EMPTY_LIST, configState.tasks(CONNECTOR_IDS.get(0))); + assertEquals(Collections.emptyList(), configState.tasks(CONNECTOR_IDS.get(0))); // Both TASK_CONFIG_STRUCTS[0] -> SAMPLE_CONFIGS[0] assertNull(configState.taskConfig(TASK_IDS.get(0))); assertNull(configState.taskConfig(TASK_IDS.get(1))); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java index 36649a3da71..ff9f2c9d1ba 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java @@ -60,7 +60,7 @@ import static org.junit.Assert.fail; @RunWith(PowerMockRunner.class) @PrepareForTest(KafkaOffsetBackingStore.class) @PowerMockIgnore("javax.management.*") -@SuppressWarnings("unchecked") +@SuppressWarnings({"unchecked", "deprecation"}) public class KafkaOffsetBackingStoreTest { private static final String TOPIC = "connect-offsets"; private static final short TOPIC_PARTITIONS = 2; @@ -117,7 +117,7 @@ public class KafkaOffsetBackingStoreTest { @Test public void testStartStop() throws Exception { expectConfigure(); - expectStart(Collections.EMPTY_LIST); + expectStart(Collections.emptyList()); expectStop(); PowerMock.replayAll(); @@ -166,18 +166,15 @@ public class KafkaOffsetBackingStoreTest { @Test public void testGetSet() throws Exception { expectConfigure(); - expectStart(Collections.EMPTY_LIST); + expectStart(Collections.emptyList()); expectStop(); // First get() against an empty store final Capture> firstGetReadToEndCallback = EasyMock.newCapture(); storeLog.readToEnd(EasyMock.capture(firstGetReadToEndCallback)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - firstGetReadToEndCallback.getValue().onCompletion(null, null); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + firstGetReadToEndCallback.getValue().onCompletion(null, null); + return null; }); // Set offsets @@ -284,7 +281,7 @@ public class KafkaOffsetBackingStoreTest { @Test public void testGetSetNull() throws Exception { expectConfigure(); - expectStart(Collections.EMPTY_LIST); + expectStart(Collections.emptyList()); // Set offsets Capture callback0 = EasyMock.newCapture(); @@ -297,14 +294,11 @@ public class KafkaOffsetBackingStoreTest { // Second get() should get the produced data and return the new values final Capture> secondGetReadToEndCallback = EasyMock.newCapture(); storeLog.readToEnd(EasyMock.capture(secondGetReadToEndCallback)); - PowerMock.expectLastCall().andAnswer(new IAnswer() { - @Override - public Object answer() throws Throwable { - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, (byte[]) null, TP0_VALUE.array())); - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY.array(), (byte[]) null)); - secondGetReadToEndCallback.getValue().onCompletion(null, null); - return null; - } + PowerMock.expectLastCall().andAnswer(() -> { + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, (byte[]) null, TP0_VALUE.array())); + capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY.array(), (byte[]) null)); + secondGetReadToEndCallback.getValue().onCompletion(null, null); + return null; }); expectStop(); @@ -354,7 +348,7 @@ public class KafkaOffsetBackingStoreTest { @Test public void testSetFailure() throws Exception { expectConfigure(); - expectStart(Collections.EMPTY_LIST); + expectStart(Collections.emptyList()); expectStop(); // Set offsets diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java index 53ca0dccd5f..1af6e343521 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java @@ -131,6 +131,7 @@ public class KafkaBasedLogTest { } }; + @SuppressWarnings("unchecked") @Before public void setUp() { store = PowerMock.createPartialMock(KafkaBasedLog.class, new String[]{"createConsumer", "createProducer"}, diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java index ee089454166..f071bdad315 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java @@ -55,6 +55,7 @@ public abstract class ReplaceField> implements Transf .define(ConfigName.WHITELIST, ConfigDef.Type.LIST, Collections.emptyList(), ConfigDef.Importance.MEDIUM, "Fields to include. If specified, only these fields will be used.") .define(ConfigName.RENAME, ConfigDef.Type.LIST, Collections.emptyList(), new ConfigDef.Validator() { + @SuppressWarnings("unchecked") @Override public void ensureValid(String name, Object value) { parseRenameMappings((List) value); @@ -83,7 +84,7 @@ public abstract class ReplaceField> implements Transf renames = parseRenameMappings(config.getList(ConfigName.RENAME)); reverseRenames = invert(renames); - schemaUpdateCache = new SynchronizedCache<>(new LRUCache(16)); + schemaUpdateCache = new SynchronizedCache<>(new LRUCache<>(16)); } static Map parseRenameMappings(List mappings) { diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java index d709054dbe0..b0549fbfc9d 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/FlattenTest.java @@ -141,6 +141,7 @@ public class FlattenTest { assertNull(transformed.valueSchema()); assertTrue(transformed.value() instanceof Map); + @SuppressWarnings("unchecked") Map transformedMap = (Map) transformed.value(); assertEquals(9, transformedMap.size()); assertEquals((byte) 8, transformedMap.get("A#B#int8")); @@ -196,6 +197,7 @@ public class FlattenTest { assertNull(transformed.valueSchema()); assertTrue(transformed.value() instanceof Map); + @SuppressWarnings("unchecked") Map transformedMap = (Map) transformed.value(); assertNull(transformedMap.get("B.opt_int32")); @@ -211,6 +213,7 @@ public class FlattenTest { assertNull(transformed.keySchema()); assertTrue(transformed.key() instanceof Map); + @SuppressWarnings("unchecked") Map transformedMap = (Map) transformed.key(); assertEquals(12, transformedMap.get("A.B")); } diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java index 9211a462103..52849f9bb0d 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java @@ -71,6 +71,7 @@ public class MaskFieldTest { final List maskFields = new ArrayList<>(value.keySet()); maskFields.remove("magic"); + @SuppressWarnings("unchecked") final Map updatedValue = (Map) transform(maskFields).apply(record(null, value)).value(); assertEquals(42, updatedValue.get("magic")); diff --git a/core/src/main/scala/kafka/admin/AclCommand.scala b/core/src/main/scala/kafka/admin/AclCommand.scala index ad375d20572..e4cde132585 100644 --- a/core/src/main/scala/kafka/admin/AclCommand.scala +++ b/core/src/main/scala/kafka/admin/AclCommand.scala @@ -33,6 +33,7 @@ import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceP import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.io.StdIn object AclCommand extends Logging { @@ -449,7 +450,7 @@ object AclCommand extends Logging { if (opts.options.has(opts.forceOpt)) return true println(msg) - Console.readLine().equalsIgnoreCase("y") + StdIn.readLine().equalsIgnoreCase("y") } private def validateOperation(opts: AclCommandOptions, resourceToAcls: Map[ResourcePatternFilter, Set[Acl]]): Unit = { diff --git a/core/src/main/scala/kafka/admin/AdminClient.scala b/core/src/main/scala/kafka/admin/AdminClient.scala index ae2ffb16b6e..aa5dfec9721 100644 --- a/core/src/main/scala/kafka/admin/AdminClient.scala +++ b/core/src/main/scala/kafka/admin/AdminClient.scala @@ -36,7 +36,7 @@ import org.apache.kafka.common.requests.DescribeGroupsResponse.GroupMetadata import org.apache.kafka.common.requests.OffsetFetchResponse import org.apache.kafka.common.utils.LogContext import org.apache.kafka.common.utils.{KafkaThread, Time, Utils} -import org.apache.kafka.common.{Cluster, Node, TopicPartition} +import org.apache.kafka.common.{Node, TopicPartition} import scala.collection.JavaConverters._ import scala.util.{Failure, Success, Try} diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index e62e5a80596..cd479691294 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -376,6 +376,7 @@ object AdminUtils extends Logging with AdminUtilities { } } + @deprecated("This method is deprecated and will be replaced by kafka.zk.AdminZkClient.", "1.1.0") def topicExists(zkUtils: ZkUtils, topic: String): Boolean = zkUtils.pathExists(getTopicPath(topic)) @@ -467,6 +468,7 @@ object AdminUtils extends Logging with AdminUtilities { writeTopicPartitionAssignment(zkUtils, topic, partitionReplicaAssignment, update) } + @deprecated("This method is deprecated and will be replaced by kafka.zk.AdminZkClient.", "1.1.0") private def writeTopicPartitionAssignment(zkUtils: ZkUtils, topic: String, replicaAssignment: Map[Int, Seq[Int]], update: Boolean) { try { val zkPath = getTopicPath(topic) @@ -523,6 +525,7 @@ object AdminUtils extends Logging with AdminUtilities { changeEntityConfig(zkUtils, ConfigType.User, sanitizedEntityName, configs) } + @deprecated("This method is deprecated and will be replaced by kafka.zk.AdminZkClient.", "1.1.0") def validateTopicConfig(zkUtils: ZkUtils, topic: String, configs: Properties): Unit = { Topic.validate(topic) if (!topicExists(zkUtils, topic)) @@ -562,6 +565,7 @@ object AdminUtils extends Logging with AdminUtilities { } } + @deprecated("This method is deprecated and will be replaced by kafka.zk.AdminZkClient.", "1.1.0") private def changeEntityConfig(zkUtils: ZkUtils, rootEntityType: String, fullSanitizedEntityName: String, configs: Properties) { val sanitizedEntityPath = rootEntityType + '/' + fullSanitizedEntityName val entityConfigPath = getEntityConfigPath(rootEntityType, fullSanitizedEntityName) @@ -574,13 +578,14 @@ object AdminUtils extends Logging with AdminUtilities { zkUtils.createSequentialPersistentPath(seqNode, content) } - def getConfigChangeZnodeData(sanitizedEntityPath: String) : Map[String, Any] = { + def getConfigChangeZnodeData(sanitizedEntityPath: String): Map[String, Any] = { Map("version" -> 2, "entity_path" -> sanitizedEntityPath) } /** * Write out the entity config to zk, if there is any */ + @deprecated("This method is deprecated and will be replaced by kafka.zk.AdminZkClient.", "1.1.0") private def writeEntityConfig(zkUtils: ZkUtils, entityPath: String, config: Properties) { val map = Map("version" -> 1, "config" -> config.asScala) zkUtils.updatePersistentPath(entityPath, Json.legacyEncodeAsString(map)) @@ -590,7 +595,7 @@ object AdminUtils extends Logging with AdminUtilities { * Read the entity (topic, broker, client, user or ) config (if any) from zk * sanitizedEntityName is , , , or /clients/. */ - @deprecated("This method is deprecated and will be replaced by kafka.zk.AdminZkClient.", "1.1.0") + @deprecated("This method is deprecated and will be replaced by kafka.zk.AdminZkClient.", "1.1.0") def fetchEntityConfig(zkUtils: ZkUtils, rootEntityType: String, sanitizedEntityName: String): Properties = { val entityConfigPath = getEntityConfigPath(rootEntityType, sanitizedEntityName) // readDataMaybeNull returns Some(null) if the path exists, but there is no data diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 3ba6c1fecd9..92cde7e4dc2 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -36,6 +36,7 @@ import org.apache.kafka.common.TopicPartition import scala.collection.JavaConverters._ import scala.collection._ +import scala.io.StdIn object TopicCommand extends Logging { @@ -390,7 +391,7 @@ object TopicCommand extends Logging { def askToProceed(): Unit = { println("Are you sure you want to continue? [y/n]") - if (!Console.readLine().equalsIgnoreCase("y")) { + if (!StdIn.readLine().equalsIgnoreCase("y")) { println("Ending your session") Exit.exit(0) } diff --git a/core/src/main/scala/kafka/controller/ControllerContext.scala b/core/src/main/scala/kafka/controller/ControllerContext.scala index 20c3de0c1e5..de881842b29 100644 --- a/core/src/main/scala/kafka/controller/ControllerContext.scala +++ b/core/src/main/scala/kafka/controller/ControllerContext.scala @@ -31,7 +31,7 @@ class ControllerContext { var epoch: Int = KafkaController.InitialControllerEpoch var epochZkVersion: Int = KafkaController.InitialControllerEpochZkVersion var allTopics: Set[String] = Set.empty - private var partitionReplicaAssignmentUnderlying: mutable.Map[String, mutable.Map[Int, Seq[Int]]] = mutable.Map.empty + private val partitionReplicaAssignmentUnderlying: mutable.Map[String, mutable.Map[Int, Seq[Int]]] = mutable.Map.empty val partitionLeadershipInfo: mutable.Map[TopicPartition, LeaderIsrAndControllerEpoch] = mutable.Map.empty val partitionsBeingReassigned: mutable.Map[TopicPartition, ReassignedPartitionsContext] = mutable.Map.empty val replicasOnOfflineDirs: mutable.Map[Int, Set[TopicPartition]] = mutable.Map.empty diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index a52f3f02363..101ff390a53 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -274,7 +274,7 @@ class KafkaController(val config: KafkaConfig, zkClient: KafkaZkClient, time: Ti info("starting the token expiry check scheduler") tokenCleanScheduler.startup() tokenCleanScheduler.schedule(name = "delete-expired-tokens", - fun = tokenManager.expireTokens, + fun = () => tokenManager.expireTokens, period = config.delegationTokenExpiryCheckIntervalMs, unit = TimeUnit.MILLISECONDS) } diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala index 260b8024eba..edec22857e1 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala @@ -140,7 +140,7 @@ class GroupMetadataManager(brokerId: Int, scheduler.startup() if (enableMetadataExpiration) { scheduler.schedule(name = "delete-expired-group-metadata", - fun = cleanupGroupMetadata, + fun = () => cleanupGroupMetadata, period = config.offsetsRetentionCheckIntervalMs, unit = TimeUnit.MILLISECONDS) } @@ -691,7 +691,7 @@ class GroupMetadataManager(brokerId: Int, onGroupUnloaded: GroupMetadata => Unit) { val topicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, offsetsPartition) info(s"Scheduling unloading of offsets and group metadata from $topicPartition") - scheduler.schedule(topicPartition.toString, removeGroupsAndOffsets) + scheduler.schedule(topicPartition.toString, () => removeGroupsAndOffsets) def removeGroupsAndOffsets() { var numOffsetsRemoved = 0 diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala index 2cc719d2f58..9d4eed69fd3 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala @@ -497,7 +497,7 @@ class TransactionCoordinator(brokerId: Int, info("Starting up.") scheduler.startup() scheduler.schedule("transaction-abort", - abortTimedOutTransactions, + () => abortTimedOutTransactions, txnConfig.abortTimedOutTransactionsIntervalMs, txnConfig.abortTimedOutTransactionsIntervalMs ) diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index 87e6d1310e2..b45953f861e 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -423,7 +423,7 @@ class TransactionStateManager(brokerId: Int, } } - scheduler.schedule(s"load-txns-for-partition-$topicPartition", loadTransactions) + scheduler.schedule(s"load-txns-for-partition-$topicPartition", () => loadTransactions) } /** @@ -458,7 +458,7 @@ class TransactionStateManager(brokerId: Int, } } - scheduler.schedule(s"remove-txns-for-partition-$topicPartition", removeTransactions) + scheduler.schedule(s"remove-txns-for-partition-$topicPartition", () => removeTransactions) } private def validateTransactionTopicPartitionCountIsStable(): Unit = { diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 19e2f2f5424..688736c7d66 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -335,7 +335,7 @@ class Log(@volatile var dir: File, /** The name of this log */ def name = dir.getName() - def leaderEpochCache = _leaderEpochCache + def leaderEpochCache: LeaderEpochFileCache = _leaderEpochCache private def initializeLeaderEpochCache(): LeaderEpochFileCache = { // create the log directory if it doesn't exist diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 8449e39d581..7fc70dabfd3 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -354,7 +354,7 @@ class LogCleaner(initialConfig: CleanerConfig, case _: LogCleaningAbortedException => // task can be aborted, let it go. case _: KafkaStorageException => // partition is already offline. let it go. case e: IOException => - var logDirectory = cleanable.log.dir.getParent + val logDirectory = cleanable.log.dir.getParent val msg = s"Failed to clean up log for ${cleanable.topicPartition} in dir ${logDirectory} due to IOException" logDirFailureChannel.maybeAddOfflineLogDir(logDirectory, msg, e) } finally { diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index 2fc7b749852..e4559b8323c 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -75,9 +75,6 @@ private[log] class LogCleanerManager(val logDirs: Seq[File], * for each log directory */ private val uncleanablePartitions = mutable.HashMap[String, mutable.Set[TopicPartition]]() - /* the set of directories marked as uncleanable and therefore offline */ - private val uncleanableDirs = mutable.HashSet[String]() - /* a global lock used to control all access to the in-progress set and the offset checkpoints */ private val lock = new ReentrantLock diff --git a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala index 8d3ac4d1682..10ae8df1329 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherManager.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherManager.scala @@ -93,7 +93,6 @@ abstract class AbstractFetcherManager[T <: AbstractFetcherThread](val name: Stri // Visible for testing private[server] def getFetcher(topicPartition: TopicPartition): Option[T] = { lock synchronized { - val fetcherId = getFetcherId(topicPartition) fetcherThreadMap.values.find { fetcherThread => fetcherThread.fetchState(topicPartition).isDefined } diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index 60d397d6cf2..2cee83c2e74 100755 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -142,8 +142,8 @@ abstract class AbstractFetcherThread(name: String, * on latest epochs of the future replicas (the one that is fetching) */ private def buildLeaderEpochRequest(): ResultWithPartitions[Map[TopicPartition, EpochData]] = inLock(partitionMapLock) { - var partitionsWithoutEpochs = mutable.Set.empty[TopicPartition] - var partitionsWithEpochs = mutable.Map.empty[TopicPartition, EpochData] + val partitionsWithoutEpochs = mutable.Set.empty[TopicPartition] + val partitionsWithEpochs = mutable.Map.empty[TopicPartition, EpochData] partitionStates.partitionStates.asScala.foreach { state => val tp = state.topicPartition diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index be6736d9c07..f2e3e01af83 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -767,7 +767,7 @@ class KafkaApis(val requestChannel: RequestChannel, maxNumOffsets = partitionData.maxNumOffsets, isFromConsumer = offsetRequest.replicaId == ListOffsetRequest.CONSUMER_REPLICA_ID, fetchOnlyFromLeader = offsetRequest.replicaId != ListOffsetRequest.DEBUGGING_REPLICA_ID) - (topicPartition, new ListOffsetResponse.PartitionData(Errors.NONE, offsets.map(new JLong(_)).asJava)) + (topicPartition, new ListOffsetResponse.PartitionData(Errors.NONE, offsets.map(JLong.valueOf).asJava)) } catch { // NOTE: UnknownTopicOrPartitionException and NotLeaderForPartitionException are special cased since these error messages // are typically transient and there is no value in logging the entire stack trace for the same diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index be50be14cf2..17f340d783c 100755 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -302,7 +302,7 @@ object ConsoleConsumer extends Logging { val keyDeserializer = options.valueOf(keyDeserializerOpt) val valueDeserializer = options.valueOf(valueDeserializerOpt) val isolationLevel = options.valueOf(isolationLevelOpt).toString - val formatter: MessageFormatter = messageFormatterClass.newInstance().asInstanceOf[MessageFormatter] + val formatter: MessageFormatter = messageFormatterClass.getDeclaredConstructor().newInstance().asInstanceOf[MessageFormatter] if (keyDeserializer != null && !keyDeserializer.isEmpty) { formatterArgs.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer) @@ -480,12 +480,14 @@ class DefaultMessageFormatter extends MessageFormatter { lineSeparator = props.getProperty("line.separator").getBytes(StandardCharsets.UTF_8) // Note that `toString` will be called on the instance returned by `Deserializer.deserialize` if (props.containsKey("key.deserializer")) { - keyDeserializer = Some(Class.forName(props.getProperty("key.deserializer")).newInstance().asInstanceOf[Deserializer[_]]) + keyDeserializer = Some(Class.forName(props.getProperty("key.deserializer")).getDeclaredConstructor() + .newInstance().asInstanceOf[Deserializer[_]]) keyDeserializer.get.configure(propertiesWithKeyPrefixStripped("key.deserializer.", props).asScala.asJava, true) } // Note that `toString` will be called on the instance returned by `Deserializer.deserialize` if (props.containsKey("value.deserializer")) { - valueDeserializer = Some(Class.forName(props.getProperty("value.deserializer")).newInstance().asInstanceOf[Deserializer[_]]) + valueDeserializer = Some(Class.forName(props.getProperty("value.deserializer")).getDeclaredConstructor() + .newInstance().asInstanceOf[Deserializer[_]]) valueDeserializer.get.configure(propertiesWithKeyPrefixStripped("value.deserializer.", props).asScala.asJava, false) } } diff --git a/core/src/main/scala/kafka/tools/ConsoleProducer.scala b/core/src/main/scala/kafka/tools/ConsoleProducer.scala index a9099954564..9df53f5291c 100644 --- a/core/src/main/scala/kafka/tools/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleProducer.scala @@ -39,7 +39,7 @@ object ConsoleProducer { try { val config = new ProducerConfig(args) - val reader = Class.forName(config.readerClass).newInstance().asInstanceOf[MessageReader] + val reader = Class.forName(config.readerClass).getDeclaredConstructor().newInstance().asInstanceOf[MessageReader] reader.init(System.in, getReaderProps(config)) val producer = new KafkaProducer[Array[Byte], Array[Byte]](producerProps(config)) diff --git a/core/src/main/scala/kafka/utils/Mx4jLoader.scala b/core/src/main/scala/kafka/utils/Mx4jLoader.scala index f2c8644b9b3..e49f3a57f1e 100644 --- a/core/src/main/scala/kafka/utils/Mx4jLoader.scala +++ b/core/src/main/scala/kafka/utils/Mx4jLoader.scala @@ -45,7 +45,7 @@ object Mx4jLoader extends Logging { val processorName = new ObjectName("Server:name=XSLTProcessor") val httpAdaptorClass = Class.forName("mx4j.tools.adaptor.http.HttpAdaptor") - val httpAdaptor = httpAdaptorClass.newInstance() + val httpAdaptor = httpAdaptorClass.getDeclaredConstructor().newInstance() httpAdaptorClass.getMethod("setHost", classOf[String]).invoke(httpAdaptor, address.asInstanceOf[AnyRef]) httpAdaptorClass.getMethod("setPort", Integer.TYPE).invoke(httpAdaptor, port.asInstanceOf[AnyRef]) @@ -53,7 +53,7 @@ object Mx4jLoader extends Logging { mbs.registerMBean(httpAdaptor, httpName) val xsltProcessorClass = Class.forName("mx4j.tools.adaptor.http.XSLTProcessor") - val xsltProcessor = xsltProcessorClass.newInstance() + val xsltProcessor = xsltProcessorClass.getDeclaredConstructor().newInstance() httpAdaptorClass.getMethod("setProcessor", Class.forName("mx4j.tools.adaptor.http.ProcessorMBean")).invoke(httpAdaptor, xsltProcessor.asInstanceOf[AnyRef]) mbs.registerMBean(xsltProcessor, processorName) httpAdaptorClass.getMethod("start").invoke(httpAdaptor) diff --git a/core/src/main/scala/kafka/utils/json/DecodeJson.scala b/core/src/main/scala/kafka/utils/json/DecodeJson.scala index eab1f2a2789..0b57fa7b94f 100644 --- a/core/src/main/scala/kafka/utils/json/DecodeJson.scala +++ b/core/src/main/scala/kafka/utils/json/DecodeJson.scala @@ -43,11 +43,7 @@ trait DecodeJson[T] { def decode(node: JsonNode): T = decodeEither(node) match { case Right(x) => x - case Left(x) => - // Non-deprecated constructors were only introduced in Jackson 2.7, so stick with the deprecated one in case - // people have older versions of Jackson in their classpath. Once the Scala clients are removed, we can loosen - // this restriction. - throw new JsonMappingException(x) + case Left(x) => throw new JsonMappingException(null, x) } } diff --git a/core/src/main/scala/kafka/utils/json/JsonObject.scala b/core/src/main/scala/kafka/utils/json/JsonObject.scala index 8feb08b9a9e..9b48cb8a77a 100644 --- a/core/src/main/scala/kafka/utils/json/JsonObject.scala +++ b/core/src/main/scala/kafka/utils/json/JsonObject.scala @@ -31,7 +31,7 @@ import scala.collection.Iterator class JsonObject private[json] (protected val node: ObjectNode) extends JsonValue { def apply(name: String): JsonValue = - get(name).getOrElse(throw new JsonMappingException(s"No such field exists: `$name`")) + get(name).getOrElse(throw new JsonMappingException(null, s"No such field exists: `$name`")) def get(name: String): Option[JsonValue] = Option(node.get(name)).map(JsonValue(_)) diff --git a/core/src/main/scala/kafka/utils/json/JsonValue.scala b/core/src/main/scala/kafka/utils/json/JsonValue.scala index cbc82c03d1e..ff62c6c12d1 100644 --- a/core/src/main/scala/kafka/utils/json/JsonValue.scala +++ b/core/src/main/scala/kafka/utils/json/JsonValue.scala @@ -59,7 +59,7 @@ trait JsonValue { * If this is a JSON object, return an instance of JsonObject. Otherwise, throw a JsonMappingException. */ def asJsonObject: JsonObject = - asJsonObjectOption.getOrElse(throw new JsonMappingException(s"Expected JSON object, received $node")) + asJsonObjectOption.getOrElse(throw new JsonMappingException(null, s"Expected JSON object, received $node")) /** * If this is a JSON object, return a JsonObject wrapped by a `Some`. Otherwise, return None. @@ -76,7 +76,7 @@ trait JsonValue { * If this is a JSON array, return an instance of JsonArray. Otherwise, throw a JsonMappingException. */ def asJsonArray: JsonArray = - asJsonArrayOption.getOrElse(throw new JsonMappingException(s"Expected JSON array, received $node")) + asJsonArrayOption.getOrElse(throw new JsonMappingException(null, s"Expected JSON array, received $node")) /** * If this is a JSON array, return a JsonArray wrapped by a `Some`. Otherwise, return None. diff --git a/core/src/test/scala/integration/kafka/api/AdminClientIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientIntegrationTest.scala index ca23e1fd4c4..ee69ae42a0f 100644 --- a/core/src/test/scala/integration/kafka/api/AdminClientIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AdminClientIntegrationTest.scala @@ -918,26 +918,17 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { val producer = createProducer() sendRecords(producer, 10, topicPartition) var messageCount = 0 - TestUtils.waitUntilTrue(() => { - messageCount += consumer.poll(0).count - messageCount == 10 - }, "Expected 10 messages", 3000L) + TestUtils.consumeRecords(consumer, 10) client.deleteRecords(Map(topicPartition -> RecordsToDelete.beforeOffset(3L)).asJava).all.get consumer.seek(topicPartition, 1) messageCount = 0 - TestUtils.waitUntilTrue(() => { - messageCount += consumer.poll(0).count - messageCount == 7 - }, "Expected 7 messages", 3000L) + TestUtils.consumeRecords(consumer, 7) client.deleteRecords(Map(topicPartition -> RecordsToDelete.beforeOffset(8L)).asJava).all.get consumer.seek(topicPartition, 1) messageCount = 0 - TestUtils.waitUntilTrue(() => { - messageCount += consumer.poll(0).count - messageCount == 2 - }, "Expected 2 messages", 3000L) + TestUtils.consumeRecords(consumer, 2) } @Test @@ -988,10 +979,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { private def subscribeAndWaitForAssignment(topic: String, consumer: KafkaConsumer[Array[Byte], Array[Byte]]): Unit = { consumer.subscribe(Collections.singletonList(topic)) - TestUtils.waitUntilTrue(() => { - consumer.poll(0) - !consumer.assignment.isEmpty - }, "Expected non-empty assignment") + TestUtils.pollUntilTrue(consumer, () => !consumer.assignment.isEmpty, "Expected non-empty assignment") } private def sendRecords(producer: KafkaProducer[Array[Byte], Array[Byte]], @@ -1147,8 +1135,7 @@ class AdminClientIntegrationTest extends IntegrationTestHarness with Logging { consumerThread.start // Test that we can list the new group. TestUtils.waitUntilTrue(() => { - val matching = client.listConsumerGroups().all().get().asScala. - filter(listing => listing.groupId().equals(testGroupId)) + val matching = client.listConsumerGroups.all.get().asScala.filter(_.groupId == testGroupId) !matching.isEmpty }, s"Expected to be able to list $testGroupId") diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 9ef721491cc..23f66e1a703 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -16,7 +16,7 @@ import java.nio.ByteBuffer import java.util import java.util.concurrent.ExecutionException import java.util.regex.Pattern -import java.util.{ArrayList, Collections, Optional, Properties} +import java.util.{Collections, Optional, Properties} import java.time.Duration import kafka.admin.ConsumerGroupCommand.{ConsumerGroupCommandOptions, ConsumerGroupService} @@ -317,7 +317,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { private def createOffsetCommitRequest = { new requests.OffsetCommitRequest.Builder( - group, Map(tp -> new requests.OffsetCommitRequest.PartitionData(0, 27, "metadata")).asJava). + group, Map(tp -> new requests.OffsetCommitRequest.PartitionData(0L, Optional.empty[Integer](), "metadata")).asJava). setMemberId("").setGenerationId(1). build() } @@ -1516,20 +1516,14 @@ class AuthorizerIntegrationTest extends BaseRequestTest { startingOffset: Int = 0, topic: String = topic, part: Int = part) { - val records = new ArrayList[ConsumerRecord[Array[Byte], Array[Byte]]]() - - TestUtils.waitUntilTrue(() => { - for (record <- consumer.poll(50).asScala) - records.add(record) - records.size == numRecords - }, "Failed to receive all expected records from the consumer") + val records = TestUtils.consumeRecords(consumer, numRecords) for (i <- 0 until numRecords) { - val record = records.get(i) + val record = records(i) val offset = startingOffset + i - assertEquals(topic, record.topic()) - assertEquals(part, record.partition()) - assertEquals(offset.toLong, record.offset()) + assertEquals(topic, record.topic) + assertEquals(part, record.partition) + assertEquals(offset.toLong, record.offset) } } diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index 3e67b1876ec..4d2e1a980a4 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -306,7 +306,6 @@ abstract class BaseConsumerTest extends IntegrationTestHarness { val allNonEmptyAssignments = assignments.forall(assignment => assignment.nonEmpty) if (!allNonEmptyAssignments) { // at least one consumer got empty assignment - val uniqueAssignedPartitions = (Set[TopicPartition]() /: assignments) (_ ++ _) return false } diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index 7a2394e7b30..09a6188d067 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -26,7 +26,7 @@ import kafka.integration.KafkaServerTestHarness import kafka.log.LogConfig import kafka.server.KafkaConfig import kafka.utils.TestUtils -import org.apache.kafka.clients.consumer.{ConsumerRecord, KafkaConsumer} +import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer._ import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.record.TimestampType @@ -34,7 +34,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.junit.Assert._ import org.junit.{After, Before, Test} -import scala.collection.mutable.{ArrayBuffer, Buffer} +import scala.collection.mutable.Buffer import scala.concurrent.ExecutionException abstract class BaseProducerSendTest extends KafkaServerTestHarness { @@ -86,15 +86,6 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { producer } - private def pollUntilNumRecords(numRecords: Int) : Seq[ConsumerRecord[Array[Byte], Array[Byte]]] = { - val records = new ArrayBuffer[ConsumerRecord[Array[Byte], Array[Byte]]]() - TestUtils.waitUntilTrue(() => { - records ++= consumer.poll(50).asScala - records.size == numRecords - }, s"Consumed ${records.size} records until timeout, but expected $numRecords records.") - records - } - /** * testSendOffset checks the basic send API behavior * @@ -329,7 +320,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { consumer.assign(List(new TopicPartition(topic, partition)).asJava) // make sure the fetched messages also respect the partitioning and ordering - val records = pollUntilNumRecords(numRecords) + val records = TestUtils.consumeRecords(consumer, numRecords) records.zipWithIndex.foreach { case (record, i) => assertEquals(topic, record.topic) @@ -496,7 +487,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { producer.flush() assertTrue("All requests are complete.", responses.forall(_.isDone())) // Check the messages received by broker. - pollUntilNumRecords(numRecords) + TestUtils.pollUntilAtLeastNumRecords(consumer, numRecords) } finally { producer.close() } diff --git a/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala index 2ab2b1a89cb..29c7507449b 100644 --- a/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala @@ -19,7 +19,8 @@ package kafka.api import java.util import kafka.server.KafkaConfig -import kafka.utils.{JaasTestUtils, TestUtils, ZkUtils} +import kafka.utils.{JaasTestUtils, TestUtils} +import kafka.zk.ConfigEntityChangeNotificationZNode import org.apache.kafka.clients.admin.{AdminClient, AdminClientConfig} import org.apache.kafka.common.config.SaslConfigs import org.apache.kafka.common.security.auth.SecurityProtocol @@ -47,7 +48,7 @@ class DelegationTokenEndToEndAuthorizationTest extends EndToEndAuthorizationTest override def configureSecurityBeforeServersStart() { super.configureSecurityBeforeServersStart() - zkClient.makeSurePersistentPathExists(ZkUtils.ConfigChangesPath) + zkClient.makeSurePersistentPathExists(ConfigEntityChangeNotificationZNode.path) // Create broker admin credentials before starting brokers createScramCredentials(zkConnect, kafkaPrincipal, kafkaPassword) } diff --git a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala index 7fd68c2b5ac..854e3381342 100644 --- a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala @@ -21,18 +21,17 @@ import com.yammer.metrics.Metrics import com.yammer.metrics.core.Gauge import java.io.File -import java.util.ArrayList import java.util.concurrent.ExecutionException import kafka.admin.AclCommand import kafka.security.auth._ import kafka.server._ import kafka.utils._ -import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerRecord} +import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.{KafkaException, TopicPartition} -import org.apache.kafka.common.errors.{GroupAuthorizationException, TimeoutException, TopicAuthorizationException} +import org.apache.kafka.common.errors.{GroupAuthorizationException, TopicAuthorizationException} import org.apache.kafka.common.resource.PatternType import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} import org.junit.Assert._ @@ -458,22 +457,14 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas topic: String = topic, part: Int = part, timeout: Long = 10000) { - val records = new ArrayList[ConsumerRecord[Array[Byte], Array[Byte]]]() - - val deadlineMs = System.currentTimeMillis() + timeout - while (records.size < numRecords && System.currentTimeMillis() < deadlineMs) { - for (record <- consumer.poll(50).asScala) - records.add(record) - } - if (records.size < numRecords) - throw new TimeoutException + val records = TestUtils.consumeRecords(consumer, numRecords, timeout) for (i <- 0 until numRecords) { - val record = records.get(i) + val record = records(i) val offset = startingOffset + i - assertEquals(topic, record.topic()) - assertEquals(part, record.partition()) - assertEquals(offset.toLong, record.offset()) + assertEquals(topic, record.topic) + assertEquals(part, record.partition) + assertEquals(offset.toLong, record.offset) } } } diff --git a/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala index f6cb7572f79..04dcf28a680 100644 --- a/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala +++ b/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala @@ -35,8 +35,6 @@ import org.junit.{Before, Test} import scala.collection.JavaConverters._ import org.apache.kafka.test.TestUtils.isValidClusterId -import scala.collection.mutable.ArrayBuffer - /** The test cases here verify the following conditions. * 1. The ProducerInterceptor receives the cluster id after the onSend() method is called and before onAcknowledgement() method is called. * 2. The Serializer receives the cluster id before the serialize() method is called. @@ -203,17 +201,8 @@ class EndToEndClusterIdTest extends KafkaServerTestHarness { startingOffset: Int = 0, topic: String = topic, part: Int = part) { - val records = new ArrayBuffer[ConsumerRecord[Array[Byte], Array[Byte]]]() - val maxIters = numRecords * 50 - var iters = 0 - while (records.size < numRecords) { - for (record <- consumer.poll(50).asScala) { - records += record - } - if (iters > maxIters) - throw new IllegalStateException("Failed to consume the expected records after " + iters + " iterations.") - iters += 1 - } + val records = TestUtils.consumeRecords(consumer, numRecords) + for (i <- 0 until numRecords) { val record = records(i) val offset = startingOffset + i diff --git a/core/src/test/scala/integration/kafka/api/LegacyAdminClientTest.scala b/core/src/test/scala/integration/kafka/api/LegacyAdminClientTest.scala index f1eca543b0c..08a0224704d 100644 --- a/core/src/test/scala/integration/kafka/api/LegacyAdminClientTest.scala +++ b/core/src/test/scala/integration/kafka/api/LegacyAdminClientTest.scala @@ -35,6 +35,8 @@ import scala.collection.JavaConverters._ /** * Tests for the deprecated Scala AdminClient. */ +@deprecated("The Scala AdminClient has been deprecated in favour of org.apache.kafka.clients.admin.AdminClient", + since = "0.11.0") class LegacyAdminClientTest extends IntegrationTestHarness with Logging { val producerCount = 1 diff --git a/core/src/test/scala/integration/kafka/api/LogAppendTimeTest.scala b/core/src/test/scala/integration/kafka/api/LogAppendTimeTest.scala index c36a3f178ae..795f954a32d 100644 --- a/core/src/test/scala/integration/kafka/api/LogAppendTimeTest.scala +++ b/core/src/test/scala/integration/kafka/api/LogAppendTimeTest.scala @@ -21,15 +21,11 @@ import java.util.concurrent.TimeUnit import kafka.server.KafkaConfig import kafka.utils.TestUtils -import org.apache.kafka.clients.consumer.ConsumerRecord import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.common.record.TimestampType import org.junit.{Before, Test} import org.junit.Assert.{assertEquals, assertNotEquals, assertTrue} -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer - /** * Tests where the broker is configured to use LogAppendTime. For tests where LogAppendTime is configured via topic * level configs, see the *ProducerSendTest classes. @@ -66,11 +62,7 @@ class LogAppendTimeTest extends IntegrationTestHarness { val consumer = createConsumer() consumer.subscribe(Collections.singleton(topic)) - val consumerRecords = new ArrayBuffer[ConsumerRecord[Array[Byte], Array[Byte]]] - TestUtils.waitUntilTrue(() => { - consumerRecords ++= consumer.poll(50).asScala - consumerRecords.size == producerRecords.size - }, s"Consumed ${consumerRecords.size} records until timeout instead of the expected ${producerRecords.size} records") + val consumerRecords = TestUtils.consumeRecords(consumer, producerRecords.size) consumerRecords.zipWithIndex.foreach { case (consumerRecord, index) => val producerRecord = producerRecords(index) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 42b3984e305..2aee15a8381 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -126,6 +126,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(numRecords, records.size) } + @deprecated("poll(Duration) is the replacement", since = "2.0") @Test def testDeprecatedPollBlocksForAssignment(): Unit = { val consumer = createConsumer() @@ -134,6 +135,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(Set(tp, tp2), consumer.assignment().asScala) } + @deprecated("Serializer now includes a default method that provides the headers", since = "2.1") @Test def testHeadersExtendedSerializerDeserializer(): Unit = { val extendedSerializer = new ExtendedSerializer[Array[Byte]] with SerializerImpl @@ -1522,7 +1524,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { consumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testPerPartitionLagMetricsCleanUpWithAssign") val consumer = createConsumer() consumer.assign(List(tp).asJava) - val records = awaitNonEmptyRecords(consumer, tp) + awaitNonEmptyRecords(consumer, tp) // Verify the metric exist. val tags = new util.HashMap[String, String]() tags.put("client-id", "testPerPartitionLagMetricsCleanUpWithAssign") diff --git a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala index d2ff2a85829..8ab32af2ed2 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala @@ -37,7 +37,7 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer") producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer") val producer = registerProducer(new KafkaProducer(producerProps)) - val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, new Integer(0), "key".getBytes, "value".getBytes) + val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, 0, "key".getBytes, "value".getBytes) producer.send(record) } diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 9402584252e..853f99944ff 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -22,6 +22,7 @@ import java.io.{Closeable, File, FileWriter} import java.nio.file.{Files, Paths, StandardCopyOption} import java.lang.management.ManagementFactory import java.security.KeyStore +import java.time.Duration import java.util import java.util.{Collections, Properties} import java.util.concurrent._ @@ -147,7 +148,7 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet clientThreads.foreach(_.join(5 * 1000)) executors.foreach(_.shutdownNow()) producers.foreach(_.close(0, TimeUnit.MILLISECONDS)) - consumers.foreach(_.close(0, TimeUnit.MILLISECONDS)) + consumers.foreach(_.close(Duration.ofMillis(0))) adminClients.foreach(_.close()) TestUtils.shutdownServers(servers) super.tearDown() @@ -993,10 +994,8 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet } private def awaitInitialPositions(consumer: KafkaConsumer[_, _]): Unit = { - do { - consumer.poll(1) - } while (consumer.assignment.isEmpty) - consumer.assignment.asScala.foreach(tp => consumer.position(tp)) + TestUtils.pollUntilTrue(consumer, () => !consumer.assignment.isEmpty, "Timed out while waiting for assignment") + consumer.assignment.asScala.foreach(consumer.position) } private def clientProps(securityProtocol: SecurityProtocol, saslMechanism: Option[String] = None): Properties = { @@ -1025,12 +1024,7 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet topic: String): Unit = { val producerRecords = (1 to numRecords).map(i => new ProducerRecord(topic, s"key$i", s"value$i")) producerRecords.map(producer.send).map(_.get(10, TimeUnit.SECONDS)) - var received = 0 - TestUtils.waitUntilTrue(() => { - received += consumer.poll(50).count - received >= numRecords - }, s"Consumed $received records until timeout instead of the expected $numRecords records") - assertEquals(numRecords, received) + TestUtils.pollUntilAtLeastNumRecords(consumer, numRecords) } private def verifyAuthenticationFailure(producer: KafkaProducer[_, _]): Unit = { diff --git a/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala b/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala index 851ac5178e6..95a4843eb65 100644 --- a/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala +++ b/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala @@ -28,7 +28,7 @@ import kafka.utils.JaasTestUtils.JaasSection import kafka.utils.{JaasTestUtils, TestUtils} import kafka.utils.Implicits._ import kafka.zk.ZooKeeperTestHarness -import org.apache.kafka.clients.consumer.{ConsumerRecord, KafkaConsumer} +import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.config.SslConfigs import org.apache.kafka.common.internals.Topic @@ -38,7 +38,6 @@ import org.junit.{After, Before, Test} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import scala.collection.JavaConverters._ object MultipleListenersWithSameSecurityProtocolBaseTest { val SecureInternal = "SECURE_INTERNAL" @@ -169,11 +168,7 @@ abstract class MultipleListenersWithSameSecurityProtocolBaseTest extends ZooKeep val consumer = consumers(clientMetadata) consumer.subscribe(Collections.singleton(clientMetadata.topic)) - val records = new ArrayBuffer[ConsumerRecord[Array[Byte], Array[Byte]]] - TestUtils.waitUntilTrue(() => { - records ++= consumer.poll(50).asScala - records.size == producerRecords.size - }, s"Consumed ${records.size} records until timeout instead of the expected ${producerRecords.size} records with mechanism ${clientMetadata.saslMechanism}") + TestUtils.consumeRecords(consumer, producerRecords.size) } } diff --git a/core/src/test/scala/kafka/common/InterBrokerSendThreadTest.scala b/core/src/test/scala/kafka/common/InterBrokerSendThreadTest.scala index 5c0ea2d8d48..ae372e061c4 100644 --- a/core/src/test/scala/kafka/common/InterBrokerSendThreadTest.scala +++ b/core/src/test/scala/kafka/common/InterBrokerSendThreadTest.scala @@ -16,13 +16,14 @@ */ package kafka.common +import java.util + import kafka.utils.MockTime import org.apache.kafka.clients.{ClientRequest, ClientResponse, NetworkClient, RequestCompletionHandler} import org.apache.kafka.common.Node import org.apache.kafka.common.errors.AuthenticationException import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests.AbstractRequest -import org.apache.kafka.common.utils.Utils import org.easymock.EasyMock import org.junit.{Assert, Test} @@ -43,7 +44,7 @@ class InterBrokerSendThreadTest { // poll is always called but there should be no further invocations on NetworkClient EasyMock.expect(networkClient.poll(EasyMock.anyLong(), EasyMock.anyLong())) - .andReturn(Utils.mkList()) + .andReturn(new util.ArrayList()) EasyMock.replay(networkClient) @@ -80,7 +81,7 @@ class InterBrokerSendThreadTest { EasyMock.expect(networkClient.send(clientRequest, time.milliseconds())) EasyMock.expect(networkClient.poll(EasyMock.anyLong(), EasyMock.anyLong())) - .andReturn(Utils.mkList()) + .andReturn(new util.ArrayList()) EasyMock.replay(networkClient) @@ -118,7 +119,7 @@ class InterBrokerSendThreadTest { .andReturn(0) EasyMock.expect(networkClient.poll(EasyMock.anyLong(), EasyMock.anyLong())) - .andReturn(Utils.mkList()) + .andReturn(new util.ArrayList()) EasyMock.expect(networkClient.connectionFailed(node)) .andReturn(true) @@ -164,7 +165,7 @@ class InterBrokerSendThreadTest { .andReturn(0) EasyMock.expect(networkClient.poll(EasyMock.anyLong(), EasyMock.anyLong())) - .andReturn(Utils.mkList()) + .andReturn(new util.ArrayList()) // rule out disconnects so the request stays for the expiry check EasyMock.expect(networkClient.connectionFailed(node)) diff --git a/core/src/test/scala/unit/kafka/admin/TestAdminUtils.scala b/core/src/test/scala/unit/kafka/admin/TestAdminUtils.scala index 55fe587f74d..7cbad05bbe5 100644 --- a/core/src/test/scala/unit/kafka/admin/TestAdminUtils.scala +++ b/core/src/test/scala/unit/kafka/admin/TestAdminUtils.scala @@ -19,6 +19,7 @@ package kafka.admin import java.util.Properties import kafka.utils.ZkUtils +@deprecated("This class is deprecated since AdminUtilities will be replaced by kafka.zk.AdminZkClient.", "1.1.0") class TestAdminUtils extends AdminUtilities { override def changeBrokerConfig(zkUtils: ZkUtils, brokerIds: Seq[Int], configs: Properties): Unit = {} override def fetchEntityConfig(zkUtils: ZkUtils, entityType: String, entityName: String): Properties = {new Properties} diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala index 44d5c5f4b75..1311aa4f017 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerChannelManagerTest.scala @@ -16,6 +16,7 @@ */ package kafka.coordinator.transaction +import java.util.Arrays.asList import java.util.concurrent.locks.ReentrantReadWriteLock import kafka.server.{DelayedOperationPurgatory, KafkaConfig, MetadataCache} @@ -23,7 +24,7 @@ import kafka.utils.timer.MockTimer import kafka.utils.TestUtils import org.apache.kafka.clients.{ClientResponse, NetworkClient} import org.apache.kafka.common.requests.{RequestHeader, TransactionResult, WriteTxnMarkersRequest, WriteTxnMarkersResponse} -import org.apache.kafka.common.utils.{MockTime, Utils} +import org.apache.kafka.common.utils.MockTime import org.apache.kafka.common.{Node, TopicPartition} import org.easymock.{Capture, EasyMock, IAnswer} import org.junit.Assert._ @@ -129,10 +130,10 @@ class TransactionMarkerChannelManagerTest { assertEquals(0, channelManager.queueForBroker(broker2.id).get.totalNumMarkers(txnTopicPartition2)) val expectedBroker1Request = new WriteTxnMarkersRequest.Builder( - Utils.mkList(new WriteTxnMarkersRequest.TxnMarkerEntry(producerId1, producerEpoch, coordinatorEpoch, txnResult, Utils.mkList(partition1)), - new WriteTxnMarkersRequest.TxnMarkerEntry(producerId2, producerEpoch, coordinatorEpoch, txnResult, Utils.mkList(partition1)))).build() + asList(new WriteTxnMarkersRequest.TxnMarkerEntry(producerId1, producerEpoch, coordinatorEpoch, txnResult, asList(partition1)), + new WriteTxnMarkersRequest.TxnMarkerEntry(producerId2, producerEpoch, coordinatorEpoch, txnResult, asList(partition1)))).build() val expectedBroker2Request = new WriteTxnMarkersRequest.Builder( - Utils.mkList(new WriteTxnMarkersRequest.TxnMarkerEntry(producerId1, producerEpoch, coordinatorEpoch, txnResult, Utils.mkList(partition2)))).build() + asList(new WriteTxnMarkersRequest.TxnMarkerEntry(producerId1, producerEpoch, coordinatorEpoch, txnResult, asList(partition2)))).build() val requests: Map[Node, WriteTxnMarkersRequest] = channelManager.generateRequests().map { handler => (handler.destination, handler.request.asInstanceOf[WriteTxnMarkersRequest.Builder].build()) @@ -206,10 +207,10 @@ class TransactionMarkerChannelManagerTest { assertEquals(1, channelManager.queueForUnknownBroker.totalNumMarkers(txnTopicPartition2)) val expectedBroker1Request = new WriteTxnMarkersRequest.Builder( - Utils.mkList(new WriteTxnMarkersRequest.TxnMarkerEntry(producerId1, producerEpoch, coordinatorEpoch, txnResult, Utils.mkList(partition1)), - new WriteTxnMarkersRequest.TxnMarkerEntry(producerId2, producerEpoch, coordinatorEpoch, txnResult, Utils.mkList(partition1)))).build() + asList(new WriteTxnMarkersRequest.TxnMarkerEntry(producerId1, producerEpoch, coordinatorEpoch, txnResult, asList(partition1)), + new WriteTxnMarkersRequest.TxnMarkerEntry(producerId2, producerEpoch, coordinatorEpoch, txnResult, asList(partition1)))).build() val expectedBroker2Request = new WriteTxnMarkersRequest.Builder( - Utils.mkList(new WriteTxnMarkersRequest.TxnMarkerEntry(producerId1, producerEpoch, coordinatorEpoch, txnResult, Utils.mkList(partition2)))).build() + asList(new WriteTxnMarkersRequest.TxnMarkerEntry(producerId1, producerEpoch, coordinatorEpoch, txnResult, asList(partition2)))).build() val firstDrainedRequests: Map[Node, WriteTxnMarkersRequest] = channelManager.generateRequests().map { handler => (handler.destination, handler.request.asInstanceOf[WriteTxnMarkersRequest.Builder].build()) diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandlerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandlerTest.scala index 85159c31b29..84f3dffbfbd 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandlerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMarkerRequestCompletionHandlerTest.scala @@ -17,12 +17,12 @@ package kafka.coordinator.transaction import java.{lang, util} +import java.util.Arrays.asList import org.apache.kafka.clients.ClientResponse import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{RequestHeader, TransactionResult, WriteTxnMarkersRequest, WriteTxnMarkersResponse} -import org.apache.kafka.common.utils.Utils import org.easymock.{EasyMock, IAnswer} import org.junit.Assert._ import org.junit.Test @@ -40,9 +40,8 @@ class TransactionMarkerRequestCompletionHandlerTest { private val coordinatorEpoch = 0 private val txnResult = TransactionResult.COMMIT private val topicPartition = new TopicPartition("topic1", 0) - private val txnIdAndMarkers = - Utils.mkList( - TxnIdAndMarkerEntry(transactionalId, new WriteTxnMarkersRequest.TxnMarkerEntry(producerId, producerEpoch, coordinatorEpoch, txnResult, Utils.mkList(topicPartition)))) + private val txnIdAndMarkers = asList( + TxnIdAndMarkerEntry(transactionalId, new WriteTxnMarkersRequest.TxnMarkerEntry(producerId, producerEpoch, coordinatorEpoch, txnResult, asList(topicPartition)))) private val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerEpoch, txnTimeoutMs, PrepareCommit, mutable.Set[TopicPartition](topicPartition), 0L, 0L) diff --git a/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala b/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala index 74dae7c5303..b66324e169b 100755 --- a/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala @@ -64,22 +64,22 @@ class FetchSessionTest { def testSessionCache(): Unit = { val cache = new FetchSessionCache(3, 100) assertEquals(0, cache.size) - val id1 = cache.maybeCreateSession(0, false, 10, dummyCreate(10)) - val id2 = cache.maybeCreateSession(10, false, 20, dummyCreate(20)) - val id3 = cache.maybeCreateSession(20, false, 30, dummyCreate(30)) - assertEquals(INVALID_SESSION_ID, cache.maybeCreateSession(30, false, 40, dummyCreate(40))) - assertEquals(INVALID_SESSION_ID, cache.maybeCreateSession(40, false, 5, dummyCreate(5))) + val id1 = cache.maybeCreateSession(0, false, 10, () => dummyCreate(10)) + val id2 = cache.maybeCreateSession(10, false, 20, () => dummyCreate(20)) + val id3 = cache.maybeCreateSession(20, false, 30, () => dummyCreate(30)) + assertEquals(INVALID_SESSION_ID, cache.maybeCreateSession(30, false, 40, () => dummyCreate(40))) + assertEquals(INVALID_SESSION_ID, cache.maybeCreateSession(40, false, 5, () => dummyCreate(5))) assertCacheContains(cache, id1, id2, id3) cache.touch(cache.get(id1).get, 200) - val id4 = cache.maybeCreateSession(210, false, 11, dummyCreate(11)) + val id4 = cache.maybeCreateSession(210, false, 11, () => dummyCreate(11)) assertCacheContains(cache, id1, id3, id4) cache.touch(cache.get(id1).get, 400) cache.touch(cache.get(id3).get, 390) cache.touch(cache.get(id4).get, 400) - val id5 = cache.maybeCreateSession(410, false, 50, dummyCreate(50)) + val id5 = cache.maybeCreateSession(410, false, 50, () => dummyCreate(50)) assertCacheContains(cache, id3, id4, id5) - assertEquals(INVALID_SESSION_ID, cache.maybeCreateSession(410, false, 5, dummyCreate(5))) - val id6 = cache.maybeCreateSession(410, true, 5, dummyCreate(5)) + assertEquals(INVALID_SESSION_ID, cache.maybeCreateSession(410, false, 5, () => dummyCreate(5))) + val id6 = cache.maybeCreateSession(410, true, 5, () => dummyCreate(5)) assertCacheContains(cache, id3, id5, id6) } @@ -89,7 +89,7 @@ class FetchSessionTest { assertEquals(0, cache.totalPartitions) assertEquals(0, cache.size) assertEquals(0, cache.evictionsMeter.count) - val id1 = cache.maybeCreateSession(0, false, 2, dummyCreate(2)) + val id1 = cache.maybeCreateSession(0, false, 2, () => dummyCreate(2)) assertTrue(id1 > 0) assertCacheContains(cache, id1) val session1 = cache.get(id1).get @@ -97,7 +97,7 @@ class FetchSessionTest { assertEquals(2, cache.totalPartitions) assertEquals(1, cache.size) assertEquals(0, cache.evictionsMeter.count) - val id2 = cache.maybeCreateSession(0, false, 4, dummyCreate(4)) + val id2 = cache.maybeCreateSession(0, false, 4, () => dummyCreate(4)) val session2 = cache.get(id2).get assertTrue(id2 > 0) assertCacheContains(cache, id1, id2) @@ -106,7 +106,7 @@ class FetchSessionTest { assertEquals(0, cache.evictionsMeter.count) cache.touch(session1, 200) cache.touch(session2, 200) - val id3 = cache.maybeCreateSession(200, false, 5, dummyCreate(5)) + val id3 = cache.maybeCreateSession(200, false, 5, () => dummyCreate(5)) assertTrue(id3 > 0) assertCacheContains(cache, id2, id3) assertEquals(9, cache.totalPartitions) diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index b18511e7ce9..822d3ea67cf 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -20,6 +20,7 @@ package kafka.server import java.net.InetAddress import java.util import java.util.{Collections, Optional} +import java.util.Arrays.asList import kafka.api.{ApiVersion, KAFKA_0_10_2_IV0} import kafka.controller.KafkaController @@ -44,7 +45,6 @@ import org.apache.kafka.common.requests.UpdateMetadataRequest.{Broker, EndPoint} import org.apache.kafka.common.requests.WriteTxnMarkersRequest.TxnMarkerEntry import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} -import org.apache.kafka.common.utils.Utils import org.easymock.{Capture, EasyMock, IAnswer} import org.junit.Assert.{assertEquals, assertTrue} import org.junit.{After, Test} @@ -116,7 +116,7 @@ class KafkaApisTest { EasyMock.reset(replicaManager, clientRequestQuotaManager, requestChannel) val invalidTopicPartition = new TopicPartition(topic, invalidPartitionId) - val partitionOffsetCommitData = new OffsetCommitRequest.PartitionData(15L, 23, "") + val partitionOffsetCommitData = new OffsetCommitRequest.PartitionData(15L, Optional.empty[Integer](), "") val (offsetCommitRequest, request) = buildRequest(new OffsetCommitRequest.Builder("groupId", Map(invalidTopicPartition -> partitionOffsetCommitData).asJava)) @@ -213,7 +213,7 @@ class KafkaApisTest { @Test def shouldRespondWithUnsupportedForMessageFormatOnHandleWriteTxnMarkersWhenMagicLowerThanRequired(): Unit = { val topicPartition = new TopicPartition("t", 0) - val (writeTxnMarkersRequest, request) = createWriteTxnMarkersRequest(Utils.mkList(topicPartition)) + val (writeTxnMarkersRequest, request) = createWriteTxnMarkersRequest(asList(topicPartition)) val expectedErrors = Map(topicPartition -> Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT).asJava val capturedResponse: Capture[RequestChannel.Response] = EasyMock.newCapture() @@ -232,7 +232,7 @@ class KafkaApisTest { @Test def shouldRespondWithUnknownTopicWhenPartitionIsNotHosted(): Unit = { val topicPartition = new TopicPartition("t", 0) - val (writeTxnMarkersRequest, request) = createWriteTxnMarkersRequest(Utils.mkList(topicPartition)) + val (writeTxnMarkersRequest, request) = createWriteTxnMarkersRequest(asList(topicPartition)) val expectedErrors = Map(topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION).asJava val capturedResponse: Capture[RequestChannel.Response] = EasyMock.newCapture() @@ -252,7 +252,7 @@ class KafkaApisTest { def shouldRespondWithUnsupportedMessageFormatForBadPartitionAndNoErrorsForGoodPartition(): Unit = { val tp1 = new TopicPartition("t", 0) val tp2 = new TopicPartition("t1", 0) - val (writeTxnMarkersRequest, request) = createWriteTxnMarkersRequest(Utils.mkList(tp1, tp2)) + val (writeTxnMarkersRequest, request) = createWriteTxnMarkersRequest(asList(tp1, tp2)) val expectedErrors = Map(tp1 -> Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT, tp2 -> Errors.NONE).asJava val capturedResponse: Capture[RequestChannel.Response] = EasyMock.newCapture() @@ -291,7 +291,7 @@ class KafkaApisTest { def shouldRespondWithUnknownTopicOrPartitionForBadPartitionAndNoErrorsForGoodPartition(): Unit = { val tp1 = new TopicPartition("t", 0) val tp2 = new TopicPartition("t1", 0) - val (writeTxnMarkersRequest, request) = createWriteTxnMarkersRequest(Utils.mkList(tp1, tp2)) + val (writeTxnMarkersRequest, request) = createWriteTxnMarkersRequest(asList(tp1, tp2)) val expectedErrors = Map(tp1 -> Errors.UNKNOWN_TOPIC_OR_PARTITION, tp2 -> Errors.NONE).asJava val capturedResponse: Capture[RequestChannel.Response] = EasyMock.newCapture() @@ -329,7 +329,7 @@ class KafkaApisTest { @Test def shouldAppendToLogOnWriteTxnMarkersWhenCorrectMagicVersion(): Unit = { val topicPartition = new TopicPartition("t", 0) - val request = createWriteTxnMarkersRequest(Utils.mkList(topicPartition))._2 + val request = createWriteTxnMarkersRequest(asList(topicPartition))._2 EasyMock.expect(replicaManager.getMagic(topicPartition)) .andReturn(Some(RecordBatch.MAGIC_VALUE_V2)) @@ -486,7 +486,7 @@ class KafkaApisTest { } private def createWriteTxnMarkersRequest(partitions: util.List[TopicPartition]) = { - val requestBuilder = new WriteTxnMarkersRequest.Builder(Utils.mkList( + val requestBuilder = new WriteTxnMarkersRequest.Builder(asList( new TxnMarkerEntry(1, 1.toShort, 0, TransactionResult.COMMIT, partitions))) buildRequest(requestBuilder) } diff --git a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala index 47092820fbd..d56a9f0559c 100644 --- a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala @@ -139,9 +139,7 @@ class LogDirFailureTest extends IntegrationTestHarness { // The first send() should succeed producer.send(record).get() - TestUtils.waitUntilTrue(() => { - consumer.poll(0).count() == 1 - }, "Expected the first message", 3000L) + TestUtils.consumeRecords(consumer, 1) // Make log directory of the partition on the leader broker inaccessible by replacing it with a file val replica = leaderServer.replicaManager.localReplicaOrException(partition) @@ -188,9 +186,7 @@ class LogDirFailureTest extends IntegrationTestHarness { producer.send(record).get(6000L, TimeUnit.MILLISECONDS) // Consumer should receive some messages - TestUtils.waitUntilTrue(() => { - consumer.poll(0).count() > 0 - }, "Expected some messages", 3000L) + TestUtils.pollUntilAtLeastNumRecords(consumer, 1) // There should be no remaining LogDirEventNotification znode assertTrue(zkClient.getAllLogDirEventNotifications.isEmpty) @@ -203,10 +199,7 @@ class LogDirFailureTest extends IntegrationTestHarness { private def subscribeAndWaitForAssignment(topic: String, consumer: KafkaConsumer[Array[Byte], Array[Byte]]) { consumer.subscribe(Collections.singletonList(topic)) - TestUtils.waitUntilTrue(() => { - consumer.poll(0) - !consumer.assignment.isEmpty - }, "Expected non-empty assignment") + TestUtils.pollUntilTrue(consumer, () => !consumer.assignment.isEmpty, "Expected non-empty assignment") } } diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index 50449dc3e36..04b34675d86 100755 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -50,6 +50,7 @@ class LogOffsetTest extends BaseRequestTest { props.put("log.segment.bytes", "140") } + @deprecated("ListOffsetsRequest V0", since = "") @Test def testGetOffsetsForUnknownTopic() { val topicPartition = new TopicPartition("foo", 0) @@ -60,6 +61,7 @@ class LogOffsetTest extends BaseRequestTest { assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, response.responseData.get(topicPartition).error) } + @deprecated("ListOffsetsRequest V0", since = "") @Test def testGetOffsetsAfterDeleteRecords() { val topic = "kafka-" @@ -151,6 +153,7 @@ class LogOffsetTest extends BaseRequestTest { assertFalse(offsetChanged) } + @deprecated("legacyFetchOffsetsBefore", since = "") @Test def testGetOffsetsBeforeNow() { val random = new Random @@ -180,6 +183,7 @@ class LogOffsetTest extends BaseRequestTest { assertEquals(Seq(20L, 18L, 16L, 14L, 12L, 10L, 8L, 6L, 4L, 2L, 0L), consumerOffsets) } + @deprecated("legacyFetchOffsetsBefore", since = "") @Test def testGetOffsetsBeforeEarliestTime() { val random = new Random diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index f2d3b4a1cad..7b2fde3993d 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -240,7 +240,7 @@ class RequestQuotaTest extends BaseRequestTest { case ApiKeys.OFFSET_COMMIT => new OffsetCommitRequest.Builder("test-group", - Map(tp -> new OffsetCommitRequest.PartitionData(0, 15, "metadata")).asJava). + Map(tp -> new OffsetCommitRequest.PartitionData(0, Optional.empty[Integer](), "metadata")).asJava). setMemberId("").setGenerationId(1) case ApiKeys.OFFSET_FETCH => diff --git a/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala b/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala index 5c37891c937..1d70539a91b 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala @@ -20,7 +20,7 @@ package kafka.server.epoch import java.io.{File, RandomAccessFile} import java.util.Properties -import kafka.api.KAFKA_0_11_0_IV1 +import kafka.api.ApiVersion import kafka.log.Log import kafka.server.KafkaConfig._ import kafka.server.{KafkaConfig, KafkaServer} @@ -32,7 +32,7 @@ import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.record.RecordBatch -import org.apache.kafka.common.serialization.Deserializer +import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.junit.Assert.{assertEquals, assertTrue} import org.junit.{After, Before, Test} @@ -42,7 +42,7 @@ import scala.collection.Seq /** * These tests were written to assert the addition of leader epochs to the replication protocol fix the problems - * described in KIP-101. There is a boolean KIP_101_ENABLED which can be toggled to demonstrate the tests failing in the pre-KIP-101 case + * described in KIP-101. * * https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Epoch+rather+than+High+Watermark+for+Truncation * @@ -50,6 +50,8 @@ import scala.collection.Seq */ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness with Logging { + // Set this to KAFKA_0_11_0_IV1 to demonstrate the tests failing in the pre-KIP-101 case + val apiVersion = ApiVersion.latestVersion val topic = "topic1" val msg = new Array[Byte](1000) val msgBigger = new Array[Byte](10000) @@ -57,8 +59,6 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness var producer: KafkaProducer[Array[Byte], Array[Byte]] = null var consumer: KafkaConsumer[Array[Byte], Array[Byte]] = null - val KIP_101_ENABLED = true - @Before override def setUp() { super.setUp() @@ -239,7 +239,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness //Search to see if we have non-monotonic offsets in the log startConsumer() - val records = consumer.poll(1000).asScala + val records = TestUtils.pollUntilAtLeastNumRecords(consumer, 100) var prevOffset = -1L records.foreach { r => assertTrue(s"Offset $prevOffset came before ${r.offset} ", r.offset > prevOffset) @@ -397,7 +397,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, getBrokerListStrFromServers(brokers)) consumerConfig.put(ConsumerConfig.FETCH_MAX_BYTES_CONFIG, String.valueOf(getLogFile(brokers(1), 0).length() * 2)) consumerConfig.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, String.valueOf(getLogFile(brokers(1), 0).length() * 2)) - consumer = new KafkaConsumer(consumerConfig, new StubDeserializer, new StubDeserializer) + consumer = new KafkaConsumer(consumerConfig, new ByteArrayDeserializer, new ByteArrayDeserializer) consumer.assign(List(new TopicPartition(topic, 0)).asJava) consumer.seek(new TopicPartition(topic, 0), 0) consumer @@ -435,9 +435,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness producer = createProducer //TODO not sure why we need to recreate the producer, but it doesn't reconnect if we don't } - private def epochCache(broker: KafkaServer): LeaderEpochFileCache = { - getLog(broker, 0).leaderEpochCache.asInstanceOf[LeaderEpochFileCache] - } + private def epochCache(broker: KafkaServer): LeaderEpochFileCache = getLog(broker, 0).leaderEpochCache private def latestRecord(leader: KafkaServer, offset: Int = -1, partition: Int = 0): RecordBatch = { getLog(leader, partition).activeSegment.read(0, None, Integer.MAX_VALUE) @@ -468,19 +466,9 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness private def createBroker(id: Int, enableUncleanLeaderElection: Boolean = false): KafkaServer = { val config = createBrokerConfig(id, zkConnect) - if(!KIP_101_ENABLED) { - config.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, KAFKA_0_11_0_IV1.version) - config.setProperty(KafkaConfig.LogMessageFormatVersionProp, KAFKA_0_11_0_IV1.version) - } + config.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, apiVersion.version) + config.setProperty(KafkaConfig.LogMessageFormatVersionProp, apiVersion.version) config.setProperty(KafkaConfig.UncleanLeaderElectionEnableProp, enableUncleanLeaderElection.toString) createServer(fromProps(config)) } - - private class StubDeserializer extends Deserializer[Array[Byte]] { - override def configure(configs: java.util.Map[String, _], isKey: Boolean): Unit = {} - - override def deserialize(topic: String, data: Array[Byte]): Array[Byte] = { data } - - override def close(): Unit = {} - } } diff --git a/core/src/test/scala/unit/kafka/tools/ConsoleProducerTest.scala b/core/src/test/scala/unit/kafka/tools/ConsoleProducerTest.scala index 86aae548e54..e69a5e67ae8 100644 --- a/core/src/test/scala/unit/kafka/tools/ConsoleProducerTest.scala +++ b/core/src/test/scala/unit/kafka/tools/ConsoleProducerTest.scala @@ -63,7 +63,7 @@ class ConsoleProducerTest { @Test def testParseKeyProp(): Unit = { val config = new ConsoleProducer.ProducerConfig(validArgs) - val reader = Class.forName(config.readerClass).newInstance().asInstanceOf[LineMessageReader] + val reader = Class.forName(config.readerClass).getDeclaredConstructor().newInstance().asInstanceOf[LineMessageReader] reader.init(System.in,ConsoleProducer.getReaderProps(config)) assert(reader.keySeparator == "#") assert(reader.parseKey) diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 47d45ad70a8..c3e7312f5af 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -1230,7 +1230,7 @@ object TestUtils extends Logging { } finally consumer.close() } - def pollUntilAtLeastNumRecords[K, V](consumer: KafkaConsumer[K, V], + def pollUntilAtLeastNumRecords[K, V](consumer: Consumer[K, V], numRecords: Int, waitTimeMs: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Seq[ConsumerRecord[K, V]] = { val records = new ArrayBuffer[ConsumerRecord[K, V]]() @@ -1244,7 +1244,7 @@ object TestUtils extends Logging { records } - def consumeRecords[K, V](consumer: KafkaConsumer[K, V], + def consumeRecords[K, V](consumer: Consumer[K, V], numRecords: Int, waitTimeMs: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Seq[ConsumerRecord[K, V]] = { val records = pollUntilAtLeastNumRecords(consumer, numRecords, waitTimeMs) diff --git a/core/src/test/scala/unit/kafka/utils/ZkUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/ZkUtilsTest.scala index 292db8b8895..c0c3c6b8ef0 100755 --- a/core/src/test/scala/unit/kafka/utils/ZkUtilsTest.scala +++ b/core/src/test/scala/unit/kafka/utils/ZkUtilsTest.scala @@ -25,6 +25,7 @@ import org.apache.kafka.common.security.JaasUtils import org.junit.Assert._ import org.junit.{After, Before, Test} +@deprecated("Deprecated given that ZkUtils is deprecated", since = "2.0.0") class ZkUtilsTest extends ZooKeeperTestHarness { val path = "/path" diff --git a/core/src/test/scala/unit/kafka/zk/AdminZkClientTest.scala b/core/src/test/scala/unit/kafka/zk/AdminZkClientTest.scala index ec4eed29536..b15f8ac47b5 100644 --- a/core/src/test/scala/unit/kafka/zk/AdminZkClientTest.scala +++ b/core/src/test/scala/unit/kafka/zk/AdminZkClientTest.scala @@ -170,7 +170,7 @@ class AdminZkClientTest extends ZooKeeperTestHarness with Logging with RackAware assertEquals(props, savedProps) } - TestUtils.assertConcurrent("Concurrent topic creation failed", Seq(createTopic, createTopic), + TestUtils.assertConcurrent("Concurrent topic creation failed", Seq(() => createTopic, () => createTopic), JTestUtils.DEFAULT_MAX_WAIT_MS.toInt) } diff --git a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala index 0088c657de0..fd3f59cefc2 100644 --- a/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala +++ b/core/src/test/scala/unit/kafka/zookeeper/ZooKeeperClientTest.scala @@ -367,11 +367,11 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { } } + val goodCalls = new AtomicInteger(0) val goodHandler = new StateChangeHandler { - val calls = new AtomicInteger(0) override val name = this.getClass.getName override def beforeInitializingSession(): Unit = { - calls.incrementAndGet() + goodCalls.incrementAndGet() } } @@ -382,7 +382,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness { client.forceReinitialize() - assertEquals(1, goodHandler.calls.get) + assertEquals(1, goodCalls.get) // Client should be usable even if the callback throws an error val createResponse = zooKeeperClient.handleRequest(CreateRequest(mockPath, Array.empty[Byte], diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRecordBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRecordBenchmark.java index e7b39827d0c..83d5c2b19c9 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRecordBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRecordBenchmark.java @@ -41,7 +41,7 @@ public class ProducerRecordBenchmark { @Benchmark @OutputTimeUnit(TimeUnit.NANOSECONDS) public ProducerRecord constructorBenchmark() { - return new ProducerRecord("topic", "value"); + return new ProducerRecord<>("topic", "value"); } } diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java index f8af99f6911..18b4912789a 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java @@ -33,7 +33,6 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.TimeWindows; import java.io.IOException; @@ -171,6 +170,7 @@ public class PageViewTypedDemo { public String region; } + @SuppressWarnings("deprecation") public static void main(final String[] args) { final Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pageview-typed"); @@ -206,7 +206,7 @@ public class PageViewTypedDemo { return viewByRegion; }) .map((user, viewRegion) -> new KeyValue<>(viewRegion.region, viewRegion)) - .groupByKey(Serialized.with(Serdes.String(), new JSONSerde<>())) + .groupByKey(org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), new JSONSerde<>())) .windowedBy(TimeWindows.of(Duration.ofDays(7)).advanceBy(Duration.ofSeconds(1))) .count() .toStream() diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java index 25b10e8fa06..d49223891f0 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java @@ -35,7 +35,6 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.Produced; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.TimeWindows; import java.util.Properties; @@ -55,6 +54,7 @@ import java.util.Properties; */ public class PageViewUntypedDemo { + @SuppressWarnings("deprecation") public static void main(final String[] args) throws Exception { final Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pageview-untyped"); @@ -87,7 +87,7 @@ public class PageViewUntypedDemo { }) .map((user, viewRegion) -> new KeyValue<>(viewRegion.get("region").textValue(), viewRegion)) - .groupByKey(Serialized.with(Serdes.String(), jsonSerde)) + .groupByKey(org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), jsonSerde)) .windowedBy(TimeWindows.of(Duration.ofDays(7)).advanceBy(Duration.ofSeconds(1))) .count() .toStream() diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index f1a875479a7..3b691513004 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -30,7 +30,6 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Predicate; import org.apache.kafka.streams.kstream.Printed; import org.apache.kafka.streams.kstream.Produced; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.TransformerSupplier; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueMapper; @@ -797,7 +796,7 @@ public class KStreamImpl extends AbstractStream implements KStream KGroupedStream groupBy(final KeyValueMapper selector, - final Serialized serialized) { + final org.apache.kafka.streams.kstream.Serialized serialized) { Objects.requireNonNull(selector, "selector can't be null"); Objects.requireNonNull(serialized, "serialized can't be null"); final SerializedInternal serializedInternal = new SerializedInternal<>(serialized); @@ -832,7 +831,7 @@ public class KStreamImpl extends AbstractStream implements KStream groupByKey(final Serialized serialized) { + public KGroupedStream groupByKey(final org.apache.kafka.streams.kstream.Serialized serialized) { final SerializedInternal serializedInternal = new SerializedInternal<>(serialized); return groupByKey(Grouped.with(serializedInternal.keySerde(), serializedInternal.valueSerde())); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index f49d109c0fc..9caac30e1b3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -26,7 +26,6 @@ import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.Predicate; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.Suppressed; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueMapper; @@ -493,7 +492,7 @@ public class KTableImpl extends AbstractStream implements KTable< final boolean rightOuter, final String joinMergeName, final String internalQueryableName, - final MaterializedInternal materializedInternal) { + final MaterializedInternal> materializedInternal) { final Set allSourceNodes = ensureJoinableWith(other); if (leftOuter) { @@ -523,7 +522,7 @@ public class KTableImpl extends AbstractStream implements KTable< final KTableKTableJoinMerger joinMerge = new KTableKTableJoinMerger<>(joinThis, joinOther, internalQueryableName); - final KTableKTableJoinNode.KTableKTableJoinNodeBuilder, Change, Change> kTableJoinNodeBuilder = KTableKTableJoinNode.kTableKTableJoinNodeBuilder(); + final KTableKTableJoinNode.KTableKTableJoinNodeBuilder kTableJoinNodeBuilder = KTableKTableJoinNode.kTableKTableJoinNodeBuilder(); // only materialize if specified in Materialized if (materializedInternal != null) { @@ -543,7 +542,7 @@ public class KTableImpl extends AbstractStream implements KTable< .withOtherJoinSideNodeName(((KTableImpl) other).name) .withThisJoinSideNodeName(name); - final KTableKTableJoinNode, Change, Change> kTableKTableJoinNode = kTableJoinNodeBuilder.build(); + final KTableKTableJoinNode kTableKTableJoinNode = kTableJoinNodeBuilder.build(); builder.addGraphNode(this.streamsGraphNode, kTableKTableJoinNode); // we can inherit parent key serde if user do not provide specific overrides @@ -568,7 +567,7 @@ public class KTableImpl extends AbstractStream implements KTable< @Override @Deprecated public KGroupedTable groupBy(final KeyValueMapper> selector, - final Serialized serialized) { + final org.apache.kafka.streams.kstream.Serialized serialized) { Objects.requireNonNull(selector, "selector can't be null"); Objects.requireNonNull(serialized, "serialized can't be null"); final SerializedInternal serializedInternal = new SerializedInternal<>(serialized); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SerializedInternal.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SerializedInternal.java index 0cb7050cd8e..09fd9e5bd0c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SerializedInternal.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SerializedInternal.java @@ -17,11 +17,10 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.streams.kstream.Serialized; @Deprecated -public class SerializedInternal extends Serialized { - public SerializedInternal(final Serialized serialized) { +public class SerializedInternal extends org.apache.kafka.streams.kstream.Serialized { + public SerializedInternal(final org.apache.kafka.streams.kstream.Serialized serialized) { super(serialized); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java index 41c27bac5f9..796468e8e33 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java @@ -17,7 +17,9 @@ package org.apache.kafka.streams.kstream.internals.graph; +import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.streams.kstream.internals.Change; import org.apache.kafka.streams.kstream.internals.KeyValueStoreMaterializer; import org.apache.kafka.streams.kstream.internals.MaterializedInternal; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; @@ -29,18 +31,18 @@ import java.util.Arrays; /** * Too much specific information to generalize so the KTable-KTable join requires a specific node. */ -public class KTableKTableJoinNode extends BaseJoinProcessorNode { +public class KTableKTableJoinNode extends BaseJoinProcessorNode, Change, Change> { private final String[] joinThisStoreNames; private final String[] joinOtherStoreNames; - private final MaterializedInternal materializedInternal; + private final MaterializedInternal> materializedInternal; KTableKTableJoinNode(final String nodeName, - final ValueJoiner valueJoiner, - final ProcessorParameters joinThisProcessorParameters, - final ProcessorParameters joinOtherProcessorParameters, - final ProcessorParameters joinMergeProcessorParameters, - final MaterializedInternal materializedInternal, + final ValueJoiner, ? super Change, ? extends Change> valueJoiner, + final ProcessorParameters> joinThisProcessorParameters, + final ProcessorParameters> joinOtherProcessorParameters, + final ProcessorParameters> joinMergeProcessorParameters, + final MaterializedInternal> materializedInternal, final String thisJoinSide, final String otherJoinSide, final String[] joinThisStoreNames, @@ -107,12 +109,12 @@ public class KTableKTableJoinNode extends BaseJoinProcessorNode joinThisProcessorParameters; + private ProcessorParameters> joinThisProcessorParameters; private String[] joinOtherStoreNames; - private MaterializedInternal materializedInternal; - private ProcessorParameters joinOtherProcessorParameters; - private ProcessorParameters joinMergeProcessorParameters; - private ValueJoiner valueJoiner; + private MaterializedInternal> materializedInternal; + private ProcessorParameters> joinOtherProcessorParameters; + private ProcessorParameters> joinMergeProcessorParameters; + private ValueJoiner, ? super Change, ? extends Change> valueJoiner; private String thisJoinSide; private String otherJoinSide; @@ -124,7 +126,7 @@ public class KTableKTableJoinNode extends BaseJoinProcessorNode withJoinThisProcessorParameters(final ProcessorParameters joinThisProcessorParameters) { + public KTableKTableJoinNodeBuilder withJoinThisProcessorParameters(final ProcessorParameters> joinThisProcessorParameters) { this.joinThisProcessorParameters = joinThisProcessorParameters; return this; } @@ -139,17 +141,17 @@ public class KTableKTableJoinNode extends BaseJoinProcessorNode withJoinOtherProcessorParameters(final ProcessorParameters joinOtherProcessorParameters) { + public KTableKTableJoinNodeBuilder withJoinOtherProcessorParameters(final ProcessorParameters> joinOtherProcessorParameters) { this.joinOtherProcessorParameters = joinOtherProcessorParameters; return this; } - public KTableKTableJoinNodeBuilder withJoinMergeProcessorParameters(final ProcessorParameters joinMergeProcessorParameters) { + public KTableKTableJoinNodeBuilder withJoinMergeProcessorParameters(final ProcessorParameters> joinMergeProcessorParameters) { this.joinMergeProcessorParameters = joinMergeProcessorParameters; return this; } - public KTableKTableJoinNodeBuilder withValueJoiner(final ValueJoiner valueJoiner) { + public KTableKTableJoinNodeBuilder withValueJoiner(final ValueJoiner, ? super Change, ? extends Change> valueJoiner) { this.valueJoiner = valueJoiner; return this; } @@ -164,7 +166,8 @@ public class KTableKTableJoinNode extends BaseJoinProcessorNode withMaterializedInternal(final MaterializedInternal materializedInternal) { + public KTableKTableJoinNodeBuilder withMaterializedInternal( + final MaterializedInternal> materializedInternal) { this.materializedInternal = materializedInternal; return this; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/OptimizableRepartitionNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/OptimizableRepartitionNode.java index 05ec6fb89b5..4797a21399d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/OptimizableRepartitionNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/OptimizableRepartitionNode.java @@ -24,7 +24,7 @@ import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.processor.FailOnInvalidTimestamp; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; -public class OptimizableRepartitionNode extends BaseRepartitionNode { +public class OptimizableRepartitionNode extends BaseRepartitionNode { OptimizableRepartitionNode(final String nodeName, final String sourceName, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSinkNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSinkNode.java index 0f8ac2cb9b4..95076c844af 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSinkNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSinkNode.java @@ -58,6 +58,7 @@ public class StreamSinkNode extends StreamsGraphNode { final String[] parentNames = parentNodeNames(); if (partitioner == null && keySerializer instanceof WindowedSerializer) { + @SuppressWarnings("unchecked") final StreamPartitioner windowedPartitioner = (StreamPartitioner) new WindowedStreamPartitioner((WindowedSerializer) keySerializer); topologyBuilder.addSink(nodeName(), topicNameExtractor, keySerializer, valSerializer, windowedPartitioner, parentNames); } else { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNode.java index 14281237175..0e6435aa289 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNode.java @@ -53,6 +53,7 @@ public class TableProcessorNode extends StreamsGraph "} " + super.toString(); } + @SuppressWarnings("unchecked") @Override public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { final boolean shouldMaterialize = materializedInternal != null && materializedInternal.isQueryable(); @@ -65,7 +66,8 @@ public class TableProcessorNode extends StreamsGraph } if (shouldMaterialize) { - topologyBuilder.addStateStore(new KeyValueStoreMaterializer<>((MaterializedInternal>) materializedInternal).materialize(), processorName); + topologyBuilder.addStateStore(new KeyValueStoreMaterializer<>( + (MaterializedInternal>) materializedInternal).materialize(), processorName); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java index 8576ee25841..90fb5d47784 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java @@ -73,6 +73,7 @@ public class TableSourceNode extends StreamSourceNod } @Override + @SuppressWarnings("unchecked") public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { final String topicName = getTopicNames().iterator().next(); @@ -129,7 +130,7 @@ public class TableSourceNode extends StreamSourceNod return this; } - public TableSourceNodeBuilder withConsumedInternal(final ConsumedInternal consumedInternal) { + public TableSourceNodeBuilder withConsumedInternal(final ConsumedInternal consumedInternal) { this.consumedInternal = consumedInternal; return this; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index 7fa2851c59b..938ed91c468 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -514,12 +514,12 @@ public class InternalTopologyBuilder { nodeGroups = null; } - public final void addStateStore(final StoreBuilder storeBuilder, + public final void addStateStore(final StoreBuilder storeBuilder, final String... processorNames) { addStateStore(storeBuilder, false, processorNames); } - public final void addStateStore(final StoreBuilder storeBuilder, + public final void addStateStore(final StoreBuilder storeBuilder, final boolean allowOverride, final String... processorNames) { Objects.requireNonNull(storeBuilder, "storeBuilder can't be null"); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java index 50ce386f13f..57792b61fbe 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java @@ -88,6 +88,7 @@ public interface WindowStore extends StateStore, ReadOnlyWindowStore * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException If {@code null} is used for key. */ + @SuppressWarnings("deprecation") WindowStoreIterator fetch(K key, long timeFrom, long timeTo); @Override @@ -110,6 +111,7 @@ public interface WindowStore extends StateStore, ReadOnlyWindowStore * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException If {@code null} is used for any key. */ + @SuppressWarnings("deprecation") KeyValueIterator, V> fetch(K from, K to, long timeFrom, long timeTo); @Override @@ -128,6 +130,7 @@ public interface WindowStore extends StateStore, ReadOnlyWindowStore * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException if {@code null} is used for any key */ + @SuppressWarnings("deprecation") KeyValueIterator, V> fetchAll(long timeFrom, long timeTo); @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index b55e5448139..c04921b1db3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -178,6 +178,7 @@ class CachingWindowStore extends WrappedStateStore.AbstractStateStore impl } } + @SuppressWarnings("deprecation") @Override public synchronized WindowStoreIterator fetch(final Bytes key, final long timeFrom, final long timeTo) { // since this function may not access the underlying inner store, we need to validate @@ -203,6 +204,7 @@ class CachingWindowStore extends WrappedStateStore.AbstractStateStore impl return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator); } + @SuppressWarnings("deprecation") @Override public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to, final long timeFrom, final long timeTo) { // since this function may not access the underlying inner store, we need to validate @@ -255,7 +257,8 @@ class CachingWindowStore extends WrappedStateStore.AbstractStateStore impl cacheFunction ); } - + + @SuppressWarnings("deprecation") @Override public KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo) { validateStoreOpen(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java index 9808ca967cf..a592471bee9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java @@ -51,11 +51,13 @@ class ChangeLoggingWindowBytesStore extends WrappedStateStore.AbstractStateStore return bytesStore.fetch(key, timestamp); } + @SuppressWarnings("deprecation") @Override public WindowStoreIterator fetch(final Bytes key, final long from, final long to) { return bytesStore.fetch(key, from, to); } + @SuppressWarnings("deprecation") @Override public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, final Bytes keyTo, final long from, final long to) { return bytesStore.fetch(keyFrom, keyTo, from, to); @@ -65,7 +67,8 @@ class ChangeLoggingWindowBytesStore extends WrappedStateStore.AbstractStateStore public KeyValueIterator, byte[]> all() { return bytesStore.all(); } - + + @SuppressWarnings("deprecation") @Override public KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo) { return bytesStore.fetchAll(timeFrom, timeTo); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java index d95b44222e2..84b589d348a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java @@ -94,6 +94,7 @@ public class CompositeReadOnlyWindowStore implements ReadOnlyWindowStore, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { Objects.requireNonNull(from, "from can't be null"); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java index 5162eac8848..fefa772ac15 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java @@ -138,6 +138,7 @@ public class MeteredWindowStore extends WrappedStateStore.AbstractStateSto } } + @SuppressWarnings("deprecation") @Override public WindowStoreIterator fetch(final K key, final long timeFrom, final long timeTo) { return new MeteredWindowStoreIterator<>(inner.fetch(keyBytes(key), timeFrom, timeTo), @@ -152,6 +153,7 @@ public class MeteredWindowStore extends WrappedStateStore.AbstractStateSto return new MeteredWindowedKeyValueIterator<>(inner.all(), fetchTime, metrics, serdes, time); } + @SuppressWarnings("deprecation") @Override public KeyValueIterator, V> fetchAll(final long timeFrom, final long timeTo) { return new MeteredWindowedKeyValueIterator<>(inner.fetchAll(timeFrom, timeTo), @@ -161,6 +163,7 @@ public class MeteredWindowStore extends WrappedStateStore.AbstractStateSto time); } + @SuppressWarnings("deprecation") @Override public KeyValueIterator, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { return new MeteredWindowedKeyValueIterator<>(inner.fetch(keyBytes(from), keyBytes(to), timeFrom, timeTo), diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index d7bb523b049..bb13c74fbe9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -85,12 +85,14 @@ public class RocksDBWindowStore extends WrappedStateStore.AbstractStateSto return serdes.valueFrom(bytesValue); } + @SuppressWarnings("deprecation") @Override public WindowStoreIterator fetch(final K key, final long timeFrom, final long timeTo) { final KeyValueIterator bytesIterator = bytesStore.fetch(Bytes.wrap(serdes.rawKey(key)), timeFrom, timeTo); return new WindowStoreIteratorWrapper<>(bytesIterator, serdes, windowSize).valuesIterator(); } + @SuppressWarnings("deprecation") @Override public KeyValueIterator, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { final KeyValueIterator bytesIterator = bytesStore.fetch(Bytes.wrap(serdes.rawKey(from)), Bytes.wrap(serdes.rawKey(to)), timeFrom, timeTo); @@ -103,6 +105,7 @@ public class RocksDBWindowStore extends WrappedStateStore.AbstractStateSto return new WindowStoreIteratorWrapper<>(bytesIterator, serdes, windowSize).keyValueIterator(); } + @SuppressWarnings("deprecation") @Override public KeyValueIterator, V> fetchAll(final long timeFrom, final long timeTo) { final KeyValueIterator bytesIterator = bytesStore.fetchAll(timeFrom, timeTo); diff --git a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java index b9d542bc9b6..fde5bff9728 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java @@ -28,7 +28,6 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.network.Selectable; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; @@ -49,7 +48,6 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import java.io.File; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -60,6 +58,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -157,7 +156,7 @@ public class KafkaStreamsTest { @Test public void shouldCleanupResourcesOnCloseWithoutPreviousStart() throws Exception { builder.globalTable("anyTopic"); - final List nodes = Arrays.asList(new Node(0, "localhost", 8121)); + final List nodes = asList(new Node(0, "localhost", 8121)); final Cluster cluster = new Cluster("mockClusterId", nodes, Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), nodes.get(0)); @@ -482,7 +481,7 @@ public class KafkaStreamsTest { assertEquals(2, threadMetadata.size()); for (final ThreadMetadata metadata : threadMetadata) { assertTrue("#threadState() was: " + metadata.threadState() + "; expected either RUNNING, PARTITIONS_REVOKED, PARTITIONS_ASSIGNED, or CREATED", - Utils.mkList("RUNNING", "PARTITIONS_REVOKED", "PARTITIONS_ASSIGNED", "CREATED").contains(metadata.threadState())); + asList("RUNNING", "PARTITIONS_REVOKED", "PARTITIONS_ASSIGNED", "CREATED").contains(metadata.threadState())); assertEquals(0, metadata.standbyTasks().size()); assertEquals(0, metadata.activeTasks().size()); } 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 2531591869d..331323d34b1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java @@ -44,6 +44,7 @@ import java.util.Iterator; import java.util.Map; import java.util.Properties; +import static java.util.Arrays.asList; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; @@ -173,7 +174,7 @@ public class StreamsBuilderTest { } // no exception was thrown - assertEquals(Utils.mkList("A:aa"), processorSupplier.theCapturedProcessor().processed); + assertEquals(asList("A:aa"), processorSupplier.theCapturedProcessor().processed); } @Test @@ -192,8 +193,8 @@ public class StreamsBuilderTest { driver.pipeInput(recordFactory.create("topic-source", "A", "aa")); } - assertEquals(Utils.mkList("A:aa"), sourceProcessorSupplier.theCapturedProcessor().processed); - assertEquals(Utils.mkList("A:aa"), throughProcessorSupplier.theCapturedProcessor().processed); + assertEquals(asList("A:aa"), sourceProcessorSupplier.theCapturedProcessor().processed); + assertEquals(asList("A:aa"), throughProcessorSupplier.theCapturedProcessor().processed); } @Test @@ -216,7 +217,7 @@ public class StreamsBuilderTest { driver.pipeInput(recordFactory.create(topic1, "D", "dd")); } - assertEquals(Utils.mkList("A:aa", "B:bb", "C:cc", "D:dd"), processorSupplier.theCapturedProcessor().processed); + assertEquals(asList("A:aa", "B:bb", "C:cc", "D:dd"), processorSupplier.theCapturedProcessor().processed); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java index de856bba236..7a58e1fee7b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java @@ -652,7 +652,7 @@ public class TopologyTest { public void topologyWithDynamicRoutingShouldDescribeExtractorClass() { final StreamsBuilder builder = new StreamsBuilder(); - final TopicNameExtractor topicNameExtractor = new TopicNameExtractor() { + final TopicNameExtractor topicNameExtractor = new TopicNameExtractor() { @Override public String extract(final Object key, final Object value, final RecordContext recordContext) { return recordContext.topic() + "-" + key; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java index ac5a41837fa..6539168489f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java @@ -38,7 +38,6 @@ import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.Produced; import org.apache.kafka.test.IntegrationTest; import org.apache.kafka.test.StreamsTestUtils; -import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; import org.junit.Before; import org.junit.BeforeClass; @@ -229,7 +228,7 @@ public class FineGrainedAutoResetIntegrationTest { } private void commitInvalidOffsets() { - final KafkaConsumer consumer = new KafkaConsumer(TestUtils.consumerConfig( + final KafkaConsumer consumer = new KafkaConsumer<>(TestUtils.consumerConfig( CLUSTER.bootstrapServers(), streamsConfiguration.getProperty(StreamsConfig.APPLICATION_ID_CONFIG), StringDeserializer.class, @@ -301,16 +300,10 @@ public class FineGrainedAutoResetIntegrationTest { final TestingUncaughtExceptionHandler uncaughtExceptionHandler = new TestingUncaughtExceptionHandler(); - final TestCondition correctExceptionThrownCondition = new TestCondition() { - @Override - public boolean conditionMet() { - return uncaughtExceptionHandler.correctExceptionThrown; - } - }; - streams.setUncaughtExceptionHandler(uncaughtExceptionHandler); streams.start(); - TestUtils.waitForCondition(correctExceptionThrownCondition, "The expected NoOffsetForPartitionException was never thrown"); + TestUtils.waitForCondition(() -> uncaughtExceptionHandler.correctExceptionThrown, + "The expected NoOffsetForPartitionException was never thrown"); streams.close(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java index 08aa245b616..1db23a55d2c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java @@ -37,7 +37,6 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.Produced; import org.apache.kafka.streams.kstream.Reducer; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.test.IntegrationTest; import org.apache.kafka.test.MockMapper; @@ -79,7 +78,7 @@ public class KStreamAggregationDedupIntegrationTest { private Reducer reducer; private KStream stream; - + @SuppressWarnings("deprecation") @Before public void before() throws InterruptedException { testNo++; @@ -100,7 +99,7 @@ public class KStreamAggregationDedupIntegrationTest { groupedStream = stream .groupBy( mapper, - Serialized.with(Serdes.String(), Serdes.String())); + org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.String())); reducer = (value1, value2) -> value1 + ":" + value2; } @@ -174,13 +173,14 @@ public class KStreamAggregationDedupIntegrationTest { ); } + @SuppressWarnings("deprecation") @Test public void shouldGroupByKey() throws Exception { final long timestamp = mockTime.milliseconds(); produceMessages(timestamp); produceMessages(timestamp); - stream.groupByKey(Serialized.with(Serdes.Integer(), Serdes.String())) + stream.groupByKey(org.apache.kafka.streams.kstream.Serialized.with(Serdes.Integer(), Serdes.String())) .windowedBy(TimeWindows.of(ofMillis(500L))) .count(Materialized.as("count-windows")) .toStream((windowedKey, value) -> windowedKey.key() + "@" + windowedKey.window().start()) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java index 7642f69b8e0..718483becc6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java @@ -43,7 +43,6 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.Produced; import org.apache.kafka.streams.kstream.Reducer; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.SessionWindowedDeserializer; import org.apache.kafka.streams.kstream.SessionWindows; import org.apache.kafka.streams.kstream.TimeWindowedDeserializer; @@ -93,6 +92,7 @@ import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +@SuppressWarnings({"unchecked", "deprecation"}) @Category({IntegrationTest.class}) public class KStreamAggregationIntegrationTest { private static final int NUM_BROKERS = 1; @@ -138,7 +138,7 @@ public class KStreamAggregationIntegrationTest { groupedStream = stream .groupBy( mapper, - Serialized.with(Serdes.String(), Serdes.String())); + org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.String())); reducer = (value1, value2) -> value1 + ":" + value2; initializer = () -> 0; @@ -428,7 +428,7 @@ public class KStreamAggregationIntegrationTest { produceMessages(timestamp); produceMessages(timestamp); - stream.groupByKey(Serialized.with(Serdes.Integer(), Serdes.String())) + stream.groupByKey(org.apache.kafka.streams.kstream.Serialized.with(Serdes.Integer(), Serdes.String())) .windowedBy(TimeWindows.of(ofMillis(500L))) .count() .toStream((windowedKey, value) -> windowedKey.key() + "@" + windowedKey.window().start()).to(outputTopic, Produced.with(Serdes.String(), Serdes.Long())); @@ -521,7 +521,7 @@ public class KStreamAggregationIntegrationTest { final CountDownLatch latch = new CountDownLatch(11); builder.stream(userSessionsStream, Consumed.with(Serdes.String(), Serdes.String())) - .groupByKey(Serialized.with(Serdes.String(), Serdes.String())) + .groupByKey(org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.String())) .windowedBy(SessionWindows.with(ofMillis(sessionGap))) .count() .toStream() @@ -619,7 +619,7 @@ public class KStreamAggregationIntegrationTest { final CountDownLatch latch = new CountDownLatch(11); final String userSessionsStore = "UserSessionsStore"; builder.stream(userSessionsStream, Consumed.with(Serdes.String(), Serdes.String())) - .groupByKey(Serialized.with(Serdes.String(), Serdes.String())) + .groupByKey(org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.String())) .windowedBy(SessionWindows.with(ofMillis(sessionGap))) .reduce((value1, value2) -> value1 + ":" + value2, Materialized.as(userSessionsStore)) .toStream() @@ -706,7 +706,7 @@ public class KStreamAggregationIntegrationTest { final CountDownLatch latch = new CountDownLatch(5); builder.stream(userSessionsStream, Consumed.with(Serdes.String(), Serdes.String())) - .groupByKey(Serialized.with(Serdes.String(), Serdes.String())) + .groupByKey(org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.String())) .windowedBy(UnlimitedWindows.of().startOn(ofEpochMilli(startTime))) .count() .toStream() diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java index 3fd82202a2d..81938f82d1a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java @@ -351,6 +351,7 @@ public class RestoreIntegrationTest { this.processorLatch = processorLatch; } + @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context) { this.store = (KeyValueStore) context.getStateStore(topic); @@ -391,7 +392,7 @@ public class RestoreIntegrationTest { consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class); consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class); - final Consumer consumer = new KafkaConsumer(consumerConfig); + final Consumer consumer = new KafkaConsumer<>(consumerConfig); final List partitions = Arrays.asList( new TopicPartition(topic, 0), new TopicPartition(topic, 1)); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java index 6239a681961..3aff0a2336a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java @@ -35,6 +35,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Properties; import java.util.Set; @@ -157,7 +158,7 @@ public class EmbeddedKafkaCluster extends ExternalResource { */ public void createTopics(final String... topics) throws InterruptedException { for (final String topic : topics) { - createTopic(topic, 1, 1, new Properties()); + createTopic(topic, 1, 1, Collections.emptyMap()); } } @@ -167,7 +168,7 @@ public class EmbeddedKafkaCluster extends ExternalResource { * @param topic The name of the topic. */ public void createTopic(final String topic) throws InterruptedException { - createTopic(topic, 1, 1, new Properties()); + createTopic(topic, 1, 1, Collections.emptyMap()); } /** @@ -178,7 +179,7 @@ public class EmbeddedKafkaCluster extends ExternalResource { * @param replication The replication factor for (the partitions of) this topic. */ public void createTopic(final String topic, final int partitions, final int replication) throws InterruptedException { - createTopic(topic, partitions, replication, new Properties()); + createTopic(topic, partitions, replication, Collections.emptyMap()); } /** @@ -192,7 +193,7 @@ public class EmbeddedKafkaCluster extends ExternalResource { public void createTopic(final String topic, final int partitions, final int replication, - final Properties topicConfig) throws InterruptedException { + final Map topicConfig) throws InterruptedException { brokers[0].createTopic(topic, partitions, replication, topicConfig); final List topicPartitions = new ArrayList<>(); for (int partition = 0; partition < partitions; partition++) { diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java index 6c2c464d5ac..badc63c6763 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java @@ -148,7 +148,7 @@ public class KafkaEmbedded { * @param topic The name of the topic. */ public void createTopic(final String topic) { - createTopic(topic, 1, 1, new Properties()); + createTopic(topic, 1, 1, Collections.emptyMap()); } /** @@ -159,7 +159,7 @@ public class KafkaEmbedded { * @param replication The replication factor for (the partitions of) this topic. */ public void createTopic(final String topic, final int partitions, final int replication) { - createTopic(topic, partitions, replication, new Properties()); + createTopic(topic, partitions, replication, Collections.emptyMap()); } /** @@ -173,11 +173,11 @@ public class KafkaEmbedded { public void createTopic(final String topic, final int partitions, final int replication, - final Properties topicConfig) { + final Map topicConfig) { log.debug("Creating topic { name: {}, partitions: {}, replication: {}, config: {} }", topic, partitions, replication, topicConfig); final NewTopic newTopic = new NewTopic(topic, partitions, (short) replication); - newTopic.configs((Map) topicConfig); + newTopic.configs(topicConfig); try (final AdminClient adminClient = createAdminClient()) { adminClient.createTopics(Collections.singletonList(newTopic)).all().get(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java index 1b3c1f1b7da..7127c9f703e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java @@ -24,7 +24,7 @@ import static org.apache.kafka.streams.EqualityCheck.verifyInEquality; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; - +@SuppressWarnings("deprecation") public class JoinWindowsTest { private static final long ANY_SIZE = 123L; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java index 872ae5ccd42..3f43691f206 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java @@ -38,6 +38,7 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +@SuppressWarnings("deprecation") public class RepartitionTopicNamingTest { private final KeyValueMapper kvMapper = (k, v) -> k + v; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java index 6fc0caca960..369bbad7393 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java @@ -24,6 +24,7 @@ import static org.apache.kafka.streams.EqualityCheck.verifyInEquality; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; +@SuppressWarnings("deprecation") public class SessionWindowsTest { @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java index b87a9928497..2bdb3a05716 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java @@ -28,6 +28,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.fail; +@SuppressWarnings("deprecation") public class TimeWindowsTest { private static final long ANY_SIZE = 123L; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java index 023e83fdde1..b9cad4c2f93 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java @@ -42,6 +42,7 @@ public class UnlimitedWindowsTest { UnlimitedWindows.of().startOn(ofEpochMilli(-1)); } + @SuppressWarnings("deprecation") @Test public void shouldThrowOnUntil() { final UnlimitedWindows windowSpec = UnlimitedWindows.of(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java index b360ceced03..f4ede7c78e2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java @@ -48,10 +48,10 @@ import static org.junit.Assert.assertTrue; public class AbstractStreamTest { @Test - public void testToInternlValueTransformerSupplierSuppliesNewTransformers() { - final ValueTransformerSupplier valueTransformerSupplier = createMock(ValueTransformerSupplier.class); + public void testToInternalValueTransformerSupplierSuppliesNewTransformers() { + final ValueTransformerSupplier valueTransformerSupplier = createMock(ValueTransformerSupplier.class); expect(valueTransformerSupplier.get()).andReturn(null).times(3); - final ValueTransformerWithKeySupplier valueTransformerWithKeySupplier = + final ValueTransformerWithKeySupplier valueTransformerWithKeySupplier = AbstractStream.toValueTransformerWithKeySupplier(valueTransformerSupplier); replay(valueTransformerSupplier); valueTransformerWithKeySupplier.get(); @@ -61,8 +61,9 @@ public class AbstractStreamTest { } @Test - public void testToInternalValueTransformerSupplierSuppliesNewTransformers() { - final ValueTransformerWithKeySupplier valueTransformerWithKeySupplier = createMock(ValueTransformerWithKeySupplier.class); + public void testToInternalValueTransformerWithKeySupplierSuppliesNewTransformers() { + final ValueTransformerWithKeySupplier valueTransformerWithKeySupplier = + createMock(ValueTransformerWithKeySupplier.class); expect(valueTransformerWithKeySupplier.get()).andReturn(null).times(3); replay(valueTransformerWithKeySupplier); valueTransformerWithKeySupplier.get(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java index 2bf6971ff7f..7756846fae2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilderTest.java @@ -44,6 +44,7 @@ import java.util.Map; import java.util.Set; import java.util.regex.Pattern; +import static java.util.Arrays.asList; import static org.apache.kafka.streams.Topology.AutoOffsetReset; import static org.hamcrest.core.IsInstanceOf.instanceOf; import static org.junit.Assert.assertEquals; @@ -52,6 +53,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +@SuppressWarnings("unchecked") public class InternalStreamsBuilderTest { private static final String APP_ID = "app-id"; @@ -122,7 +124,7 @@ public class InternalStreamsBuilderTest { merged.groupByKey().count(Materialized.>as("my-table")); builder.buildAndOptimizeTopology(); final Map> actual = builder.internalTopologyBuilder.stateStoreNameToSourceTopics(); - assertEquals(Utils.mkList("topic-1", "topic-2", "topic-3"), actual.get("my-table")); + assertEquals(asList("topic-1", "topic-2", "topic-3"), actual.get("my-table")); } @Test @@ -409,4 +411,4 @@ public class InternalStreamsBuilderTest { public static InternalTopologyBuilder internalTopologyBuilder(final InternalStreamsBuilder internalStreamsBuilder) { return internalStreamsBuilder.internalTopologyBuilder; } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java index 18f36aa8bc5..26b7e243744 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java @@ -35,7 +35,6 @@ import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.Merger; import org.apache.kafka.streams.kstream.Reducer; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.SessionWindows; import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.Windowed; @@ -68,6 +67,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNull; +@SuppressWarnings("unchecked") public class KGroupedStreamImplTest { private static final String TOPIC = "topic"; @@ -78,10 +78,11 @@ public class KGroupedStreamImplTest { private final ConsumerRecordFactory recordFactory = new ConsumerRecordFactory<>(new StringSerializer(), new StringSerializer()); private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String()); + @SuppressWarnings("deprecation") @Before public void before() { final KStream stream = builder.stream(TOPIC, Consumed.with(Serdes.String(), Serdes.String())); - groupedStream = stream.groupByKey(Serialized.with(Serdes.String(), Serdes.String())); + groupedStream = stream.groupByKey(org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.String())); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java index 662ede7d330..c06cce4c919 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java @@ -30,7 +30,6 @@ import org.apache.kafka.streams.kstream.KGroupedTable; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.test.ConsumerRecordFactory; import org.apache.kafka.test.MockAggregator; @@ -205,12 +204,12 @@ public class KGroupedTableImplTest { } } - @SuppressWarnings("unchecked") + @SuppressWarnings({"unchecked", "deprecation"}) @Test public void shouldCountAndMaterializeResults() { final KTable table = builder.table(topic, Consumed.with(Serdes.String(), Serdes.String())); - table.groupBy(MockMapper.selectValueKeyValueMapper(), - Serialized.with(Serdes.String(), + table.groupBy(MockMapper.selectValueKeyValueMapper(), + org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.String())) .count(Materialized.>as("count") .withKeySerde(Serdes.String()) @@ -224,12 +223,12 @@ public class KGroupedTableImplTest { } } - @SuppressWarnings("unchecked") + @SuppressWarnings({"unchecked", "deprecation"}) @Test public void shouldAggregateAndMaterializeResults() { final KTable table = builder.table(topic, Consumed.with(Serdes.String(), Serdes.String())); table.groupBy(MockMapper.selectValueKeyValueMapper(), - Serialized.with(Serdes.String(), + org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.String())) .aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, 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 d033e49f240..1f3492ca5c8 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 @@ -19,7 +19,6 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.TopologyTestDriver; @@ -33,7 +32,6 @@ import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Predicate; import org.apache.kafka.streams.kstream.Produced; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.Transformer; import org.apache.kafka.streams.kstream.TransformerSupplier; import org.apache.kafka.streams.kstream.ValueJoiner; @@ -64,6 +62,7 @@ import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; import static java.time.Duration.ofMillis; +import static java.util.Arrays.asList; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.core.IsInstanceOf.instanceOf; import static org.junit.Assert.assertEquals; @@ -71,7 +70,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; - +@SuppressWarnings("unchecked") public class KStreamImplTest { private final Consumed stringConsumed = Consumed.with(Serdes.String(), Serdes.String()); @@ -189,6 +188,7 @@ public class KStreamImplTest { TopologyWrapper.getInternalTopologyBuilder(builder.build()).setApplicationId("X").build(null).processors().size()); } + @SuppressWarnings("deprecation") @Test public void shouldPreserveSerdesForOperators() { final StreamsBuilder builder = new StreamsBuilder(); @@ -264,13 +264,13 @@ public class KStreamImplTest { assertEquals(((AbstractStream) stream1.groupByKey()).keySerde(), consumedInternal.keySerde()); assertEquals(((AbstractStream) stream1.groupByKey()).valueSerde(), consumedInternal.valueSerde()); - assertEquals(((AbstractStream) stream1.groupByKey(Serialized.with(mySerde, mySerde))).keySerde(), mySerde); - assertEquals(((AbstractStream) stream1.groupByKey(Serialized.with(mySerde, mySerde))).valueSerde(), mySerde); + assertEquals(((AbstractStream) stream1.groupByKey(org.apache.kafka.streams.kstream.Serialized.with(mySerde, mySerde))).keySerde(), mySerde); + assertEquals(((AbstractStream) stream1.groupByKey(org.apache.kafka.streams.kstream.Serialized.with(mySerde, mySerde))).valueSerde(), mySerde); assertEquals(((AbstractStream) stream1.groupBy(selector)).keySerde(), null); assertEquals(((AbstractStream) stream1.groupBy(selector)).valueSerde(), consumedInternal.valueSerde()); - assertEquals(((AbstractStream) stream1.groupBy(selector, Serialized.with(mySerde, mySerde))).keySerde(), mySerde); - assertEquals(((AbstractStream) stream1.groupBy(selector, Serialized.with(mySerde, mySerde))).valueSerde(), mySerde); + assertEquals(((AbstractStream) stream1.groupBy(selector, org.apache.kafka.streams.kstream.Serialized.with(mySerde, mySerde))).keySerde(), mySerde); + assertEquals(((AbstractStream) stream1.groupBy(selector, org.apache.kafka.streams.kstream.Serialized.with(mySerde, mySerde))).valueSerde(), mySerde); assertEquals(((AbstractStream) stream1.join(stream1, joiner, JoinWindows.of(100L))).keySerde(), null); assertEquals(((AbstractStream) stream1.join(stream1, joiner, JoinWindows.of(100L))).valueSerde(), null); @@ -364,7 +364,7 @@ public class KStreamImplTest { driver.pipeInput(recordFactory.create(input, "b", "v1")); } final List> mockProcessors = processorSupplier.capturedProcessors(2); - assertThat(mockProcessors.get(0).processed, equalTo(Utils.mkList("a:v1", "a:v2"))); + assertThat(mockProcessors.get(0).processed, equalTo(asList("a:v1", "a:v2"))); assertThat(mockProcessors.get(1).processed, equalTo(Collections.singletonList("b:v1"))); } @@ -682,7 +682,7 @@ public class KStreamImplTest { driver.pipeInput(recordFactory.create(topic1, "D", "dd")); } - assertEquals(Utils.mkList("A:aa", "B:bb", "C:cc", "D:dd"), processorSupplier.theCapturedProcessor().processed); + assertEquals(asList("A:aa", "B:bb", "C:cc", "D:dd"), processorSupplier.theCapturedProcessor().processed); } @Test @@ -711,7 +711,7 @@ public class KStreamImplTest { driver.pipeInput(recordFactory.create(topic1, "H", "hh")); } - assertEquals(Utils.mkList("A:aa", "B:bb", "C:cc", "D:dd", "E:ee", "F:ff", "G:gg", "H:hh"), + assertEquals(asList("A:aa", "B:bb", "C:cc", "D:dd", "E:ee", "F:ff", "G:gg", "H:hh"), processorSupplier.theCapturedProcessor().processed); } @@ -729,7 +729,7 @@ public class KStreamImplTest { driver.pipeInput(recordFactory.create("topic-7", "E", "ee")); } - assertEquals(Utils.mkList("A:aa", "B:bb", "C:cc", "D:dd", "E:ee"), + assertEquals(asList("A:aa", "B:bb", "C:cc", "D:dd", "E:ee"), processorSupplier.theCapturedProcessor().processed); } @@ -752,7 +752,7 @@ public class KStreamImplTest { driver.pipeInput(recordFactory.create(topic3, "E", "ee")); } - assertEquals(Utils.mkList("A:aa", "B:bb", "C:cc", "D:dd", "E:ee"), + assertEquals(asList("A:aa", "B:bb", "C:cc", "D:dd", "E:ee"), processorSupplier.theCapturedProcessor().processed); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java index 74ad19cb2d5..0fb9a5da41e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java @@ -30,7 +30,6 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.test.ConsumerRecordFactory; -import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.StreamsTestUtils; import org.junit.Rule; @@ -47,8 +46,9 @@ public class KStreamTransformTest { private final ConsumerRecordFactory recordFactory = new ConsumerRecordFactory<>(new IntegerSerializer(), new IntegerSerializer()); private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.Integer()); + @SuppressWarnings("deprecation") @Rule - public final KStreamTestDriver kstreamDriver = new KStreamTestDriver(); + public final org.apache.kafka.test.KStreamTestDriver kstreamDriver = new org.apache.kafka.test.KStreamTestDriver(); @Test public void testTransform() { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java index 570053c0c85..94d06eb2dcd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java @@ -142,6 +142,7 @@ public class KStreamTransformValuesTest { assertArrayEquals(expected, supplier.theCapturedProcessor().processed.toArray()); } + @SuppressWarnings("unchecked") @Test public void shouldInitializeTransformerWithForwardDisabledProcessorContext() { final SingletonNoOpValueTransformer transformer = new SingletonNoOpValueTransformer<>(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java index 1e39bd3339d..b82c8fef0a6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.TopologyTestDriver; @@ -31,7 +30,6 @@ import org.apache.kafka.streams.kstream.Grouped; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; @@ -50,6 +48,7 @@ import java.util.List; import java.util.Properties; import static java.time.Duration.ofMillis; +import static java.util.Arrays.asList; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.test.StreamsTestUtils.getMetricByName; @@ -100,7 +99,7 @@ public class KStreamWindowAggregateTest { } assertEquals( - Utils.mkList( + asList( "[A@0/10]:0+1", "[B@0/10]:0+2", "[C@0/10]:0+3", @@ -261,7 +260,7 @@ public class KStreamWindowAggregateTest { final String topic = "topic"; final KStream stream1 = builder.stream(topic, Consumed.with(Serdes.String(), Serdes.String())); - stream1.groupByKey(Serialized.with(Serdes.String(), Serdes.String())) + stream1.groupByKey(org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.String())) .windowedBy(TimeWindows.of(ofMillis(10)).advanceBy(ofMillis(5)).until(100)) .aggregate( () -> "", diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduceTest.java index 913710fd460..790c563fa0a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduceTest.java @@ -26,7 +26,6 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.kstream.Consumed; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.streams.test.ConsumerRecordFactory; @@ -52,13 +51,14 @@ public class KStreamWindowReduceTest { private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String()); private final ConsumerRecordFactory recordFactory = new ConsumerRecordFactory<>(new StringSerializer(), new StringSerializer()); + @SuppressWarnings("deprecation") @Test public void shouldLogAndMeterOnNullKey() { final StreamsBuilder builder = new StreamsBuilder(); builder .stream("TOPIC", Consumed.with(Serdes.String(), Serdes.String())) - .groupByKey(Serialized.with(Serdes.String(), Serdes.String())) + .groupByKey(org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.String())) .windowedBy(TimeWindows.of(ofMillis(500L))) .reduce((value1, value2) -> value1 + "+" + value2); @@ -80,7 +80,7 @@ public class KStreamWindowReduceTest { final StreamsBuilder builder = new StreamsBuilder(); builder .stream("TOPIC", Consumed.with(Serdes.String(), Serdes.String())) - .groupByKey(Serialized.with(Serdes.String(), Serdes.String())) + .groupByKey(org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.String())) .windowedBy(TimeWindows.of(ofMillis(5L)).until(100)) .reduce((value1, value2) -> value1 + "+" + value2) .toStream() 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 b75b2392f86..01b2609b2bf 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 @@ -19,7 +19,6 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; @@ -31,11 +30,9 @@ import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.Reducer; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.streams.state.KeyValueStore; -import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockAggregator; import org.apache.kafka.test.MockInitializer; import org.apache.kafka.test.MockMapper; @@ -50,13 +47,16 @@ import java.io.File; import java.util.HashMap; import java.util.Map; +import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; +@SuppressWarnings("deprecation") public class KTableAggregateTest { private final Serde stringSerde = Serdes.String(); private final Consumed consumed = Consumed.with(stringSerde, stringSerde); - private final Serialized stringSerialzied = Serialized.with(stringSerde, stringSerde); + private final org.apache.kafka.streams.kstream.Serialized stringSerialzied = + org.apache.kafka.streams.kstream.Serialized.with(stringSerde, stringSerde); private final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); private File stateDir = null; @@ -64,7 +64,7 @@ public class KTableAggregateTest { @Rule public EmbeddedKafkaCluster cluster = null; @Rule - public final KStreamTestDriver driver = new KStreamTestDriver(); + public final org.apache.kafka.test.KStreamTestDriver driver = new org.apache.kafka.test.KStreamTestDriver(); @Before public void setUp() { @@ -106,7 +106,7 @@ public class KTableAggregateTest { driver.process(topic1, "C", "8"); driver.flushState(); - assertEquals(Utils.mkList( + assertEquals(asList( "A:0+1", "B:0+2", "A:0+1-1+3", @@ -139,7 +139,7 @@ public class KTableAggregateTest { driver.process(topic1, "A", "3"); driver.process(topic1, "A", "4"); driver.flushState(); - assertEquals(Utils.mkList( + assertEquals(asList( "A:0+4"), supplier.theCapturedProcessor().processed); } @@ -192,7 +192,7 @@ public class KTableAggregateTest { driver.process(topic1, "B", "7"); driver.flushState(); - assertEquals(Utils.mkList( + assertEquals(asList( "1:0+1", "1:0+1-1", "1:0+1-1+1", @@ -220,7 +220,7 @@ public class KTableAggregateTest { driver.flushState(); - assertEquals(Utils.mkList( + assertEquals(asList( "green:1", "green:2", "green:1", "blue:1", @@ -281,7 +281,7 @@ public class KTableAggregateTest { driver.flushState(); - assertEquals(Utils.mkList( + assertEquals(asList( "blue:1", "yellow:1", "green:2" @@ -337,7 +337,7 @@ public class KTableAggregateTest { driver.process(input, "12", "C"); driver.flushState(); - assertEquals(Utils.mkList( + assertEquals(asList( "1:1", "1:12", "1:2", @@ -362,7 +362,7 @@ public class KTableAggregateTest { public KeyValue apply(final Long key, final String value) { return new KeyValue<>(value, key); } - }, Serialized.with(Serdes.String(), Serdes.Long())) + }, org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.Long())) .reduce(new Reducer() { @Override public Long apply(final Long value1, final Long value2) { 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 3e143f5807b..e717b0863a2 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 @@ -45,6 +45,7 @@ import java.util.Properties; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +@SuppressWarnings("unchecked") public class KTableFilterTest { private final Consumed consumed = Consumed.with(Serdes.String(), Serdes.Integer()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java index 6e666c94691..5747fed49f6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.Topology; @@ -34,7 +33,6 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.Predicate; import org.apache.kafka.streams.kstream.Produced; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.streams.kstream.ValueMapperWithKey; @@ -61,11 +59,13 @@ import java.lang.reflect.Field; import java.util.List; import java.util.Properties; +import static java.util.Arrays.asList; import static org.easymock.EasyMock.mock; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +@SuppressWarnings("unchecked") public class KTableImplTest { private final Consumed stringConsumed = Consumed.with(Serdes.String(), Serdes.String()); @@ -128,13 +128,14 @@ public class KTableImplTest { } final List> processors = supplier.capturedProcessors(4); - assertEquals(Utils.mkList("A:01", "B:02", "C:03", "D:04"), processors.get(0).processed); - assertEquals(Utils.mkList("A:1", "B:2", "C:3", "D:4"), processors.get(1).processed); - assertEquals(Utils.mkList("A:null", "B:2", "C:null", "D:4"), processors.get(2).processed); - assertEquals(Utils.mkList("A:01", "B:02", "C:03", "D:04"), processors.get(3).processed); + assertEquals(asList("A:01", "B:02", "C:03", "D:04"), processors.get(0).processed); + assertEquals(asList("A:1", "B:2", "C:3", "D:4"), processors.get(1).processed); + assertEquals(asList("A:null", "B:2", "C:null", "D:4"), processors.get(2).processed); + assertEquals(asList("A:01", "B:02", "C:03", "D:04"), processors.get(3).processed); } @Test + @SuppressWarnings("deprecation") public void shouldPreserveSerdesForOperators() { final StreamsBuilder builder = new StreamsBuilder(); final KTable table1 = builder.table("topic-2", stringConsumed); @@ -183,8 +184,8 @@ public class KTableImplTest { assertEquals(((AbstractStream) table1.groupBy(KeyValue::new)).keySerde(), null); assertEquals(((AbstractStream) table1.groupBy(KeyValue::new)).valueSerde(), null); - assertEquals(((AbstractStream) table1.groupBy(KeyValue::new, Serialized.with(mySerde, mySerde))).keySerde(), mySerde); - assertEquals(((AbstractStream) table1.groupBy(KeyValue::new, Serialized.with(mySerde, mySerde))).valueSerde(), mySerde); + assertEquals(((AbstractStream) table1.groupBy(KeyValue::new, org.apache.kafka.streams.kstream.Serialized.with(mySerde, mySerde))).keySerde(), mySerde); + assertEquals(((AbstractStream) table1.groupBy(KeyValue::new, org.apache.kafka.streams.kstream.Serialized.with(mySerde, mySerde))).valueSerde(), mySerde); assertEquals(((AbstractStream) table1.join(table1, joiner)).keySerde(), consumedInternal.keySerde()); assertEquals(((AbstractStream) table1.join(table1, joiner)).valueSerde(), null); 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 cd29b50f668..5ecbe83be78 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 @@ -29,7 +29,6 @@ import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.processor.MockProcessorContext; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; -import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; @@ -63,8 +62,10 @@ public class KTableKTableInnerJoinTest { private final Materialized> materialized = Materialized.with(intSerde, stringSerde); private File stateDir = null; + + @SuppressWarnings("deprecation") @Rule - public final KStreamTestDriver driver = new KStreamTestDriver(); + public final org.apache.kafka.test.KStreamTestDriver driver = new org.apache.kafka.test.KStreamTestDriver(); @Before public void setUp() { @@ -80,6 +81,7 @@ public class KTableKTableInnerJoinTest { assertEquals(1, copartitionGroups.size()); assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); + @SuppressWarnings("unchecked") final KTableValueGetterSupplier getterSupplier = ((KTableImpl) joined).valueGetterSupplier(); driver.setUp(builder, stateDir, Serdes.Integer(), Serdes.String()); @@ -354,6 +356,7 @@ public class KTableKTableInnerJoinTest { public void shouldLogAndMeterSkippedRecordsDueToNullLeftKey() { final StreamsBuilder builder = new StreamsBuilder(); + @SuppressWarnings("unchecked") final Processor> join = new KTableKTableInnerJoin<>( (KTableImpl) builder.table("left", Consumed.with(stringSerde, stringSerde)), (KTableImpl) builder.table("right", Consumed.with(stringSerde, stringSerde)), 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 9be61899946..1cd360a4b4f 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 @@ -26,13 +26,11 @@ import org.apache.kafka.streams.TopologyWrapper; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.streams.processor.MockProcessorContext; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.streams.state.KeyValueStore; -import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockReducer; @@ -66,8 +64,10 @@ public class KTableKTableLeftJoinTest { final private Serde intSerde = Serdes.Integer(); final private Serde stringSerde = Serdes.String(); private File stateDir = null; + + @SuppressWarnings("deprecation") @Rule - public final KStreamTestDriver driver = new KStreamTestDriver(); + public final org.apache.kafka.test.KStreamTestDriver driver = new org.apache.kafka.test.KStreamTestDriver(); private final Consumed consumed = Consumed.with(intSerde, stringSerde); @Before @@ -92,6 +92,7 @@ public class KTableKTableLeftJoinTest { assertEquals(1, copartitionGroups.size()); assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); + @SuppressWarnings("unchecked") final KTableValueGetterSupplier getterSupplier = ((KTableImpl) joined).valueGetterSupplier(); driver.setUp(builder, stateDir); @@ -343,6 +344,7 @@ public class KTableKTableLeftJoinTest { * It is based on a fairly complicated join used by the developer that reported the bug. * Before the fix this would trigger an IllegalStateException. */ + @SuppressWarnings("deprecation") @Test public void shouldNotThrowIllegalStateExceptionWhenMultiCacheEvictions() { final String agg = "agg"; @@ -365,7 +367,7 @@ public class KTableKTableLeftJoinTest { return new KeyValue<>(key, value); } }, - Serialized.with(Serdes.Long(), Serdes.String()) + org.apache.kafka.streams.kstream.Serialized.with(Serdes.Long(), Serdes.String()) ) .reduce(MockReducer.STRING_ADDER, MockReducer.STRING_ADDER, Materialized.>as("agg-store")); @@ -417,6 +419,7 @@ public class KTableKTableLeftJoinTest { public void shouldLogAndMeterSkippedRecordsDueToNullLeftKey() { final StreamsBuilder builder = new StreamsBuilder(); + @SuppressWarnings("unchecked") final Processor> join = new KTableKTableLeftJoin<>( (KTableImpl) builder.table("left", Consumed.with(stringSerde, stringSerde)), (KTableImpl) builder.table("right", Consumed.with(stringSerde, stringSerde)), 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 3995fcf8f93..1500ba37e76 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 @@ -26,7 +26,6 @@ import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.processor.MockProcessorContext; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; -import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; @@ -57,8 +56,10 @@ public class KTableKTableOuterJoinTest { final private Serde intSerde = Serdes.Integer(); final private Serde stringSerde = Serdes.String(); private File stateDir = null; + + @SuppressWarnings("deprecation") @Rule - public final KStreamTestDriver driver = new KStreamTestDriver(); + public final org.apache.kafka.test.KStreamTestDriver driver = new org.apache.kafka.test.KStreamTestDriver(); private final Consumed consumed = Consumed.with(intSerde, stringSerde); @Before @@ -88,6 +89,7 @@ public class KTableKTableOuterJoinTest { assertEquals(1, copartitionGroups.size()); assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); + @SuppressWarnings("unchecked") final KTableValueGetterSupplier getterSupplier = ((KTableImpl) joined).valueGetterSupplier(); driver.setUp(builder, stateDir); @@ -359,6 +361,7 @@ public class KTableKTableOuterJoinTest { public void shouldLogAndMeterSkippedRecordsDueToNullLeftKey() { final StreamsBuilder builder = new StreamsBuilder(); + @SuppressWarnings("unchecked") final Processor> join = new KTableKTableOuterJoin<>( (KTableImpl) builder.table("left", Consumed.with(stringSerde, stringSerde)), (KTableImpl) builder.table("right", Consumed.with(stringSerde, stringSerde)), diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoinTest.java index d4805a20897..8f234b6f95e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoinTest.java @@ -34,6 +34,7 @@ public class KTableKTableRightJoinTest { public void shouldLogAndMeterSkippedRecordsDueToNullLeftKey() { final StreamsBuilder builder = new StreamsBuilder(); + @SuppressWarnings("unchecked") final Processor> join = new KTableKTableRightJoin<>( (KTableImpl) builder.table("left", Consumed.with(Serdes.String(), Serdes.String())), (KTableImpl) builder.table("right", Consumed.with(Serdes.String(), Serdes.String())), 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 0f56043be50..21ef0389707 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 @@ -19,7 +19,6 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.TopologyTestDriver; @@ -41,11 +40,13 @@ import org.junit.Test; import java.util.Properties; +import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +@SuppressWarnings("unchecked") public class KTableMapValuesTest { private final Consumed consumed = Consumed.with(Serdes.String(), Serdes.String()); @@ -59,7 +60,7 @@ public class KTableMapValuesTest { driver.pipeInput(recordFactory.create(topic1, "B", "2")); driver.pipeInput(recordFactory.create(topic1, "C", "3")); driver.pipeInput(recordFactory.create(topic1, "D", "4")); - assertEquals(Utils.mkList("A:1", "B:2", "C:3", "D:4"), supplier.theCapturedProcessor().processed); + assertEquals(asList("A:1", "B:2", "C:3", "D:4"), supplier.theCapturedProcessor().processed); } } 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 2055f9cf319..c9db537080a 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 @@ -19,7 +19,6 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.TopologyTestDriver; @@ -37,6 +36,7 @@ import org.junit.Test; import java.util.Properties; +import static java.util.Arrays.asList; import static org.apache.kafka.test.StreamsTestUtils.getMetricByName; import static org.hamcrest.CoreMatchers.hasItem; import static org.junit.Assert.assertEquals; @@ -71,7 +71,7 @@ public class KTableSourceTest { driver.pipeInput(integerFactory.create(topic1, "B", null)); } - assertEquals(Utils.mkList("A:1", "B:2", "C:3", "D:4", "A:null", "B:null"), supplier.theCapturedProcessor().processed); + assertEquals(asList("A:1", "B:2", "C:3", "D:4", "A:null", "B:null"), supplier.theCapturedProcessor().processed); } @Test @@ -96,6 +96,7 @@ public class KTableSourceTest { final String topic1 = "topic1"; + @SuppressWarnings("unchecked") final KTableImpl table1 = (KTableImpl) builder.table(topic1, stringConsumed); final Topology topology = builder.build(); @@ -146,6 +147,7 @@ public class KTableSourceTest { final String topic1 = "topic1"; + @SuppressWarnings("unchecked") final KTableImpl table1 = (KTableImpl) builder.table(topic1, stringConsumed); final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); @@ -184,6 +186,7 @@ public class KTableSourceTest { final String topic1 = "topic1"; + @SuppressWarnings("unchecked") final KTableImpl table1 = (KTableImpl) builder.table(topic1, stringConsumed); table1.enableSendingOldValues(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java index dbcce90bf25..82ada52ce37 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java @@ -26,7 +26,6 @@ import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.streams.kstream.ValueTransformerWithKey; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; @@ -136,6 +135,7 @@ public class KTableTransformValuesTest { } } + @SuppressWarnings("unchecked") @Test public void shouldInitializeTransformerWithForwardDisabledProcessorContext() { final SingletonNoOpValueTransformer transformer = new SingletonNoOpValueTransformer<>(); @@ -359,6 +359,7 @@ public class KTableTransformValuesTest { assertThat(keyValueStore.get("C"), is("C->null!")); } + @SuppressWarnings("deprecation") @Test public void shouldCalculateCorrectOldValuesIfMaterializedEvenIfStateful() { builder @@ -368,7 +369,7 @@ public class KTableTransformValuesTest { Materialized.>as(QUERYABLE_NAME) .withKeySerde(Serdes.String()) .withValueSerde(Serdes.Integer())) - .groupBy(toForceSendingOfOldValues(), Serialized.with(Serdes.String(), Serdes.Integer())) + .groupBy(toForceSendingOfOldValues(), org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.Integer())) .reduce(MockReducer.INTEGER_ADDER, MockReducer.INTEGER_SUBTRACTOR) .mapValues(mapBackToStrings()) .toStream() @@ -386,12 +387,13 @@ public class KTableTransformValuesTest { assertThat(keyValueStore.get("A"), is(3)); } + @SuppressWarnings("deprecation") @Test public void shouldCalculateCorrectOldValuesIfNotStatefulEvenIfNotMaterialized() { builder .table(INPUT_TOPIC, CONSUMED) .transformValues(new StatelessTransformerSupplier()) - .groupBy(toForceSendingOfOldValues(), Serialized.with(Serdes.String(), Serdes.Integer())) + .groupBy(toForceSendingOfOldValues(), org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.Integer())) .reduce(MockReducer.INTEGER_ADDER, MockReducer.INTEGER_SUBTRACTOR) .mapValues(mapBackToStrings()) .toStream() @@ -544,4 +546,4 @@ public class KTableTransformValuesTest { public void close() { } } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java index 9ee918c1c4b..749d5d67c94 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java @@ -28,7 +28,6 @@ import org.apache.kafka.streams.kstream.ForeachAction; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.Merger; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.SessionWindowedKStream; import org.apache.kafka.streams.kstream.SessionWindows; import org.apache.kafka.streams.kstream.Windowed; @@ -66,10 +65,11 @@ public class SessionWindowedKStreamImplTest { }; private SessionWindowedKStream stream; + @SuppressWarnings("deprecation") @Before public void before() { final KStream stream = builder.stream(TOPIC, Consumed.with(Serdes.String(), Serdes.String())); - this.stream = stream.groupByKey(Serialized.with(Serdes.String(), Serdes.String())) + this.stream = stream.groupByKey(org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.String())) .windowedBy(SessionWindows.with(ofMillis(500))); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressScenarioTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressScenarioTest.java index f23bf0b83d0..57b860e4186 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressScenarioTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SuppressScenarioTest.java @@ -352,6 +352,7 @@ public class SuppressScenarioTest { } } + @SuppressWarnings("deprecation") @Test public void shouldSupportFinalResultsForTimeWindows() { final StreamsBuilder builder = new StreamsBuilder(); @@ -402,6 +403,7 @@ public class SuppressScenarioTest { } } + @SuppressWarnings("deprecation") @Test public void shouldSupportFinalResultsForTimeWindowsWithLargeJump() { final StreamsBuilder builder = new StreamsBuilder(); @@ -457,6 +459,7 @@ public class SuppressScenarioTest { } } + @SuppressWarnings("deprecation") @Test public void shouldSupportFinalResultsForSessionWindows() { final StreamsBuilder builder = new StreamsBuilder(); 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 f951ebb4155..c67b9e67151 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 @@ -27,7 +27,6 @@ import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.ForeachAction; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.TimeWindowedKStream; import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.Windowed; @@ -59,10 +58,11 @@ public class TimeWindowedKStreamImplTest { private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String()); private TimeWindowedKStream windowedStream; + @SuppressWarnings("deprecation") @Before public void before() { final KStream stream = builder.stream(TOPIC, Consumed.with(Serdes.String(), Serdes.String())); - windowedStream = stream.groupByKey(Serialized.with(Serdes.String(), Serdes.String())) + windowedStream = stream.groupByKey(org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.String())) .windowedBy(TimeWindows.of(ofMillis(500L))); } diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/YahooBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/YahooBenchmark.java index 1109f9d9be3..de1a621eeee 100644 --- a/streams/src/test/java/org/apache/kafka/streams/perf/YahooBenchmark.java +++ b/streams/src/test/java/org/apache/kafka/streams/perf/YahooBenchmark.java @@ -39,7 +39,6 @@ import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.Predicate; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.streams.state.WindowStore; @@ -252,7 +251,7 @@ public class YahooBenchmark { } } - + @SuppressWarnings("deprecation") private KafkaStreams createYahooBenchmarkStreams(final Properties streamConfig, final String campaignsTopic, final String eventsTopic, final CountDownLatch latch, final int numRecords) { final Map serdeProps = new HashMap<>(); @@ -334,7 +333,7 @@ public class YahooBenchmark { // calculate windowed counts keyedByCampaign - .groupByKey(Serialized.with(Serdes.String(), Serdes.String())) + .groupByKey(org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.String())) .windowedBy(TimeWindows.of(Duration.ofMillis(10 * 1000))) .count(Materialized.>as("time-windows")); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestampTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestampTest.java index d46f3d7a544..5781e676728 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestampTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestampTest.java @@ -47,7 +47,7 @@ public class UsePreviousTimeOnInvalidTimestampTest extends TimestampExtractorTes @Test public void shouldThrowStreamsException() { final TimestampExtractor extractor = new UsePreviousTimeOnInvalidTimestamp(); - final ConsumerRecord record = new ConsumerRecord<>("anyTopic", 0, 0, null, null); + final ConsumerRecord record = new ConsumerRecord<>("anyTopic", 0, 0, null, null); try { extractor.extract(record, -1); fail("should have thrown StreamsException"); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java index 4ce9a9f4f17..ee79e40f820 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContextTest.java @@ -198,6 +198,7 @@ public class AbstractProcessorContextTest { return null; } + @SuppressWarnings("deprecation") @Override public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) { return null; @@ -216,9 +217,11 @@ public class AbstractProcessorContextTest { @Override public void forward(final K key, final V value, final To to) {} + @SuppressWarnings("deprecation") @Override public void forward(final K key, final V value, final int childIndex) {} + @SuppressWarnings("deprecation") @Override public void forward(final K key, final V value, final String childName) {} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java index 32ac23e39d6..8789e2ba29a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java @@ -32,7 +32,6 @@ import org.apache.kafka.streams.errors.LockException; import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.StateRestoreCallback; -import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.internals.OffsetCheckpoint; import org.apache.kafka.test.InternalMockProcessorContext; import org.apache.kafka.test.MockStateRestoreListener; @@ -56,6 +55,7 @@ import java.util.Properties; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import static java.util.Arrays.asList; import static org.apache.kafka.test.MockStateRestoreListener.RESTORE_BATCH; import static org.apache.kafka.test.MockStateRestoreListener.RESTORE_END; import static org.apache.kafka.test.MockStateRestoreListener.RESTORE_START; @@ -103,7 +103,7 @@ public class GlobalStateManagerImplTest { store3 = new NoOpReadOnlyStore<>(storeName3); store4 = new NoOpReadOnlyStore<>(storeName4); - topology = ProcessorTopology.withGlobalStores(Utils.mkList(store1, store2, store3, store4), storeToTopic); + topology = ProcessorTopology.withGlobalStores(asList(store1, store2, store3, store4), storeToTopic); streamsConfig = new StreamsConfig(new Properties() { { @@ -649,7 +649,7 @@ public class GlobalStateManagerImplTest { assertTrue(testFile4.exists()); // only delete and recreate store 1 and 3 -- 2 and 4 must be untouched - stateManager.reinitializeStateStoresForPartitions(Utils.mkList(t1, t3), processorContext); + stateManager.reinitializeStateStoresForPartitions(asList(t1, t3), processorContext); assertFalse(testFile1.exists()); assertTrue(testFile2.exists()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java index f3e369fb5fc..53d374b862b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java @@ -42,6 +42,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Set; +import static java.util.Arrays.asList; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; @@ -84,7 +85,7 @@ public class GlobalStateTaskTest { storeToTopic.put("t1-store", topic1); storeToTopic.put("t2-store", topic2); topology = ProcessorTopology.with( - Utils.mkList(sourceOne, sourceTwo, processorOne, processorTwo), + asList(sourceOne, sourceTwo, processorOne, processorTwo), sourceByTopics, Collections.emptyList(), storeToTopic); @@ -225,4 +226,4 @@ public class GlobalStateTaskTest { assertThat(stateMgr.checkpointed(), equalTo(expectedOffsets)); } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java index d49dd9d0878..1be5231f59a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java @@ -48,7 +48,7 @@ import java.util.Set; import java.util.regex.Pattern; import static java.time.Duration.ofSeconds; -import static org.apache.kafka.common.utils.Utils.mkList; +import static java.util.Arrays.asList; import static org.apache.kafka.common.utils.Utils.mkSet; import static org.hamcrest.core.IsInstanceOf.instanceOf; @@ -170,7 +170,7 @@ public class InternalTopologyBuilderTest { @Test(expected = NullPointerException.class) public void testAddProcessorWithNullParents() { - builder.addProcessor("processor", new MockProcessorSupplier(), null); + builder.addProcessor("processor", new MockProcessorSupplier(), (String) null); } @Test @@ -201,7 +201,7 @@ public class InternalTopologyBuilderTest { @Test(expected = NullPointerException.class) public void testAddSinkWithNullParents() { - builder.addSink("sink", "topic", null, null, null, null); + builder.addSink("sink", "topic", null, null, null, (String) null); } @Test @@ -346,7 +346,7 @@ public class InternalTopologyBuilderTest { builder.addProcessor("processor-1", new MockProcessorSupplier(), "source-1"); builder.addProcessor("processor-2", new MockProcessorSupplier(), "source-2", "processor-1"); - builder.copartitionSources(mkList("source-1", "source-2")); + builder.copartitionSources(asList("source-1", "source-2")); builder.addProcessor("processor-3", new MockProcessorSupplier(), "source-3", "source-4"); @@ -494,7 +494,7 @@ public class InternalTopologyBuilderTest { @Test(expected = NullPointerException.class) public void shouldNotAllowNullTopicChooserWhenAddingSink() { - builder.addSink("name", (TopicNameExtractor) null, null, null, null); + builder.addSink("name", (TopicNameExtractor) null, null, null, null); } @Test(expected = NullPointerException.class) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java index c4e58be7c1f..e63751899f2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java @@ -214,7 +214,7 @@ public class RecordCollectorTest { final Metrics metrics = new Metrics(); final Sensor sensor = metrics.sensor("skipped-records"); final LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(); - final MetricName metricName = new MetricName("name", "group", "description", Collections.EMPTY_MAP); + final MetricName metricName = new MetricName("name", "group", "description", Collections.emptyMap()); sensor.add(metricName, new Sum()); final RecordCollector collector = new RecordCollectorImpl( "test", @@ -335,7 +335,7 @@ public class RecordCollectorTest { collector.init(new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) { @Override public List partitionsFor(final String topic) { - return Collections.EMPTY_LIST; + return Collections.emptyList(); } }); @@ -353,7 +353,7 @@ public class RecordCollectorTest { collector.init(new MockProducer(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer) { @Override public List partitionsFor(final String topic) { - return Collections.EMPTY_LIST; + return Collections.emptyList(); } }); @@ -364,7 +364,7 @@ public class RecordCollectorTest { public void testRecordHeaderPassThroughSerializer() { final CustomStringSerializer keySerializer = new CustomStringSerializer(); final CustomStringSerializer valueSerializer = new CustomStringSerializer(); - keySerializer.configure(Collections.EMPTY_MAP, true); + keySerializer.configure(Collections.emptyMap(), true); final RecordCollectorImpl collector = new RecordCollectorImpl( "test", diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java index 269983f6380..f209c7a5208 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SinkNodeTest.java @@ -48,7 +48,12 @@ public class SinkNodeTest { anyStateSerde, recordCollector ); - private final SinkNode sink = new SinkNode<>("anyNodeName", new StaticTopicNameExtractor("any-output-topic"), anySerializer, anySerializer, null); + private final SinkNode sink = new SinkNode<>("anyNodeName", + new StaticTopicNameExtractor<>("any-output-topic"), anySerializer, anySerializer, null); + + // Used to verify that the correct exceptions are thrown if the compiler checks are bypassed + @SuppressWarnings("unchecked") + private final SinkNode illTypedSink = (SinkNode) sink; @Before public void before() { @@ -57,7 +62,6 @@ public class SinkNodeTest { } @Test - @SuppressWarnings("unchecked") public void shouldThrowStreamsExceptionOnInputRecordWithInvalidTimestamp() { final Bytes anyKey = new Bytes("any key".getBytes()); final Bytes anyValue = new Bytes("any value".getBytes()); @@ -65,7 +69,7 @@ public class SinkNodeTest { // When/Then context.setTime(-1); // ensures a negative timestamp is set for the record we send next try { - sink.process(anyKey, anyValue); + illTypedSink.process(anyKey, anyValue); fail("Should have thrown StreamsException"); } catch (final StreamsException ignored) { // expected @@ -73,7 +77,6 @@ public class SinkNodeTest { } @Test - @SuppressWarnings("unchecked") public void shouldThrowStreamsExceptionOnKeyValueTypeSerializerMismatch() { final String keyOfDifferentTypeThanSerializer = "key with different type"; final String valueOfDifferentTypeThanSerializer = "value with different type"; @@ -81,7 +84,7 @@ public class SinkNodeTest { // When/Then context.setTime(0); try { - sink.process(keyOfDifferentTypeThanSerializer, valueOfDifferentTypeThanSerializer); + illTypedSink.process(keyOfDifferentTypeThanSerializer, valueOfDifferentTypeThanSerializer); fail("Should have thrown StreamsException"); } catch (final StreamsException e) { assertThat(e.getCause(), instanceOf(ClassCastException.class)); @@ -89,14 +92,13 @@ public class SinkNodeTest { } @Test - @SuppressWarnings("unchecked") public void shouldHandleNullKeysWhenThrowingStreamsExceptionOnKeyValueTypeSerializerMismatch() { final String invalidValueToTriggerSerializerMismatch = ""; // When/Then context.setTime(1); try { - sink.process(null, invalidValueToTriggerSerializerMismatch); + illTypedSink.process(null, invalidValueToTriggerSerializerMismatch); fail("Should have thrown StreamsException"); } catch (final StreamsException e) { assertThat(e.getCause(), instanceOf(ClassCastException.class)); @@ -105,14 +107,13 @@ public class SinkNodeTest { } @Test - @SuppressWarnings("unchecked") public void shouldHandleNullValuesWhenThrowingStreamsExceptionOnKeyValueTypeSerializerMismatch() { final String invalidKeyToTriggerSerializerMismatch = ""; // When/Then context.setTime(1); try { - sink.process(invalidKeyToTriggerSerializerMismatch, null); + illTypedSink.process(invalidKeyToTriggerSerializerMismatch, null); fail("Should have thrown StreamsException"); } catch (final StreamsException e) { assertThat(e.getCause(), instanceOf(ClassCastException.class)); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeTest.java index 4c7c3a6fa14..452dd7b71d8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeTest.java @@ -18,7 +18,7 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeaders; -import org.apache.kafka.common.serialization.ExtendedDeserializer; +import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.test.MockSourceNode; import org.junit.Test; @@ -31,7 +31,7 @@ import static org.hamcrest.MatcherAssert.assertThat; public class SourceNodeTest { @Test public void shouldProvideTopicHeadersAndDataToKeyDeserializer() { - final SourceNode sourceNode = new MockSourceNode<>(new String[]{""}, new TheExtendedDeserializer(), new TheExtendedDeserializer()); + final SourceNode sourceNode = new MockSourceNode<>(new String[]{""}, new TheDeserializer(), new TheDeserializer()); final RecordHeaders headers = new RecordHeaders(); final String deserializeKey = sourceNode.deserializeKey("topic", headers, "data".getBytes(StandardCharsets.UTF_8)); assertThat(deserializeKey, is("topic" + headers + "data")); @@ -39,13 +39,13 @@ public class SourceNodeTest { @Test public void shouldProvideTopicHeadersAndDataToValueDeserializer() { - final SourceNode sourceNode = new MockSourceNode<>(new String[]{""}, new TheExtendedDeserializer(), new TheExtendedDeserializer()); + final SourceNode sourceNode = new MockSourceNode<>(new String[]{""}, new TheDeserializer(), new TheDeserializer()); final RecordHeaders headers = new RecordHeaders(); final String deserializedValue = sourceNode.deserializeValue("topic", headers, "data".getBytes(StandardCharsets.UTF_8)); assertThat(deserializedValue, is("topic" + headers + "data")); } - public static class TheExtendedDeserializer implements ExtendedDeserializer { + public static class TheDeserializer implements Deserializer { @Override public String deserialize(final String topic, final Headers headers, final byte[] data) { return topic + headers + new String(data, StandardCharsets.UTF_8); @@ -62,4 +62,4 @@ public class SourceNodeTest { @Override public void close() { } } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index 639ebf861f1..0c24e2d637f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -63,7 +63,6 @@ import java.io.File; import java.io.IOException; import java.time.Duration; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -73,11 +72,11 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import static java.time.Duration.ofMillis; +import static java.util.Arrays.asList; import static java.util.Collections.emptyList; import static java.util.Collections.emptySet; import static java.util.Collections.singletonList; import static org.apache.kafka.common.utils.Utils.mkEntry; -import static org.apache.kafka.common.utils.Utils.mkList; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkProperties; import static org.hamcrest.CoreMatchers.containsString; @@ -106,7 +105,7 @@ public class StandbyTaskTest { private final Set topicPartitions = Collections.emptySet(); private final ProcessorTopology topology = ProcessorTopology.withLocalStores( - mkList(new MockKeyValueStoreBuilder(storeName1, false).build(), new MockKeyValueStoreBuilder(storeName2, true).build()), + asList(new MockKeyValueStoreBuilder(storeName1, false).build(), new MockKeyValueStoreBuilder(storeName2, true).build()), mkMap( mkEntry(storeName1, storeChangelogTopicName1), mkEntry(storeName2, storeChangelogTopicName2) @@ -149,13 +148,13 @@ public class StandbyTaskTest { @Before public void setup() throws Exception { restoreStateConsumer.reset(); - restoreStateConsumer.updatePartitions(storeChangelogTopicName1, mkList( + restoreStateConsumer.updatePartitions(storeChangelogTopicName1, asList( new PartitionInfo(storeChangelogTopicName1, 0, Node.noNode(), new Node[0], new Node[0]), new PartitionInfo(storeChangelogTopicName1, 1, Node.noNode(), new Node[0], new Node[0]), new PartitionInfo(storeChangelogTopicName1, 2, Node.noNode(), new Node[0], new Node[0]) )); - restoreStateConsumer.updatePartitions(storeChangelogTopicName2, mkList( + restoreStateConsumer.updatePartitions(storeChangelogTopicName2, asList( new PartitionInfo(storeChangelogTopicName2, 0, Node.noNode(), new Node[0], new Node[0]), new PartitionInfo(storeChangelogTopicName2, 1, Node.noNode(), new Node[0], new Node[0]), new PartitionInfo(storeChangelogTopicName2, 2, Node.noNode(), new Node[0], new Node[0]) @@ -204,7 +203,7 @@ public class StandbyTaskTest { final Set partition = Collections.singleton(partition2); restoreStateConsumer.assign(partition); - for (final ConsumerRecord record : Arrays.asList( + for (final ConsumerRecord record : asList( new ConsumerRecord<>(partition2.topic(), partition2.partition(), 10, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, 1, 100), new ConsumerRecord<>(partition2.topic(), partition2.partition(), 20, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, 2, 100), new ConsumerRecord<>(partition2.topic(), partition2.partition(), 30, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, 3, 100))) { @@ -219,7 +218,7 @@ public class StandbyTaskTest { final MockKeyValueStore store2 = (MockKeyValueStore) context.getStateMgr().getStore(storeName2); assertEquals(Collections.emptyList(), store1.keys); - assertEquals(mkList(1, 2, 3), store2.keys); + assertEquals(asList(1, 2, 3), store2.keys); } @Test @@ -229,7 +228,7 @@ public class StandbyTaskTest { final TopicPartition topicPartition = new TopicPartition(changelogName, 1); - final List partitions = mkList(topicPartition); + final List partitions = asList(topicPartition); consumer.assign(partitions); @@ -268,7 +267,7 @@ public class StandbyTaskTest { final List> remaining1 = task.update( topicPartition, - Arrays.asList( + asList( makeWindowedConsumerRecord(changelogName, 10, 1, 0L, 60_000L), makeWindowedConsumerRecord(changelogName, 20, 2, 60_000L, 120_000), makeWindowedConsumerRecord(changelogName, 30, 3, 120_000L, 180_000), @@ -277,7 +276,7 @@ public class StandbyTaskTest { ); assertEquals( - Arrays.asList( + asList( new KeyValue<>(new Windowed<>(1, new TimeWindow(0, 60_000)), 100L), new KeyValue<>(new Windowed<>(2, new TimeWindow(60_000, 120_000)), 100L), new KeyValue<>(new Windowed<>(3, new TimeWindow(120_000, 180_000)), 100L) @@ -293,7 +292,7 @@ public class StandbyTaskTest { // the first record's window should have expired. assertEquals( - Arrays.asList( + asList( new KeyValue<>(new Windowed<>(2, new TimeWindow(60_000, 120_000)), 100L), new KeyValue<>(new Windowed<>(3, new TimeWindow(120_000, 180_000)), 100L), new KeyValue<>(new Windowed<>(4, new TimeWindow(180_000, 240_000)), 100L) @@ -330,7 +329,7 @@ public class StandbyTaskTest { final String changelogName = applicationId + "-" + storeName + "-changelog"; final TopicPartition topicPartition = new TopicPartition(changelogName, 1); - final List partitions = mkList(topicPartition); + final List partitions = asList(topicPartition); final InternalTopologyBuilder internalTopologyBuilder = new InternalTopologyBuilder().setApplicationId(applicationId); @@ -394,7 +393,7 @@ public class StandbyTaskTest { @Test public void shouldRestoreToKTable() throws IOException { - consumer.assign(mkList(globalTopicPartition)); + consumer.assign(asList(globalTopicPartition)); consumer.commitSync(mkMap(mkEntry(globalTopicPartition, new OffsetAndMetadata(0L)))); final StandbyTask task = new StandbyTask( @@ -412,7 +411,7 @@ public class StandbyTaskTest { // The commit offset is at 0L. Records should not be processed List> remaining = task.update( globalTopicPartition, - Arrays.asList( + asList( makeConsumerRecord(globalTopicPartition, 10, 1), makeConsumerRecord(globalTopicPartition, 20, 2), makeConsumerRecord(globalTopicPartition, 30, 3), @@ -514,14 +513,14 @@ public class StandbyTaskTest { @Test public void shouldCheckpointStoreOffsetsOnCommit() throws IOException { - consumer.assign(mkList(globalTopicPartition)); + consumer.assign(asList(globalTopicPartition)); final Map committedOffsets = new HashMap<>(); committedOffsets.put(new TopicPartition(globalTopicPartition.topic(), globalTopicPartition.partition()), new OffsetAndMetadata(100L)); consumer.commitSync(committedOffsets); restoreStateConsumer.updatePartitions( globalStoreName, - mkList(new PartitionInfo(globalStoreName, 0, Node.noNode(), new Node[0], new Node[0])) + asList(new PartitionInfo(globalStoreName, 0, Node.noNode(), new Node[0], new Node[0])) ); final TaskId taskId = new TaskId(0, 0); @@ -561,14 +560,14 @@ public class StandbyTaskTest { @Test public void shouldCloseStateMangerOnTaskCloseWhenCommitFailed() throws Exception { - consumer.assign(mkList(globalTopicPartition)); + consumer.assign(asList(globalTopicPartition)); final Map committedOffsets = new HashMap<>(); committedOffsets.put(new TopicPartition(globalTopicPartition.topic(), globalTopicPartition.partition()), new OffsetAndMetadata(100L)); consumer.commitSync(committedOffsets); restoreStateConsumer.updatePartitions( globalStoreName, - mkList(new PartitionInfo(globalStoreName, 0, Node.noNode(), new Node[0], new Node[0])) + asList(new PartitionInfo(globalStoreName, 0, Node.noNode(), new Node[0], new Node[0])) ); final StreamsConfig config = createConfig(baseDir); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java index ae48f57db31..34f0a32b88c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StoreChangelogReaderTest.java @@ -47,6 +47,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import static java.util.Arrays.asList; import static org.apache.kafka.test.MockStateRestoreListener.RESTORE_BATCH; import static org.apache.kafka.test.MockStateRestoreListener.RESTORE_END; import static org.apache.kafka.test.MockStateRestoreListener.RESTORE_START; @@ -358,7 +359,7 @@ public class StoreChangelogReaderTest { replay(active, task); changelogReader.restore(active); - assertThat(callback.restored, CoreMatchers.equalTo(Utils.mkList(KeyValue.pair(bytes, bytes), KeyValue.pair(bytes, bytes)))); + assertThat(callback.restored, CoreMatchers.equalTo(asList(KeyValue.pair(bytes, bytes), KeyValue.pair(bytes, bytes)))); } @Test @@ -546,4 +547,4 @@ public class StoreChangelogReaderTest { consumer.assign(Collections.singletonList(topicPartition)); } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 7ff7c708395..b042e3c6a54 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -62,12 +62,12 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.time.Duration; -import java.util.Arrays; import java.util.Collections; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import static java.util.Arrays.asList; import static java.util.Collections.singletonList; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; @@ -114,7 +114,7 @@ public class StreamTaskTest { private final Long offset = 543L; private final ProcessorTopology topology = ProcessorTopology.withSources( - Utils.mkList(source1, source2, processorStreamTime, processorSystemTime), + asList(source1, source2, processorStreamTime, processorSystemTime), mkMap(mkEntry(topic1, source1), mkEntry(topic2, source2)) ); @@ -166,7 +166,7 @@ public class StreamTaskTest { @Before public void setup() { - consumer.assign(Arrays.asList(partition1, partition2)); + consumer.assign(asList(partition1, partition2)); stateDirectory = new StateDirectory(createConfig(false), new MockTime()); } @@ -190,13 +190,13 @@ public class StreamTaskTest { public void testProcessOrder() { task = createStatelessTask(createConfig(false)); - task.addRecords(partition1, Arrays.asList( + task.addRecords(partition1, asList( getConsumerRecord(partition1, 10), getConsumerRecord(partition1, 20), getConsumerRecord(partition1, 30) )); - task.addRecords(partition2, Arrays.asList( + task.addRecords(partition2, asList( getConsumerRecord(partition2, 25), getConsumerRecord(partition2, 35), getConsumerRecord(partition2, 45) @@ -261,12 +261,12 @@ public class StreamTaskTest { public void testPauseResume() { task = createStatelessTask(createConfig(false)); - task.addRecords(partition1, Arrays.asList( + task.addRecords(partition1, asList( getConsumerRecord(partition1, 10), getConsumerRecord(partition1, 20) )); - task.addRecords(partition2, Arrays.asList( + task.addRecords(partition2, asList( getConsumerRecord(partition2, 35), getConsumerRecord(partition2, 45), getConsumerRecord(partition2, 55), @@ -280,7 +280,7 @@ public class StreamTaskTest { assertEquals(1, consumer.paused().size()); assertTrue(consumer.paused().contains(partition2)); - task.addRecords(partition1, Arrays.asList( + task.addRecords(partition1, asList( getConsumerRecord(partition1, 30), getConsumerRecord(partition1, 40), getConsumerRecord(partition1, 50) @@ -318,14 +318,14 @@ public class StreamTaskTest { task.initializeStateStores(); task.initializeTopology(); - task.addRecords(partition1, Arrays.asList( + task.addRecords(partition1, asList( getConsumerRecord(partition1, 20), getConsumerRecord(partition1, 142), getConsumerRecord(partition1, 155), getConsumerRecord(partition1, 160) )); - task.addRecords(partition2, Arrays.asList( + task.addRecords(partition2, asList( getConsumerRecord(partition2, 25), getConsumerRecord(partition2, 145), getConsumerRecord(partition2, 159), @@ -403,13 +403,13 @@ public class StreamTaskTest { task.initializeStateStores(); task.initializeTopology(); - task.addRecords(partition1, Arrays.asList( + task.addRecords(partition1, asList( getConsumerRecord(partition1, 20), getConsumerRecord(partition1, 30), getConsumerRecord(partition1, 40) )); - task.addRecords(partition2, Arrays.asList( + task.addRecords(partition2, asList( getConsumerRecord(partition2, 25), getConsumerRecord(partition2, 35), getConsumerRecord(partition2, 45) @@ -1215,11 +1215,11 @@ public class StreamTaskTest { final TopicPartition repartition = new TopicPartition("repartition", 1); final ProcessorTopology topology = ProcessorTopology.withRepartitionTopics( - Utils.mkList(source1, source2), + asList(source1, source2), mkMap(mkEntry(topic1, source1), mkEntry(repartition.topic(), source2)), Collections.singleton(repartition.topic()) ); - consumer.assign(Arrays.asList(partition1, repartition)); + consumer.assign(asList(partition1, repartition)); task = new StreamTask( taskId00, @@ -1287,7 +1287,7 @@ public class StreamTaskTest { private StreamTask createStatefulTask(final StreamsConfig config, final boolean logged) { final ProcessorTopology topology = ProcessorTopology.with( - Utils.mkList(source1, source2), + asList(source1, source2), mkMap(mkEntry(topic1, source1), mkEntry(topic2, source2)), singletonList(stateStore), logged ? Collections.singletonMap(storeName, storeName + "-changelog") : Collections.emptyMap()); @@ -1309,7 +1309,7 @@ public class StreamTaskTest { private StreamTask createStatefulTaskThatThrowsExceptionOnClose() { final ProcessorTopology topology = ProcessorTopology.with( - Utils.mkList(source1, source3), + asList(source1, source3), mkMap(mkEntry(topic1, source1), mkEntry(topic2, source3)), singletonList(stateStore), Collections.emptyMap()); @@ -1331,7 +1331,7 @@ public class StreamTaskTest { private StreamTask createStatelessTask(final StreamsConfig streamsConfig) { final ProcessorTopology topology = ProcessorTopology.withSources( - Utils.mkList(source1, source2, processorStreamTime, processorSystemTime), + asList(source1, source2, processorStreamTime, processorSystemTime), mkMap(mkEntry(topic1, source1), mkEntry(topic2, source2)) ); @@ -1358,7 +1358,7 @@ public class StreamTaskTest { // this task will throw exception when processing (on partition2), flushing, suspending and closing private StreamTask createTaskThatThrowsException(final boolean enableEos) { final ProcessorTopology topology = ProcessorTopology.withSources( - Utils.mkList(source1, source3, processorStreamTime, processorSystemTime), + asList(source1, source3, processorStreamTime, processorSystemTime), mkMap(mkEntry(topic1, source1), mkEntry(topic2, source3)) ); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index b4de5ec0057..11474a38663 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -76,7 +76,6 @@ import org.junit.Test; import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -87,6 +86,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; +import static java.util.Arrays.asList; import static java.util.Collections.singletonList; import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; @@ -1130,7 +1130,7 @@ public class StreamThreadTest { final StreamThread thread = createStreamThread(clientId, config, false); final MockConsumer restoreConsumer = clientSupplier.restoreConsumer; restoreConsumer.updatePartitions("stream-thread-test-count-one-changelog", - Utils.mkList( + asList( new PartitionInfo("stream-thread-test-count-one-changelog", 0, null, @@ -1452,7 +1452,7 @@ public class StreamThreadTest { public void adminClientMetricsVerification() { final Node broker1 = new Node(0, "dummyHost-1", 1234); final Node broker2 = new Node(1, "dummyHost-2", 1234); - final List cluster = Arrays.asList(broker1, broker2); + final List cluster = asList(broker1, broker2); final MockAdminClient adminClient = new MockAdminClient(cluster, broker1, null); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java index 5b1da16c3cb..bdc7a65c386 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java @@ -269,7 +269,7 @@ public class StreamsMetadataStateTest { @Test(expected = NullPointerException.class) public void shouldThrowWhenSerializerIsNull() { - metadataState.getMetadataWithKey("table-three", "key", (Serializer) null); + metadataState.getMetadataWithKey("table-three", "key", (Serializer) null); } @Test(expected = NullPointerException.class) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java index 649aa191ee0..7c5abbeee09 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java @@ -50,7 +50,6 @@ import org.junit.Test; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -61,6 +60,7 @@ import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import static java.time.Duration.ofMillis; +import static java.util.Arrays.asList; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.not; import static org.junit.Assert.assertEquals; @@ -68,6 +68,7 @@ import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +@SuppressWarnings("unchecked") public class StreamsPartitionAssignorTest { private final TopicPartition t1p0 = new TopicPartition("topic1", 0); @@ -85,7 +86,7 @@ public class StreamsPartitionAssignorTest { private final Set allTopics = Utils.mkSet("topic1", "topic2"); - private final List infos = Arrays.asList( + private final List infos = asList( new PartitionInfo("topic1", 0, Node.noNode(), new Node[0], new Node[0]), new PartitionInfo("topic1", 1, Node.noNode(), new Node[0], new Node[0]), new PartitionInfo("topic1", 2, Node.noNode(), new Node[0], new Node[0]), @@ -158,12 +159,12 @@ public class StreamsPartitionAssignorTest { final TaskId taskIdC0 = new TaskId(2, 0); final TaskId taskIdC1 = new TaskId(2, 1); - final List expectedSubList1 = Arrays.asList(taskIdA0, taskIdA3, taskIdB2); - final List expectedSubList2 = Arrays.asList(taskIdA1, taskIdB0, taskIdC0); - final List expectedSubList3 = Arrays.asList(taskIdA2, taskIdB1, taskIdC1); - final List> embeddedList = Arrays.asList(expectedSubList1, expectedSubList2, expectedSubList3); + final List expectedSubList1 = asList(taskIdA0, taskIdA3, taskIdB2); + final List expectedSubList2 = asList(taskIdA1, taskIdB0, taskIdC0); + final List expectedSubList3 = asList(taskIdA2, taskIdB1, taskIdC1); + final List> embeddedList = asList(expectedSubList1, expectedSubList2, expectedSubList3); - final List tasks = Arrays.asList(taskIdC0, taskIdC1, taskIdB0, taskIdB1, taskIdB2, taskIdA0, taskIdA1, taskIdA2, taskIdA3); + final List tasks = asList(taskIdC0, taskIdC1, taskIdB0, taskIdB1, taskIdB2, taskIdA0, taskIdA1, taskIdA2, taskIdA3); Collections.shuffle(tasks); final List> interleavedTaskIds = partitionAssignor.interleaveTasksByGroupId(tasks, 3); @@ -190,7 +191,7 @@ public class StreamsPartitionAssignorTest { final PartitionAssignor.Subscription subscription = partitionAssignor.subscription(Utils.mkSet("topic1", "topic2")); Collections.sort(subscription.topics()); - assertEquals(Utils.mkList("topic1", "topic2"), subscription.topics()); + assertEquals(asList("topic1", "topic2"), subscription.topics()); final Set standbyTasks = new HashSet<>(cachedTasks); standbyTasks.removeAll(prevTasks); @@ -204,7 +205,7 @@ public class StreamsPartitionAssignorTest { builder.addSource(null, "source1", null, null, null, "topic1"); builder.addSource(null, "source2", null, null, null, "topic2"); builder.addProcessor("processor", new MockProcessorSupplier(), "source1", "source2"); - final List topics = Utils.mkList("topic1", "topic2"); + final List topics = asList("topic1", "topic2"); final Set allTasks = Utils.mkSet(task0, task1, task2); final Set prevTasks10 = Utils.mkSet(task0); @@ -268,7 +269,7 @@ public class StreamsPartitionAssignorTest { builder.addProcessor("processor", new MockProcessorSupplier(), "source1"); builder.addProcessor("processorII", new MockProcessorSupplier(), "source2"); - final List localInfos = Arrays.asList( + final List localInfos = asList( new PartitionInfo("topic1", 0, Node.noNode(), new Node[0], new Node[0]), new PartitionInfo("topic1", 1, Node.noNode(), new Node[0], new Node[0]), new PartitionInfo("topic1", 2, Node.noNode(), new Node[0], new Node[0]), @@ -286,7 +287,7 @@ public class StreamsPartitionAssignorTest { Collections.emptySet(), Collections.emptySet()); - final List topics = Utils.mkList("topic1", "topic2"); + final List topics = asList("topic1", "topic2"); final TaskId taskIdA0 = new TaskId(0, 0); final TaskId taskIdA1 = new TaskId(0, 1); @@ -320,12 +321,12 @@ public class StreamsPartitionAssignorTest { // the first consumer final AssignmentInfo info10 = AssignmentInfo.decode(assignments.get("consumer10").userData()); - final List expectedInfo10TaskIds = Arrays.asList(taskIdA1, taskIdA3, taskIdB1, taskIdB3); + final List expectedInfo10TaskIds = asList(taskIdA1, taskIdA3, taskIdB1, taskIdB3); assertEquals(expectedInfo10TaskIds, info10.activeTasks()); // the second consumer final AssignmentInfo info11 = AssignmentInfo.decode(assignments.get("consumer11").userData()); - final List expectedInfo11TaskIds = Arrays.asList(taskIdA0, taskIdA2, taskIdB0, taskIdB2); + final List expectedInfo11TaskIds = asList(taskIdA0, taskIdA2, taskIdB0, taskIdB2); assertEquals(expectedInfo11TaskIds, info11.activeTasks()); } @@ -338,7 +339,7 @@ public class StreamsPartitionAssignorTest { builder.addSource(null, "source2", null, null, null, "topic2"); builder.addProcessor("processor2", new MockProcessorSupplier(), "source2"); builder.addStateStore(new MockKeyValueStoreBuilder("store2", false), "processor2"); - final List topics = Utils.mkList("topic1", "topic2"); + final List topics = asList("topic1", "topic2"); final Set allTasks = Utils.mkSet(task0, task1, task2); final UUID uuid1 = UUID.randomUUID(); @@ -369,7 +370,7 @@ public class StreamsPartitionAssignorTest { builder.addSource(null, "source1", null, null, null, "topic1"); builder.addSource(null, "source2", null, null, null, "topic2"); builder.addProcessor("processor", new MockProcessorSupplier(), "source1", "source2"); - final List topics = Utils.mkList("topic1", "topic2"); + final List topics = asList("topic1", "topic2"); final Set allTasks = Utils.mkSet(task0, task1, task2); final Set prevTasks10 = Utils.mkSet(task0); @@ -424,7 +425,7 @@ public class StreamsPartitionAssignorTest { builder.addSource(null, "source2", null, null, null, "topic2"); builder.addSource(null, "source3", null, null, null, "topic3"); builder.addProcessor("processor", new MockProcessorSupplier(), "source1", "source2", "source3"); - final List topics = Utils.mkList("topic1", "topic2", "topic3"); + final List topics = asList("topic1", "topic2", "topic3"); final Set allTasks = Utils.mkSet(task0, task1, task2, task3); // assuming that previous tasks do not have topic3 @@ -481,7 +482,7 @@ public class StreamsPartitionAssignorTest { builder.addStateStore(new MockKeyValueStoreBuilder("store2", false), "processor-2"); builder.addStateStore(new MockKeyValueStoreBuilder("store3", false), "processor-2"); - final List topics = Utils.mkList("topic1", "topic2"); + final List topics = asList("topic1", "topic2"); final TaskId task00 = new TaskId(0, 0); final TaskId task01 = new TaskId(0, 1); @@ -489,7 +490,7 @@ public class StreamsPartitionAssignorTest { final TaskId task10 = new TaskId(1, 0); final TaskId task11 = new TaskId(1, 1); final TaskId task12 = new TaskId(1, 2); - final List tasks = Utils.mkList(task00, task01, task02, task10, task11, task12); + final List tasks = asList(task00, task01, task02, task10, task11, task12); final UUID uuid1 = UUID.randomUUID(); final UUID uuid2 = UUID.randomUUID(); @@ -568,7 +569,7 @@ public class StreamsPartitionAssignorTest { builder.addSource(null, "source1", null, null, null, "topic1"); builder.addSource(null, "source2", null, null, null, "topic2"); builder.addProcessor("processor", new MockProcessorSupplier(), "source1", "source2"); - final List topics = Utils.mkList("topic1", "topic2"); + final List topics = asList("topic1", "topic2"); final Set allTasks = Utils.mkSet(task0, task1, task2); @@ -632,7 +633,7 @@ public class StreamsPartitionAssignorTest { public void testOnAssignment() { configurePartitionAssignor(Collections.emptyMap()); - final List activeTaskList = Utils.mkList(task0, task3); + final List activeTaskList = asList(task0, task3); final Map> activeTasks = new HashMap<>(); final Map> standbyTasks = new HashMap<>(); final Map> hostState = Collections.singletonMap( @@ -644,7 +645,7 @@ public class StreamsPartitionAssignorTest { standbyTasks.put(task2, Utils.mkSet(t3p2)); final AssignmentInfo info = new AssignmentInfo(activeTaskList, standbyTasks, hostState); - final PartitionAssignor.Assignment assignment = new PartitionAssignor.Assignment(Utils.mkList(t3p0, t3p3), info.encode()); + final PartitionAssignor.Assignment assignment = new PartitionAssignor.Assignment(asList(t3p0, t3p3), info.encode()); final Capture capturedCluster = EasyMock.newCapture(); taskManager.setPartitionsByHostState(hostState); @@ -672,7 +673,7 @@ public class StreamsPartitionAssignorTest { builder.addSink("sink1", "topicX", null, null, null, "processor1"); builder.addSource(null, "source2", null, null, null, "topicX"); builder.addProcessor("processor2", new MockProcessorSupplier(), "source2"); - final List topics = Utils.mkList("topic1", applicationId + "-topicX"); + final List topics = asList("topic1", applicationId + "-topicX"); final Set allTasks = Utils.mkSet(task0, task1, task2); final UUID uuid1 = UUID.randomUUID(); @@ -706,7 +707,7 @@ public class StreamsPartitionAssignorTest { builder.addProcessor("processor2", new MockProcessorSupplier(), "source2"); builder.addSink("sink2", "topicZ", null, null, null, "processor2"); builder.addSource(null, "source3", null, null, null, "topicZ"); - final List topics = Utils.mkList("topic1", "test-topicX", "test-topicZ"); + final List topics = asList("topic1", "test-topicX", "test-topicZ"); final Set allTasks = Utils.mkSet(task0, task1, task2); final UUID uuid1 = UUID.randomUUID(); @@ -789,7 +790,7 @@ public class StreamsPartitionAssignorTest { subscriptions.put( client, new PartitionAssignor.Subscription( - Utils.mkList("topic1", "topic3"), + asList("topic1", "topic3"), new SubscriptionInfo(uuid, emptyTasks, emptyTasks, userEndPoint).encode() ) ); @@ -805,7 +806,7 @@ public class StreamsPartitionAssignorTest { // check if all internal topics were created as expected assertThat(mockInternalTopicManager.readyTopics, equalTo(expectedCreatedInternalTopics)); - final List expectedAssignment = Arrays.asList( + final List expectedAssignment = asList( new TopicPartition("topic1", 0), new TopicPartition("topic1", 1), new TopicPartition("topic1", 2), @@ -853,7 +854,7 @@ public class StreamsPartitionAssignorTest { builder.addProcessor("processor", new MockProcessorSupplier(), "source"); builder.addSink("sink", "output", null, null, null, "processor"); - final List topics = Utils.mkList("topic1"); + final List topics = asList("topic1"); final UUID uuid1 = UUID.randomUUID(); @@ -1275,7 +1276,7 @@ public class StreamsPartitionAssignorTest { standbyTaskMap, Collections.>emptyMap() ))); - assertThat(assignment.get("consumer1").partitions(), equalTo(Utils.mkList(t1p0, t1p1))); + assertThat(assignment.get("consumer1").partitions(), equalTo(asList(t1p0, t1p1))); assertThat(AssignmentInfo.decode(assignment.get("future-consumer").userData()), equalTo(new AssignmentInfo())); assertThat(assignment.get("future-consumer").partitions().size(), equalTo(0)); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java index b0e7fce2bbe..00af1002578 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java @@ -40,7 +40,6 @@ import org.junit.runner.RunWith; import java.io.File; import java.io.IOException; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -50,6 +49,7 @@ import java.util.Set; import java.util.UUID; import java.util.regex.Pattern; +import static java.util.Arrays.asList; import static org.easymock.EasyMock.checkOrder; import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.expectLastCall; @@ -153,7 +153,7 @@ public class TaskManagerTest { topologyBuilder, subscriptionUpdates); - taskManager.updateSubscriptionsFromAssignment(Utils.mkList(t1p1, t2p1)); + taskManager.updateSubscriptionsFromAssignment(asList(t1p1, t2p1)); EasyMock.verify(activeTaskCreator, topologyBuilder, @@ -169,7 +169,7 @@ public class TaskManagerTest { topologyBuilder, subscriptionUpdates); - taskManager.updateSubscriptionsFromAssignment(Utils.mkList(t1p1)); + taskManager.updateSubscriptionsFromAssignment(asList(t1p1)); EasyMock.verify(activeTaskCreator, topologyBuilder, @@ -212,7 +212,7 @@ public class TaskManagerTest { @Test public void shouldReturnCachedTaskIdsFromDirectory() throws IOException { - final File[] taskFolders = Utils.mkList(testFolder.newFolder("0_1"), + final File[] taskFolders = asList(testFolder.newFolder("0_1"), testFolder.newFolder("0_2"), testFolder.newFolder("0_3"), testFolder.newFolder("1_1"), @@ -618,7 +618,7 @@ public class TaskManagerTest { @Test public void shouldNotResumeConsumptionUntilAllStoresRestored() { EasyMock.expect(active.allTasksRunning()).andReturn(false); - final Consumer consumer = (Consumer) EasyMock.createStrictMock(Consumer.class); + final Consumer consumer = EasyMock.createStrictMock(Consumer.class); taskManager.setConsumer(consumer); EasyMock.replay(active, consumer); @@ -636,11 +636,11 @@ public class TaskManagerTest { assertTrue(taskManager.assignedActiveTasks().isEmpty()); // assign two active tasks with two partitions each - activeTasks.put(task01, new HashSet<>(Arrays.asList(t1p1, t2p1))); - activeTasks.put(task02, new HashSet<>(Arrays.asList(t1p2, t2p2))); + activeTasks.put(task01, new HashSet<>(asList(t1p1, t2p1))); + activeTasks.put(task02, new HashSet<>(asList(t1p2, t2p2))); // assign one standby task with two partitions - standbyTasks.put(task03, new HashSet<>(Arrays.asList(t1p3, t2p3))); + standbyTasks.put(task03, new HashSet<>(asList(t1p3, t2p3))); taskManager.setAssignmentMetadata(activeTasks, standbyTasks); assertThat(taskManager.assignedActiveTasks(), equalTo(activeTasks)); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/StateSerdesTest.java b/streams/src/test/java/org/apache/kafka/streams/state/StateSerdesTest.java index 88b69974e50..56ff71d74df 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/StateSerdesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/StateSerdesTest.java @@ -24,6 +24,7 @@ import org.junit.Test; import java.nio.ByteBuffer; +@SuppressWarnings("unchecked") public class StateSerdesTest { @Test(expected = NullPointerException.class) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java index 71147319fb3..44fa62fd917 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java @@ -52,7 +52,7 @@ import java.util.UUID; import static java.time.Duration.ofHours; import static java.time.Duration.ofMinutes; import static java.time.Instant.ofEpochMilli; -import static org.apache.kafka.common.utils.Utils.mkList; +import static java.util.Arrays.asList; import static org.apache.kafka.streams.state.internals.ThreadCacheTest.memoryCacheEntrySize; import static org.apache.kafka.test.StreamsTestUtils.toList; import static org.apache.kafka.test.StreamsTestUtils.verifyKeyValueList; @@ -117,6 +117,7 @@ public class CachingWindowStoreTest { private WindowStore store; private int numRecordsProcessed; + @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext processorContext) { this.store = (WindowStore) processorContext.getStateStore("store-name"); @@ -413,7 +414,7 @@ public class CachingWindowStoreTest { cachingStore.put(bytesKey("aa"), bytesValue("0004"), 1); cachingStore.put(bytesKey("a"), bytesValue("0005"), SEGMENT_INTERVAL); - final List> expected = mkList( + final List> expected = asList( KeyValue.pair(0L, bytesValue("0001")), KeyValue.pair(1L, bytesValue("0003")), KeyValue.pair(SEGMENT_INTERVAL, bytesValue("0005")) @@ -431,7 +432,7 @@ public class CachingWindowStoreTest { cachingStore.put(bytesKey("a"), bytesValue("0005"), SEGMENT_INTERVAL); verifyKeyValueList( - mkList( + asList( windowedPair("a", "0001", 0), windowedPair("a", "0003", 1), windowedPair("a", "0005", SEGMENT_INTERVAL) @@ -440,12 +441,12 @@ public class CachingWindowStoreTest { ); verifyKeyValueList( - mkList(windowedPair("aa", "0002", 0), windowedPair("aa", "0004", 1)), + asList(windowedPair("aa", "0002", 0), windowedPair("aa", "0004", 1)), toList(cachingStore.fetch(bytesKey("aa"), bytesKey("aa"), ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))) ); verifyKeyValueList( - mkList( + asList( windowedPair("a", "0001", 0), windowedPair("a", "0003", 1), windowedPair("aa", "0002", 0), diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java index 117fd8ff391..5fdfd46ccd3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java @@ -45,7 +45,7 @@ public class ChangeLoggingKeyValueBytesStoreTest { private InternalMockProcessorContext context; private final InMemoryKeyValueStore inner = new InMemoryKeyValueStore<>("kv", Serdes.Bytes(), Serdes.ByteArray()); private final ChangeLoggingKeyValueBytesStore store = new ChangeLoggingKeyValueBytesStore(inner); - private final Map sent = new HashMap<>(); + private final Map sent = new HashMap<>(); private final Bytes hi = Bytes.wrap("hi".getBytes()); private final Bytes hello = Bytes.wrap("hello".getBytes()); private final byte[] there = "there".getBytes(); 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 6d2d994c1c6..a36b101c3c6 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 @@ -43,7 +43,7 @@ import static org.junit.Assert.assertArrayEquals; public class ChangeLoggingWindowBytesStoreTest { private final TaskId taskId = new TaskId(0, 0); - private final Map sent = new HashMap<>(); + private final Map sent = new HashMap<>(); private final NoOpRecordCollector collector = new NoOpRecordCollector() { @Override public void send(final String topic, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/FilteredCacheIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/FilteredCacheIteratorTest.java index 08b6b237ccc..4a0796d3ea4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/FilteredCacheIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/FilteredCacheIteratorTest.java @@ -18,7 +18,6 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.state.KeyValueIterator; import org.junit.Before; @@ -26,6 +25,7 @@ import org.junit.Test; import java.util.List; +import static java.util.Arrays.asList; import static org.apache.kafka.test.StreamsTestUtils.toList; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; @@ -50,7 +50,7 @@ public class FilteredCacheIteratorTest { private final InMemoryKeyValueStore store = new InMemoryKeyValueStore("name", null, null); private final KeyValue firstEntry = KeyValue.pair(Bytes.wrap("a".getBytes()), new LRUCacheEntry("1".getBytes())); - private final List> entries = Utils.mkList( + private final List> entries = asList( firstEntry, KeyValue.pair(Bytes.wrap("b".getBytes()), new LRUCacheEntry("2".getBytes())), @@ -119,7 +119,7 @@ public class FilteredCacheIteratorTest { @Test public void shouldFilterEntriesNotMatchingHasNextCondition() { final List> keyValues = toList(firstEntryIterator); - assertThat(keyValues, equalTo(Utils.mkList(firstEntry))); + assertThat(keyValues, equalTo(asList(firstEntry))); } @Test(expected = UnsupportedOperationException.class) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java index 99abdc4746b..e2757a92992 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java @@ -60,6 +60,7 @@ public class ReadOnlyWindowStoreStub implements ReadOnlyWindowStore, } } + @SuppressWarnings("deprecation") @Override public WindowStoreIterator fetch(final K key, final long timeFrom, final long timeTo) { if (!open) { @@ -126,7 +127,8 @@ public class ReadOnlyWindowStoreStub implements ReadOnlyWindowStore, } }; } - + + @SuppressWarnings("deprecation") @Override public KeyValueIterator, V> fetchAll(final long timeFrom, final long timeTo) { if (!open) { @@ -180,6 +182,7 @@ public class ReadOnlyWindowStoreStub implements ReadOnlyWindowStore, return fetchAll(from.toEpochMilli(), to.toEpochMilli()); } + @SuppressWarnings("deprecation") @Override public KeyValueIterator, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) { if (!open) { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java index c41b0942301..09c5e5b78e5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java @@ -47,7 +47,6 @@ import org.junit.Test; import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -56,6 +55,7 @@ import java.util.Set; import static java.time.Duration.ofMillis; import static java.time.Instant.ofEpochMilli; +import static java.util.Arrays.asList; import static java.util.Objects.requireNonNull; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; @@ -179,12 +179,12 @@ public class RocksDBWindowStoreTest { assertEquals("four", windowStore.fetch(4, startTime + 4L)); assertEquals("five", windowStore.fetch(5, startTime + 5L)); - assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize)))); - assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L + windowSize)))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L + windowSize)))); - assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L + windowSize)))); - assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L + windowSize)))); + assertEquals(asList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize)))); + assertEquals(asList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L + windowSize)))); + assertEquals(asList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L + windowSize)))); + assertEquals(asList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L + windowSize)))); + assertEquals(asList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L + windowSize)))); putSecondBatch(windowStore, startTime, context); @@ -195,21 +195,21 @@ public class RocksDBWindowStoreTest { assertEquals("two+5", windowStore.fetch(2, startTime + 7L)); assertEquals("two+6", windowStore.fetch(2, startTime + 8L)); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime - 2L - windowSize), ofEpochMilli(startTime - 2L + windowSize)))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime - 1L - windowSize), ofEpochMilli(startTime - 1L + windowSize)))); - assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); - assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L + windowSize)))); - assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L + windowSize)))); - assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L + windowSize)))); - assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L + windowSize)))); - assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L + windowSize)))); - assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 6L - windowSize), ofEpochMilli(startTime + 6L + windowSize)))); - assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 7L - windowSize), ofEpochMilli(startTime + 7L + windowSize)))); - assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 8L - windowSize), ofEpochMilli(startTime + 8L + windowSize)))); - assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 9L - windowSize), ofEpochMilli(startTime + 9L + windowSize)))); - assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 10L - windowSize), ofEpochMilli(startTime + 10L + windowSize)))); - assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 11L - windowSize), ofEpochMilli(startTime + 11L + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 12L - windowSize), ofEpochMilli(startTime + 12L + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(2, ofEpochMilli(startTime - 2L - windowSize), ofEpochMilli(startTime - 2L + windowSize)))); + assertEquals(asList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime - 1L - windowSize), ofEpochMilli(startTime - 1L + windowSize)))); + assertEquals(asList("two", "two+1"), toList(windowStore.fetch(2, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); + assertEquals(asList("two", "two+1", "two+2"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L + windowSize)))); + assertEquals(asList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L + windowSize)))); + assertEquals(asList("two", "two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L + windowSize)))); + assertEquals(asList("two", "two+1", "two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L + windowSize)))); + assertEquals(asList("two", "two+1", "two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L + windowSize)))); + assertEquals(asList("two+1", "two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 6L - windowSize), ofEpochMilli(startTime + 6L + windowSize)))); + assertEquals(asList("two+2", "two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 7L - windowSize), ofEpochMilli(startTime + 7L + windowSize)))); + assertEquals(asList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 8L - windowSize), ofEpochMilli(startTime + 8L + windowSize)))); + assertEquals(asList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 9L - windowSize), ofEpochMilli(startTime + 9L + windowSize)))); + assertEquals(asList("two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 10L - windowSize), ofEpochMilli(startTime + 10L + windowSize)))); + assertEquals(asList("two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 11L - windowSize), ofEpochMilli(startTime + 11L + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 12L - windowSize), ofEpochMilli(startTime + 12L + windowSize)))); // Flush the store and verify all current entries were properly flushed ... windowStore.flush(); @@ -239,7 +239,7 @@ public class RocksDBWindowStoreTest { final KeyValue, String> five = windowedPair(5, "five", startTime + 5); assertEquals( - Utils.mkList(zero, one, two, four, five), + asList(zero, one, two, four, five), StreamsTestUtils.toList(windowStore.all()) ); } @@ -258,17 +258,17 @@ public class RocksDBWindowStoreTest { final KeyValue, String> five = windowedPair(5, "five", startTime + 5); assertEquals( - Utils.mkList(one, two, four), + asList(one, two, four), StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 4))) ); assertEquals( - Utils.mkList(zero, one, two), + asList(zero, one, two), StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(startTime + 0), ofEpochMilli(startTime + 3))) ); assertEquals( - Utils.mkList(one, two, four, five), + asList(one, two, four, five), StreamsTestUtils.toList(windowStore.fetchAll(ofEpochMilli(startTime + 1), ofEpochMilli(startTime + 5))) ); } @@ -287,36 +287,36 @@ public class RocksDBWindowStoreTest { final KeyValue, String> five = windowedPair(5, "five", startTime + 5); assertEquals( - Utils.mkList(zero, one), + asList(zero, one), StreamsTestUtils.toList(windowStore.fetch(0, 1, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize))) ); assertEquals( - Utils.mkList(one), + asList(one), StreamsTestUtils.toList(windowStore.fetch(1, 1, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize))) ); assertEquals( - Utils.mkList(one, two), + asList(one, two), StreamsTestUtils.toList(windowStore.fetch(1, 3, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize))) ); assertEquals( - Utils.mkList(zero, one, two), + asList(zero, one, two), StreamsTestUtils.toList(windowStore.fetch(0, 5, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize))) ); assertEquals( - Utils.mkList(zero, one, two, + asList(zero, one, two, four, five), StreamsTestUtils.toList(windowStore.fetch(0, 5, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L + windowSize + 5L))) ); assertEquals( - Utils.mkList(two, four, five), + asList(two, four, five), StreamsTestUtils.toList(windowStore.fetch(0, 5, ofEpochMilli(startTime + 2L), ofEpochMilli(startTime + 0L + windowSize + 5L))) ); assertEquals( - Utils.mkList(), + asList(), StreamsTestUtils.toList(windowStore.fetch(4, 5, ofEpochMilli(startTime + 2L), ofEpochMilli(startTime + windowSize))) ); assertEquals( - Utils.mkList(), + asList(), StreamsTestUtils.toList(windowStore.fetch(0, 3, ofEpochMilli(startTime + 3L), ofEpochMilli(startTime + windowSize + 5))) ); } @@ -328,30 +328,30 @@ public class RocksDBWindowStoreTest { putFirstBatch(windowStore, startTime, context); - assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L)))); - assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L)))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L)))); - assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L)))); - assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L)))); + assertEquals(asList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L)))); + assertEquals(asList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L)))); + assertEquals(asList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L)))); + assertEquals(asList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L)))); + assertEquals(asList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L)))); + assertEquals(asList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L)))); putSecondBatch(windowStore, startTime, context); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime - 1L - windowSize), ofEpochMilli(startTime - 1L)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L)))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L)))); - assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L)))); - assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L)))); - assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L)))); - assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 6L - windowSize), ofEpochMilli(startTime + 6L)))); - assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 7L - windowSize), ofEpochMilli(startTime + 7L)))); - assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 8L - windowSize), ofEpochMilli(startTime + 8L)))); - assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 9L - windowSize), ofEpochMilli(startTime + 9L)))); - assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 10L - windowSize), ofEpochMilli(startTime + 10L)))); - assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 11L - windowSize), ofEpochMilli(startTime + 11L)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 12L - windowSize), ofEpochMilli(startTime + 12L)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 13L - windowSize), ofEpochMilli(startTime + 13L)))); + assertEquals(asList(), toList(windowStore.fetch(2, ofEpochMilli(startTime - 1L - windowSize), ofEpochMilli(startTime - 1L)))); + assertEquals(asList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 0L - windowSize), ofEpochMilli(startTime + 0L)))); + assertEquals(asList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L)))); + assertEquals(asList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L)))); + assertEquals(asList("two", "two+1"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L)))); + assertEquals(asList("two", "two+1", "two+2"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L)))); + assertEquals(asList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 5L - windowSize), ofEpochMilli(startTime + 5L)))); + assertEquals(asList("two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 6L - windowSize), ofEpochMilli(startTime + 6L)))); + assertEquals(asList("two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 7L - windowSize), ofEpochMilli(startTime + 7L)))); + assertEquals(asList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 8L - windowSize), ofEpochMilli(startTime + 8L)))); + assertEquals(asList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 9L - windowSize), ofEpochMilli(startTime + 9L)))); + assertEquals(asList("two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 10L - windowSize), ofEpochMilli(startTime + 10L)))); + assertEquals(asList("two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 11L - windowSize), ofEpochMilli(startTime + 11L)))); + assertEquals(asList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 12L - windowSize), ofEpochMilli(startTime + 12L)))); + assertEquals(asList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 13L - windowSize), ofEpochMilli(startTime + 13L)))); // Flush the store and verify all current entries were properly flushed ... windowStore.flush(); @@ -374,30 +374,30 @@ public class RocksDBWindowStoreTest { putFirstBatch(windowStore, startTime, context); - assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 0L), ofEpochMilli(startTime + 0L + windowSize)))); - assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + 1L), ofEpochMilli(startTime + 1L + windowSize)))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L), ofEpochMilli(startTime + 2L + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + 3L), ofEpochMilli(startTime + 3L + windowSize)))); - assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + 4L), ofEpochMilli(startTime + 4L + windowSize)))); - assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + 5L), ofEpochMilli(startTime + 5L + windowSize)))); + assertEquals(asList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 0L), ofEpochMilli(startTime + 0L + windowSize)))); + assertEquals(asList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + 1L), ofEpochMilli(startTime + 1L + windowSize)))); + assertEquals(asList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L), ofEpochMilli(startTime + 2L + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + 3L), ofEpochMilli(startTime + 3L + windowSize)))); + assertEquals(asList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + 4L), ofEpochMilli(startTime + 4L + windowSize)))); + assertEquals(asList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + 5L), ofEpochMilli(startTime + 5L + windowSize)))); putSecondBatch(windowStore, startTime, context); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime - 2L), ofEpochMilli(startTime - 2L + windowSize)))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime - 1L), ofEpochMilli(startTime - 1L + windowSize)))); - assertEquals(Utils.mkList("two", "two+1"), toList(windowStore.fetch(2, ofEpochMilli(startTime), ofEpochMilli(startTime + windowSize)))); - assertEquals(Utils.mkList("two", "two+1", "two+2"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 1L), ofEpochMilli(startTime + 1L + windowSize)))); - assertEquals(Utils.mkList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L), ofEpochMilli(startTime + 2L + windowSize)))); - assertEquals(Utils.mkList("two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 3L), ofEpochMilli(startTime + 3L + windowSize)))); - assertEquals(Utils.mkList("two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 4L), ofEpochMilli(startTime + 4L + windowSize)))); - assertEquals(Utils.mkList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 5L), ofEpochMilli(startTime + 5L + windowSize)))); - assertEquals(Utils.mkList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 6L), ofEpochMilli(startTime + 6L + windowSize)))); - assertEquals(Utils.mkList("two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 7L), ofEpochMilli(startTime + 7L + windowSize)))); - assertEquals(Utils.mkList("two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 8L), ofEpochMilli(startTime + 8L + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 9L), ofEpochMilli(startTime + 9L + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 10L), ofEpochMilli(startTime + 10L + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 11L), ofEpochMilli(startTime + 11L + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 12L), ofEpochMilli(startTime + 12L + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(2, ofEpochMilli(startTime - 2L), ofEpochMilli(startTime - 2L + windowSize)))); + assertEquals(asList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime - 1L), ofEpochMilli(startTime - 1L + windowSize)))); + assertEquals(asList("two", "two+1"), toList(windowStore.fetch(2, ofEpochMilli(startTime), ofEpochMilli(startTime + windowSize)))); + assertEquals(asList("two", "two+1", "two+2"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 1L), ofEpochMilli(startTime + 1L + windowSize)))); + assertEquals(asList("two", "two+1", "two+2", "two+3"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 2L), ofEpochMilli(startTime + 2L + windowSize)))); + assertEquals(asList("two+1", "two+2", "two+3", "two+4"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 3L), ofEpochMilli(startTime + 3L + windowSize)))); + assertEquals(asList("two+2", "two+3", "two+4", "two+5"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 4L), ofEpochMilli(startTime + 4L + windowSize)))); + assertEquals(asList("two+3", "two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 5L), ofEpochMilli(startTime + 5L + windowSize)))); + assertEquals(asList("two+4", "two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 6L), ofEpochMilli(startTime + 6L + windowSize)))); + assertEquals(asList("two+5", "two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 7L), ofEpochMilli(startTime + 7L + windowSize)))); + assertEquals(asList("two+6"), toList(windowStore.fetch(2, ofEpochMilli(startTime + 8L), ofEpochMilli(startTime + 8L + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 9L), ofEpochMilli(startTime + 9L + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 10L), ofEpochMilli(startTime + 10L + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 11L), ofEpochMilli(startTime + 11L + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + 12L), ofEpochMilli(startTime + 12L + windowSize)))); // Flush the store and verify all current entries were properly flushed ... windowStore.flush(); @@ -421,17 +421,17 @@ public class RocksDBWindowStoreTest { setCurrentTime(startTime); windowStore.put(0, "zero"); - assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); + assertEquals(asList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); windowStore.put(0, "zero"); windowStore.put(0, "zero+"); windowStore.put(0, "zero++"); - assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); - assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L + windowSize)))); - assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L + windowSize)))); - assertEquals(Utils.mkList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L + windowSize)))); + assertEquals(asList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); + assertEquals(asList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 1L - windowSize), ofEpochMilli(startTime + 1L + windowSize)))); + assertEquals(asList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 2L - windowSize), ofEpochMilli(startTime + 2L + windowSize)))); + assertEquals(asList("zero", "zero", "zero+", "zero++"), toList(windowStore.fetch(0, ofEpochMilli(startTime + 3L - windowSize), ofEpochMilli(startTime + 3L + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(0, ofEpochMilli(startTime + 4L - windowSize), ofEpochMilli(startTime + 4L + windowSize)))); // Flush the store and verify all current entries were properly flushed ... windowStore.flush(); @@ -489,12 +489,12 @@ public class RocksDBWindowStoreTest { segmentDirs(baseDir) ); - assertEquals(Utils.mkList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); - assertEquals(Utils.mkList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); - assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); - assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); + assertEquals(asList("zero"), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); + assertEquals(asList("one"), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); + assertEquals(asList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); + assertEquals(asList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); + assertEquals(asList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); setCurrentTime(startTime + increment * 6); windowStore.put(6, "six"); @@ -508,13 +508,13 @@ public class RocksDBWindowStoreTest { ); - assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); - assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); - assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); - assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); + assertEquals(asList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); + assertEquals(asList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); + assertEquals(asList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); + assertEquals(asList("six"), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize)))); setCurrentTime(startTime + increment * 7); @@ -528,14 +528,14 @@ public class RocksDBWindowStoreTest { segmentDirs(baseDir) ); - assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); - assertEquals(Utils.mkList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); - assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); - assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); - assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize)))); - assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, ofEpochMilli(startTime + increment * 7 - windowSize), ofEpochMilli(startTime + increment * 7 + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); + assertEquals(asList("two"), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); + assertEquals(asList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); + assertEquals(asList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); + assertEquals(asList("six"), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize)))); + assertEquals(asList("seven"), toList(windowStore.fetch(7, ofEpochMilli(startTime + increment * 7 - windowSize), ofEpochMilli(startTime + increment * 7 + windowSize)))); setCurrentTime(startTime + increment * 8); windowStore.put(8, "eight"); @@ -549,15 +549,15 @@ public class RocksDBWindowStoreTest { ); - assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); - assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); - assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); - assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize)))); - assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, ofEpochMilli(startTime + increment * 7 - windowSize), ofEpochMilli(startTime + increment * 7 + windowSize)))); - assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, ofEpochMilli(startTime + increment * 8 - windowSize), ofEpochMilli(startTime + increment * 8 + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); + assertEquals(asList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); + assertEquals(asList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); + assertEquals(asList("six"), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize)))); + assertEquals(asList("seven"), toList(windowStore.fetch(7, ofEpochMilli(startTime + increment * 7 - windowSize), ofEpochMilli(startTime + increment * 7 + windowSize)))); + assertEquals(asList("eight"), toList(windowStore.fetch(8, ofEpochMilli(startTime + increment * 8 - windowSize), ofEpochMilli(startTime + increment * 8 + windowSize)))); // check segment directories windowStore.flush(); @@ -605,27 +605,27 @@ public class RocksDBWindowStoreTest { Utils.delete(baseDir); windowStore = createWindowStore(context, false); - assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(7, ofEpochMilli(startTime + increment * 7 - windowSize), ofEpochMilli(startTime + increment * 7 + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(8, ofEpochMilli(startTime + increment * 8 - windowSize), ofEpochMilli(startTime + increment * 8 + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(7, ofEpochMilli(startTime + increment * 7 - windowSize), ofEpochMilli(startTime + increment * 7 + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(8, ofEpochMilli(startTime + increment * 8 - windowSize), ofEpochMilli(startTime + increment * 8 + windowSize)))); context.restore(windowName, changeLog); - assertEquals(Utils.mkList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); - assertEquals(Utils.mkList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); - assertEquals(Utils.mkList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); - assertEquals(Utils.mkList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); - assertEquals(Utils.mkList("six"), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize)))); - assertEquals(Utils.mkList("seven"), toList(windowStore.fetch(7, ofEpochMilli(startTime + increment * 7 - windowSize), ofEpochMilli(startTime + increment * 7 + windowSize)))); - assertEquals(Utils.mkList("eight"), toList(windowStore.fetch(8, ofEpochMilli(startTime + increment * 8 - windowSize), ofEpochMilli(startTime + increment * 8 + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(0, ofEpochMilli(startTime - windowSize), ofEpochMilli(startTime + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(1, ofEpochMilli(startTime + increment - windowSize), ofEpochMilli(startTime + increment + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(2, ofEpochMilli(startTime + increment * 2 - windowSize), ofEpochMilli(startTime + increment * 2 + windowSize)))); + assertEquals(asList(), toList(windowStore.fetch(3, ofEpochMilli(startTime + increment * 3 - windowSize), ofEpochMilli(startTime + increment * 3 + windowSize)))); + assertEquals(asList("four"), toList(windowStore.fetch(4, ofEpochMilli(startTime + increment * 4 - windowSize), ofEpochMilli(startTime + increment * 4 + windowSize)))); + assertEquals(asList("five"), toList(windowStore.fetch(5, ofEpochMilli(startTime + increment * 5 - windowSize), ofEpochMilli(startTime + increment * 5 + windowSize)))); + assertEquals(asList("six"), toList(windowStore.fetch(6, ofEpochMilli(startTime + increment * 6 - windowSize), ofEpochMilli(startTime + increment * 6 + windowSize)))); + assertEquals(asList("seven"), toList(windowStore.fetch(7, ofEpochMilli(startTime + increment * 7 - windowSize), ofEpochMilli(startTime + increment * 7 + windowSize)))); + assertEquals(asList("eight"), toList(windowStore.fetch(8, ofEpochMilli(startTime + increment * 8 - windowSize), ofEpochMilli(startTime + increment * 8 + windowSize)))); // check segment directories windowStore.flush(); @@ -730,7 +730,7 @@ public class RocksDBWindowStoreTest { context.setStreamTime(segmentInterval * 6L); windowStore = createWindowStore(context, false); - final List expected = Utils.mkList(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)); + final List expected = asList(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L)); expected.sort(String::compareTo); final List actual = Utils.toList(segmentDirs(baseDir).iterator()); @@ -785,19 +785,19 @@ public class RocksDBWindowStoreTest { windowStore.put("a", "0005", 0x7a00000000000000L - 1); - final List expected = Utils.mkList("0001", "0003", "0005"); + final List expected = asList("0001", "0003", "0005"); assertThat(toList(windowStore.fetch("a", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), equalTo(expected)); List, String>> list = StreamsTestUtils.toList(windowStore.fetch("a", "a", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); - assertThat(list, equalTo(Utils.mkList( + assertThat(list, equalTo(asList( windowedPair("a", "0001", 0, windowSize), windowedPair("a", "0003", 1, windowSize), windowedPair("a", "0005", 0x7a00000000000000L - 1, windowSize) ))); list = StreamsTestUtils.toList(windowStore.fetch("aa", "aa", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))); - assertThat(list, equalTo(Utils.mkList( + assertThat(list, equalTo(asList( windowedPair("aa", "0002", 0, windowSize), windowedPair("aa", "0004", 1, windowSize) ))); @@ -868,11 +868,11 @@ public class RocksDBWindowStoreTest { windowStore.put(key2, "8", 59999); windowStore.put(key3, "9", 59999); - final List expectedKey1 = Utils.mkList("1", "4", "7"); + final List expectedKey1 = asList("1", "4", "7"); assertThat(toList(windowStore.fetch(key1, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), equalTo(expectedKey1)); - final List expectedKey2 = Utils.mkList("2", "5", "8"); + final List expectedKey2 = asList("2", "5", "8"); assertThat(toList(windowStore.fetch(key2, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), equalTo(expectedKey2)); - final List expectedKey3 = Utils.mkList("3", "6", "9"); + final List expectedKey3 = asList("3", "6", "9"); assertThat(toList(windowStore.fetch(key3, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), equalTo(expectedKey3)); } @@ -919,7 +919,7 @@ public class RocksDBWindowStoreTest { private Set segmentDirs(final File baseDir) { final File windowDir = new File(baseDir, windowName); - return new HashSet<>(Arrays.asList(requireNonNull(windowDir.list()))); + return new HashSet<>(asList(requireNonNull(windowDir.list()))); } private Map> entriesByKey(final List> changeLog, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java index 4f1e181c327..a7a64c423b2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java @@ -358,7 +358,7 @@ public class ThreadCacheTest { cache.put(namespace2, Bytes.wrap(new byte[]{4}), cleanEntry(new byte[]{4})); cache.flush(namespace1); - assertEquals(Collections.EMPTY_LIST, received); + assertEquals(Collections.emptyList(), received); } @@ -531,4 +531,4 @@ public class ThreadCacheTest { } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java index 6f801c9e93a..623c3e36f76 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java @@ -33,7 +33,6 @@ import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.ValueMapper; import java.io.IOException; @@ -47,6 +46,7 @@ public class BrokerCompatibilityTest { private static final String SOURCE_TOPIC = "brokerCompatibilitySourceTopic"; private static final String SINK_TOPIC = "brokerCompatibilitySinkTopic"; + @SuppressWarnings("deprecation") public static void main(final String[] args) throws IOException { if (args.length < 2) { System.err.println("BrokerCompatibilityTest are expecting two parameters: propFile, eosEnabled; but only see " + args.length + " parameter"); @@ -83,7 +83,7 @@ public class BrokerCompatibilityTest { final StreamsBuilder builder = new StreamsBuilder(); - builder.stream(SOURCE_TOPIC).groupByKey(Serialized.with(stringSerde, stringSerde)) + builder.stream(SOURCE_TOPIC).groupByKey(org.apache.kafka.streams.kstream.Serialized.with(stringSerde, stringSerde)) .count() .toStream() .mapValues(new ValueMapper() { diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java index ddff7a892e2..ef908f42c17 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -33,7 +33,6 @@ import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.Predicate; import org.apache.kafka.streams.kstream.Produced; -import org.apache.kafka.streams.kstream.Serialized; import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.state.KeyValueStore; @@ -111,6 +110,7 @@ public class SmokeTestClient extends SmokeTestUtil { return fullProps; } + @SuppressWarnings("deprecation") private static KafkaStreams createKafkaStreams(final Properties props) { final StreamsBuilder builder = new StreamsBuilder(); final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde); @@ -126,7 +126,7 @@ public class SmokeTestClient extends SmokeTestUtil { // min final KGroupedStream groupedData = - data.groupByKey(Serialized.with(stringSerde, intSerde)); + data.groupByKey(org.apache.kafka.streams.kstream.Serialized.with(stringSerde, intSerde)); groupedData .windowedBy(TimeWindows.of(Duration.ofDays(1))) @@ -239,7 +239,7 @@ public class SmokeTestClient extends SmokeTestUtil { // test repartition final Agg agg = new Agg(); - cntTable.groupBy(agg.selector(), Serialized.with(stringSerde, longSerde)) + cntTable.groupBy(agg.selector(), org.apache.kafka.streams.kstream.Serialized.with(stringSerde, longSerde)) .aggregate(agg.init(), agg.adder(), agg.remover(), Materialized.as(Stores.inMemoryKeyValueStore("cntByCnt")) .withKeySerde(Serdes.String()) diff --git a/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java index 2f356bff3a1..3b332b55266 100644 --- a/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java @@ -205,6 +205,7 @@ public class InternalMockProcessorContext extends AbstractProcessorContext imple return storeMap.get(name); } + @SuppressWarnings("deprecation") @Override public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) { throw new UnsupportedOperationException("schedule() not supported."); @@ -227,13 +228,13 @@ public class InternalMockProcessorContext extends AbstractProcessorContext imple } @Override - @SuppressWarnings("unchecked") + @SuppressWarnings({"unchecked", "deprecation"}) public void forward(final K key, final V value, final int childIndex) { forward(key, value, To.child(((List) currentNode().children()).get(childIndex).name())); } @Override - @SuppressWarnings("unchecked") + @SuppressWarnings({"unchecked", "deprecation"}) public void forward(final K key, final V value, final String childName) { forward(key, value, To.child(childName)); } diff --git a/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java index 36d049c58ba..22820848863 100644 --- a/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java @@ -35,7 +35,7 @@ import java.util.Properties; public class NoOpProcessorContext extends AbstractProcessorContext { public boolean initialized; - public Map forwardedValues = new HashMap(); + public Map forwardedValues = new HashMap<>(); public NoOpProcessorContext() { super(new TaskId(1, 1), streamsConfig(), new MockStreamsMetrics(new Metrics()), null, null); @@ -53,6 +53,7 @@ public class NoOpProcessorContext extends AbstractProcessorContext { return null; } + @SuppressWarnings("deprecation") @Override public Cancellable schedule(final long interval, final PunctuationType type, final Punctuator callback) { return null; @@ -75,11 +76,13 @@ public class NoOpProcessorContext extends AbstractProcessorContext { forwardedValues.put(key, value); } + @SuppressWarnings("deprecation") @Override public void forward(final K key, final V value, final int childIndex) { forward(key, value); } + @SuppressWarnings("deprecation") @Override public void forward(final K key, final V value, final String childName) { forward(key, value); diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KGroupedTable.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KGroupedTable.scala index 54b1e1f94bb..e30c8f30e41 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KGroupedTable.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KGroupedTable.scala @@ -20,7 +20,7 @@ package org.apache.kafka.streams.scala package kstream -import org.apache.kafka.streams.kstream.{KGroupedTable => KGroupedTableJ, _} +import org.apache.kafka.streams.kstream.{KGroupedTable => KGroupedTableJ} import org.apache.kafka.streams.scala.ImplicitConversions._ import org.apache.kafka.streams.scala.FunctionsCompatConversions._ diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java index ba5286783b0..7fdfdbe354f 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java @@ -232,7 +232,7 @@ public class MockProcessorContextTest { final AbstractProcessor processor = new AbstractProcessor() { @Override public void process(final String key, final Long value) { - //noinspection unchecked + @SuppressWarnings("unchecked") final KeyValueStore stateStore = (KeyValueStore) context().getStateStore("my-state"); stateStore.put(key, (stateStore.get(key) == null ? 0 : stateStore.get(key)) + value); stateStore.put("all", (stateStore.get("all") == null ? 0 : stateStore.get("all")) + value); 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 58f6e02cc9f..bead079ce8d 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 @@ -901,9 +901,9 @@ public class TopologyTestDriverTest { return new Processor() { private KeyValueStore store; + @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context) { - //noinspection unchecked this.store = (KeyValueStore) context.getStateStore("storeProcessorStore"); } diff --git a/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java b/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java index a9537a7641d..61827449e88 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java +++ b/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java @@ -52,6 +52,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; +import java.time.Duration; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -413,7 +414,7 @@ public class ClientCompatibilityTest { if (curTime - prodTimeMs > TIMEOUT_MS) throw new RuntimeException("Timed out after " + TIMEOUT_MS + " ms."); if (recordIter == null) { - ConsumerRecords records = consumer.poll(100); + ConsumerRecords records = consumer.poll(Duration.ofMillis(100)); recordIter = records.iterator(); } if (recordIter.hasNext()) diff --git a/tools/src/test/java/org/apache/kafka/trogdor/common/JsonSerializationTest.java b/tools/src/test/java/org/apache/kafka/trogdor/common/JsonSerializationTest.java index e7809cd0c80..5e6ff81e28b 100644 --- a/tools/src/test/java/org/apache/kafka/trogdor/common/JsonSerializationTest.java +++ b/tools/src/test/java/org/apache/kafka/trogdor/common/JsonSerializationTest.java @@ -68,6 +68,7 @@ public class JsonSerializationTest { private void verify(T val1) throws Exception { byte[] bytes = JsonUtil.JSON_SERDE.writeValueAsBytes(val1); + @SuppressWarnings("unchecked") Class clazz = (Class) val1.getClass(); T val2 = JsonUtil.JSON_SERDE.readValue(bytes, clazz); for (Field field : clazz.getDeclaredFields()) {