MINOR: fix rawtype warning in StandbyTask (#17203)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
Matthias J. Sax 2024-09-16 11:49:19 -07:00 committed by GitHub
parent f1c011a8b5
commit aaf3fc05f8
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
3 changed files with 5 additions and 12 deletions

View File

@ -49,8 +49,7 @@ public class StandbyTask extends AbstractTask implements Task {
private final Sensor updateSensor;
private final StreamsMetricsImpl streamsMetrics;
@SuppressWarnings("rawtypes")
protected final InternalProcessorContext processorContext;
protected final InternalProcessorContext<?, ?> processorContext;
/**
* @param id the ID of this task
@ -61,7 +60,6 @@ public class StandbyTask extends AbstractTask implements Task {
* @param stateMgr the {@link ProcessorStateManager} for this task
* @param stateDirectory the {@link StateDirectory} created by the thread
*/
@SuppressWarnings("rawtypes")
StandbyTask(final TaskId id,
final Set<TopicPartition> inputPartitions,
final ProcessorTopology topology,
@ -70,7 +68,7 @@ public class StandbyTask extends AbstractTask implements Task {
final ProcessorStateManager stateMgr,
final StateDirectory stateDirectory,
final ThreadCache cache,
final InternalProcessorContext processorContext) {
final InternalProcessorContext<?, ?> processorContext) {
super(
id,
topology,
@ -333,11 +331,6 @@ public class StandbyTask extends AbstractTask implements Task {
throw new IllegalStateException("Attempted to add records to task " + id() + " for invalid input partition " + partition);
}
@SuppressWarnings("rawtypes")
InternalProcessorContext processorContext() {
return processorContext;
}
/**
* Produces a string representation containing useful information about a Task.
* This is useful in debugging scenarios.

View File

@ -92,7 +92,7 @@ class StandbyTaskCreator {
partitions,
stateUpdaterEnabled);
final InternalProcessorContext<Object, Object> context = new ProcessorContextImpl(
final InternalProcessorContext<?, ?> context = new ProcessorContextImpl(
taskId,
applicationConfig,
stateManager,
@ -147,7 +147,7 @@ class StandbyTaskCreator {
final Set<TopicPartition> inputPartitions,
final ProcessorTopology topology,
final ProcessorStateManager stateManager,
final InternalProcessorContext<Object, Object> context) {
final InternalProcessorContext<?, ?> context) {
final StandbyTask task = new StandbyTask(
taskId,
inputPartitions,

View File

@ -598,7 +598,7 @@ public class StandbyTaskTest {
streamsMetrics
);
final InternalProcessorContext context = new ProcessorContextImpl(
final InternalProcessorContext<?, ?> context = new ProcessorContextImpl(
taskId,
config,
stateManager,