You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ij...@apache.org on 2018/11/13 06:19:11 UTC
[kafka] branch trunk updated: KAFKA-7612: Fix javac warnings and
enable warnings as errors (#5900)
This is an automated email from the ASF dual-hosted git repository.
ijuma pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/trunk by this push:
new 12f310d KAFKA-7612: Fix javac warnings and enable warnings as errors (#5900)
12f310d is described below
commit 12f310d50e7f5b1c18c4f61a119a6cd830da3bc0
Author: Ismael Juma <is...@juma.me.uk>
AuthorDate: Mon Nov 12 22:18:59 2018 -0800
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 <mj...@apache.org>, Manikumar Reddy <ma...@gmail.com>, Randall Hauch <rh...@gmail.com>, Bill Bejeck <bi...@confluent.io>, Stanislav Kozlovski <st...@outlook.com>
---
build.gradle | 9 +-
.../apache/kafka/clients/FetchSessionHandler.java | 8 +-
.../apache/kafka/clients/admin/ConfigEntry.java | 1 +
.../kafka/clients/consumer/KafkaConsumer.java | 2 +-
.../kafka/clients/consumer/MockConsumer.java | 1 +
.../kafka/clients/consumer/internals/Fetcher.java | 13 +-
.../kafka/common/network/SslTransportLayer.java | 1 +
.../common/record/LazyDownConversionRecords.java | 8 +-
.../record/LazyDownConversionRecordsSend.java | 6 +-
.../security/authenticator/CredentialCache.java | 3 +-
.../security/authenticator/LoginManager.java | 1 +
.../authenticator/SaslClientAuthenticator.java | 1 +
.../authenticator/SaslClientCallbackHandler.java | 1 +
.../authenticator/SaslServerAuthenticator.java | 1 +
.../java/org/apache/kafka/common/utils/Crc32.java | 1 +
.../apache/kafka/common/utils/PureJavaCrc32C.java | 1 +
.../java/org/apache/kafka/common/utils/Utils.java | 17 +-
.../kafka/clients/FetchSessionHandlerTest.java | 1 +
.../kafka/clients/admin/KafkaAdminClientTest.java | 1 +
.../kafka/clients/consumer/KafkaConsumerTest.java | 24 +-
.../internals/ConsumerCoordinatorTest.java | 3 +-
.../clients/consumer/internals/FetcherTest.java | 6 +-
.../kafka/clients/producer/KafkaProducerTest.java | 7 +-
.../org/apache/kafka/common/KafkaFutureTest.java | 32 +--
.../apache/kafka/common/config/ConfigDefTest.java | 2 +-
.../apache/kafka/common/metrics/MetricsTest.java | 1 +
.../kafka/common/network/ChannelBuildersTest.java | 3 +-
.../kafka/common/record/FileRecordsTest.java | 4 +-
.../kafka/common/requests/RequestResponseTest.java | 1 +
.../common/serialization/SerializationTest.java | 7 +-
.../kafka/connect/util/ConnectorUtilsTest.java | 4 +-
.../kafka/connect/json/JsonConverterTest.java | 2 +-
.../kafka/connect/converters/NumberConverter.java | 1 +
.../kafka/connect/runtime/ConnectorConfig.java | 10 +-
.../org/apache/kafka/connect/runtime/Worker.java | 1 +
.../kafka/connect/runtime/WorkerConnector.java | 1 +
.../runtime/errors/RetryWithToleranceOperator.java | 1 +
.../runtime/isolation/DelegatingClassLoader.java | 10 +-
.../kafka/connect/runtime/isolation/Plugins.java | 11 +-
.../runtime/rest/ConnectRestConfigurable.java | 14 +-
.../kafka/connect/runtime/rest/RestServer.java | 5 +-
.../kafka/connect/runtime/rest/util/SSLUtils.java | 1 +
.../kafka/connect/storage/OffsetStorageWriter.java | 1 +
.../kafka/connect/runtime/AbstractHerderTest.java | 2 +-
.../kafka/connect/runtime/ConnectMetricsTest.java | 1 +
.../kafka/connect/runtime/MockConnectMetrics.java | 1 +
.../runtime/SourceTaskOffsetCommitterTest.java | 9 +-
.../runtime/WorkerSinkTaskThreadedTest.java | 12 +-
.../apache/kafka/connect/runtime/WorkerTest.java | 7 +-
.../runtime/distributed/DistributedHerderTest.java | 2 +
.../connect/runtime/isolation/PluginsTest.java | 2 +
.../kafka/connect/runtime/rest/RestServerTest.java | 7 +-
.../resources/ConnectorPluginsResourceTest.java | 9 +-
.../connect/runtime/rest/util/SSLUtilsTest.java | 1 +
.../storage/FileOffsetBackingStoreTest.java | 1 +
.../storage/KafkaConfigBackingStoreTest.java | 20 +-
.../storage/KafkaOffsetBackingStoreTest.java | 32 +--
.../kafka/connect/util/KafkaBasedLogTest.java | 1 +
.../kafka/connect/transforms/ReplaceField.java | 3 +-
.../kafka/connect/transforms/FlattenTest.java | 3 +
.../kafka/connect/transforms/MaskFieldTest.java | 1 +
core/src/main/scala/kafka/admin/AclCommand.scala | 3 +-
core/src/main/scala/kafka/admin/AdminClient.scala | 2 +-
core/src/main/scala/kafka/admin/AdminUtils.scala | 9 +-
core/src/main/scala/kafka/admin/TopicCommand.scala | 3 +-
.../scala/kafka/controller/ControllerContext.scala | 2 +-
.../scala/kafka/controller/KafkaController.scala | 2 +-
.../coordinator/group/GroupMetadataManager.scala | 4 +-
.../transaction/TransactionCoordinator.scala | 2 +-
.../transaction/TransactionStateManager.scala | 4 +-
core/src/main/scala/kafka/log/Log.scala | 2 +-
core/src/main/scala/kafka/log/LogCleaner.scala | 2 +-
.../main/scala/kafka/log/LogCleanerManager.scala | 3 -
.../kafka/server/AbstractFetcherManager.scala | 1 -
.../scala/kafka/server/AbstractFetcherThread.scala | 4 +-
core/src/main/scala/kafka/server/KafkaApis.scala | 2 +-
.../main/scala/kafka/tools/ConsoleConsumer.scala | 8 +-
.../main/scala/kafka/tools/ConsoleProducer.scala | 2 +-
core/src/main/scala/kafka/utils/Mx4jLoader.scala | 4 +-
.../main/scala/kafka/utils/json/DecodeJson.scala | 6 +-
.../main/scala/kafka/utils/json/JsonObject.scala | 2 +-
.../main/scala/kafka/utils/json/JsonValue.scala | 4 +-
.../kafka/api/AdminClientIntegrationTest.scala | 23 +-
.../kafka/api/AuthorizerIntegrationTest.scala | 20 +-
.../integration/kafka/api/BaseConsumerTest.scala | 1 -
.../kafka/api/BaseProducerSendTest.scala | 17 +-
.../DelegationTokenEndToEndAuthorizationTest.scala | 5 +-
.../kafka/api/EndToEndAuthorizationTest.scala | 23 +-
.../kafka/api/EndToEndClusterIdTest.scala | 15 +-
.../kafka/api/LegacyAdminClientTest.scala | 2 +
.../integration/kafka/api/LogAppendTimeTest.scala | 10 +-
.../kafka/api/PlaintextConsumerTest.scala | 4 +-
.../kafka/api/PlaintextProducerSendTest.scala | 2 +-
.../server/DynamicBrokerReconfigurationTest.scala | 16 +-
...ListenersWithSameSecurityProtocolBaseTest.scala | 9 +-
.../kafka/common/InterBrokerSendThreadTest.scala | 11 +-
.../scala/unit/kafka/admin/TestAdminUtils.scala | 1 +
.../TransactionMarkerChannelManagerTest.scala | 15 +-
...sactionMarkerRequestCompletionHandlerTest.scala | 7 +-
.../scala/unit/kafka/server/FetchSessionTest.scala | 24 +-
.../scala/unit/kafka/server/KafkaApisTest.scala | 16 +-
.../unit/kafka/server/LogDirFailureTest.scala | 13 +-
.../scala/unit/kafka/server/LogOffsetTest.scala | 4 +
.../scala/unit/kafka/server/RequestQuotaTest.scala | 2 +-
...chDrivenReplicationProtocolAcceptanceTest.scala | 32 +--
.../unit/kafka/tools/ConsoleProducerTest.scala | 2 +-
.../test/scala/unit/kafka/utils/TestUtils.scala | 4 +-
.../test/scala/unit/kafka/utils/ZkUtilsTest.scala | 1 +
.../scala/unit/kafka/zk/AdminZkClientTest.scala | 2 +-
.../unit/kafka/zookeeper/ZooKeeperClientTest.scala | 6 +-
.../jmh/producer/ProducerRecordBenchmark.java | 2 +-
.../examples/pageview/PageViewTypedDemo.java | 4 +-
.../examples/pageview/PageViewUntypedDemo.java | 4 +-
.../streams/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 +-
.../kstream/internals/graph/StreamSinkNode.java | 1 +
.../internals/graph/TableProcessorNode.java | 4 +-
.../kstream/internals/graph/TableSourceNode.java | 3 +-
.../internals/InternalTopologyBuilder.java | 4 +-
.../apache/kafka/streams/state/WindowStore.java | 3 +
.../state/internals/CachingWindowStore.java | 5 +-
.../internals/ChangeLoggingWindowBytesStore.java | 5 +-
.../internals/CompositeReadOnlyWindowStore.java | 1 +
.../state/internals/MeteredWindowStore.java | 3 +
.../state/internals/RocksDBWindowStore.java | 3 +
.../org/apache/kafka/streams/KafkaStreamsTest.java | 7 +-
.../apache/kafka/streams/StreamsBuilderTest.java | 9 +-
.../org/apache/kafka/streams/TopologyTest.java | 2 +-
.../FineGrainedAutoResetIntegrationTest.java | 13 +-
.../KStreamAggregationDedupIntegrationTest.java | 8 +-
.../KStreamAggregationIntegrationTest.java | 12 +-
.../integration/RestoreIntegrationTest.java | 3 +-
.../integration/utils/EmbeddedKafkaCluster.java | 9 +-
.../streams/integration/utils/KafkaEmbedded.java | 8 +-
.../kafka/streams/kstream/JoinWindowsTest.java | 2 +-
.../kstream/RepartitionTopicNamingTest.java | 1 +
.../kafka/streams/kstream/SessionWindowsTest.java | 1 +
.../kafka/streams/kstream/TimeWindowsTest.java | 1 +
.../streams/kstream/UnlimitedWindowsTest.java | 1 +
.../kstream/internals/AbstractStreamTest.java | 11 +-
.../internals/InternalStreamsBuilderTest.java | 6 +-
.../kstream/internals/KGroupedStreamImplTest.java | 5 +-
.../kstream/internals/KGroupedTableImplTest.java | 11 +-
.../streams/kstream/internals/KStreamImplTest.java | 24 +-
.../kstream/internals/KStreamTransformTest.java | 4 +-
.../internals/KStreamTransformValuesTest.java | 1 +
.../internals/KStreamWindowAggregateTest.java | 7 +-
.../kstream/internals/KStreamWindowReduceTest.java | 6 +-
.../kstream/internals/KTableAggregateTest.java | 24 +-
.../kstream/internals/KTableFilterTest.java | 1 +
.../streams/kstream/internals/KTableImplTest.java | 17 +-
.../internals/KTableKTableInnerJoinTest.java | 7 +-
.../internals/KTableKTableLeftJoinTest.java | 11 +-
.../internals/KTableKTableOuterJoinTest.java | 7 +-
.../internals/KTableKTableRightJoinTest.java | 1 +
.../kstream/internals/KTableMapValuesTest.java | 5 +-
.../kstream/internals/KTableSourceTest.java | 7 +-
.../internals/KTableTransformValuesTest.java | 10 +-
.../internals/SessionWindowedKStreamImplTest.java | 4 +-
.../kstream/internals/SuppressScenarioTest.java | 3 +
.../internals/TimeWindowedKStreamImplTest.java | 4 +-
.../apache/kafka/streams/perf/YahooBenchmark.java | 5 +-
.../UsePreviousTimeOnInvalidTimestampTest.java | 2 +-
.../internals/AbstractProcessorContextTest.java | 3 +
.../internals/GlobalStateManagerImplTest.java | 6 +-
.../processor/internals/GlobalStateTaskTest.java | 5 +-
.../internals/InternalTopologyBuilderTest.java | 10 +-
.../processor/internals/RecordCollectorTest.java | 8 +-
.../streams/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 +-
.../internals/StreamsPartitionAssignorTest.java | 55 ++--
.../processor/internals/TaskManagerTest.java | 16 +-
.../kafka/streams/state/StateSerdesTest.java | 1 +
.../state/internals/CachingWindowStoreTest.java | 11 +-
.../ChangeLoggingKeyValueBytesStoreTest.java | 2 +-
.../ChangeLoggingWindowBytesStoreTest.java | 2 +-
.../state/internals/FilteredCacheIteratorTest.java | 6 +-
.../state/internals/ReadOnlyWindowStoreStub.java | 5 +-
.../state/internals/RocksDBWindowStoreTest.java | 276 ++++++++++-----------
.../streams/state/internals/ThreadCacheTest.java | 4 +-
.../streams/tests/BrokerCompatibilityTest.java | 4 +-
.../kafka/streams/tests/SmokeTestClient.java | 6 +-
.../kafka/test/InternalMockProcessorContext.java | 5 +-
.../apache/kafka/test/NoOpProcessorContext.java | 5 +-
.../streams/scala/kstream/KGroupedTable.scala | 2 +-
.../kafka/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 bba3d83..4d514df 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 16990ac..30ae65f 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<TopicPartition> 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<TopicPartition> 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 e8da646..9976108 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 065e663..3a75672 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<K, V> implements Consumer<K, V> {
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 9eee6da..f1dcb32 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<K, V> implements Consumer<K, V> {
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 93abc40..265fc99 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<K, V> implements SubscriptionState.Listener, Closeable {
@Override
public void onSuccess(ClientResponse resp) {
synchronized (Fetcher.this) {
+ @SuppressWarnings("unchecked")
FetchResponse<Records> response = (FetchResponse<Records>) resp.responseBody();
FetchSessionHandler handler = sessionHandler(fetchTarget.id());
if (handler == null) {
@@ -239,7 +240,7 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
for (Map.Entry<TopicPartition, FetchResponse.PartitionData<Records>> entry : response.responseData().entrySet()) {
TopicPartition partition = entry.getKey();
long fetchOffset = data.sessionPartitions().get(partition).fetchOffset;
- FetchResponse.PartitionData fetchData = entry.getValue();
+ FetchResponse.PartitionData<Records> fetchData = entry.getValue();
log.debug("Fetch {} at offset {} for partition {} returned fetch data {}",
isolationLevel, fetchOffset, partition, fetchData);
@@ -1280,18 +1281,12 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
return batch.isTransactional() && abortedProducerIds.contains(batch.producerId());
}
- private PriorityQueue<FetchResponse.AbortedTransaction> abortedTransactions(FetchResponse.PartitionData partition) {
+ private PriorityQueue<FetchResponse.AbortedTransaction> abortedTransactions(FetchResponse.PartitionData<?> partition) {
if (partition.abortedTransactions == null || partition.abortedTransactions.isEmpty())
return null;
PriorityQueue<FetchResponse.AbortedTransaction> abortedTransactions = new PriorityQueue<>(
- partition.abortedTransactions.size(),
- new Comparator<FetchResponse.AbortedTransaction>() {
- @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 a6696f7..9410fdd 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 4e84473..217870d 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<ConvertedRecords> it = iterator(0);
+ java.util.Iterator<ConvertedRecords<?>> 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<ConvertedRecords> iterator(long maximumReadSize) {
+ public java.util.Iterator<ConvertedRecords<?>> 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<ConvertedRecords> {
+ private class Iterator extends AbstractIterator<ConvertedRecords<?>> {
private final AbstractIterator<? extends RecordBatch> 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 ad1f97f..62551d4 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<LazyDownCon
private RecordConversionStats recordConversionStats;
private RecordsSend convertedRecordsWriter;
- private Iterator<ConvertedRecords> convertedRecordsIterator;
+ private Iterator<ConvertedRecords<?>> convertedRecordsIterator;
public LazyDownConversionRecordsSend(String destination, LazyDownConversionRecords records) {
super(destination, records, records.sizeInBytes());
@@ -74,8 +74,8 @@ public final class LazyDownConversionRecordsSend extends RecordsSend<LazyDownCon
// Check if we have more chunks left to down-convert
if (convertedRecordsIterator.hasNext()) {
// Get next chunk of down-converted messages
- ConvertedRecords<MemoryRecords> 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 28923ab..ecf3ea9 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 <C> Cache<C> createCache(String mechanism, Class<C> credentialClass) {
Cache<C> cache = new Cache<>(credentialClass);
+ @SuppressWarnings("unchecked")
Cache<C> oldCache = (Cache<C>) 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 7bb7be7..3d9481c 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<? extends T> defaultClass) {
String prefix = jaasContext.type() == JaasContext.Type.SERVER ? ListenerName.saslMechanismPrefix(saslMechanism) : "";
+ @SuppressWarnings("unchecked")
Class<? extends T> clazz = (Class<? extends T>) 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 02a4261..78428ee 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 8b830c0..e141bb6 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<String, String> extensions = (Map<String, String>) 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 2d2ca25..0583f66 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 9bfc576..777ea2b 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 3489ddd..8abc93d 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 f94858c..9848f8e 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 <T> Set<T> mkSet(T... elems) {
- return new HashSet<>(Arrays.asList(elems));
- }
-
- /*
- * Creates a list
- * @param elems the elements
- * @param <T> the type of element
- * @return List
- */
- @SafeVarargs
- public static <T> List<T> mkList(T... elems) {
- return Arrays.asList(elems);
+ Set<T> 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 4c12fc6..ec1b062 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<TopicPartition, FetchRequest.PartitionData> expected,
Map<TopicPartition, FetchRequest.PartitionData>... actuals) {
for (Map<TopicPartition, FetchRequest.PartitionData> 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 1f62d39..82c5b1d 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 <T> void assertCollectionIs(Collection<T> 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 89fca84..987bad2 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<String, String> 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<String, String> 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<String, String> 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 b430078..1afe0ee 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<TopicPartition, OffsetCommitRequest.PartitionData> 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<ClientResponse, Object>() {
@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 c7b0b30..f4439ce 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<TopicPartition, FetchRequest.PartitionData> sessionPartitions =
- (LinkedHashMap<TopicPartition, FetchRequest.PartitionData>) field.get(handler);
- for (Map.Entry<TopicPartition, FetchRequest.PartitionData> 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 b544a65..872d390 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 6f9efca..37cfd1e 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<String> future = new KafkaFutureImpl<>();
- CompleterThread myThread = new CompleterThread(future, "You must construct additional pylons.");
+ CompleterThread<String> 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<Integer> future = new KafkaFutureImpl<>();
- KafkaFuture<Integer> doubledFuture = future.thenApply(new KafkaFuture.BaseFunction<Integer, Integer>() {
- @Override
- public Integer apply(Integer integer) {
- return 2 * integer;
- }
- });
+ KafkaFuture<Integer> doubledFuture = future.thenApply(integer -> 2 * integer);
assertFalse(doubledFuture.isDone());
- KafkaFuture<Integer> tripledFuture = future.thenApply(new KafkaFuture.Function<Integer, Integer>() {
- @Override
- public Integer apply(Integer integer) {
- return 3 * integer;
- }
- });
+ KafkaFuture<Integer> 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<Integer> quadrupledFuture = future.thenApply(new KafkaFuture.BaseFunction<Integer, Integer>() {
- @Override
- public Integer apply(Integer integer) {
- return 4 * integer;
- }
- });
+ KafkaFuture<Integer> quadrupledFuture = future.thenApply(integer -> 4 * integer);
assertEquals(Integer.valueOf(84), quadrupledFuture.getNow(-1));
KafkaFutureImpl<Integer> futureFail = new KafkaFutureImpl<>();
- KafkaFuture<Integer> futureAppliedFail = futureFail.thenApply(new KafkaFuture.BaseFunction<Integer, Integer>() {
- @Override
- public Integer apply(Integer integer) {
- return 2 * integer;
- }
- });
+ KafkaFuture<Integer> 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<KafkaFutureImpl<Integer>> futures = new ArrayList<>();
for (int i = 0; i < numThreads; i++) {
- futures.add(new KafkaFutureImpl<Integer>());
+ futures.add(new KafkaFutureImpl<>());
}
KafkaFuture<Void> allFuture = KafkaFuture.allOf(futures.toArray(new KafkaFuture[0]));
final List<CompleterThread> 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 affa5dd..974d39f 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<String> configsInHtml = new HashSet();
+ Set<String> 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 eb3f775..e70be27 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 630cba1..55fa998 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 1945bcc..20ecba1 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<ConvertedRecords> it = lazyRecords.iterator(16 * 1024L);
+ Iterator<ConvertedRecords<?>> 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<ConvertedRecords> it = lazyRecords.iterator(readSize);
+ Iterator<ConvertedRecords<?>> 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 e9b36a0..cdb6bb4 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<TopicPartition, OffsetCommitRequest.PartitionData> 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 16c35a8..f4b15e8 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<Class<Object>, List<Object>> testData = new HashMap() {
+ final private Map<Class<?>, List<Object>> testData = new HashMap<Class<?>, List<Object>>() {
{
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<Class<Object>, List<Object>> test : testData.entrySet()) {
- try (Serde<Object> serde = Serdes.serdeFrom(test.getKey())) {
+ for (Map.Entry<Class<?>, List<Object>> test : testData.entrySet()) {
+ try (Serde<Object> serde = Serdes.serdeFrom((Class<Object>) 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 ea53084..771bba0 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 5bc27db..1510a37 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 131a097..0af4aac 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<T extends Number> 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 10096a5..efcc01d 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<String> transformAliases = (List<String>) value;
@@ -247,14 +248,15 @@ public class ConnectorConfig extends AbstractConfig {
final List<Transformation<R>> transformations = new ArrayList<>(transformAliases.size());
for (String alias : transformAliases) {
final String prefix = TRANSFORMS_CONFIG + "." + alias + ".";
- final Transformation<R> transformation;
try {
- transformation = getClass(prefix + "type").asSubclass(Transformation.class).newInstance();
+ @SuppressWarnings("unchecked")
+ final Transformation<R> 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 df73a43..6e021b9 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 55d4860..b7fe74f 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 eadf276..2513514 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 6104dd4..460df39 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<PluginClassLoader>) () -> new PluginClassLoader(pluginLocation, urls, parent)
);
}
@@ -331,6 +326,7 @@ public class DelegatingClassLoader extends URLClassLoader {
return result;
}
+ @SuppressWarnings("unchecked")
private <T> Collection<PluginDesc<T>> getServiceLoaderPluginDesc(Class<T> klass, ClassLoader loader) {
ServiceLoader<T> serviceLoader = ServiceLoader.load(klass, loader);
Collection<PluginDesc<T>> 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 e766cbd..e7cb16d 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<String> paths) {
- return (DelegatingClassLoader) AccessController.doPrivileged(
- new PrivilegedAction() {
- @Override
- public Object run() {
- return new DelegatingClassLoader(paths);
- }
- }
+ return AccessController.doPrivileged(
+ (PrivilegedAction<DelegatingClassLoader>) () -> 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 c9c2c3b..0d5cbd6 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<ResourceConfig> {
}
@Override
- public ResourceConfig register(Object component, Map contracts) {
+ public ResourceConfig register(Object component, Map<Class<?>, Integer> contracts) {
if (allowedToRegister(component)) {
resourceConfig.register(component, contracts);
}
@@ -81,7 +81,7 @@ public class ConnectRestConfigurable implements Configurable<ResourceConfig> {
}
@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<ResourceConfig> {
}
@Override
- public ResourceConfig register(Class componentClass, Map contracts) {
+ public ResourceConfig register(Class<?> componentClass, Map<Class<?>, Integer> contracts) {
if (allowedToRegister(componentClass)) {
resourceConfig.register(componentClass, contracts);
}
@@ -97,7 +97,7 @@ public class ConnectRestConfigurable implements Configurable<ResourceConfig> {
}
@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<ResourceConfig> {
}
@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<ResourceConfig> {
}
@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<ResourceConfig> {
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 5a589db..1538643 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<ConnectRestExtension> connectRestExtensions = Collections.EMPTY_LIST;
+ private List<ConnectRestExtension> 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<String> parseListeners() {
List<String> 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 980d7c2..a50a0b0 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<String, Object> sslConfigValues) {
List<String> sslEnabledProtocols = (List<String>) 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 3239b67..c360d43 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<String, ?> partition, Map<String, ?> offset) {
data.put((Map<String, Object>) partition, (Map<String, Object>) 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 b65612a..fc557c6 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<String, String> config = new HashMap();
+ Map<String, String> 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 25395f9..fe15c01 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<String, String> 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 3f65029..64ea526 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<String, String> 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 c7cb08b..baf0d8e 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<ConnectorTaskId, ScheduledFuture<?>> 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<Runnable> 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 d0089e9..d49c1cd 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<SinkRecord> transformationChain;
private WorkerSinkTask workerTask;
@Mock private KafkaConsumer<byte[], byte[]> consumer;
private Capture<ConsumerRebalanceListener> 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<SinkRecord> recordCapture = EasyMock.newCapture();
- EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture))).andAnswer(new IAnswer<SinkRecord>() {
- @Override
- public SinkRecord answer() {
- return recordCapture.getValue();
- }
- }).anyTimes();
+ EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture))).andAnswer(
+ (IAnswer<SinkRecord>) () -> recordCapture.getValue()).anyTimes();
Capture<Collection<SinkRecord>> 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 4dd560a..f3cacc4 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<? extends Converter> 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 5f450b1..1217ef9 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 5c8aa29..1100910 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 2f8704a..c66ce36 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<String, String> baseWorkerProps() {
Map<String, String> 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<Callback<Collection<String>>> 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 f9b5ad7..a3aee6a 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<? extends Connector>) klass, "0.0.0"));
+ @SuppressWarnings("unchecked")
+ MockConnectorPluginDesc pluginDesc = new MockConnectorPluginDesc((Class<? extends Connector>) klass, "0.0.0");
+ CONNECTOR_PLUGINS.add(pluginDesc);
}
for (Class<?> klass : connectorClasses) {
- CONNECTOR_PLUGINS.add(new MockConnectorPluginDesc((Class<? extends Connector>) klass));
+ @SuppressWarnings("unchecked")
+ MockConnectorPluginDesc pluginDesc = new MockConnectorPluginDesc((Class<? extends Connector>) 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 b8b7114..63595d6 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<String, String> 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 c6b61b4..df955f8 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 8a02c1e..68c447a 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<String, byte[]>());
+ 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<String, byte[]>());
+ 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 36649a3..ff9f2c9 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<Callback<Void>> firstGetReadToEndCallback = EasyMock.newCapture();
storeLog.readToEnd(EasyMock.capture(firstGetReadToEndCallback));
- PowerMock.expectLastCall().andAnswer(new IAnswer<Object>() {
- @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<org.apache.kafka.clients.producer.Callback> callback0 = EasyMock.newCapture();
@@ -297,14 +294,11 @@ public class KafkaOffsetBackingStoreTest {
// Second get() should get the produced data and return the new values
final Capture<Callback<Void>> secondGetReadToEndCallback = EasyMock.newCapture();
storeLog.readToEnd(EasyMock.capture(secondGetReadToEndCallback));
- PowerMock.expectLastCall().andAnswer(new IAnswer<Object>() {
- @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 53ca0dc..1af6e34 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 ee08945..f071bda 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<R extends ConnectRecord<R>> 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<String>) value);
@@ -83,7 +84,7 @@ public abstract class ReplaceField<R extends ConnectRecord<R>> implements Transf
renames = parseRenameMappings(config.getList(ConfigName.RENAME));
reverseRenames = invert(renames);
- schemaUpdateCache = new SynchronizedCache<>(new LRUCache<Schema, Schema>(16));
+ schemaUpdateCache = new SynchronizedCache<>(new LRUCache<>(16));
}
static Map<String, String> parseRenameMappings(List<String> 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 d709054..b0549fb 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<String, Object> transformedMap = (Map<String, Object>) 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<String, Object> transformedMap = (Map<String, Object>) 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<String, Object> transformedMap = (Map<String, Object>) 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 9211a46..52849f9 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<String> maskFields = new ArrayList<>(value.keySet());
maskFields.remove("magic");
+ @SuppressWarnings("unchecked")
final Map<String, Object> 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 ad375d2..e4cde13 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 ae2ffb1..aa5dfec 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 e62e5a8..cd47969 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 <user, client>) config (if any) from zk
* sanitizedEntityName is <topic>, <broker>, <client-id>, <user> or <user>/clients/<client-id>.
*/
- @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 3ba6c1f..92cde7e 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 20c3de0..de88184 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 a52f3f0..101ff39 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 260b802..edec228 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 2cc719d..9d4eed6 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 87e6d13..b45953f 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 19e2f2f..688736c 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 8449e39..7fc70da 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 2fc7b74..e4559b8 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 8d3ac4d..10ae8df 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 60d397d..2cee83c 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 be6736d..f2e3e01 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 be50be1..17f340d 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 a909995..9df53f5 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 f2c8644..e49f3a5 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 eab1f2a..0b57fa7 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 8feb08b..9b48cb8 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 cbc82c0..ff62c6c 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 ca23e1f..ee69ae4 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 9ef7214..23f66e1 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 3e67b18..4d2e1a9 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 7a2394e..09a6188 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 2ab2b1a..29c7507 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 7fd68c2..854e338 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 f6cb757..04dcf28 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 f1eca54..08a0224 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 c36a3f1..795f954 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 42b3984..2aee15a 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 d2ff2a8..8ab32af 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 9402584..853f999 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 851ac51..95a4843 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 5c0ea2d..ae372e0 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 55fe587..7cbad05 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 44d5c5f..1311aa4 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 85159c3..84f3dff 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 74dae7c..b66324e 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 b18511e..822d3ea 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 4709282..d56a9f0 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 50449dc..04b3467 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 f2d3b4a..7b2fde3 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 5c37891..1d70539 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 86aae54..e69a5e6 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 47d45ad..c3e7312 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 292db8b..c0c3c6b 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 ec4eed2..b15f8ac 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 0088c65..fd3f59c 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 e7b3982..83d5c2b 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<String, String> 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 f8af99f..18b4912 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 25b10e8..d492238 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 f1a8754..3b69151 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<K, V> extends AbstractStream<K, V> implements KStream<K
@Override
@Deprecated
public <KR> KGroupedStream<KR, V> groupBy(final KeyValueMapper<? super K, ? super V, KR> selector,
- final Serialized<KR, V> serialized) {
+ final org.apache.kafka.streams.kstream.Serialized<KR, V> serialized) {
Objects.requireNonNull(selector, "selector can't be null");
Objects.requireNonNull(serialized, "serialized can't be null");
final SerializedInternal<KR, V> serializedInternal = new SerializedInternal<>(serialized);
@@ -832,7 +831,7 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
@Override
@Deprecated
- public KGroupedStream<K, V> groupByKey(final Serialized<K, V> serialized) {
+ public KGroupedStream<K, V> groupByKey(final org.apache.kafka.streams.kstream.Serialized<K, V> serialized) {
final SerializedInternal<K, V> 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 f49d109..9caac30 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<K, S, V> extends AbstractStream<K, V> implements KTable<
final boolean rightOuter,
final String joinMergeName,
final String internalQueryableName,
- final MaterializedInternal materializedInternal) {
+ final MaterializedInternal<K, R, KeyValueStore<Bytes, byte[]>> materializedInternal) {
final Set<String> allSourceNodes = ensureJoinableWith(other);
if (leftOuter) {
@@ -523,7 +522,7 @@ public class KTableImpl<K, S, V> extends AbstractStream<K, V> implements KTable<
final KTableKTableJoinMerger<K, R> joinMerge = new KTableKTableJoinMerger<>(joinThis, joinOther, internalQueryableName);
- final KTableKTableJoinNode.KTableKTableJoinNodeBuilder<K, Change<V>, Change<V1>, Change<R>> kTableJoinNodeBuilder = KTableKTableJoinNode.kTableKTableJoinNodeBuilder();
+ final KTableKTableJoinNode.KTableKTableJoinNodeBuilder<K, V, V1, R> kTableJoinNodeBuilder = KTableKTableJoinNode.kTableKTableJoinNodeBuilder();
// only materialize if specified in Materialized
if (materializedInternal != null) {
@@ -543,7 +542,7 @@ public class KTableImpl<K, S, V> extends AbstractStream<K, V> implements KTable<
.withOtherJoinSideNodeName(((KTableImpl) other).name)
.withThisJoinSideNodeName(name);
- final KTableKTableJoinNode<K, Change<V>, Change<V1>, Change<R>> kTableKTableJoinNode = kTableJoinNodeBuilder.build();
+ final KTableKTableJoinNode<K, V, V1, R> 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<K, S, V> extends AbstractStream<K, V> implements KTable<
@Override
@Deprecated
public <K1, V1> KGroupedTable<K1, V1> groupBy(final KeyValueMapper<? super K, ? super V, KeyValue<K1, V1>> selector,
- final Serialized<K1, V1> serialized) {
+ final org.apache.kafka.streams.kstream.Serialized<K1, V1> serialized) {
Objects.requireNonNull(selector, "selector can't be null");
Objects.requireNonNull(serialized, "serialized can't be null");
final SerializedInternal<K1, V1> 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 0cb7050..09fd9e5 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<K, V> extends Serialized<K, V> {
- public SerializedInternal(final Serialized<K, V> serialized) {
+public class SerializedInternal<K, V> extends org.apache.kafka.streams.kstream.Serialized<K, V> {
+ public SerializedInternal(final org.apache.kafka.streams.kstream.Serialized<K, V> 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 41c27ba..796468e 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<K, V1, V2, VR> extends BaseJoinProcessorNode<K, V1, V2, VR> {
+public class KTableKTableJoinNode<K, V1, V2, VR> extends BaseJoinProcessorNode<K, Change<V1>, Change<V2>, Change<VR>> {
private final String[] joinThisStoreNames;
private final String[] joinOtherStoreNames;
- private final MaterializedInternal materializedInternal;
+ private final MaterializedInternal<K, VR, KeyValueStore<Bytes, byte[]>> materializedInternal;
KTableKTableJoinNode(final String nodeName,
- final ValueJoiner<? super V1, ? super V2, ? extends VR> valueJoiner,
- final ProcessorParameters<K, V1> joinThisProcessorParameters,
- final ProcessorParameters<K, V2> joinOtherProcessorParameters,
- final ProcessorParameters<K, VR> joinMergeProcessorParameters,
- final MaterializedInternal materializedInternal,
+ final ValueJoiner<? super Change<V1>, ? super Change<V2>, ? extends Change<VR>> valueJoiner,
+ final ProcessorParameters<K, Change<V1>> joinThisProcessorParameters,
+ final ProcessorParameters<K, Change<V2>> joinOtherProcessorParameters,
+ final ProcessorParameters<K, Change<VR>> joinMergeProcessorParameters,
+ final MaterializedInternal<K, VR, KeyValueStore<Bytes, byte[]>> materializedInternal,
final String thisJoinSide,
final String otherJoinSide,
final String[] joinThisStoreNames,
@@ -107,12 +109,12 @@ public class KTableKTableJoinNode<K, V1, V2, VR> extends BaseJoinProcessorNode<K
private String nodeName;
private String[] joinThisStoreNames;
- private ProcessorParameters<K, V1> joinThisProcessorParameters;
+ private ProcessorParameters<K, Change<V1>> joinThisProcessorParameters;
private String[] joinOtherStoreNames;
- private MaterializedInternal materializedInternal;
- private ProcessorParameters<K, V2> joinOtherProcessorParameters;
- private ProcessorParameters<K, VR> joinMergeProcessorParameters;
- private ValueJoiner<? super V1, ? super V2, ? extends VR> valueJoiner;
+ private MaterializedInternal<K, VR, KeyValueStore<Bytes, byte[]>> materializedInternal;
+ private ProcessorParameters<K, Change<V2>> joinOtherProcessorParameters;
+ private ProcessorParameters<K, Change<VR>> joinMergeProcessorParameters;
+ private ValueJoiner<? super Change<V1>, ? super Change<V2>, ? extends Change<VR>> valueJoiner;
private String thisJoinSide;
private String otherJoinSide;
@@ -124,7 +126,7 @@ public class KTableKTableJoinNode<K, V1, V2, VR> extends BaseJoinProcessorNode<K
return this;
}
- public KTableKTableJoinNodeBuilder<K, V1, V2, VR> withJoinThisProcessorParameters(final ProcessorParameters<K, V1> joinThisProcessorParameters) {
+ public KTableKTableJoinNodeBuilder<K, V1, V2, VR> withJoinThisProcessorParameters(final ProcessorParameters<K, Change<V1>> joinThisProcessorParameters) {
this.joinThisProcessorParameters = joinThisProcessorParameters;
return this;
}
@@ -139,17 +141,17 @@ public class KTableKTableJoinNode<K, V1, V2, VR> extends BaseJoinProcessorNode<K
return this;
}
- public KTableKTableJoinNodeBuilder<K, V1, V2, VR> withJoinOtherProcessorParameters(final ProcessorParameters<K, V2> joinOtherProcessorParameters) {
+ public KTableKTableJoinNodeBuilder<K, V1, V2, VR> withJoinOtherProcessorParameters(final ProcessorParameters<K, Change<V2>> joinOtherProcessorParameters) {
this.joinOtherProcessorParameters = joinOtherProcessorParameters;
return this;
}
- public KTableKTableJoinNodeBuilder<K, V1, V2, VR> withJoinMergeProcessorParameters(final ProcessorParameters<K, VR> joinMergeProcessorParameters) {
+ public KTableKTableJoinNodeBuilder<K, V1, V2, VR> withJoinMergeProcessorParameters(final ProcessorParameters<K, Change<VR>> joinMergeProcessorParameters) {
this.joinMergeProcessorParameters = joinMergeProcessorParameters;
return this;
}
- public KTableKTableJoinNodeBuilder<K, V1, V2, VR> withValueJoiner(final ValueJoiner<? super V1, ? super V2, ? extends VR> valueJoiner) {
+ public KTableKTableJoinNodeBuilder<K, V1, V2, VR> withValueJoiner(final ValueJoiner<? super Change<V1>, ? super Change<V2>, ? extends Change<VR>> valueJoiner) {
this.valueJoiner = valueJoiner;
return this;
}
@@ -164,7 +166,8 @@ public class KTableKTableJoinNode<K, V1, V2, VR> extends BaseJoinProcessorNode<K
return this;
}
- public KTableKTableJoinNodeBuilder<K, V1, V2, VR> withMaterializedInternal(final MaterializedInternal materializedInternal) {
+ public KTableKTableJoinNodeBuilder<K, V1, V2, VR> withMaterializedInternal(
+ final MaterializedInternal<K, VR, KeyValueStore<Bytes, byte[]>> 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 05ec6fb..4797a21 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<K, V> extends BaseRepartitionNode {
+public class OptimizableRepartitionNode<K, V> extends BaseRepartitionNode<K, V> {
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 0f8ac2c..95076c8 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<K, V> extends StreamsGraphNode {
final String[] parentNames = parentNodeNames();
if (partitioner == null && keySerializer instanceof WindowedSerializer) {
+ @SuppressWarnings("unchecked")
final StreamPartitioner<K, V> windowedPartitioner = (StreamPartitioner<K, V>) new WindowedStreamPartitioner<Object, V>((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 1428123..0e6435a 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<K, V, S extends StateStore> 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<K, V, S extends StateStore> extends StreamsGraph
}
if (shouldMaterialize) {
- topologyBuilder.addStateStore(new KeyValueStoreMaterializer<>((MaterializedInternal<K, V, KeyValueStore<Bytes, byte[]>>) materializedInternal).materialize(), processorName);
+ topologyBuilder.addStateStore(new KeyValueStoreMaterializer<>(
+ (MaterializedInternal<K, V, KeyValueStore<Bytes, byte[]>>) 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 8576ee2..90fb5d4 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<K, V, S extends StateStore> extends StreamSourceNod
}
@Override
+ @SuppressWarnings("unchecked")
public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
final String topicName = getTopicNames().iterator().next();
@@ -129,7 +130,7 @@ public class TableSourceNode<K, V, S extends StateStore> extends StreamSourceNod
return this;
}
- public TableSourceNodeBuilder<K, V, S> withConsumedInternal(final ConsumedInternal consumedInternal) {
+ public TableSourceNodeBuilder<K, V, S> withConsumedInternal(final ConsumedInternal<K, V> 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 7fa2851..938ed91 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 50ce386..57792b6 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<K, V> extends StateStore, ReadOnlyWindowStore<K, V>
* @throws InvalidStateStoreException if the store is not initialized
* @throws NullPointerException If {@code null} is used for key.
*/
+ @SuppressWarnings("deprecation")
WindowStoreIterator<V> fetch(K key, long timeFrom, long timeTo);
@Override
@@ -110,6 +111,7 @@ public interface WindowStore<K, V> extends StateStore, ReadOnlyWindowStore<K, V>
* @throws InvalidStateStoreException if the store is not initialized
* @throws NullPointerException If {@code null} is used for any key.
*/
+ @SuppressWarnings("deprecation")
KeyValueIterator<Windowed<K>, V> fetch(K from, K to, long timeFrom, long timeTo);
@Override
@@ -128,6 +130,7 @@ public interface WindowStore<K, V> extends StateStore, ReadOnlyWindowStore<K, V>
* @throws InvalidStateStoreException if the store is not initialized
* @throws NullPointerException if {@code null} is used for any key
*/
+ @SuppressWarnings("deprecation")
KeyValueIterator<Windowed<K>, 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 b55e544..c04921b 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<K, V> extends WrappedStateStore.AbstractStateStore impl
}
}
+ @SuppressWarnings("deprecation")
@Override
public synchronized WindowStoreIterator<byte[]> 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<K, V> extends WrappedStateStore.AbstractStateStore impl
return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator);
}
+ @SuppressWarnings("deprecation")
@Override
public KeyValueIterator<Windowed<Bytes>, 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<K, V> extends WrappedStateStore.AbstractStateStore impl
cacheFunction
);
}
-
+
+ @SuppressWarnings("deprecation")
@Override
public KeyValueIterator<Windowed<Bytes>, 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 9808ca9..a592471 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<byte[]> fetch(final Bytes key, final long from, final long to) {
return bytesStore.fetch(key, from, to);
}
+ @SuppressWarnings("deprecation")
@Override
public KeyValueIterator<Windowed<Bytes>, 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<Windowed<Bytes>, byte[]> all() {
return bytesStore.all();
}
-
+
+ @SuppressWarnings("deprecation")
@Override
public KeyValueIterator<Windowed<Bytes>, 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 d95b442..84b589d 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<K, V> implements ReadOnlyWindowStore<K
return fetch(key, from.toEpochMilli(), to.toEpochMilli());
}
+ @SuppressWarnings("deprecation")
@Override
public KeyValueIterator<Windowed<K>, 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 5162eac..fefa772 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<K, V> extends WrappedStateStore.AbstractStateSto
}
}
+ @SuppressWarnings("deprecation")
@Override
public WindowStoreIterator<V> 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<K, V> extends WrappedStateStore.AbstractStateSto
return new MeteredWindowedKeyValueIterator<>(inner.all(), fetchTime, metrics, serdes, time);
}
+ @SuppressWarnings("deprecation")
@Override
public KeyValueIterator<Windowed<K>, V> fetchAll(final long timeFrom, final long timeTo) {
return new MeteredWindowedKeyValueIterator<>(inner.fetchAll(timeFrom, timeTo),
@@ -161,6 +163,7 @@ public class MeteredWindowStore<K, V> extends WrappedStateStore.AbstractStateSto
time);
}
+ @SuppressWarnings("deprecation")
@Override
public KeyValueIterator<Windowed<K>, 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 d7bb523..bb13c74 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<K, V> extends WrappedStateStore.AbstractStateSto
return serdes.valueFrom(bytesValue);
}
+ @SuppressWarnings("deprecation")
@Override
public WindowStoreIterator<V> fetch(final K key, final long timeFrom, final long timeTo) {
final KeyValueIterator<Bytes, byte[]> bytesIterator = bytesStore.fetch(Bytes.wrap(serdes.rawKey(key)), timeFrom, timeTo);
return new WindowStoreIteratorWrapper<>(bytesIterator, serdes, windowSize).valuesIterator();
}
+ @SuppressWarnings("deprecation")
@Override
public KeyValueIterator<Windowed<K>, V> fetch(final K from, final K to, final long timeFrom, final long timeTo) {
final KeyValueIterator<Bytes, byte[]> bytesIterator = bytesStore.fetch(Bytes.wrap(serdes.rawKey(from)), Bytes.wrap(serdes.rawKey(to)), timeFrom, timeTo);
@@ -103,6 +105,7 @@ public class RocksDBWindowStore<K, V> extends WrappedStateStore.AbstractStateSto
return new WindowStoreIteratorWrapper<>(bytesIterator, serdes, windowSize).keyValueIterator();
}
+ @SuppressWarnings("deprecation")
@Override
public KeyValueIterator<Windowed<K>, V> fetchAll(final long timeFrom, final long timeTo) {
final KeyValueIterator<Bytes, byte[]> 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 b9d542b..fde5bff 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<Node> nodes = Arrays.asList(new Node(0, "localhost", 8121));
+ final List<Node> nodes = asList(new Node(0, "localhost", 8121));
final Cluster cluster = new Cluster("mockClusterId", nodes,
Collections.emptySet(), Collections.<String>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 2531591..331323d 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 de856bb..7a58e1f 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<Object, Object> topicNameExtractor = new TopicNameExtractor<Object, Object>() {
@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 ac5a418..6539168 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<String, String> 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 08aa245..1db23a5 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<String> reducer;
private KStream<Integer, String> 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 7642f69..718483b 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 3fd8220..81938f8 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<Integer, Integer>) 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<Integer, Integer> consumer = new KafkaConsumer<>(consumerConfig);
final List<TopicPartition> 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 6239a68..3aff0a2 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<String, String> topicConfig) throws InterruptedException {
brokers[0].createTopic(topic, partitions, replication, topicConfig);
final List<TopicPartition> 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 6c2c464..badc63c 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<String, String> 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 1b3c1f1..7127c9f 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 872ae5c..3f43691 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<String, String, String> 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 6fc0cac..369bbad 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 b87a992..2bdb3a0 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 023e83f..b9cad4c 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 b360cec..f4ede7c 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 2bf6971..7756846 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.<String, Long, KeyValueStore<Bytes, byte[]>>as("my-table"));
builder.buildAndOptimizeTopology();
final Map<String, List<String>> 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 18f36aa..26b7e24 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<String, String> 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<String, String> 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 662ede7..c06cce4 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<String, String> table = builder.table(topic, Consumed.with(Serdes.String(), Serdes.String()));
- table.groupBy(MockMapper.<String, String>selectValueKeyValueMapper(),
- Serialized.with(Serdes.String(),
+ table.groupBy(MockMapper.selectValueKeyValueMapper(),
+ org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(),
Serdes.String()))
.count(Materialized.<String, Long, KeyValueStore<Bytes, byte[]>>as("count")
.withKeySerde(Serdes.String())
@@ -224,12 +223,12 @@ public class KGroupedTableImplTest {
}
}
- @SuppressWarnings("unchecked")
+ @SuppressWarnings({"unchecked", "deprecation"})
@Test
public void shouldAggregateAndMaterializeResults() {
final KTable<String, String> table = builder.table(topic, Consumed.with(Serdes.String(), Serdes.String()));
table.groupBy(MockMapper.<String, String>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 d033e49..1f3492c 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<String, String> 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<MockProcessor<String, String>> 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 74ad19c..0fb9a5d 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<Integer, Integer> 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 570053c..94d06eb 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<String, String> 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 1e39bd3..b82c8fe 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<String, String> 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 913710f..790c563 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<String, String> 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 b75b239..01b2609 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<String> stringSerde = Serdes.String();
private final Consumed<String, String> consumed = Consumed.with(stringSerde, stringSerde);
- private final Serialized<String, String> stringSerialzied = Serialized.with(stringSerde, stringSerde);
+ private final org.apache.kafka.streams.kstream.Serialized<String, String> stringSerialzied =
+ org.apache.kafka.streams.kstream.Serialized.with(stringSerde, stringSerde);
private final MockProcessorSupplier<String, Object> 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<String, Long> 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<Long>() {
@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 3e143f5..e717b08 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<String, Integer> 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 6e666c9..5747fed 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<String, String> stringConsumed = Consumed.with(Serdes.String(), Serdes.String());
@@ -128,13 +128,14 @@ public class KTableImplTest {
}
final List<MockProcessor<String, Object>> 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<String, String> 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 cd29b50..5ecbe83 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<Integer, String, KeyValueStore<Bytes, byte[]>> 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<Integer, String> getterSupplier = ((KTableImpl<Integer, String, String>) 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<String, Change<String>> join = new KTableKTableInnerJoin<>(
(KTableImpl<String, String, String>) builder.table("left", Consumed.with(stringSerde, stringSerde)),
(KTableImpl<String, String, String>) 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 9be6189..1cd360a 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<Integer> intSerde = Serdes.Integer();
final private Serde<String> 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<Integer, String> 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<Integer, String> getterSupplier = ((KTableImpl<Integer, String, String>) 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.<Long, String, KeyValueStore<Bytes, byte[]>>as("agg-store"));
@@ -417,6 +419,7 @@ public class KTableKTableLeftJoinTest {
public void shouldLogAndMeterSkippedRecordsDueToNullLeftKey() {
final StreamsBuilder builder = new StreamsBuilder();
+ @SuppressWarnings("unchecked")
final Processor<String, Change<String>> join = new KTableKTableLeftJoin<>(
(KTableImpl<String, String, String>) builder.table("left", Consumed.with(stringSerde, stringSerde)),
(KTableImpl<String, String, String>) 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 3995fcf..1500ba3 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<Integer> intSerde = Serdes.Integer();
final private Serde<String> 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<Integer, String> 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<Integer, String> getterSupplier = ((KTableImpl<Integer, String, String>) 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<String, Change<String>> join = new KTableKTableOuterJoin<>(
(KTableImpl<String, String, String>) builder.table("left", Consumed.with(stringSerde, stringSerde)),
(KTableImpl<String, String, String>) 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 d4805a2..8f234b6 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<String, Change<String>> join = new KTableKTableRightJoin<>(
(KTableImpl<String, String, String>) builder.table("left", Consumed.with(Serdes.String(), Serdes.String())),
(KTableImpl<String, String, String>) 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 0f56043..21ef038 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<String, String> 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 2055f9c..c9db537 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<String, String, String> table1 = (KTableImpl<String, String, String>) builder.table(topic1, stringConsumed);
final Topology topology = builder.build();
@@ -146,6 +147,7 @@ public class KTableSourceTest {
final String topic1 = "topic1";
+ @SuppressWarnings("unchecked")
final KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>) builder.table(topic1, stringConsumed);
final MockProcessorSupplier<String, Integer> supplier = new MockProcessorSupplier<>();
@@ -184,6 +186,7 @@ public class KTableSourceTest {
final String topic1 = "topic1";
+ @SuppressWarnings("unchecked")
final KTableImpl<String, String, String> table1 = (KTableImpl<String, String, String>) 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 dbcce90..82ada52 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<String, String> 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.<String, Integer, KeyValueStore<Bytes, byte[]>>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 9ee918c..749d5d6 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<String, String> stream;
+ @SuppressWarnings("deprecation")
@Before
public void before() {
final KStream<String, String> 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 f23bf0b..57b860e 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 f951ebb..c67b9e6 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<String, String> windowedStream;
+ @SuppressWarnings("deprecation")
@Before
public void before() {
final KStream<String, String> 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 1109f9d..de1a621 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<String, Object> 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.<String, Long, WindowStore<Bytes, byte[]>>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 d46f3d7..5781e67 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<Object, Object> 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 4ce9a9f..ee79e40 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 <K, V> void forward(final K key, final V value, final To to) {}
+ @SuppressWarnings("deprecation")
@Override
public <K, V> void forward(final K key, final V value, final int childIndex) {}
+ @SuppressWarnings("deprecation")
@Override
public <K, V> 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 32ac23e..8789e2b 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.<StateStore>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 f3e369f..53d374b 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.<StateStore>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 d49dd9d..1be5231 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<Object, Object>) 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 c4e58be..e637518 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<PartitionInfo> 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<PartitionInfo> 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 269983f..f209c7a 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<byte[], byte[]> 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<Object, Object> 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 4c7c3a6..452dd7b 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<String, String> sourceNode = new MockSourceNode<>(new String[]{""}, new TheExtendedDeserializer(), new TheExtendedDeserializer());
+ final SourceNode<String, String> 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<String, String> sourceNode = new MockSourceNode<>(new String[]{""}, new TheExtendedDeserializer(), new TheExtendedDeserializer());
+ final SourceNode<String, String> 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<String> {
+ public static class TheDeserializer implements Deserializer<String> {
@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 639ebf8..0c24e2d 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<TopicPartition> 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<TopicPartition> partition = Collections.singleton(partition2);
restoreStateConsumer.assign(partition);
- for (final ConsumerRecord<Integer, Integer> record : Arrays.asList(
+ for (final ConsumerRecord<Integer, Integer> 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<TopicPartition> partitions = mkList(topicPartition);
+ final List<TopicPartition> partitions = asList(topicPartition);
consumer.assign(partitions);
@@ -268,7 +267,7 @@ public class StandbyTaskTest {
final List<ConsumerRecord<byte[], byte[]>> 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<TopicPartition> partitions = mkList(topicPartition);
+ final List<TopicPartition> 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<ConsumerRecord<byte[], byte[]>> 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<TopicPartition, OffsetAndMetadata> 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<TopicPartition, OffsetAndMetadata> 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 ae48f57..34f0a32 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 7ff7c70..b042e3c 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 b4de5ec..11474a3 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<byte[], byte[]> 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<Node> cluster = Arrays.asList(broker1, broker2);
+ final List<Node> 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 5b1da16..bdc7a65 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<Object>) 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 649aa19..7c5abbe 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<String> allTopics = Utils.mkSet("topic1", "topic2");
- private final List<PartitionInfo> infos = Arrays.asList(
+ private final List<PartitionInfo> 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<TaskId> expectedSubList1 = Arrays.asList(taskIdA0, taskIdA3, taskIdB2);
- final List<TaskId> expectedSubList2 = Arrays.asList(taskIdA1, taskIdB0, taskIdC0);
- final List<TaskId> expectedSubList3 = Arrays.asList(taskIdA2, taskIdB1, taskIdC1);
- final List<List<TaskId>> embeddedList = Arrays.asList(expectedSubList1, expectedSubList2, expectedSubList3);
+ final List<TaskId> expectedSubList1 = asList(taskIdA0, taskIdA3, taskIdB2);
+ final List<TaskId> expectedSubList2 = asList(taskIdA1, taskIdB0, taskIdC0);
+ final List<TaskId> expectedSubList3 = asList(taskIdA2, taskIdB1, taskIdC1);
+ final List<List<TaskId>> embeddedList = asList(expectedSubList1, expectedSubList2, expectedSubList3);
- final List<TaskId> tasks = Arrays.asList(taskIdC0, taskIdC1, taskIdB0, taskIdB1, taskIdB2, taskIdA0, taskIdA1, taskIdA2, taskIdA3);
+ final List<TaskId> tasks = asList(taskIdC0, taskIdC1, taskIdB0, taskIdB1, taskIdB2, taskIdA0, taskIdA1, taskIdA2, taskIdA3);
Collections.shuffle(tasks);
final List<List<TaskId>> 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<TaskId> 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<String> topics = Utils.mkList("topic1", "topic2");
+ final List<String> topics = asList("topic1", "topic2");
final Set<TaskId> allTasks = Utils.mkSet(task0, task1, task2);
final Set<TaskId> 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<PartitionInfo> localInfos = Arrays.asList(
+ final List<PartitionInfo> 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.<String>emptySet(),
Collections.<String>emptySet());
- final List<String> topics = Utils.mkList("topic1", "topic2");
+ final List<String> 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<TaskId> expectedInfo10TaskIds = Arrays.asList(taskIdA1, taskIdA3, taskIdB1, taskIdB3);
+ final List<TaskId> expectedInfo10TaskIds = asList(taskIdA1, taskIdA3, taskIdB1, taskIdB3);
assertEquals(expectedInfo10TaskIds, info10.activeTasks());
// the second consumer
final AssignmentInfo info11 = AssignmentInfo.decode(assignments.get("consumer11").userData());
- final List<TaskId> expectedInfo11TaskIds = Arrays.asList(taskIdA0, taskIdA2, taskIdB0, taskIdB2);
+ final List<TaskId> 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<String> topics = Utils.mkList("topic1", "topic2");
+ final List<String> topics = asList("topic1", "topic2");
final Set<TaskId> 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<String> topics = Utils.mkList("topic1", "topic2");
+ final List<String> topics = asList("topic1", "topic2");
final Set<TaskId> allTasks = Utils.mkSet(task0, task1, task2);
final Set<TaskId> 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<String> topics = Utils.mkList("topic1", "topic2", "topic3");
+ final List<String> topics = asList("topic1", "topic2", "topic3");
final Set<TaskId> 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<String> topics = Utils.mkList("topic1", "topic2");
+ final List<String> 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<TaskId> tasks = Utils.mkList(task00, task01, task02, task10, task11, task12);
+ final List<TaskId> 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<String> topics = Utils.mkList("topic1", "topic2");
+ final List<String> topics = asList("topic1", "topic2");
final Set<TaskId> allTasks = Utils.mkSet(task0, task1, task2);
@@ -632,7 +633,7 @@ public class StreamsPartitionAssignorTest {
public void testOnAssignment() {
configurePartitionAssignor(Collections.<String, Object>emptyMap());
- final List<TaskId> activeTaskList = Utils.mkList(task0, task3);
+ final List<TaskId> activeTaskList = asList(task0, task3);
final Map<TaskId, Set<TopicPartition>> activeTasks = new HashMap<>();
final Map<TaskId, Set<TopicPartition>> standbyTasks = new HashMap<>();
final Map<HostInfo, Set<TopicPartition>> 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<Cluster> 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<String> topics = Utils.mkList("topic1", applicationId + "-topicX");
+ final List<String> topics = asList("topic1", applicationId + "-topicX");
final Set<TaskId> 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<String> topics = Utils.mkList("topic1", "test-topicX", "test-topicZ");
+ final List<String> topics = asList("topic1", "test-topicX", "test-topicZ");
final Set<TaskId> 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<TopicPartition> expectedAssignment = Arrays.asList(
+ final List<TopicPartition> 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<String> topics = Utils.mkList("topic1");
+ final List<String> topics = asList("topic1");
final UUID uuid1 = UUID.randomUUID();
@@ -1275,7 +1276,7 @@ public class StreamsPartitionAssignorTest {
standbyTaskMap,
Collections.<HostInfo, Set<TopicPartition>>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 b0e7fce..00af100 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<byte[], byte[]> consumer = (Consumer<byte[], byte[]>) EasyMock.createStrictMock(Consumer.class);
+ final Consumer<byte[], byte[]> 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 88b6997..56ff71d 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 7114731..44fa62f 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<String, String> store;
private int numRecordsProcessed;
+ @SuppressWarnings("unchecked")
@Override
public void init(final ProcessorContext processorContext) {
this.store = (WindowStore<String, String>) 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<KeyValue<Long, byte[]>> expected = mkList(
+ final List<KeyValue<Long, byte[]>> 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 117fd8f..5fdfd46 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<Bytes, byte[]> inner = new InMemoryKeyValueStore<>("kv", Serdes.Bytes(), Serdes.ByteArray());
private final ChangeLoggingKeyValueBytesStore store = new ChangeLoggingKeyValueBytesStore(inner);
- private final Map sent = new HashMap<>();
+ private final Map<Object, Object> 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 6d2d994..a36b101 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<Object, Object> sent = new HashMap<>();
private final NoOpRecordCollector collector = new NoOpRecordCollector() {
@Override
public <K, V> 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 08b6b23..4a0796d 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<Bytes, LRUCacheEntry> store = new InMemoryKeyValueStore("name", null, null);
private final KeyValue<Bytes, LRUCacheEntry> firstEntry = KeyValue.pair(Bytes.wrap("a".getBytes()),
new LRUCacheEntry("1".getBytes()));
- private final List<KeyValue<Bytes, LRUCacheEntry>> entries = Utils.mkList(
+ private final List<KeyValue<Bytes, LRUCacheEntry>> 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<KeyValue<Bytes, LRUCacheEntry>> 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 99abdc4..e2757a9 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<K, V> implements ReadOnlyWindowStore<K, V>,
}
}
+ @SuppressWarnings("deprecation")
@Override
public WindowStoreIterator<V> fetch(final K key, final long timeFrom, final long timeTo) {
if (!open) {
@@ -126,7 +127,8 @@ public class ReadOnlyWindowStoreStub<K, V> implements ReadOnlyWindowStore<K, V>,
}
};
}
-
+
+ @SuppressWarnings("deprecation")
@Override
public KeyValueIterator<Windowed<K>, V> fetchAll(final long timeFrom, final long timeTo) {
if (!open) {
@@ -180,6 +182,7 @@ public class ReadOnlyWindowStoreStub<K, V> implements ReadOnlyWindowStore<K, V>,
return fetchAll(from.toEpochMilli(), to.toEpochMilli());
}
+ @SuppressWarnings("deprecation")
@Override
public KeyValueIterator<Windowed<K>, 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 c41b094..09c5e5b 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<Windowed<Integer>, 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<Windowed<Integer>, 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<Windowed<Integer>, 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<String> expected = Utils.mkList(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L));
+ final List<String> expected = asList(segments.segmentName(4L), segments.segmentName(5L), segments.segmentName(6L));
expected.sort(String::compareTo);
final List<String> 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<KeyValue<Windowed<String>, 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<String> 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<Integer, Set<String>> entriesByKey(final List<KeyValue<byte[], byte[]>> 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 4f1e181..a7a64c4 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 6f801c9..623c3e3 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.<String, String>stream(SOURCE_TOPIC).groupByKey(Serialized.with(stringSerde, stringSerde))
+ builder.<String, String>stream(SOURCE_TOPIC).groupByKey(org.apache.kafka.streams.kstream.Serialized.with(stringSerde, stringSerde))
.count()
.toStream()
.mapValues(new ValueMapper<Long, String>() {
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 ddff7a8..ef908f4 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<String, Integer> stringIntConsumed = Consumed.with(stringSerde, intSerde);
@@ -126,7 +126,7 @@ public class SmokeTestClient extends SmokeTestUtil {
// min
final KGroupedStream<String, Integer> 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.<String, Long>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 2f356bf..3b332b5 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 <K, V> void forward(final K key, final V value, final int childIndex) {
forward(key, value, To.child(((List<ProcessorNode>) currentNode().children()).get(childIndex).name()));
}
@Override
- @SuppressWarnings("unchecked")
+ @SuppressWarnings({"unchecked", "deprecation"})
public <K, V> 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 36d049c..2282084 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<Object, Object> 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 <K, V> void forward(final K key, final V value, final int childIndex) {
forward(key, value);
}
+ @SuppressWarnings("deprecation")
@Override
public <K, V> 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 54b1e1f..e30c8f3 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 ba52867..7fdfdbe 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<String, Long> processor = new AbstractProcessor<String, Long>() {
@Override
public void process(final String key, final Long value) {
- //noinspection unchecked
+ @SuppressWarnings("unchecked")
final KeyValueStore<String, Long> stateStore = (KeyValueStore<String, Long>) 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 58f6e02..bead079 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<String, Long>() {
private KeyValueStore<String, Long> store;
+ @SuppressWarnings("unchecked")
@Override
public void init(final ProcessorContext context) {
- //noinspection unchecked
this.store = (KeyValueStore<String, Long>) 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 a9537a7..6182744 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<byte[], byte[]> records = consumer.poll(100);
+ ConsumerRecords<byte[], byte[]> 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 e7809cd..5e6ff81 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 <T> void verify(T val1) throws Exception {
byte[] bytes = JsonUtil.JSON_SERDE.writeValueAsBytes(val1);
+ @SuppressWarnings("unchecked")
Class<T> clazz = (Class<T>) val1.getClass();
T val2 = JsonUtil.JSON_SERDE.readValue(bytes, clazz);
for (Field field : clazz.getDeclaredFields()) {