Browse Source

KAFKA-12499: add transaction timeout verification (#10482)

This PR tries to add the check for transaction timeout for a comparison against commit interval of streams. If transaction timeout is smaller than commit interval, stream should crash and inform user to update their commit interval to be larger or equal to the given transaction timeout, or vise versa.

Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>, A. Sophie Blee-Goldman <sophie@confluent.io>
pull/10698/head
Boyang Chen 4 years ago committed by GitHub
parent
commit
ae8b784537
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 2
      streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java
  2. 24
      streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
  3. 27
      streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
  4. 2
      streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java
  5. 2
      streams/src/test/java/org/apache/kafka/streams/integration/EmitOnChangeIntegrationTest.java
  6. 11
      streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java
  7. 5
      streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java
  8. 4
      streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java
  9. 2
      streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java
  10. 2
      streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java
  11. 2
      streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java
  12. 2
      streams/src/test/java/org/apache/kafka/streams/integration/HighAvailabilityTaskAssignorIntegrationTest.java
  13. 2
      streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java
  14. 2
      streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
  15. 2
      streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionIntegrationTest.java
  16. 2
      streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java
  17. 2
      streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java
  18. 2
      streams/src/test/java/org/apache/kafka/streams/integration/LagFetchIntegrationTest.java
  19. 2
      streams/src/test/java/org/apache/kafka/streams/integration/OptimizedKTableIntegrationTest.java
  20. 2
      streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java
  21. 2
      streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java
  22. 2
      streams/src/test/java/org/apache/kafka/streams/integration/ResetPartitionTimeIntegrationTest.java
  23. 2
      streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java
  24. 2
      streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java
  25. 2
      streams/src/test/java/org/apache/kafka/streams/integration/StoreQueryIntegrationTest.java
  26. 2
      streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java
  27. 2
      streams/src/test/java/org/apache/kafka/streams/integration/StreamTableJoinTopologyOptimizationIntegrationTest.java
  28. 7
      streams/src/test/java/org/apache/kafka/streams/integration/SuppressionDurabilityIntegrationTest.java
  29. 2
      streams/src/test/java/org/apache/kafka/streams/integration/SuppressionIntegrationTest.java
  30. 2
      streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java
  31. 2
      streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionWithMergeOptimizingTest.java
  32. 2
      streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
  33. 2
      streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java
  34. 2
      streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java
  35. 2
      streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java
  36. 6
      streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java
  37. 2
      streams/src/test/java/org/apache/kafka/streams/tests/StaticMemberTestClient.java
  38. 2
      streams/src/test/java/org/apache/kafka/streams/tests/StreamsBrokerDownResilienceTest.java
  39. 2
      streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java
  40. 2
      streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
  41. 2
      streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java
  42. 2
      streams/upgrade-system-tests-0100/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
  43. 2
      streams/upgrade-system-tests-0100/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java
  44. 2
      streams/upgrade-system-tests-0101/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
  45. 2
      streams/upgrade-system-tests-0101/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java
  46. 2
      streams/upgrade-system-tests-0102/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
  47. 2
      streams/upgrade-system-tests-0102/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java
  48. 2
      streams/upgrade-system-tests-0110/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
  49. 2
      streams/upgrade-system-tests-0110/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java
  50. 2
      streams/upgrade-system-tests-10/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
  51. 2
      streams/upgrade-system-tests-10/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java
  52. 2
      streams/upgrade-system-tests-11/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
  53. 2
      streams/upgrade-system-tests-11/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java
  54. 2
      streams/upgrade-system-tests-20/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
  55. 2
      streams/upgrade-system-tests-20/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java
  56. 2
      streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
  57. 2
      streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java
  58. 2
      streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
  59. 2
      streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java
  60. 2
      streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
  61. 2
      streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java
  62. 2
      streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
  63. 2
      streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
  64. 2
      streams/upgrade-system-tests-26/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
  65. 2
      streams/upgrade-system-tests-27/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java

2
streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java

@ -180,7 +180,7 @@ public class PageViewTypedDemo { @@ -180,7 +180,7 @@ public class PageViewTypedDemo {
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, JSONSerde.class);
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, JSONSerde.class);
props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
// setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data
props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");

24
streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java

@ -58,6 +58,7 @@ import static org.apache.kafka.common.IsolationLevel.READ_COMMITTED; @@ -58,6 +58,7 @@ import static org.apache.kafka.common.IsolationLevel.READ_COMMITTED;
import static org.apache.kafka.common.config.ConfigDef.Range.atLeast;
import static org.apache.kafka.common.config.ConfigDef.Range.between;
import static org.apache.kafka.common.config.ConfigDef.ValidString.in;
import static org.apache.kafka.common.config.ConfigDef.parseType;
/**
* Configuration for a {@link KafkaStreams} instance.
@ -144,6 +145,7 @@ public class StreamsConfig extends AbstractConfig { @@ -144,6 +145,7 @@ public class StreamsConfig extends AbstractConfig {
private final boolean eosEnabled;
private static final long DEFAULT_COMMIT_INTERVAL_MS = 30000L;
private static final long EOS_DEFAULT_COMMIT_INTERVAL_MS = 100L;
private static final int DEFAULT_TRANSACTION_TIMEOUT = 10000;
public static final int DUMMY_THREAD_INDEX = 1;
public static final long MAX_TASK_IDLE_MS_DISABLED = -1;
@ -907,7 +909,7 @@ public class StreamsConfig extends AbstractConfig { @@ -907,7 +909,7 @@ public class StreamsConfig extends AbstractConfig {
tempProducerDefaultOverrides.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, Integer.MAX_VALUE);
tempProducerDefaultOverrides.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, true);
// Reduce the transaction timeout for quicker pending offset expiration on broker side.
tempProducerDefaultOverrides.put(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, 10000);
tempProducerDefaultOverrides.put(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, DEFAULT_TRANSACTION_TIMEOUT);
PRODUCER_EOS_OVERRIDES = Collections.unmodifiableMap(tempProducerDefaultOverrides);
}
@ -1078,6 +1080,26 @@ public class StreamsConfig extends AbstractConfig { @@ -1078,6 +1080,26 @@ public class StreamsConfig extends AbstractConfig {
if (props.containsKey(RETRIES_CONFIG)) {
log.warn("Configuration parameter `{}` is deprecated and will be removed in the 4.0.0 release.", RETRIES_CONFIG);
}
if (eosEnabled) {
verifyEOSTransactionTimeoutCompatibility();
}
}
private void verifyEOSTransactionTimeoutCompatibility() {
final long commitInterval = getLong(COMMIT_INTERVAL_MS_CONFIG);
final String transactionTimeoutConfigKey = producerPrefix(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG);
final int transactionTimeout = originals().containsKey(transactionTimeoutConfigKey) ? (int) parseType(
transactionTimeoutConfigKey, originals().get(transactionTimeoutConfigKey), Type.INT) : DEFAULT_TRANSACTION_TIMEOUT;
if (transactionTimeout < commitInterval) {
throw new IllegalArgumentException(String.format("Transaction timeout %d was set lower than " +
"streams commit interval %d. This will cause ongoing transaction always timeout due to inactivity " +
"caused by long commit interval. Consider reconfiguring commit interval to match " +
"transaction timeout by tuning 'commit.interval.ms' config, or increase the transaction timeout to match " +
"commit interval by tuning `producer.transaction.timeout.ms` config.",
transactionTimeout, commitInterval));
}
}
@Override

27
streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java

@ -47,6 +47,7 @@ import java.util.Properties; @@ -47,6 +47,7 @@ import java.util.Properties;
import static org.apache.kafka.common.IsolationLevel.READ_COMMITTED;
import static org.apache.kafka.common.IsolationLevel.READ_UNCOMMITTED;
import static org.apache.kafka.streams.StreamsConfig.AT_LEAST_ONCE;
import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE;
import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE_BETA;
import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE_V2;
@ -398,6 +399,32 @@ public class StreamsConfigTest { @@ -398,6 +399,32 @@ public class StreamsConfigTest {
assertEquals("30000", producerConfigs.get(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG));
}
@SuppressWarnings("deprecation")
@Test
public void shouldThrowIfTransactionTimeoutSmallerThanCommitIntervalForEOSAlpha() {
assertThrows(IllegalArgumentException.class,
() -> testTransactionTimeoutSmallerThanCommitInterval(EXACTLY_ONCE));
}
@SuppressWarnings("deprecation")
@Test
public void shouldThrowIfTransactionTimeoutSmallerThanCommitIntervalForEOSBeta() {
assertThrows(IllegalArgumentException.class,
() -> testTransactionTimeoutSmallerThanCommitInterval(EXACTLY_ONCE_BETA));
}
@Test
public void shouldNotThrowIfTransactionTimeoutSmallerThanCommitIntervalForAtLeastOnce() {
testTransactionTimeoutSmallerThanCommitInterval(AT_LEAST_ONCE);
}
private void testTransactionTimeoutSmallerThanCommitInterval(final String processingGuarantee) {
props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, processingGuarantee);
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10000L);
props.put(StreamsConfig.producerPrefix(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG), 3000);
new StreamsConfig(props);
}
@Test
public void shouldOverrideStreamsDefaultConsumerConifgsOnRestoreConsumer() {
props.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_RECORDS_CONFIG), "10");

2
streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java

@ -148,7 +148,7 @@ public abstract class AbstractResetIntegrationTest { @@ -148,7 +148,7 @@ public abstract class AbstractResetIntegrationTest {
streamsConfig.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Long().getClass());
streamsConfig.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
streamsConfig.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
streamsConfig.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
streamsConfig.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
streamsConfig.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 100);
streamsConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
streamsConfig.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, Integer.toString(STREAMS_CONSUMER_TIMEOUT));

2
streams/src/test/java/org/apache/kafka/streams/integration/EmitOnChangeIntegrationTest.java

@ -90,7 +90,7 @@ public class EmitOnChangeIntegrationTest { @@ -90,7 +90,7 @@ public class EmitOnChangeIntegrationTest {
mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()),
mkEntry(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1),
mkEntry(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0),
mkEntry(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 300000),
mkEntry(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 300000L),
mkEntry(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class),
mkEntry(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class)
)

11
streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java

@ -236,7 +236,7 @@ public class EosIntegrationTest { @@ -236,7 +236,7 @@ public class EosIntegrationTest {
final Properties properties = new Properties();
properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig);
properties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_RECORDS_CONFIG), 1);
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.METADATA_MAX_AGE_CONFIG), "1000");
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG), "earliest");
@ -325,7 +325,7 @@ public class EosIntegrationTest { @@ -325,7 +325,7 @@ public class EosIntegrationTest {
final Properties properties = new Properties();
properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig);
properties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
properties.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
@ -874,9 +874,14 @@ public class EosIntegrationTest { @@ -874,9 +874,14 @@ public class EosIntegrationTest {
.to(SINGLE_PARTITION_OUTPUT_TOPIC);
final Properties properties = new Properties();
// Set commit interval to a larger value to avoid affection of controlled stream commit,
// but not too large as we need to have a relatively low transaction timeout such
// that it should help trigger the timed out transaction in time.
final long commitIntervalMs = 20_000L;
properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig);
properties.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, numberOfStreamsThreads);
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Long.MAX_VALUE);
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, commitIntervalMs);
properties.put(StreamsConfig.producerPrefix(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG), (int) commitIntervalMs);
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.METADATA_MAX_AGE_CONFIG), "1000");
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG), "earliest");
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG), 5 * 1000);

5
streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java

@ -936,13 +936,14 @@ public class EosV2UpgradeIntegrationTest { @@ -936,13 +936,14 @@ public class EosV2UpgradeIntegrationTest {
final Properties properties = new Properties();
properties.put(StreamsConfig.CLIENT_ID_CONFIG, appDir);
properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, processingGuarantee);
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Long.MAX_VALUE);
final long commitInterval = Duration.ofMinutes(1L).toMillis();
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, commitInterval);
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.METADATA_MAX_AGE_CONFIG), Duration.ofSeconds(1L).toMillis());
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG), "earliest");
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG), (int) Duration.ofSeconds(5L).toMillis());
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), (int) Duration.ofSeconds(5L).minusMillis(1L).toMillis());
properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), MAX_POLL_INTERVAL_MS);
properties.put(StreamsConfig.producerPrefix(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG), (int) Duration.ofMinutes(1L).toMillis());
properties.put(StreamsConfig.producerPrefix(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG), (int) commitInterval);
properties.put(StreamsConfig.producerPrefix(ProducerConfig.PARTITIONER_CLASS_CONFIG), KeyPartitioner.class);
properties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
properties.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath() + File.separator + appDir);

4
streams/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java

@ -145,7 +145,7 @@ public class FineGrainedAutoResetIntegrationTest { @@ -145,7 +145,7 @@ public class FineGrainedAutoResetIntegrationTest {
final Properties props = new Properties();
props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
props.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
@ -284,7 +284,7 @@ public class FineGrainedAutoResetIntegrationTest { @@ -284,7 +284,7 @@ public class FineGrainedAutoResetIntegrationTest {
public void shouldThrowStreamsExceptionNoResetSpecified() throws InterruptedException {
final Properties props = new Properties();
props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
props.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none");

2
streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java

@ -126,7 +126,7 @@ public class GlobalKTableEOSIntegrationTest { @@ -126,7 +126,7 @@ public class GlobalKTableEOSIntegrationTest {
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
streamsConfiguration.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig);
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
streamsConfiguration.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);

2
streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java

@ -110,7 +110,7 @@ public class GlobalKTableIntegrationTest { @@ -110,7 +110,7 @@ public class GlobalKTableIntegrationTest {
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
globalTable = builder.globalTable(globalTableTopic, Consumed.with(Serdes.Long(), Serdes.String()),
Materialized.<Long, String, KeyValueStore<Bytes, byte[]>>as(globalStore)
.withKeySerde(Serdes.Long())

2
streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java

@ -119,7 +119,7 @@ public class GlobalThreadShutDownOrderTest { @@ -119,7 +119,7 @@ public class GlobalThreadShutDownOrderTest {
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
final Consumed<String, Long> stringLongConsumed = Consumed.with(Serdes.String(), Serdes.Long());

2
streams/src/test/java/org/apache/kafka/streams/integration/HighAvailabilityTaskAssignorIntegrationTest.java

@ -297,7 +297,7 @@ public class HighAvailabilityTaskAssignorIntegrationTest { @@ -297,7 +297,7 @@ public class HighAvailabilityTaskAssignorIntegrationTest {
mkEntry(StreamsConfig.MAX_WARMUP_REPLICAS_CONFIG, "2"),
mkEntry(StreamsConfig.PROBING_REBALANCE_INTERVAL_MS_CONFIG, "60000"),
mkEntry(StreamsConfig.InternalConfig.ASSIGNMENT_LISTENER, configuredAssignmentListener),
mkEntry(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, "100"),
mkEntry(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L),
mkEntry(StreamsConfig.InternalConfig.INTERNAL_TASK_ASSIGNOR_CLASS, HighAvailabilityTaskAssignor.class.getName()),
// Increasing the number of threads to ensure that a rebalance happens each time a consumer sends a rejoin (KAFKA-10455)
mkEntry(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 40)

2
streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java

@ -102,7 +102,7 @@ public class InternalTopicIntegrationTest { @@ -102,7 +102,7 @@ public class InternalTopicIntegrationTest {
streamsProp.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
streamsProp.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
streamsProp.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
streamsProp.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
streamsProp.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
streamsProp.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
streamsProp.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
}

2
streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java

@ -143,7 +143,7 @@ public class KStreamAggregationIntegrationTest { @@ -143,7 +143,7 @@ public class KStreamAggregationIntegrationTest {
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());

2
streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionIntegrationTest.java

@ -138,7 +138,7 @@ public class KStreamRepartitionIntegrationTest { @@ -138,7 +138,7 @@ public class KStreamRepartitionIntegrationTest {
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
streamsConfiguration.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, topologyOptimization);

2
streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java

@ -210,7 +210,7 @@ public class KTableKTableForeignKeyInnerJoinMultiIntegrationTest { @@ -210,7 +210,7 @@ public class KTableKTableForeignKeyInnerJoinMultiIntegrationTest {
streamsConfig.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
streamsConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
streamsConfig.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
streamsConfig.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
streamsConfig.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
// increase the session timeout value, to avoid unnecessary rebalance
streamsConfig.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 20000);
return streamsConfig;

2
streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java

@ -70,7 +70,7 @@ public class KTableSourceTopicRestartIntegrationTest { @@ -70,7 +70,7 @@ public class KTableSourceTopicRestartIntegrationTest {
STREAMS_CONFIG.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
STREAMS_CONFIG.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
STREAMS_CONFIG.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
STREAMS_CONFIG.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 5);
STREAMS_CONFIG.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 5L);
STREAMS_CONFIG.put(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, WallclockTimestampExtractor.class);
STREAMS_CONFIG.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);
STREAMS_CONFIG.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 300);

2
streams/src/test/java/org/apache/kafka/streams/integration/LagFetchIntegrationTest.java

@ -116,7 +116,7 @@ public class LagFetchIntegrationTest { @@ -116,7 +116,7 @@ public class LagFetchIntegrationTest {
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
consumerConfiguration = new Properties();
consumerConfiguration.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());

2
streams/src/test/java/org/apache/kafka/streams/integration/OptimizedKTableIntegrationTest.java

@ -197,7 +197,7 @@ public class OptimizedKTableIntegrationTest { @@ -197,7 +197,7 @@ public class OptimizedKTableIntegrationTest {
config.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
config.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
config.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
config.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
config.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 100);
config.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 200);

2
streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java

@ -226,7 +226,7 @@ public class QueryableStateIntegrationTest { @@ -226,7 +226,7 @@ public class QueryableStateIntegrationTest {
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
stringComparator = Comparator.comparing((KeyValue<String, String> o) -> o.key).thenComparing(o -> o.value);

2
streams/src/test/java/org/apache/kafka/streams/integration/RegexSourceIntegrationTest.java

@ -130,7 +130,7 @@ public class RegexSourceIntegrationTest { @@ -130,7 +130,7 @@ public class RegexSourceIntegrationTest {
outputTopic = createTopic(topicSuffixGenerator.incrementAndGet());
final Properties properties = new Properties();
properties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
properties.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
properties.put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 0L);

2
streams/src/test/java/org/apache/kafka/streams/integration/ResetPartitionTimeIntegrationTest.java

@ -125,7 +125,7 @@ public class ResetPartitionTimeIntegrationTest { @@ -125,7 +125,7 @@ public class ResetPartitionTimeIntegrationTest {
streamsConfig.put(StreamsConfig.APPLICATION_ID_CONFIG, appId);
streamsConfig.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
streamsConfig.put(StreamsConfig.POLL_MS_CONFIG, Integer.toString(DEFAULT_TIMEOUT));
streamsConfig.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Integer.toString(DEFAULT_TIMEOUT));
streamsConfig.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, (long) DEFAULT_TIMEOUT);
streamsConfig.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, processingGuarantee);
streamsConfig.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());

2
streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java

@ -126,7 +126,7 @@ public class RestoreIntegrationTest { @@ -126,7 +126,7 @@ public class RestoreIntegrationTest {
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(appId).getPath());
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
return streamsConfiguration;
}

2
streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java

@ -406,7 +406,7 @@ public class StandbyTaskEOSIntegrationTest { @@ -406,7 +406,7 @@ public class StandbyTaskEOSIntegrationTest {
streamsConfiguration.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, eosConfig);
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
// need to set to zero to get predictable active/standby task assignments
streamsConfiguration.put(StreamsConfig.ACCEPTABLE_RECOVERY_LAG_CONFIG, 0);
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");

2
streams/src/test/java/org/apache/kafka/streams/integration/StoreQueryIntegrationTest.java

@ -462,7 +462,7 @@ public class StoreQueryIntegrationTest { @@ -462,7 +462,7 @@ public class StoreQueryIntegrationTest {
config.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 100);
config.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 200);
config.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
return config;
}
}

2
streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java

@ -97,7 +97,7 @@ public class StoreUpgradeIntegrationTest { @@ -97,7 +97,7 @@ public class StoreUpgradeIntegrationTest {
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
return streamsConfiguration;
}

2
streams/src/test/java/org/apache/kafka/streams/integration/StreamTableJoinTopologyOptimizationIntegrationTest.java

@ -122,7 +122,7 @@ public class StreamTableJoinTopologyOptimizationIntegrationTest { @@ -122,7 +122,7 @@ public class StreamTableJoinTopologyOptimizationIntegrationTest {
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers());
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
streamsConfiguration.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, topologyOptimization);

7
streams/src/test/java/org/apache/kafka/streams/integration/SuppressionDurabilityIntegrationTest.java

@ -109,7 +109,7 @@ public class SuppressionDurabilityIntegrationTest { @@ -109,7 +109,7 @@ public class SuppressionDurabilityIntegrationTest {
private static final StringSerializer STRING_SERIALIZER = new StringSerializer();
private static final Serde<String> STRING_SERDE = Serdes.String();
private static final LongDeserializer LONG_DESERIALIZER = new LongDeserializer();
private static final int COMMIT_INTERVAL = 100;
private static final long COMMIT_INTERVAL = 100L;
@SuppressWarnings("deprecation")
@Parameterized.Parameters(name = "{0}")
@ -167,12 +167,13 @@ public class SuppressionDurabilityIntegrationTest { @@ -167,12 +167,13 @@ public class SuppressionDurabilityIntegrationTest {
final Properties streamsConfig = mkProperties(mkMap(
mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId),
mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()),
mkEntry(StreamsConfig.POLL_MS_CONFIG, Integer.toString(COMMIT_INTERVAL)),
mkEntry(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Integer.toString(COMMIT_INTERVAL)),
mkEntry(StreamsConfig.POLL_MS_CONFIG, Long.toString(COMMIT_INTERVAL)),
mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, processingGuaranteee),
mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath())
));
streamsConfig.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, COMMIT_INTERVAL);
KafkaStreams driver = getStartedStreams(streamsConfig, builder, true);
try {
// start by putting some stuff in the buffer

2
streams/src/test/java/org/apache/kafka/streams/integration/SuppressionIntegrationTest.java

@ -512,7 +512,7 @@ public class SuppressionIntegrationTest { @@ -512,7 +512,7 @@ public class SuppressionIntegrationTest {
mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId),
mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()),
mkEntry(StreamsConfig.POLL_MS_CONFIG, Integer.toString(COMMIT_INTERVAL)),
mkEntry(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Integer.toString(COMMIT_INTERVAL)),
mkEntry(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Long.toString(COMMIT_INTERVAL)),
mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, AT_LEAST_ONCE),
mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath())
));

2
streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java

@ -109,7 +109,7 @@ public class RepartitionOptimizingTest { @@ -109,7 +109,7 @@ public class RepartitionOptimizingTest {
public void setUp() {
streamsConfiguration = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
streamsConfiguration.setProperty(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, Integer.toString(1024 * 10));
streamsConfiguration.setProperty(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Integer.toString(5000));
streamsConfiguration.setProperty(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Long.toString(5000));
processorValueCollector.clear();
}

2
streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionWithMergeOptimizingTest.java

@ -87,7 +87,7 @@ public class RepartitionWithMergeOptimizingTest { @@ -87,7 +87,7 @@ public class RepartitionWithMergeOptimizingTest {
public void setUp() {
streamsConfiguration = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
streamsConfiguration.setProperty(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, Integer.toString(1024 * 10));
streamsConfiguration.setProperty(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Integer.toString(5000));
streamsConfiguration.setProperty(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, Long.toString(5000));
}
@After

2
streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java

@ -1388,7 +1388,7 @@ public class StreamThreadTest { @@ -1388,7 +1388,7 @@ public class StreamThreadTest {
consumer.assign(new HashSet<>(assignedPartitions));
consumer.addRecord(new ConsumerRecord<>(topic1, 1, 0, new byte[0], new byte[0]));
mockTime.sleep(config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG) + 1);
mockTime.sleep(config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG) + 1L);
thread.runOnce();
assertThat(producer.history().size(), equalTo(1));

2
streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java

@ -201,7 +201,7 @@ public class CachingPersistentWindowStoreTest { @@ -201,7 +201,7 @@ public class CachingPersistentWindowStoreTest {
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10 * 1000);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10 * 1000L);
final Instant initialWallClockTime = Instant.ofEpochMilli(0L);
final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), streamsConfiguration, initialWallClockTime);

2
streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java

@ -71,7 +71,7 @@ public class BrokerCompatibilityTest { @@ -71,7 +71,7 @@ public class BrokerCompatibilityTest {
streamsProperties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
streamsProperties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
streamsProperties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
streamsProperties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
streamsProperties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
streamsProperties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
streamsProperties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, processingMode);
final int timeout = 6000;

2
streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java

@ -105,7 +105,7 @@ public class EosTestClient extends SmokeTestUtil { @@ -105,7 +105,7 @@ public class EosTestClient extends SmokeTestUtil {
props.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 2);
props.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 3);
props.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 5000); // increase commit interval to make sure a client is killed having an open transaction
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 5000L); // increase commit interval to make sure a client is killed having an open transaction
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());

6
streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java

@ -635,17 +635,19 @@ public class RelationalSmokeTest extends SmokeTestUtil { @@ -635,17 +635,19 @@ public class RelationalSmokeTest extends SmokeTestUtil {
final String id,
final String processingGuarantee,
final String stateDir) {
return mkProperties(
final Properties properties =
mkProperties(
mkMap(
mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, broker),
mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, application),
mkEntry(StreamsConfig.CLIENT_ID_CONFIG, id),
mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, processingGuarantee),
mkEntry(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, "1000"),
mkEntry(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"),
mkEntry(StreamsConfig.STATE_DIR_CONFIG, stateDir)
)
);
properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
return properties;
}
public static KafkaStreams startSync(final String broker,

2
streams/src/test/java/org/apache/kafka/streams/tests/StaticMemberTestClient.java

@ -57,7 +57,7 @@ public class StaticMemberTestClient { @@ -57,7 +57,7 @@ public class StaticMemberTestClient {
final Properties config = new Properties();
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, testName);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);

2
streams/src/test/java/org/apache/kafka/streams/tests/StreamsBrokerDownResilienceTest.java

@ -68,7 +68,7 @@ public class StreamsBrokerDownResilienceTest { @@ -68,7 +68,7 @@ public class StreamsBrokerDownResilienceTest {
streamsProperties.put(StreamsConfig.APPLICATION_ID_CONFIG, "kafka-streams-resilience");
streamsProperties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
streamsProperties.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
streamsProperties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
streamsProperties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
// it is expected that max.poll.interval, retries, request.timeout and max.block.ms set

2
streams/src/test/java/org/apache/kafka/streams/tests/StreamsStandByReplicaTest.java

@ -65,7 +65,7 @@ public class StreamsStandByReplicaTest { @@ -65,7 +65,7 @@ public class StreamsStandByReplicaTest {
}
streamsProperties.put(StreamsConfig.APPLICATION_ID_CONFIG, "kafka-streams-standby-tasks");
streamsProperties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100);
streamsProperties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
streamsProperties.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1);
streamsProperties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0);
streamsProperties.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());

2
streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java

@ -97,7 +97,7 @@ public class StreamsUpgradeTest { @@ -97,7 +97,7 @@ public class StreamsUpgradeTest {
final Properties config = new Properties();
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest");
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
final KafkaClientSupplier kafkaClientSupplier;
if (streamsProperties.containsKey("test.future.metadata")) {

2
streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java

@ -53,7 +53,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest { @@ -53,7 +53,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest {
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "cooperative-rebalance-upgrade");
config.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final String sourceTopic = streamsProperties.getProperty("source.topic", "source");

2
streams/upgrade-system-tests-0100/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java

@ -53,7 +53,7 @@ public class StreamsUpgradeTest { @@ -53,7 +53,7 @@ public class StreamsUpgradeTest {
final Properties config = new Properties();
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest");
config.setProperty(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final KafkaStreams streams = new KafkaStreams(builder, config);

2
streams/upgrade-system-tests-0100/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java

@ -50,7 +50,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest { @@ -50,7 +50,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest {
config.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.setProperty(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final String sourceTopic = config.getProperty("source.topic", "source");

2
streams/upgrade-system-tests-0101/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java

@ -56,7 +56,7 @@ public class StreamsUpgradeTest { @@ -56,7 +56,7 @@ public class StreamsUpgradeTest {
final Properties config = new Properties();
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest");
config.setProperty(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final KafkaStreams streams = new KafkaStreams(builder, config);

2
streams/upgrade-system-tests-0101/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java

@ -49,7 +49,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest { @@ -49,7 +49,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest {
config.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.setProperty(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final String sourceTopic = config.getProperty("source.topic", "source");

2
streams/upgrade-system-tests-0102/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java

@ -49,7 +49,7 @@ public class StreamsUpgradeTest { @@ -49,7 +49,7 @@ public class StreamsUpgradeTest {
final Properties config = new Properties();
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest");
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final KafkaStreams streams = new KafkaStreams(builder, config);

2
streams/upgrade-system-tests-0102/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java

@ -45,7 +45,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest { @@ -45,7 +45,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest {
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "cooperative-rebalance-upgrade");
config.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final String sourceTopic = config.getProperty("source.topic", "source");

2
streams/upgrade-system-tests-0110/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java

@ -49,7 +49,7 @@ public class StreamsUpgradeTest { @@ -49,7 +49,7 @@ public class StreamsUpgradeTest {
final Properties config = new Properties();
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest");
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final KafkaStreams streams = new KafkaStreams(builder, config);

2
streams/upgrade-system-tests-0110/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java

@ -44,7 +44,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest { @@ -44,7 +44,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest {
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "cooperative-rebalance-upgrade");
config.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final String sourceTopic = config.getProperty("source.topic", "source");

2
streams/upgrade-system-tests-10/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java

@ -49,7 +49,7 @@ public class StreamsUpgradeTest { @@ -49,7 +49,7 @@ public class StreamsUpgradeTest {
final Properties config = new Properties();
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest");
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final KafkaStreams streams = new KafkaStreams(builder.build(), config);

2
streams/upgrade-system-tests-10/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java

@ -52,7 +52,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest { @@ -52,7 +52,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest {
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "cooperative-rebalance-upgrade");
config.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final String sourceTopic = streamsProperties.getProperty("source.topic", "source");

2
streams/upgrade-system-tests-11/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java

@ -49,7 +49,7 @@ public class StreamsUpgradeTest { @@ -49,7 +49,7 @@ public class StreamsUpgradeTest {
final Properties config = new Properties();
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest");
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final KafkaStreams streams = new KafkaStreams(builder.build(), config);

2
streams/upgrade-system-tests-11/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java

@ -52,7 +52,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest { @@ -52,7 +52,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest {
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "cooperative-rebalance-upgrade");
config.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final String sourceTopic = streamsProperties.getProperty("source.topic", "source");

2
streams/upgrade-system-tests-20/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java

@ -48,7 +48,7 @@ public class StreamsUpgradeTest { @@ -48,7 +48,7 @@ public class StreamsUpgradeTest {
final Properties config = new Properties();
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest");
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final KafkaStreams streams = new KafkaStreams(builder.build(), config);

2
streams/upgrade-system-tests-20/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java

@ -52,7 +52,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest { @@ -52,7 +52,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest {
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "cooperative-rebalance-upgrade");
config.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final String sourceTopic = streamsProperties.getProperty("source.topic", "source");

2
streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java

@ -48,7 +48,7 @@ public class StreamsUpgradeTest { @@ -48,7 +48,7 @@ public class StreamsUpgradeTest {
final Properties config = new Properties();
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest");
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final KafkaStreams streams = new KafkaStreams(builder.build(), config);

2
streams/upgrade-system-tests-21/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java

@ -52,7 +52,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest { @@ -52,7 +52,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest {
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "cooperative-rebalance-upgrade");
config.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final String sourceTopic = streamsProperties.getProperty("source.topic", "source");

2
streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java

@ -48,7 +48,7 @@ public class StreamsUpgradeTest { @@ -48,7 +48,7 @@ public class StreamsUpgradeTest {
final Properties config = new Properties();
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest");
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final KafkaStreams streams = new KafkaStreams(builder.build(), config);

2
streams/upgrade-system-tests-22/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java

@ -52,7 +52,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest { @@ -52,7 +52,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest {
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "cooperative-rebalance-upgrade");
config.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final String sourceTopic = streamsProperties.getProperty("source.topic", "source");

2
streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java

@ -48,7 +48,7 @@ public class StreamsUpgradeTest { @@ -48,7 +48,7 @@ public class StreamsUpgradeTest {
final Properties config = new Properties();
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest");
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final KafkaStreams streams = new KafkaStreams(builder.build(), config);

2
streams/upgrade-system-tests-23/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeToCooperativeRebalanceTest.java

@ -50,7 +50,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest { @@ -50,7 +50,7 @@ public class StreamsUpgradeToCooperativeRebalanceTest {
config.put(StreamsConfig.APPLICATION_ID_CONFIG, "cooperative-rebalance-upgrade");
config.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final String sourceTopic = streamsProperties.getProperty("source.topic", "source");

2
streams/upgrade-system-tests-24/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java

@ -48,7 +48,7 @@ public class StreamsUpgradeTest { @@ -48,7 +48,7 @@ public class StreamsUpgradeTest {
final Properties config = new Properties();
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest");
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final KafkaStreams streams = new KafkaStreams(builder.build(), config);

2
streams/upgrade-system-tests-25/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java

@ -48,7 +48,7 @@ public class StreamsUpgradeTest { @@ -48,7 +48,7 @@ public class StreamsUpgradeTest {
final Properties config = new Properties();
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest");
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final KafkaStreams streams = new KafkaStreams(builder.build(), config);

2
streams/upgrade-system-tests-26/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java

@ -48,7 +48,7 @@ public class StreamsUpgradeTest { @@ -48,7 +48,7 @@ public class StreamsUpgradeTest {
final Properties config = new Properties();
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest");
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final KafkaStreams streams = new KafkaStreams(builder.build(), config);

2
streams/upgrade-system-tests-27/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java

@ -48,7 +48,7 @@ public class StreamsUpgradeTest { @@ -48,7 +48,7 @@ public class StreamsUpgradeTest {
final Properties config = new Properties();
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "StreamsUpgradeTest");
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000);
config.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000L);
config.putAll(streamsProperties);
final KafkaStreams streams = new KafkaStreams(builder.build(), config);

Loading…
Cancel
Save