kafka/test-catalog/streams/integration-tests/tests.yaml

378 lines
18 KiB
YAML

org.apache.kafka.streams.integration.AdjustStreamThreadCountTest:
- shouldAddAndRemoveStreamThreadsWhileKeepingNamesCorrect
- shouldAddAndRemoveThreadsMultipleTimes
- shouldAddStreamThread
- shouldRemoveStreamThread
- shouldRemoveStreamThreadWithStaticMembership
- shouldResizeCacheAfterThreadRemovalTimesOut
- shouldResizeCacheAfterThreadReplacement
- shouldnNotRemoveStreamThreadWithinTimeout
- testConcurrentlyAccessThreads
- testRebalanceHappensBeforeStreamThreadGetDown
org.apache.kafka.streams.integration.ConsistencyVectorIntegrationTest:
- shouldHaveSamePositionBoundActiveAndStandBy
org.apache.kafka.streams.integration.DeadLetterQueueIntegrationTest:
- shouldSendToDlqAndContinueFromDeserializationError
- shouldSendToDlqAndContinueFromDsl
- shouldSendToDlqAndContinueFromProcessorAPI
- shouldSendToDlqAndFailFromDeserializationError
- shouldSendToDlqAndFailFromDsl
- shouldSendToDlqAndFailFromProcessorAPI
org.apache.kafka.streams.integration.EOSUncleanShutdownIntegrationTest:
- shouldWorkWithUncleanShutdownWipeOutStateStore
org.apache.kafka.streams.integration.EmitOnChangeIntegrationTest:
- shouldEmitSameRecordAfterFailover
org.apache.kafka.streams.integration.EosIntegrationTest:
- shouldBeAbleToCommitMultiplePartitionOffsets
- shouldBeAbleToCommitToMultiplePartitions
- shouldBeAbleToPerformMultipleTransactions
- shouldBeAbleToRestartAfterClose
- shouldBeAbleToRunWithEosEnabled
- shouldBeAbleToRunWithTwoSubtopologies
- shouldBeAbleToRunWithTwoSubtopologiesAndMultiplePartitions
- shouldCheckpointRestoredOffsetsWhenClosingCleanDuringRestoring
- shouldCommitCorrectOffsetIfInputTopicIsTransactional
- shouldNotCommitActiveTasksWithPendingInputIfRevokedTaskDidNotMakeProgress
- shouldNotViolateEosIfOneTaskFails
- shouldNotViolateEosIfOneTaskFailsWithState
- shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances
- shouldWriteLatestOffsetsToCheckpointOnShutdown
org.apache.kafka.streams.integration.FineGrainedAutoResetIntegrationTest:
- shouldFailForResetNone
- shouldOnlyReadRecordsWhereEarliestSpecifiedWithInvalidCommittedOffsets
- shouldOnlyReadRecordsWhereEarliestSpecifiedWithNoCommittedOffsetsWithDefaultGlobalAutoOffsetResetEarliest
- shouldOnlyReadRecordsWhereEarliestSpecifiedWithNoCommittedOffsetsWithGlobalAutoOffsetResetLatest
- shouldResetByDuration
- shouldThrowExceptionOverlappingPattern
- shouldThrowExceptionOverlappingTopic
- shouldThrowStreamsExceptionNoResetSpecified
org.apache.kafka.streams.integration.GlobalKTableEOSIntegrationTest:
- shouldKStreamGlobalKTableJoin
- shouldKStreamGlobalKTableLeftJoin
- shouldNotRestoreAbortedMessages
- shouldRestoreTransactionalMessages
- shouldSkipOverAbortedMessagesOnRestore
- shouldSkipOverTxMarkersOnRestore
org.apache.kafka.streams.integration.GlobalKTableIntegrationTest:
- shouldGetToRunningWithOnlyGlobalTopology
- shouldKStreamGlobalKTableJoin
- shouldKStreamGlobalKTableLeftJoin
- shouldRestoreGlobalInMemoryKTableOnRestart
org.apache.kafka.streams.integration.GlobalStateReprocessTest:
- shouldReprocessWithUserProvidedStore
org.apache.kafka.streams.integration.GlobalThreadShutDownOrderTest:
- shouldFinishGlobalStoreOperationOnShutDown
org.apache.kafka.streams.integration.HandlingSourceTopicDeletionIntegrationTest:
- shouldThrowErrorAfterSourceTopicDeleted
org.apache.kafka.streams.integration.HighAvailabilityTaskAssignorIntegrationTest:
- shouldScaleOutWithWarmupTasksAndInMemoryStores
- shouldScaleOutWithWarmupTasksAndPersistentStores
org.apache.kafka.streams.integration.IQv2EndpointToPartitionsIntegrationTest:
- CLASSIC
- STREAMS
org.apache.kafka.streams.integration.IQv2IntegrationTest:
- CLASSIC
- STREAMS
org.apache.kafka.streams.integration.IQv2StoreIntegrationTest:
- verifyStore
org.apache.kafka.streams.integration.IQv2VersionedStoreIntegrationTest:
- CLASSIC
- STREAMS
org.apache.kafka.streams.integration.InternalTopicIntegrationTest:
- shouldCompactAndDeleteTopicsForWindowStoreChangelogs
- shouldCompactTopicsForKeyValueStoreChangelogs
- shouldGetToRunningWithWindowedTableInFKJ
org.apache.kafka.streams.integration.JoinGracePeriodDurabilityIntegrationTest:
- shouldRecoverBufferAfterShutdown
org.apache.kafka.streams.integration.JoinStoreIntegrationTest:
- providingAJoinStoreNameShouldNotMakeTheJoinResultQueryable
- streamJoinChangelogTopicShouldBeConfiguredWithDeleteOnlyCleanupPolicy
org.apache.kafka.streams.integration.JoinWithIncompleteMetadataIntegrationTest:
- testShouldAutoShutdownOnJoinWithIncompleteMetadata
org.apache.kafka.streams.integration.KStreamAggregationDedupIntegrationTest:
- shouldGroupByKey
- shouldReduce
- shouldReduceWindowed
org.apache.kafka.streams.integration.KStreamAggregationIntegrationTest:
- shouldAggregate
- shouldAggregateSlidingWindows
- shouldAggregateWindowed
- shouldCount
- shouldCountSessionWindows
- shouldCountUnlimitedWindows
- shouldCountWithInternalStore
- shouldGroupByKey
- shouldReduce
- shouldReduceSessionWindows
- shouldReduceSlidingWindows
- shouldReduceWindowed
org.apache.kafka.streams.integration.KStreamKStreamIntegrationTest:
- shouldOuterJoin
org.apache.kafka.streams.integration.KStreamRepartitionIntegrationTest:
- shouldCreateOnlyOneRepartitionTopicWhenRepartitionIsFollowedByGroupByKey
- shouldCreateRepartitionTopicIfKeyChangingOperationWasNotPerformed
- shouldCreateRepartitionTopicWithSpecifiedNumberOfPartitions
- shouldDeductNumberOfPartitionsFromRepartitionOperation
- shouldDoProperJoiningWhenNumberOfPartitionsAreValidWhenUsingRepartitionOperation
- shouldGenerateRepartitionTopicWhenNameIsNotSpecified
- shouldGoThroughRebalancingCorrectly
- shouldInheritRepartitionTopicPartitionNumberFromUpstreamTopicWhenNumberOfPartitionsIsNotSpecified
- shouldPerformKeySelectOperationWhenRepartitionOperationIsUsedWithKeySelector
- shouldPerformSelectKeyWithRepartitionOperation
- shouldRepartitionToMultiplePartitions
- shouldThrowAnExceptionWhenNumberOfPartitionsOfRepartitionOperationDoNotMatchSourceTopicWhenJoining
- shouldUseStreamPartitionerForRepartitionOperation
org.apache.kafka.streams.integration.KTableEfficientRangeQueryTest:
- testStoreConfig
org.apache.kafka.streams.integration.KTableKTableForeignKeyInnerJoinCustomPartitionerIntegrationTest:
- shouldInnerJoinMultiPartitionQueryable
- shouldThrowIllegalArgumentExceptionWhenCustomPartitionerReturnsMultiplePartitions
org.apache.kafka.streams.integration.KTableKTableForeignKeyInnerJoinMultiIntegrationTest:
- shouldInnerJoinMultiPartitionQueryable
org.apache.kafka.streams.integration.KTableKTableForeignKeyJoinDistributedTest:
- shouldBeInitializedWithDefaultSerde
org.apache.kafka.streams.integration.KTableKTableForeignKeyJoinIntegrationTest:
- doJoinFromLeftThenDeleteLeftEntity
- doJoinFromLeftThenUpdateFkThenRevertBack
- doJoinFromRightThenDeleteRightEntity
- joinShouldProduceNullsWhenValueHasNonMatchingForeignKey
- shouldEmitRecordOnNullForeignKeyForLeftJoins
- shouldEmitRecordWhenOldAndNewFkDiffer
- shouldEmitTombstoneWhenDeletingNonJoiningRecords
- shouldIgnoreOutOfOrderRecordsIffVersioned
- shouldNotEmitTombstonesWhenDeletingNonExistingRecords
- shouldUnsubscribeOldForeignKeyIfLeftSideIsUpdated
org.apache.kafka.streams.integration.KTableKTableForeignKeyJoinMaterializationIntegrationTest:
- shouldEmitTombstoneWhenDeletingNonJoiningRecords
org.apache.kafka.streams.integration.KTableSourceTopicRestartIntegrationTest:
- shouldRestoreAndProgressWhenTopicNotWrittenToDuringRestoration
- shouldRestoreAndProgressWhenTopicWrittenToDuringRestorationWithEosDisabled
- shouldRestoreAndProgressWhenTopicWrittenToDuringRestorationWithEosV2Enabled
org.apache.kafka.streams.integration.KafkaStreamsCloseOptionsIntegrationTest:
- testCloseOptions
org.apache.kafka.streams.integration.KafkaStreamsTelemetryIntegrationTest:
- passedMetricsShouldNotLeakIntoClientMetrics
- shouldPassCorrectMetricsDynamicInstances
- shouldPassMetrics
- shouldPushGlobalThreadMetricsToBroker
- shouldPushMetricsToBroker
org.apache.kafka.streams.integration.LagFetchIntegrationTest:
- shouldFetchLagsDuringRebalancingWithNoOptimization
- shouldFetchLagsDuringRebalancingWithOptimization
- shouldFetchLagsDuringRestoration
org.apache.kafka.streams.integration.MetricsIntegrationTest:
- shouldAddMetricsForSessionStore
- shouldAddMetricsForWindowStoreAndSuppressionBuffer
- shouldAddMetricsOnAllLevels
org.apache.kafka.streams.integration.MetricsReporterIntegrationTest:
- shouldBeAbleToProvideInitialMetricValueToMetricsReporter
org.apache.kafka.streams.integration.NamedTopologyIntegrationTest:
- shouldAddNamedTopologyToRunningApplicationWithEmptyInitialTopology
- shouldAddNamedTopologyToRunningApplicationWithMultipleNodes
- shouldAddToEmptyInitialTopologyRemoveResetOffsetsThenAddSameNamedTopology
- shouldAddToEmptyInitialTopologyRemoveResetOffsetsThenAddSameNamedTopologyWithRepartitioning
- shouldAllowMixedCollectionAndPatternSubscriptionWithMultipleNamedTopologies
- shouldAllowPatternSubscriptionWithMultipleNamedTopologies
- shouldAllowRemovingAndAddingNamedTopologyToRunningApplicationWithMultipleNodesAndResetsOffsets
- shouldPrefixAllInternalTopicNamesWithNamedTopology
- shouldProcessMultipleIdenticalNamedTopologiesWithInMemoryAndPersistentStateStores
- shouldProcessSingleNamedTopologyAndPrefixInternalTopics
- shouldRemoveAndReplaceTopologicallyIncompatibleNamedTopology
org.apache.kafka.streams.integration.OptimizedKTableIntegrationTest:
- shouldApplyUpdatesToStandbyStore
org.apache.kafka.streams.integration.PauseResumeIntegrationTest:
- pauseResumeShouldWorkAcrossInstances
- pausedTopologyShouldNotRestoreStateStores
- shouldAllowForNamedTopologiesToStartPaused
- shouldAllowForTopologiesToStartPaused
- shouldPauseAndResumeAllKafkaStreamsWithNamedTopologies
- shouldPauseAndResumeKafkaStreams
- shouldPauseAndResumeKafkaStreamsWithNamedTopologies
org.apache.kafka.streams.integration.PositionRestartIntegrationTest:
- verifyStore
org.apache.kafka.streams.integration.ProcessingExceptionHandlerIntegrationTest:
- shouldContinueWhenProcessingExceptionOccursFromFlushingCacheIfExceptionHandlerReturnsContinue
- shouldContinueWhenProcessingExceptionOccursIfExceptionHandlerReturnsContinue
- shouldFailWhenProcessingExceptionOccursFromFlushingCacheIfExceptionHandlerReturnsFail
- shouldFailWhenProcessingExceptionOccursIfExceptionHandlerReturnsFail
- shouldStopOnFailedProcessorWhenProcessingExceptionOccursInContinueProcessingExceptionHandler
- shouldStopOnFailedProcessorWhenProcessingExceptionOccursInFailProcessingExceptionHandler
- shouldStopProcessingWhenFatalUserExceptionProcessingExceptionHandler
- shouldStopProcessingWhenProcessingExceptionHandlerReturnsNull
- shouldVerifySourceRawKeyAndSourceRawValuePresentOrNotInErrorHandlerContext
org.apache.kafka.streams.integration.PurgeRepartitionTopicIntegrationTest:
- shouldRestoreState
org.apache.kafka.streams.integration.QueryableStateIntegrationTest:
- shouldAllowToQueryAfterThreadDied
- shouldBeAbleQueryStandbyStateDuringRebalance
- shouldBeAbleToQueryDuringRebalance
- shouldBeAbleToQueryFilterState
- shouldBeAbleToQueryKeysWithGivenPrefix
- shouldBeAbleToQueryMapValuesAfterFilterState
- shouldBeAbleToQueryMapValuesState
- shouldBeAbleToQueryStateWithNonZeroSizedCache
- shouldBeAbleToQueryStateWithZeroSizedCache
- shouldNotMakeStoreAvailableUntilAllStoresAvailable
- shouldRejectNonExistentStoreName
- shouldRejectWronglyTypedStore
org.apache.kafka.streams.integration.RangeQueryIntegrationTest:
- testStoreConfig
org.apache.kafka.streams.integration.RebalanceIntegrationTest:
- shouldCommitAllTasksIfRevokedTaskTriggerPunctuation
org.apache.kafka.streams.integration.RebalanceProtocolMigrationIntegrationTest:
- shouldMigrateFromAndToStreamsRebalanceProtocol
- shouldMigrateToAndFromStreamsRebalanceProtocol
org.apache.kafka.streams.integration.RegexSourceIntegrationTest:
- shouldAddStateStoreToRegexDefinedSource
- testMultipleConsumersCanReadFromPartitionedTopic
- testNoMessagesSentExceptionFromOverlappingPatterns
- testRegexMatchesTopicsAWhenCreated
- testRegexMatchesTopicsAWhenDeleted
- testRegexRecordsAreProcessedAfterNewTopicCreatedWithMultipleSubtopologies
- testShouldReadFromRegexAndNamedTopics
org.apache.kafka.streams.integration.RelaxedNullKeyRequirementJoinTest:
- testDropNullKeyRecordsForRepartitionNodesWithNoRelaxedJoinDownstream
- testRelaxedLeftStreamGlobalTableJoin
- testRelaxedLeftStreamStreamJoin
- testRelaxedLeftStreamTableJoin
- testRelaxedOuterStreamStreamJoin
org.apache.kafka.streams.integration.ResetPartitionTimeIntegrationTest:
- shouldPreservePartitionTimeOnKafkaStreamRestart
org.apache.kafka.streams.integration.RestoreIntegrationTest:
- shouldInvokeUserDefinedGlobalStateRestoreListener
- shouldProcessDataFromStoresWithLoggingDisabled
- shouldRecordRestoreMetrics
- shouldRecycleStateFromStandbyTaskPromotedToActiveTaskAndNotRestore
- shouldRestoreNullRecord
- shouldRestoreStateFromChangelogTopic
- shouldRestoreStateFromSourceTopicForGlobalTable
- shouldRestoreStateFromSourceTopicForReadOnlyStore
- shouldSuccessfullyStartWhenLoggingDisabled
org.apache.kafka.streams.integration.RocksDBMetricsIntegrationTest:
- shouldExposeRocksDBMetricsBeforeAndAfterFailureWithEmptyStateDir
org.apache.kafka.streams.integration.SelfJoinUpgradeIntegrationTest:
- shouldRestartWithTopologyOptimizationOn
- shouldUpgradeWithTopologyOptimizationOff
org.apache.kafka.streams.integration.SlidingWindowedKStreamIntegrationTest:
- shouldAggregateWindowedWithGrace
- shouldAggregateWindowedWithNoGrace
- shouldRestoreAfterJoinRestart
org.apache.kafka.streams.integration.SmokeTestDriverIntegrationTest:
- shouldWorkWithRebalance
org.apache.kafka.streams.integration.StandbyTaskCreationIntegrationTest:
- shouldCreateStandByTasksForMaterializedAndOptimizedSourceTables
- shouldNotCreateAnyStandByTasksForStateStoreWithLoggingDisabled
org.apache.kafka.streams.integration.StandbyTaskEOSIntegrationTest:
- shouldSurviveWithOneTaskAsStandby
- shouldWipeOutStandbyStateDirectoryIfCheckpointIsMissing
org.apache.kafka.streams.integration.StandbyTaskEOSMultiRebalanceIntegrationTest:
- shouldHonorEOSWhenUsingCachingAndStandbyReplicas
org.apache.kafka.streams.integration.StateDirectoryIntegrationTest:
- testCleanUpStateDirIfEmpty
- testNotCleanUpStateDirIfNotEmpty
org.apache.kafka.streams.integration.StoreQueryIntegrationTest:
- shouldFailWithIllegalArgumentExceptionWhenIQPartitionerReturnsMultiplePartitions
- shouldQueryAllStalePartitionStores
- shouldQueryOnlyActivePartitionStoresByDefault
- shouldQuerySpecificActivePartitionStores
- shouldQuerySpecificStalePartitionStores
- shouldQuerySpecificStalePartitionStoresMultiStreamThreads
- shouldQuerySpecificStalePartitionStoresMultiStreamThreadsNamedTopology
- shouldQueryStoresAfterAddingAndRemovingStreamThread
org.apache.kafka.streams.integration.StoreUpgradeIntegrationTest:
- shouldMigrateInMemoryKeyValueStoreToTimestampedKeyValueStoreUsingPapi
- shouldMigrateInMemoryWindowStoreToTimestampedWindowStoreUsingPapi
- shouldMigratePersistentKeyValueStoreToTimestampedKeyValueStoreUsingPapi
- shouldMigratePersistentWindowStoreToTimestampedWindowStoreUsingPapi
- shouldProxyKeyValueStoreToTimestampedKeyValueStoreUsingPapi
- shouldProxyWindowStoreToTimestampedWindowStoreUsingPapi
org.apache.kafka.streams.integration.StreamStreamJoinIntegrationTest:
- testInner
- testInnerRepartitioned
- testLeft
- testLeftRepartitioned
- testMultiInner
- testOuter
- testOuterRepartitioned
- testSelfJoin
org.apache.kafka.streams.integration.StreamTableJoinIntegrationTest:
- testInner
- testInnerWithVersionedStore
- testLeft
- testLeftWithVersionedStore
org.apache.kafka.streams.integration.StreamTableJoinTopologyOptimizationIntegrationTest:
- shouldDoStreamTableJoinWithDifferentNumberOfPartitions
org.apache.kafka.streams.integration.StreamTableJoinWithGraceIntegrationTest:
- testInnerWithVersionedStore
- testLeftWithVersionedStore
org.apache.kafka.streams.integration.StreamToTableJoinScalaIntegrationTestImplicitSerdes:
- testShouldCountClicksPerRegion
- testShouldCountClicksPerRegionJava
- testShouldCountClicksPerRegionWithNamedRepartitionTopic
org.apache.kafka.streams.integration.StreamsUncaughtExceptionHandlerIntegrationTest:
- shouldEmitSameRecordAfterFailover
- shouldReplaceSingleThread
- shouldReplaceThreads
- shouldReplaceThreadsWithoutJavaHandler
- shouldShutDownClientIfGlobalStreamThreadWantsToReplaceThread
- shouldShutdownClient
- shouldShutdownMultipleThreadApplication
- shouldShutdownSingleThreadApplication
org.apache.kafka.streams.integration.StreamsUpgradeTestIntegrationTest:
- testVersionProbingUpgrade
org.apache.kafka.streams.integration.SuppressionDurabilityIntegrationTest:
- shouldRecoverBufferAfterShutdown
org.apache.kafka.streams.integration.SuppressionIntegrationTest:
- shouldAllowDisablingChangelog
- shouldAllowOverridingChangelogConfig
- shouldCreateChangelogByDefault
- shouldInheritSerdes
- shouldShutdownWhenBytesConstraintIsViolated
- shouldShutdownWhenRecordConstraintIsViolated
- shouldUseDefaultSerdes
org.apache.kafka.streams.integration.SwallowUnknownTopicErrorIntegrationTest:
- shouldThrowStreamsExceptionWithMissingTopicAndDefaultExceptionHandler
org.apache.kafka.streams.integration.TableTableJoinIntegrationTest:
- testInner
- testInnerInner
- testInnerLeft
- testInnerOuter
- testInnerWithLeftVersionedOnly
- testInnerWithRightVersionedOnly
- testInnerWithVersionedStores
- testLeft
- testLeftInner
- testLeftLeft
- testLeftOuter
- testLeftWithLeftVersionedOnly
- testLeftWithRightVersionedOnly
- testLeftWithVersionedStores
- testOuter
- testOuterInner
- testOuterLeft
- testOuterOuter
- testOuterWithLeftVersionedOnly
- testOuterWithRightVersionedOnly
- testOuterWithVersionedStores
org.apache.kafka.streams.integration.TaskAssignorIntegrationTest:
- shouldProperlyConfigureTheAssignor
org.apache.kafka.streams.integration.TaskMetadataIntegrationTest:
- shouldReportCorrectCommittedOffsetInformation
- shouldReportCorrectEndOffsetInformation
org.apache.kafka.streams.integration.TimeWindowedKStreamIntegrationTest:
- shouldAggregateWindowedWithGrace
- shouldAggregateWindowedWithNoGrace
- shouldRestoreAfterJoinRestart
- shouldThrowUnlimitedWindows
org.apache.kafka.streams.integration.VersionedKeyValueStoreIntegrationTest:
- shouldAllowCustomIQv2ForCustomStoreImplementations
- shouldManualUpgradeFromNonVersionedNonTimestampedToVersioned
- shouldManualUpgradeFromNonVersionedTimestampedToVersioned
- shouldPutGetAndDelete
- shouldRestore
- shouldSetChangelogTopicProperties
org.apache.kafka.streams.integration.WordCountTest:
- testShouldCountWords
- testShouldCountWordsJava
- testShouldCountWordsMaterialized