diff --git a/docs/connect.html b/docs/connect.html index e522c8a088f22..7b60854094971 100644 --- a/docs/connect.html +++ b/docs/connect.html @@ -311,7 +311,7 @@

REST API

  • PUT /connectors/{name}/pause - pause the connector and its tasks, which stops message processing until the connector is resumed
  • PUT /connectors/{name}/resume - resume a paused connector (or do nothing if the connector is not paused)
  • POST /connectors/{name}/restart - restart a connector (typically because it has failed)
  • -
  • POST /connectors/{name}/tasks/{taskId}/restart - restart an individual task (typically because it has failed)
  • +
  • POST /connectors/{name}/tasks/{taskInfo}/restart - restart an individual task (typically because it has failed)
  • DELETE /connectors/{name} - delete a connector, halting all tasks and deleting its configuration
  • GET /connectors/{name}/topics - get the set of topics that a specific connector is using since the connector was created or since a request to reset its set of active topics was issued
  • PUT /connectors/{name}/topics/reset - send a request to empty the set of active topics of a connector
  • diff --git a/docs/streams/developer-guide/config-streams.html b/docs/streams/developer-guide/config-streams.html index 1f8e8f3d7f86b..04d7cd074fcc5 100644 --- a/docs/streams/developer-guide/config-streams.html +++ b/docs/streams/developer-guide/config-streams.html @@ -406,8 +406,8 @@

    acceptable.recovery.lag

    Accessing Processor Context

    As we have mentioned in the Defining a Stream Processor section, a ProcessorContext control the processing workflow, such as scheduling a punctuation function, and committing the current processed state.

    This object can also be used to access the metadata related with the application like - applicationId, taskId, + applicationId, taskInfo, and stateDir, and also record related metadata as topic, partition, offset, timestamp and headers.

    diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index de747bb862f33..654fab2b4ebbd 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -984,7 +984,7 @@

    Producer's client.id naming schema:

    [client.Id] is either set via Streams configuration parameter client.id or defaults to [application.id]-[processId] ([processId] is a random UUID).

    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTaskId.java b/streams/src/main/java/org/apache/kafka/streams/TaskInfo.java similarity index 51% rename from streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTaskId.java rename to streams/src/main/java/org/apache/kafka/streams/TaskInfo.java index 82be29cdb9e6f..7785d8054b96a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTaskId.java +++ b/streams/src/main/java/org/apache/kafka/streams/TaskInfo.java @@ -14,27 +14,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.streams.processor.internals.namedtopology; +package org.apache.kafka.streams; -import org.apache.kafka.streams.processor.TaskId; - -public class NamedTaskId extends TaskId { - public NamedTaskId(final int topicGroupId, final int partition, final String namedTopology) { - super(topicGroupId, partition, namedTopology); - if (namedTopology == null) { - throw new IllegalStateException("NamedTopology is required for a NamedTaskId"); - } - } - - public String namedTopology() { - return namedTopology; - } +/** + * A wrapper class for basic task info + */ +public interface TaskInfo { + /** + * @return The ID of the subtopology that this task executes, ie the topicGroupId + */ + int subtopologyId(); - public static String namedTopology(final TaskId taskId) { - if (taskId instanceof NamedTaskId) { - return ((NamedTaskId) taskId).namedTopology(); - } else { - return null; - } - } + /** + * @return The ID number of the partition. + */ + int partition(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java index 4f9a0964405a2..bf166f83d85b6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndContinueExceptionHandler.java @@ -38,7 +38,7 @@ public DeserializationHandlerResponse handle(final ProcessorContext context, log.warn("Exception caught during Deserialization, " + "taskId: {}, topic: {}, partition: {}, offset: {}", - context.taskId(), record.topic(), record.partition(), record.offset(), + context.taskInfo(), record.topic(), record.partition(), record.offset(), exception); return DeserializationHandlerResponse.CONTINUE; diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java index 61d210649ba9a..780ab63fd58d5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/LogAndFailExceptionHandler.java @@ -38,7 +38,7 @@ public DeserializationHandlerResponse handle(final ProcessorContext context, log.error("Exception caught during Deserialization, " + "taskId: {}, topic: {}, partition: {}, offset: {}", - context.taskId(), record.topic(), record.partition(), record.offset(), + context.taskInfo(), record.topic(), record.partition(), record.offset(), exception); return DeserializationHandlerResponse.FAIL; diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/TaskCorruptedException.java b/streams/src/main/java/org/apache/kafka/streams/errors/TaskCorruptedException.java index bf5bd17fbdbc8..4945170cc0b20 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/TaskCorruptedException.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/TaskCorruptedException.java @@ -17,7 +17,7 @@ package org.apache.kafka.streams.errors; import org.apache.kafka.clients.consumer.InvalidOffsetException; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import java.util.Set; diff --git a/streams/src/main/java/org/apache/kafka/streams/errors/TaskIdFormatException.java b/streams/src/main/java/org/apache/kafka/streams/errors/TaskIdFormatException.java index 63493436816bb..27b5981fcb42a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/errors/TaskIdFormatException.java +++ b/streams/src/main/java/org/apache/kafka/streams/errors/TaskIdFormatException.java @@ -17,8 +17,10 @@ package org.apache.kafka.streams.errors; +import org.apache.kafka.streams.processor.internals.TaskId; + /** - * Indicates a run time error incurred while trying parse the {@link org.apache.kafka.streams.processor.TaskId task id} + * Indicates a run time error incurred while trying parse the {@link TaskId task id} * from the read string. * * @see org.apache.kafka.streams.processor.internals.StreamTask diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java index 6a568d5fe89e2..47b509fbd263a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java @@ -69,7 +69,7 @@ public void init(final ProcessorContext context) { super.init(context); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor( Thread.currentThread().getName(), - context.taskId().toString(), + context.taskInfo().toString(), (StreamsMetricsImpl) context.metrics()); store = (TimestampedKeyValueStore) context.getStateStore(storeName); tupleForwarder = new TimestampedTupleForwarder<>( diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index 14d0bdb97d17f..eb2eecce6b838 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -91,7 +91,7 @@ private class KStreamKStreamJoinProcessor extends AbstractProcessor { public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); - droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); + droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskInfo().toString(), metrics); otherWindowStore = context.getStateStore(otherWindowName); if (StreamsConfig.InternalConfig.getBoolean(context().appConfigs(), ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX, true)) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java index 5cf31424b8fb2..d28982eedccb7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java @@ -52,7 +52,7 @@ class KStreamKTableJoinProcessor extends AbstractProcessor { public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); - droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); + droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskInfo().toString(), metrics); store = (TimestampedKeyValueStore) context.getStateStore(storeName); tupleForwarder = new TimestampedTupleForwarder<>( store, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java index 5648e8f0a367b..887693a5d4a16 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java @@ -97,11 +97,11 @@ public void init(final ProcessorContext context) { final String threadId = Thread.currentThread().getName(); lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor( threadId, - context.taskId().toString(), + context.taskInfo().toString(), internalProcessorContext.currentNode().name(), metrics ); - droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), metrics); + droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskInfo().toString(), metrics); store = (SessionStore) context.getStateStore(storeName); tupleForwarder = new SessionTupleForwarder<>(store, context, new SessionCacheFlushListener<>(context), sendOldValues); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java index 250ea022b3855..acf1b644d1a55 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java @@ -94,11 +94,11 @@ public void init(final ProcessorContext context) { final String threadId = Thread.currentThread().getName(); lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor( threadId, - context.taskId().toString(), + context.taskInfo().toString(), internalProcessorContext.currentNode().name(), metrics ); - droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), metrics); + droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskInfo().toString(), metrics); windowStore = (TimestampedWindowStore) context.getStateStore(storeName); tupleForwarder = new TimestampedTupleForwarder<>( windowStore, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java index e1aa84bb46488..5392565dc10dd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java @@ -92,11 +92,11 @@ public void init(final ProcessorContext context) { final String threadId = Thread.currentThread().getName(); lateRecordDropSensor = droppedRecordsSensorOrLateRecordDropSensor( threadId, - context.taskId().toString(), + context.taskInfo().toString(), internalProcessorContext.currentNode().name(), metrics ); - droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), metrics); + droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskInfo().toString(), metrics); windowStore = (TimestampedWindowStore) context.getStateStore(storeName); tupleForwarder = new TimestampedTupleForwarder<>( windowStore, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java index 06701c2756d68..8d3fda6d8ac76 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java @@ -78,7 +78,7 @@ private class KTableKTableJoinProcessor extends AbstractProcessor> public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); - droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); + droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskInfo().toString(), metrics); valueGetter.init(context); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java index b6fd894ae3db5..88f6d11e16d48 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java @@ -77,7 +77,7 @@ private class KTableKTableLeftJoinProcessor extends AbstractProcessor { public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); - droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); + droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskInfo().toString(), metrics); if (queryableName != null) { store = (TimestampedKeyValueStore) context.getStateStore(queryableName); tupleForwarder = new TimestampedTupleForwarder<>( diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java index fd95105324472..8e6346fa56056 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java @@ -66,7 +66,7 @@ public void init(final ProcessorContext context) { final InternalProcessorContext internalProcessorContext = (InternalProcessorContext) context; droppedRecordsSensor = TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor( Thread.currentThread().getName(), - internalProcessorContext.taskId().toString(), + internalProcessorContext.taskInfo().toString(), internalProcessorContext.metrics() ); store = internalProcessorContext.getStateStore(storeBuilder); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java index 97878750dc1f6..6794eae0e2c7c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java @@ -90,7 +90,7 @@ public void init(final ProcessorContext context) { } droppedRecordsSensor = TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor( Thread.currentThread().getName(), - context.taskId().toString(), + context.taskInfo().toString(), (StreamsMetricsImpl) context.metrics() ); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java index 61fb1c1334774..624f31c5e349e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java @@ -64,7 +64,7 @@ public void init(final ProcessorContext context) { droppedRecordsSensor = TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor( Thread.currentThread().getName(), - internalProcessorContext.taskId().toString(), + internalProcessorContext.taskInfo().toString(), internalProcessorContext.metrics() ); store = internalProcessorContext.getStateStore(storeBuilder); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java index 6a300b879ece2..7a316341e1031 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java @@ -144,7 +144,7 @@ public void init(final ProcessorContext context) { internalProcessorContext = (InternalProcessorContext) context; suppressionEmitSensor = ProcessorNodeMetrics.suppressionEmitSensor( Thread.currentThread().getName(), - context.taskId().toString(), + context.taskInfo().toString(), internalProcessorContext.currentNode().name(), internalProcessorContext.metrics() ); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java index a75322b14aa05..0539e24cbadc5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.TaskInfo; import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.errors.StreamsException; @@ -39,12 +40,21 @@ public interface ProcessorContext { */ String applicationId(); + /** + * Returns the task info. + * + * @return the task info + */ + TaskInfo taskInfo(); + /** * Returns the task id. * * @return the task id + * @deprecated use {@link #taskInfo()} instead. */ - TaskId taskId(); + @Deprecated + org.apache.kafka.streams.processor.TaskId taskId(); /** * Returns the default key serde. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/StateStoreContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/StateStoreContext.java index 43810a2579807..418377a63dd4a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/StateStoreContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/StateStoreContext.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.TaskInfo; import org.apache.kafka.streams.errors.StreamsException; import java.io.File; @@ -35,12 +36,21 @@ public interface StateStoreContext { */ String applicationId(); + /** + * Returns the task info. + * + * @return the task info + */ + TaskInfo taskInfo(); + /** * Returns the task id. * * @return the task id + * @deprecated use {@link #taskInfo()} instead. */ - TaskId taskId(); + @Deprecated + org.apache.kafka.streams.processor.TaskId taskId(); /** * Returns the default key serde. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java b/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java index 59e7ef4bbc841..89e433cde5808 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java @@ -16,64 +16,39 @@ */ package org.apache.kafka.streams.processor; +import org.apache.kafka.streams.TaskInfo; import org.apache.kafka.streams.errors.TaskIdFormatException; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.Objects; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.MIN_NAMED_TOPOLOGY_VERSION; /** * The task ID representation composed as topic group ID plus the assigned partition ID. + * @deprecated please use {@link TaskInfo} instead. */ +@Deprecated public class TaskId implements Comparable { - private static final Logger LOG = LoggerFactory.getLogger(TaskId.class); - /** The ID of the topic group. */ public final int topicGroupId; /** The ID of the partition. */ public final int partition; - /** The namedTopology that this task belongs to, or null if it does not belong to one */ - protected final String namedTopology; public TaskId(final int topicGroupId, final int partition) { - this(topicGroupId, partition, null); - } - - public TaskId(final int topicGroupId, final int partition, final String namedTopology) { this.topicGroupId = topicGroupId; this.partition = partition; - if (namedTopology != null && namedTopology.length() == 0) { - LOG.warn("Empty string passed in for task's namedTopology, since NamedTopology name cannot be empty, we " - + "assume this task does not belong to a NamedTopology and downgrade this to null"); - this.namedTopology = null; - } else { - this.namedTopology = namedTopology; - } } - @Override public String toString() { - return namedTopology != null ? namedTopology + "_" + topicGroupId + "_" + partition : topicGroupId + "_" + partition; - } - - public String toTaskDirString() { return topicGroupId + "_" + partition; } /** - * Parse the task directory name (of the form topicGroupId_partition) and construct the TaskId with the - * optional namedTopology (may be null) - * - * @throws TaskIdFormatException if the taskIdStr is not a valid {@link TaskId} + * @throws TaskIdFormatException if the taskIdStr is not a valid {@link TaskId} */ - public static TaskId parseTaskDirectoryName(final String taskIdStr, final String namedTopology) { + public static TaskId parse(final String taskIdStr) { final int index = taskIdStr.indexOf('_'); if (index <= 0 || index + 1 >= taskIdStr.length()) { throw new TaskIdFormatException(taskIdStr); @@ -83,7 +58,7 @@ public static TaskId parseTaskDirectoryName(final String taskIdStr, final String final int topicGroupId = Integer.parseInt(taskIdStr.substring(0, index)); final int partition = Integer.parseInt(taskIdStr.substring(index + 1)); - return new TaskId(topicGroupId, partition, namedTopology); + return new TaskId(topicGroupId, partition); } catch (final Exception e) { throw new TaskIdFormatException(taskIdStr); } @@ -92,78 +67,25 @@ public static TaskId parseTaskDirectoryName(final String taskIdStr, final String /** * @throws IOException if cannot write to output stream */ - public void writeTo(final DataOutputStream out, final int version) throws IOException { + public void writeTo(final DataOutputStream out) throws IOException { out.writeInt(topicGroupId); out.writeInt(partition); - if (version >= MIN_NAMED_TOPOLOGY_VERSION) { - if (namedTopology != null) { - out.writeInt(namedTopology.length()); - out.writeChars(namedTopology); - } else { - out.writeInt(0); - } - } } /** * @throws IOException if cannot read from input stream */ - public static TaskId readFrom(final DataInputStream in, final int version) throws IOException { - final int topicGroupId = in.readInt(); - final int partition = in.readInt(); - final String namedTopology; - if (version >= MIN_NAMED_TOPOLOGY_VERSION) { - final int numNamedTopologyChars = in.readInt(); - final StringBuilder namedTopologyBuilder = new StringBuilder(); - for (int i = 0; i < numNamedTopologyChars; ++i) { - namedTopologyBuilder.append(in.readChar()); - } - namedTopology = namedTopologyBuilder.toString(); - } else { - namedTopology = null; - } - return new TaskId(topicGroupId, partition, getNamedTopologyOrElseNull(namedTopology)); + public static TaskId readFrom(final DataInputStream in) throws IOException { + return new TaskId(in.readInt(), in.readInt()); } - public void writeTo(final ByteBuffer buf, final int version) { + public void writeTo(final ByteBuffer buf) { buf.putInt(topicGroupId); buf.putInt(partition); - if (version >= MIN_NAMED_TOPOLOGY_VERSION) { - if (namedTopology != null) { - buf.putInt(namedTopology.length()); - for (final char c : namedTopology.toCharArray()) { - buf.putChar(c); - } - } else { - buf.putInt(0); - } - } } - public static TaskId readFrom(final ByteBuffer buf, final int version) { - final int topicGroupId = buf.getInt(); - final int partition = buf.getInt(); - final String namedTopology; - if (version >= MIN_NAMED_TOPOLOGY_VERSION) { - final int numNamedTopologyChars = buf.getInt(); - final StringBuilder namedTopologyBuilder = new StringBuilder(); - for (int i = 0; i < numNamedTopologyChars; ++i) { - namedTopologyBuilder.append(buf.getChar()); - } - namedTopology = namedTopologyBuilder.toString(); - } else { - namedTopology = null; - } - return new TaskId(topicGroupId, partition, getNamedTopologyOrElseNull(namedTopology)); - } - - /** - * @return the namedTopology name, or null if the passed in namedTopology is null or the empty string - */ - private static String getNamedTopologyOrElseNull(final String namedTopology) { - return (namedTopology == null || namedTopology.length() == 0) ? - null : - namedTopology; + public static TaskId readFrom(final ByteBuffer buf) { + return new TaskId(buf.getInt(), buf.getInt()); } @Override @@ -171,39 +93,24 @@ public boolean equals(final Object o) { if (this == o) { return true; } - if (o == null || getClass() != o.getClass()) { - return false; - } - final TaskId taskId = (TaskId) o; - - if (topicGroupId != taskId.topicGroupId || partition != taskId.partition) { - return false; - } - if (namedTopology != null && taskId.namedTopology != null) { - return namedTopology.equals(taskId.namedTopology); + if (o instanceof TaskId) { + final TaskId other = (TaskId) o; + return other.topicGroupId == this.topicGroupId && other.partition == this.partition; } else { - return namedTopology == null && taskId.namedTopology == null; + return false; } } @Override public int hashCode() { - return Objects.hash(topicGroupId, partition, namedTopology); + final long n = ((long) topicGroupId << 32) | (long) partition; + return (int) (n % 0xFFFFFFFFL); } @Override public int compareTo(final TaskId other) { - if (namedTopology != null && other.namedTopology != null) { - final int comparingNamedTopologies = namedTopology.compareTo(other.namedTopology); - if (comparingNamedTopologies != 0) { - return comparingNamedTopologies; - } - } else if (namedTopology != null || other.namedTopology != null) { - LOG.error("Tried to compare this = {} with other = {}, but only one had a valid named topology", this, other); - throw new IllegalStateException("Can't compare a TaskId with a namedTopology to one without"); - } - final int comparingTopicGroupId = Integer.compare(this.topicGroupId, other.topicGroupId); - return comparingTopicGroupId != 0 ? comparingTopicGroupId : Integer.compare(this.partition, other.partition); + final int compare = Integer.compare(this.topicGroupId, other.topicGroupId); + return compare != 0 ? compare : Integer.compare(this.partition, other.partition); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TaskMetadata.java b/streams/src/main/java/org/apache/kafka/streams/processor/TaskMetadata.java index 91b06eacc8ecb..e3f1b8b78c02c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TaskMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TaskMetadata.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.TaskInfo; import java.util.Collections; import java.util.Map; @@ -30,7 +31,7 @@ */ public class TaskMetadata { - private final String taskId; + private final TaskInfo taskInfo; private final Set topicPartitions; @@ -40,20 +41,32 @@ public class TaskMetadata { private final Optional timeCurrentIdlingStarted; - public TaskMetadata(final String taskId, + public TaskMetadata(final TaskInfo taskInfo, final Set topicPartitions, final Map committedOffsets, final Map endOffsets, final Optional timeCurrentIdlingStarted) { - this.taskId = taskId; + this.taskInfo = taskInfo; this.topicPartitions = Collections.unmodifiableSet(topicPartitions); this.committedOffsets = Collections.unmodifiableMap(committedOffsets); this.endOffsets = Collections.unmodifiableMap(endOffsets); this.timeCurrentIdlingStarted = timeCurrentIdlingStarted; } + /** + * @return the TaskId with additional task metadata such as partition and group id + */ + public TaskInfo id() { + return taskInfo; + } + + /** + * @return a string representing the TaskId metadata such as partition and group id + * @deprecated please use {@link #id()} instead. + */ + @Deprecated public String taskId() { - return taskId; + return taskInfo.toString(); } public Set topicPartitions() { @@ -81,19 +94,19 @@ public boolean equals(final Object o) { return false; } final TaskMetadata that = (TaskMetadata) o; - return Objects.equals(taskId, that.taskId) && + return Objects.equals(taskInfo, that.taskInfo) && Objects.equals(topicPartitions, that.topicPartitions); } @Override public int hashCode() { - return Objects.hash(taskId, topicPartitions); + return Objects.hash(taskInfo, topicPartitions); } @Override public String toString() { return "TaskMetadata{" + - "taskId=" + taskId + + "taskId=" + taskInfo + ", topicPartitions=" + topicPartitions + ", committedOffsets=" + committedOffsets + ", endOffsets=" + endOffsets + diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessorContext.java index c591d510301f3..333512a80ca7d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessorContext.java @@ -18,11 +18,11 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.TaskInfo; import org.apache.kafka.streams.processor.Cancellable; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.TimestampExtractor; import java.io.File; @@ -45,12 +45,21 @@ public interface ProcessorContext { */ String applicationId(); + /** + * Returns the task info. + * + * @return the task info + */ + TaskInfo taskInfo(); + /** * Returns the task id. * * @return the task id + * @deprecated use {@link #taskInfo()} instead. */ - TaskId taskId(); + @Deprecated + org.apache.kafka.streams.processor.TaskId taskId(); /** * The metadata of the source record, if is one. Processors may be invoked to diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java index 09a2e31f04dc9..c729211bf18f5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java @@ -22,7 +22,7 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.TaskInfo; import org.apache.kafka.streams.processor.api.RecordMetadata; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; @@ -36,7 +36,7 @@ public abstract class AbstractProcessorContext implements InternalProcessorContext { - private final TaskId taskId; + private final org.apache.kafka.streams.processor.internals.TaskId taskId; private final String applicationId; private final StreamsConfig config; private final StreamsMetricsImpl metrics; @@ -48,7 +48,7 @@ public abstract class AbstractProcessorContext implements InternalProcessorConte private long cachedSystemTimeMs; protected ThreadCache cache; - public AbstractProcessorContext(final TaskId taskId, + public AbstractProcessorContext(final org.apache.kafka.streams.processor.internals.TaskId taskId, final StreamsConfig config, final StreamsMetricsImpl metrics, final ThreadCache cache) { @@ -79,10 +79,16 @@ public String applicationId() { } @Override - public TaskId taskId() { + public TaskInfo taskInfo() { return taskId; } + @Deprecated + @Override + public org.apache.kafka.streams.processor.TaskId taskId() { + return taskId.convertToOldTaskId(); + } + @Override public Serde keySerde() { return keySerde; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java index 771a40063dd82..5b2d7db0a1fa3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java @@ -23,7 +23,7 @@ import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; + import org.slf4j.Logger; import java.util.Collection; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java index 788ae737dc00f..6b2cc03fd9ead 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java @@ -26,7 +26,6 @@ import org.apache.kafka.streams.KafkaClientSupplier; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.ThreadMetrics; @@ -143,7 +142,7 @@ Collection createTasks(final Consumer consumer, final LogContext logContext = getLogContext(taskId); - final ProcessorTopology topology = builder.buildSubtopology(taskId.topicGroupId); + final ProcessorTopology topology = builder.buildSubtopology(taskId.subtopologyId()); final ProcessorStateManager stateManager = new ProcessorStateManager( taskId, @@ -194,7 +193,7 @@ StreamTask createActiveTaskFromStandby(final StandbyTask standbyTask, inputPartitions, consumer, logContext, - builder.buildSubtopology(standbyTask.id.topicGroupId), + builder.buildSubtopology(standbyTask.id.subtopologyId()), stateManager, context ); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogReader.java index 9c62dd182e8bd..54002c2c7626a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogReader.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.streams.processor.TaskId; import java.util.Map; import java.util.Set; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogTopics.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogTopics.java index c498bd030f9c8..a9cb5020eb787 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogTopics.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogTopics.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo; import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; @@ -75,7 +74,7 @@ public void setup() { final Set changelogTopicPartitions = topicsInfo.stateChangelogTopics .keySet() .stream() - .map(topic -> new TopicPartition(topic, task.partition)) + .map(topic -> new TopicPartition(topic, task.partition())) .collect(Collectors.toSet()); changelogPartitionsForStatefulTask.put(task, changelogTopicPartitions); } @@ -84,8 +83,8 @@ public void setup() { // the expected number of partitions is the max value of TaskId.partition + 1 int numPartitions = UNKNOWN; for (final TaskId task : topicGroupTasks) { - if (numPartitions < task.partition + 1) { - numPartitions = task.partition + 1; + if (numPartitions < task.partition() + 1) { + numPartitions = task.partition() + 1; } } topicConfig.setNumberOfPartitions(numPartitions); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ClientUtils.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ClientUtils.java index a5807b50dcb4e..f7ee87a738161 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ClientUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ClientUtils.java @@ -29,7 +29,7 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.TaskId; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java index aedd9551631a9..29bc45d5b63cd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ForwardingDisabledProcessorContext.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.TaskInfo; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.Cancellable; import org.apache.kafka.streams.processor.ProcessorContext; @@ -26,7 +27,6 @@ import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.To; import java.io.File; @@ -55,7 +55,13 @@ public String applicationId() { } @Override - public TaskId taskId() { + public TaskInfo taskInfo() { + return delegate.taskInfo(); + } + + @Deprecated + @Override + public org.apache.kafka.streams.processor.TaskId taskId() { return delegate.taskId(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java index 804a30a031076..adc70cc451e99 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java @@ -23,7 +23,6 @@ import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java index 6b1378b44bc83..06d9f09870fcd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java @@ -78,7 +78,7 @@ public Map initialize() { logContext, droppedRecordsSensorOrSkippedRecordsSensor( Thread.currentThread().getName(), - processorContext.taskId().toString(), + processorContext.taskInfo().toString(), processorContext.metrics() ) ) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGrouper.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGrouper.java index 69ee09e6113d8..f1dd5fd60c1ea 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGrouper.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGrouper.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; import org.slf4j.Logger; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index a3193735a003c..6b0c3251fa21a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -24,7 +24,6 @@ import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.Task.TaskType; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index a8c32c7882c9d..7ca8d6060abb7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -133,7 +133,7 @@ public void init(final InternalProcessorContext context) { private void initSensors() { threadId = Thread.currentThread().getName(); - final String taskId = internalProcessorContext.taskId().toString(); + final String taskId = internalProcessorContext.taskInfo().toString(); final StreamsMetricsImpl streamsMetrics = internalProcessorContext.metrics(); processSensor = ProcessorNodeMetrics.processSensor(threadId, taskId, name, streamsMetrics); punctuateSensor = ProcessorNodeMetrics.punctuateSensor(threadId, taskId, name, streamsMetrics); @@ -156,7 +156,7 @@ public void close() { ); internalProcessorContext.metrics().removeAllNodeLevelSensors( threadId, - internalProcessorContext.taskId().toString(), + internalProcessorContext.taskInfo().toString(), name ); } catch (final Exception e) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index 2b75d8ddcb579..bc62772038a97 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -28,7 +28,6 @@ import org.apache.kafka.streams.processor.StateRestoreListener; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.state.internals.CachedStateStore; import org.apache.kafka.streams.state.internals.OffsetCheckpoint; @@ -612,10 +611,7 @@ public void checkpoint() { } private TopicPartition getStorePartition(final String storeName) { - // NOTE we assume the partition of the topic can always be inferred from the task id; - // if user ever use a custom partition grouper (deprecated in KIP-528) this would break and - // it is not a regression (it would always break anyways) - return new TopicPartition(changelogFor(storeName), taskId.partition); + return new TopicPartition(changelogFor(storeName), taskId.partition()); } private boolean isLoggingEnabled(final String storeName) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index 16a451d0b1550..04c0c68752736 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -43,7 +43,6 @@ import org.apache.kafka.streams.errors.TaskCorruptedException; import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.processor.StreamPartitioner; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.slf4j.Logger; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java index df7e834f092b1..d487343b7a016 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java @@ -64,7 +64,7 @@ public class RecordQueue { this.processorContext = processorContext; droppedRecordsSensor = TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor( Thread.currentThread().getName(), - processorContext.taskId().toString(), + processorContext.taskInfo().toString(), processorContext.metrics() ); recordDeserializer = new RecordDeserializer( diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java index 7198f2f17e93e..c3521dc299309 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java @@ -67,7 +67,7 @@ public void init(final InternalProcessorContext context) { // change this. processAtSourceSensor = ProcessorNodeMetrics.processorAtSourceSensorOrForwardSensor( Thread.currentThread().getName(), - context.taskId().toString(), + context.taskInfo().toString(), context.currentNode().name(), context.metrics() ); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java index b386be859b8df..cfd7a542dffbd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java @@ -24,7 +24,6 @@ import org.apache.kafka.streams.StreamsMetrics; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskMigratedException; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.ThreadMetrics; import org.apache.kafka.streams.state.internals.ThreadCache; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTaskCreator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTaskCreator.java index 56d4220bf70a9..3b0a2a130a2de 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTaskCreator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTaskCreator.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.ThreadMetrics; @@ -74,7 +73,7 @@ Collection createTasks(final Map> tasksToBeCre final TaskId taskId = newTaskAndPartitions.getKey(); final Set partitions = newTaskAndPartitions.getValue(); - final ProcessorTopology topology = builder.buildSubtopology(taskId.topicGroupId); + final ProcessorTopology topology = builder.buildSubtopology(taskId.subtopologyId()); if (topology.hasStateWithChangelogs()) { final ProcessorStateManager stateManager = new ProcessorStateManager( @@ -120,7 +119,7 @@ StandbyTask createStandbyTaskFromActive(final StreamTask streamTask, return createStandbyTask( streamTask.id(), inputPartitions, - builder.buildSubtopology(streamTask.id.topicGroupId), + builder.buildSubtopology(streamTask.id.subtopologyId()), stateManager, context ); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java index 7f01264bbdf38..93c2161e6118e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java @@ -22,7 +22,6 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.TaskId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateManagerUtil.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateManagerUtil.java index d616d60dc530f..2be42696db688 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateManagerUtil.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateManagerUtil.java @@ -26,7 +26,6 @@ import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.state.internals.RecordConverter; import org.slf4j.Logger; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index fdf027f2be776..312b6f15ef81f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -34,7 +34,6 @@ import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskCorruptedException; import org.apache.kafka.streams.processor.StateRestoreListener; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.ProcessorStateManager.StateStoreMetadata; import org.slf4j.Logger; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreToProcessorContextAdapter.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreToProcessorContextAdapter.java index a4334dfa377ac..bbfcee302e511 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreToProcessorContextAdapter.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreToProcessorContextAdapter.java @@ -26,7 +26,7 @@ import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.TaskInfo; import org.apache.kafka.streams.processor.To; import java.io.File; @@ -54,7 +54,13 @@ public String applicationId() { } @Override - public TaskId taskId() { + public TaskInfo taskInfo() { + return delegate.taskInfo(); + } + + @Deprecated + @Override + public org.apache.kafka.streams.processor.TaskId taskId() { return delegate.taskId(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index d2f7926d46d9f..8705dd73f54d9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -37,7 +37,6 @@ import org.apache.kafka.streams.processor.Cancellable; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.TimestampExtractor; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index fd6f9f87f2d05..f46dd403a9e94 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -38,7 +38,6 @@ import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.internals.metrics.ClientMetrics; import org.apache.kafka.streams.processor.StateRestoreListener; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.TaskMetadata; import org.apache.kafka.streams.processor.ThreadMetadata; import org.apache.kafka.streams.processor.internals.assignment.AssignorError; @@ -1131,7 +1130,7 @@ private void updateThreadMetadata(final Map activeTasks, final Set activeTasksMetadata = new HashSet<>(); for (final Map.Entry task : activeTasks.entrySet()) { activeTasksMetadata.add(new TaskMetadata( - task.getValue().id().toString(), + task.getValue().id(), task.getValue().inputPartitions(), task.getValue().committedOffsets(), task.getValue().highWaterMark(), @@ -1141,7 +1140,7 @@ private void updateThreadMetadata(final Map activeTasks, final Set standbyTasksMetadata = new HashSet<>(); for (final Map.Entry task : standbyTasks.entrySet()) { standbyTasksMetadata.add(new TaskMetadata( - task.getValue().id().toString(), + task.getValue().id(), task.getValue().inputPartitions(), task.getValue().committedOffsets(), task.getValue().highWaterMark(), diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java index 6c3d6571def24..6cee5962ed23e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java @@ -35,7 +35,6 @@ import org.apache.kafka.streams.errors.MissingSourceTopicException; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskAssignmentException; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo; import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; import org.apache.kafka.streams.processor.internals.assignment.AssignmentInfo; @@ -50,7 +49,6 @@ import org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignor; import org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo; import org.apache.kafka.streams.processor.internals.assignment.TaskAssignor; -import org.apache.kafka.streams.processor.internals.namedtopology.NamedTaskId; import org.apache.kafka.streams.state.HostInfo; import org.slf4j.Logger; @@ -516,7 +514,7 @@ private void populateTasksForMaps(final Map taskForParti } allAssignedPartitions.addAll(partitions); - tasksForTopicGroup.computeIfAbsent(new Subtopology(id.topicGroupId, NamedTaskId.namedTopology(id)), k -> new HashSet<>()).add(id); + tasksForTopicGroup.computeIfAbsent(new Subtopology(id.subtopologyId(), id.namedTopology()), k -> new HashSet<>()).add(id); } checkAllPartitions(allSourceTopics, partitionsForTask, allAssignedPartitions, fullMetadata); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsProducer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsProducer.java index 9b27c8b55f3bb..ab28d67d6e834 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsProducer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsProducer.java @@ -39,7 +39,7 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskMigratedException; -import org.apache.kafka.streams.processor.TaskId; + import org.slf4j.Logger; import java.util.List; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java index feedeea0480a1..8f37385965468 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java @@ -23,7 +23,6 @@ import org.apache.kafka.streams.errors.LockException; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; import java.util.Arrays; import java.util.Collection; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskId.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskId.java new file mode 100644 index 0000000000000..0b486f1345327 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskId.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.processor.internals; + +import org.apache.kafka.streams.TaskInfo; +import org.apache.kafka.streams.errors.TaskIdFormatException; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Objects; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.MIN_NAMED_TOPOLOGY_VERSION; + +/** + * The task ID representation composed as topic group ID plus the assigned partition ID. + */ +public class TaskId implements Comparable, TaskInfo { + + private static final Logger LOG = LoggerFactory.getLogger(TaskId.class); + + /** The ID of the topic group. */ + private final int topicGroupId; + /** The ID of the partition. */ + private final int partition; + /** The namedTopology that this task belongs to, or null if it does not belong to one */ + private final String namedTopology; + + public TaskId(final int topicGroupId, final int partition) { + this(topicGroupId, partition, null); + } + + public TaskId(final int topicGroupId, final int partition, final String namedTopology) { + this.topicGroupId = topicGroupId; + this.partition = partition; + if (namedTopology != null && namedTopology.length() == 0) { + LOG.warn("Empty string passed in for task's namedTopology, since NamedTopology name cannot be empty, we " + + "assume this task does not belong to a NamedTopology and downgrade this to null"); + this.namedTopology = null; + } else { + this.namedTopology = namedTopology; + } + } + + @Override + public int subtopologyId() { + return topicGroupId; + } + + @Override + public int partition() { + return partition; + } + + public String namedTopology() { + return namedTopology; + } + + @SuppressWarnings("deprecation") + public org.apache.kafka.streams.processor.TaskId convertToOldTaskId() { + return new org.apache.kafka.streams.processor.TaskId(topicGroupId, partition); + } + + @Override + public String toString() { + return namedTopology != null ? namedTopology + "_" + topicGroupId + "_" + partition : topicGroupId + "_" + partition; + } + + /** + * Parse the task directory name (of the form topicGroupId_partition) and construct the TaskId with the + * optional namedTopology (may be null) + * + * @throws TaskIdFormatException if the taskIdStr is not a valid {@link TaskId} + */ + public static TaskId parseTaskDirectoryName(final String taskIdStr, final String namedTopology) { + final int index = taskIdStr.indexOf('_'); + if (index <= 0 || index + 1 >= taskIdStr.length()) { + throw new TaskIdFormatException(taskIdStr); + } + + try { + final int topicGroupId = Integer.parseInt(taskIdStr.substring(0, index)); + final int partition = Integer.parseInt(taskIdStr.substring(index + 1)); + + return new TaskId(topicGroupId, partition, namedTopology); + } catch (final Exception e) { + throw new TaskIdFormatException(taskIdStr); + } + } + + /** + * @throws IOException if cannot write to output stream + */ + public void writeTo(final DataOutputStream out, final int version) throws IOException { + out.writeInt(topicGroupId); + out.writeInt(partition); + if (version >= MIN_NAMED_TOPOLOGY_VERSION) { + if (namedTopology != null) { + out.writeInt(namedTopology.length()); + out.writeChars(namedTopology); + } else { + out.writeInt(0); + } + } + } + + /** + * @throws IOException if cannot read from input stream + */ + public static TaskId readFrom(final DataInputStream in, final int version) throws IOException { + final int topicGroupId = in.readInt(); + final int partition = in.readInt(); + final String namedTopology; + if (version >= MIN_NAMED_TOPOLOGY_VERSION) { + final int numNamedTopologyChars = in.readInt(); + final StringBuilder namedTopologyBuilder = new StringBuilder(); + for (int i = 0; i < numNamedTopologyChars; ++i) { + namedTopologyBuilder.append(in.readChar()); + } + namedTopology = namedTopologyBuilder.toString(); + } else { + namedTopology = null; + } + return new TaskId(topicGroupId, partition, getNamedTopologyOrElseNull(namedTopology)); + } + + public void writeTo(final ByteBuffer buf, final int version) { + buf.putInt(topicGroupId); + buf.putInt(partition); + if (version >= MIN_NAMED_TOPOLOGY_VERSION) { + if (namedTopology != null) { + buf.putInt(namedTopology.length()); + for (final char c : namedTopology.toCharArray()) { + buf.putChar(c); + } + } else { + buf.putInt(0); + } + } + } + + public static TaskId readFrom(final ByteBuffer buf, final int version) { + final int topicGroupId = buf.getInt(); + final int partition = buf.getInt(); + final String namedTopology; + if (version >= MIN_NAMED_TOPOLOGY_VERSION) { + final int numNamedTopologyChars = buf.getInt(); + final StringBuilder namedTopologyBuilder = new StringBuilder(); + for (int i = 0; i < numNamedTopologyChars; ++i) { + namedTopologyBuilder.append(buf.getChar()); + } + namedTopology = namedTopologyBuilder.toString(); + } else { + namedTopology = null; + } + return new TaskId(topicGroupId, partition, getNamedTopologyOrElseNull(namedTopology)); + } + + /** + * @return the namedTopology name, or null if the passed in namedTopology is null or the empty string + */ + private static String getNamedTopologyOrElseNull(final String namedTopology) { + return (namedTopology == null || namedTopology.length() == 0) ? + null : + namedTopology; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final TaskId taskId = (TaskId) o; + + if (topicGroupId != taskId.topicGroupId || partition != taskId.partition) { + return false; + } + + if (namedTopology != null && taskId.namedTopology != null) { + return namedTopology.equals(taskId.namedTopology); + } else { + return namedTopology == null && taskId.namedTopology == null; + } + } + + @Override + public int hashCode() { + return Objects.hash(topicGroupId, partition, namedTopology); + } + + @Override + public int compareTo(final TaskId other) { + if (namedTopology != null && other.namedTopology != null) { + final int comparingNamedTopologies = namedTopology.compareTo(other.namedTopology); + if (comparingNamedTopologies != 0) { + return comparingNamedTopologies; + } + } else if (namedTopology != null || other.namedTopology != null) { + LOG.error("Tried to compare this = {} with other = {}, but only one had a valid named topology", this, other); + throw new IllegalStateException("Can't compare a TaskId with a namedTopology to one without"); + } + final int comparingTopicGroupId = Integer.compare(this.topicGroupId, other.topicGroupId); + return comparingTopicGroupId != 0 ? comparingTopicGroupId : Integer.compare(this.partition, other.partition); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java index 0de51e56efe81..2078cc563867b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java @@ -35,7 +35,6 @@ import org.apache.kafka.streams.errors.TaskCorruptedException; import org.apache.kafka.streams.errors.TaskIdFormatException; import org.apache.kafka.streams.errors.TaskMigratedException; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.Task.State; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.internals.OffsetCheckpoint; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java index 4193deb6f7d70..d219c3dae7e8d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import java.util.HashSet; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java index 4fea12b4544fe..20e125fb3b823 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.ByteBufferInputStream; import org.apache.kafka.streams.errors.TaskAssignmentException; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.state.HostInfo; import org.slf4j.Logger; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java index 0029e73337824..f179992a726b5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java @@ -17,7 +17,7 @@ package org.apache.kafka.streams.processor.internals.assignment; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.Task; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTask.java index 92769699ccccd..38ae8a9b78600 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTask.java @@ -19,7 +19,7 @@ import java.util.Map; import java.util.Set; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; class ClientStateTask { private final Map> consumerToTaskIds; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ConstrainedPrioritySet.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ConstrainedPrioritySet.java index 1de9dfc3c67b9..4e9242c58d7eb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ConstrainedPrioritySet.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ConstrainedPrioritySet.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import java.util.Collection; import java.util.Comparator; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/FallbackPriorTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/FallbackPriorTaskAssignor.java index 58456ac7ac3fc..7ab5162c385a5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/FallbackPriorTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/FallbackPriorTaskAssignor.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.assignment.AssignorConfiguration.AssignmentConfigs; import java.util.Map; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java index f6464f8ac3117..6f9e6054f100f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignor.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.Task; import org.apache.kafka.streams.processor.internals.assignment.AssignorConfiguration.AssignmentConfigs; import org.slf4j.Logger; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java index d9f7efa2a16ca..82500ea6f43e8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignor.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.assignment.AssignorConfiguration.AssignmentConfigs; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java index 5f6935f41dcb7..5a2b2c9a8212f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java @@ -29,7 +29,7 @@ import org.apache.kafka.streams.internals.generated.SubscriptionInfoData; import org.apache.kafka.streams.internals.generated.SubscriptionInfoData.PartitionToOffsetSum; import org.apache.kafka.streams.internals.generated.SubscriptionInfoData.TaskOffsetSum; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.Task; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -129,9 +129,9 @@ private void setTaskOffsetSumDataFromTaskOffsetSumMap(final Map ta final Map> topicGroupIdToPartitionOffsetSum = new HashMap<>(); for (final Map.Entry taskEntry : taskOffsetSums.entrySet()) { final TaskId task = taskEntry.getKey(); - topicGroupIdToPartitionOffsetSum.computeIfAbsent(task.topicGroupId, t -> new ArrayList<>()).add( + topicGroupIdToPartitionOffsetSum.computeIfAbsent(task.subtopologyId(), t -> new ArrayList<>()).add( new SubscriptionInfoData.PartitionToOffsetSum() - .setPartition(task.partition) + .setPartition(task.partition()) .setOffsetSum(taskEntry.getValue())); } @@ -157,14 +157,14 @@ private void setPrevAndStandbySetsFromParsedTaskOffsetSumMap(final Map { final SubscriptionInfoData.TaskId taskId = new SubscriptionInfoData.TaskId(); - taskId.setTopicGroupId(t.topicGroupId); - taskId.setPartition(t.partition); + taskId.setTopicGroupId(t.subtopologyId()); + taskId.setPartition(t.partition()); return taskId; }).collect(Collectors.toList())); data.setStandbyTasks(standbyTasks.stream().map(t -> { final SubscriptionInfoData.TaskId taskId = new SubscriptionInfoData.TaskId(); - taskId.setTopicGroupId(t.topicGroupId); - taskId.setPartition(t.partition); + taskId.setTopicGroupId(t.subtopologyId()); + taskId.setPartition(t.partition()); return taskId; }).collect(Collectors.toList())); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignor.java index aeb2192c63ef2..dbf771c287f41 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignor.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import java.util.Map; import java.util.Set; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovement.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovement.java index cbfa3daaedf90..c7c3a2838b859 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovement.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovement.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import java.util.Collections; import java.util.Comparator; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java index 51fb3ca2fd7b3..2e78dc04c08dd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java @@ -242,7 +242,7 @@ public void init(final ProcessorContext context, final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context); final String threadId = Thread.currentThread().getName(); - final String taskName = context.taskId().toString(); + final String taskName = context.taskInfo().toString(); expiredRecordSensor = TaskMetrics.droppedRecordsSensorOrExpiredWindowRecordDropSensor( threadId, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java index 224d1b222b57e..81b0a5748fdf2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java @@ -81,7 +81,7 @@ public void init(final StateStoreContext context, private void initInternal(final InternalProcessorContext context) { this.context = context; - this.cacheName = ThreadCache.nameSpaceFromTaskIdAndStore(context.taskId().toString(), name()); + this.cacheName = ThreadCache.nameSpaceFromTaskIdAndStore(context.taskInfo().toString(), name()); this.context.registerCacheFlushListener(cacheName, entries -> { for (final ThreadCache.DirtyEntry entry : entries) { putAndMaybeForward(entry, context); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java index 1cfb8ce498a64..344c17e27986a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java @@ -79,7 +79,7 @@ public void init(final StateStoreContext context, final StateStore root) { private void initInternal(final InternalProcessorContext context) { this.context = context; - cacheName = context.taskId() + "-" + name(); + cacheName = context.taskInfo() + "-" + name(); context.registerCacheFlushListener(cacheName, entries -> { for (final ThreadCache.DirtyEntry entry : entries) { putAndMaybeForward(entry, context); 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 ad5e92cec7112..b8ecb18600a84 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 @@ -90,7 +90,7 @@ private void initInternal(final InternalProcessorContext context) { topic, Serdes.Bytes(), Serdes.ByteArray()); - cacheName = context.taskId() + "-" + name(); + cacheName = context.taskInfo() + "-" + name(); context.registerCacheFlushListener(cacheName, entries -> { for (final ThreadCache.DirtyEntry entry : entries) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java index affd47ac14292..bdc00d039e049 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java @@ -76,7 +76,7 @@ public String name() { @Override public void init(final ProcessorContext context, final StateStore root) { final String threadId = Thread.currentThread().getName(); - final String taskName = context.taskId().toString(); + final String taskName = context.taskInfo().toString(); // The provided context is not required to implement InternalProcessorContext, // If it doesn't, we can't record this metric. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java index 3099e9ca3a45c..5fa487ea4870f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryTimeOrderedKeyValueBuffer.java @@ -209,7 +209,7 @@ public void init(final StateStoreContext context, final StateStore root) { } private void init(final StateStore root) { - taskId = context.taskId().toString(); + taskId = context.taskInfo().toString(); streamsMetrics = context.metrics(); threadId = Thread.currentThread().getName(); @@ -232,7 +232,7 @@ private void init(final StateStore root) { changelogTopic = ProcessorStateManager.storeChangelogTopic(context.applicationId(), storeName); updateBufferMetrics(); open = true; - partition = context.taskId().partition; + partition = context.taskInfo().partition(); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java index 5dbfc0dfe02bd..96bcff541a75e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java @@ -92,7 +92,7 @@ public void init(final ProcessorContext context, final StateStore root) { final StreamsMetricsImpl metrics = ProcessorContextUtils.getMetricsImpl(context); final String threadId = Thread.currentThread().getName(); - final String taskName = context.taskId().toString(); + final String taskName = context.taskInfo().toString(); expiredRecordSensor = TaskMetrics.droppedRecordsSensorOrExpiredWindowRecordDropSensor( threadId, taskName, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueSegments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueSegments.java index a17666e2c8d21..a1d92a00d116f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueSegments.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueSegments.java @@ -18,6 +18,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorContextUtils; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder; /** @@ -55,7 +56,7 @@ public KeyValueSegment getOrCreateSegment(final long segmentId, @Override public void openExisting(final ProcessorContext context, final long streamTime) { - metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId()); + metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), (TaskId) context.taskInfo()); super.openExisting(context, streamTime); } } \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java index 18c44e8a49faa..e98150891a061 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java @@ -95,7 +95,7 @@ public class MeteredKeyValueStore public void init(final ProcessorContext context, final StateStore root) { this.context = context instanceof InternalProcessorContext ? (InternalProcessorContext) context : null; - taskId = context.taskId().toString(); + taskId = context.taskInfo().toString(); initStoreSerde(context); streamsMetrics = (StreamsMetricsImpl) context.metrics(); @@ -111,7 +111,7 @@ public void init(final ProcessorContext context, public void init(final StateStoreContext context, final StateStore root) { this.context = context instanceof InternalProcessorContext ? (InternalProcessorContext) context : null; - taskId = context.taskId().toString(); + taskId = context.taskInfo().toString(); initStoreSerde(context); streamsMetrics = (StreamsMetricsImpl) context.metrics(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java index 1fbc8db095799..d335deacad3f9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java @@ -77,7 +77,7 @@ public void init(final ProcessorContext context, final StateStore root) { this.context = context instanceof InternalProcessorContext ? (InternalProcessorContext) context : null; initStoreSerde(context); - taskId = context.taskId().toString(); + taskId = context.taskInfo().toString(); streamsMetrics = (StreamsMetricsImpl) context.metrics(); registerMetrics(); @@ -93,7 +93,7 @@ public void init(final StateStoreContext context, final StateStore root) { this.context = context instanceof InternalProcessorContext ? (InternalProcessorContext) context : null; initStoreSerde(context); - taskId = context.taskId().toString(); + taskId = context.taskInfo().toString(); streamsMetrics = (StreamsMetricsImpl) context.metrics(); registerMetrics(); 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 91b4387fbbdc5..d406d7c7799e9 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 @@ -82,7 +82,7 @@ public void init(final ProcessorContext context, this.context = context instanceof InternalProcessorContext ? (InternalProcessorContext) context : null; initStoreSerde(context); streamsMetrics = (StreamsMetricsImpl) context.metrics(); - taskId = context.taskId().toString(); + taskId = context.taskInfo().toString(); registerMetrics(); final Sensor restoreSensor = @@ -98,7 +98,7 @@ public void init(final StateStoreContext context, this.context = context instanceof InternalProcessorContext ? (InternalProcessorContext) context : null; initStoreSerde(context); streamsMetrics = (StreamsMetricsImpl) context.metrics(); - taskId = context.taskId().toString(); + taskId = context.taskInfo().toString(); registerMetrics(); final Sensor restoreSensor = diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index f501d139378c1..7432fe11f5b7e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -28,6 +28,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreContext; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.RocksDBConfigSetter; @@ -234,7 +235,7 @@ void openRocksDB(final DBOptions dbOptions, public void init(final ProcessorContext context, final StateStore root) { // open the DB dir - metricsRecorder.init(getMetricsImpl(context), context.taskId()); + metricsRecorder.init(getMetricsImpl(context), (TaskId) context.taskInfo()); openDB(context.appConfigs(), context.stateDir()); // value getter should always read directly from rocksDB @@ -246,7 +247,7 @@ public void init(final ProcessorContext context, public void init(final StateStoreContext context, final StateStore root) { // open the DB dir - metricsRecorder.init(getMetricsImpl(context), context.taskId()); + metricsRecorder.init(getMetricsImpl(context), (TaskId) context.taskInfo()); openDB(context.appConfigs(), context.stateDir()); // value getter should always read directly from rocksDB diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java index df541d3e2c9dc..1de6b09444bca 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java @@ -19,7 +19,7 @@ import org.apache.kafka.streams.StoreQueryParameters; import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.StreamThread; import org.apache.kafka.streams.processor.internals.Task; import org.apache.kafka.streams.state.QueryableStoreType; @@ -54,7 +54,7 @@ public List stores(final StoreQueryParameters storeQueryParams) { if (storeQueryParams.partition() != null) { for (final Task task : tasks) { - if (task.id().partition == storeQueryParams.partition() && + if (task.id().partition() == storeQueryParams.partition() && task.getStore(storeName) != null && storeName.equals(task.getStore(storeName).name())) { final T typedStore = validateAndCastStores(task.getStore(storeName), queryableStoreType, storeName, task.id()); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedSegments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedSegments.java index 7318208f956a4..ff9f8f5dbb921 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedSegments.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedSegments.java @@ -18,6 +18,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorContextUtils; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder; /** @@ -55,7 +56,7 @@ public TimestampedSegment getOrCreateSegment(final long segmentId, @Override public void openExisting(final ProcessorContext context, final long streamTime) { - metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), context.taskId()); + metricsRecorder.init(ProcessorContextUtils.getMetricsImpl(context), (TaskId) context.taskInfo()); super.openExisting(context, streamTime); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java index 85412d1c5e8e0..e42bf3bafb6cb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorder.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.streams.errors.ProcessorStateException; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.internals.metrics.RocksDBMetrics.RocksDBMetricContext; import org.rocksdb.Cache; 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 90890686af4c0..56d3eda52348d 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 @@ -43,7 +43,7 @@ import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateRestoreListener; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.StateDirectory; import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier; import org.apache.kafka.streams.state.KeyValueStore; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java index 5ba607528faca..e7f81c5057095 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java @@ -32,7 +32,7 @@ import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.Transformer; import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.StateDirectory; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.QueryableStoreTypes; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java index cc1b445525f62..97e8b42a2d20e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java @@ -233,7 +233,7 @@ private static class ShutdownProcessor extends AbstractProcessor @Override public void process(final String key, final String value) { - valueList.add(value + " " + context.taskId()); + valueList.add(value + " " + context.taskInfo()); if (throwError.get()) { throw new StreamsException(Thread.currentThread().getName()); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java index 09823376cf052..ddff394f31fac 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java @@ -115,7 +115,7 @@ public void forward(final Object key, final Object value, final To to) { results.add(new KeyValueTimestamp<>((Windowed) key, (Change) value, toInternal.timestamp())); } }; - TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), streamsMetrics); + TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskInfo().toString(), streamsMetrics); initStore(enableCache); processor.init(context); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorMetricsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorMetricsTest.java index 3b46765fe20fe..a884bc6818808 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorMetricsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorMetricsTest.java @@ -28,7 +28,7 @@ import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.ProcessorNode; import org.apache.kafka.streams.state.internals.InMemoryTimeOrderedKeyValueBuffer; import org.apache.kafka.test.MockInternalProcessorContext; 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 1f4cc824e486b..3b05e7322ca58 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 @@ -28,7 +28,6 @@ import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.RocksDBConfigSetter; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java index 54c92dfef4caf..6e1ffa1dd2474 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java @@ -26,7 +26,6 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.TimestampExtractor; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.internals.ThreadCache; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ChangelogTopicsTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ChangelogTopicsTest.java index 17db61d7c5b13..4aa1b183a443f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ChangelogTopicsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ChangelogTopicsTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo; import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/HighAvailabilityStreamsPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/HighAvailabilityStreamsPartitionAssignorTest.java index d020cb37f1367..83331d20fe9ae 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/HighAvailabilityStreamsPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/HighAvailabilityStreamsPartitionAssignorTest.java @@ -33,7 +33,6 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsConfig.InternalConfig; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.assignment.AssignmentInfo; import org.apache.kafka.streams.processor.internals.assignment.AssignorError; import org.apache.kafka.streams.processor.internals.assignment.ReferenceContainer; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockChangelogReader.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockChangelogReader.java index 6ea7fc3101561..9638ecfea3863 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockChangelogReader.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MockChangelogReader.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.streams.processor.TaskId; import java.util.Collection; import java.util.Collections; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGrouperTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGrouperTest.java index 93d2dced7b25f..e4011879f3293 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGrouperTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGrouperTest.java @@ -20,7 +20,6 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; import org.junit.Test; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java index 5694c520a9dcd..17239df585335 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java @@ -26,7 +26,6 @@ import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextTest.java index 46d8f70567dfc..b56c26055b5c6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextTest.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.internals.ThreadCache; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java index 84bdf51cc8893..29d047e45e539 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java @@ -121,7 +121,7 @@ private void testMetrics(final String builtInMetricsVersion) { final String threadIdTagKey = StreamsConfig.METRICS_0100_TO_24.equals(builtInMetricsVersion) ? "client-id" : "thread-id"; metricTags.put("processor-node-id", node.name()); - metricTags.put("task-id", context.taskId().toString()); + metricTags.put("task-id", context.taskInfo().toString()); metricTags.put(threadIdTagKey, threadId); if (StreamsConfig.METRICS_0100_TO_24.equals(builtInMetricsVersion)) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java index cf6c61a9a17f8..7f5abb3820d56 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java @@ -29,7 +29,6 @@ import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.streams.processor.internals.ProcessorStateManager.StateStoreMetadata; import org.apache.kafka.streams.state.TimestampedBytesStore; @@ -499,7 +498,7 @@ public void shouldGetChangelogPartitionForRegisteredStore() { final TopicPartition changelogPartition = stateMgr.registeredChangelogPartitionFor(persistentStoreName); assertThat(changelogPartition.topic(), is(persistentStoreTopicName)); - assertThat(changelogPartition.partition(), is(taskId.partition)); + assertThat(changelogPartition.partition(), is(taskId.partition())); } @Test 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 5a83c6839d3d5..e3b96d15b87f5 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 @@ -50,7 +50,6 @@ import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.processor.StreamPartitioner; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.test.MockClientSupplier; 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 92e47194becbe..07d69f9cf602a 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 @@ -95,7 +95,7 @@ private void shouldExposeProcessMetrics(final String builtInMetricsVersion) { StreamsConfig.METRICS_0100_TO_24.equals(builtInMetricsVersion) ? "client-id" : "thread-id"; final Map metricTags = mkMap( mkEntry(threadIdTagKey, threadId), - mkEntry("task-id", context.taskId().toString()), + mkEntry("task-id", context.taskInfo().toString()), mkEntry("processor-node-id", node.name()) ); @@ -118,7 +118,7 @@ private void shouldExposeProcessMetrics(final String builtInMetricsVersion) { assertTrue(StreamsTestUtils.containsMetric(metrics, "process-rate", parentGroupName, metricTags)); assertTrue(StreamsTestUtils.containsMetric(metrics, "process-total", parentGroupName, metricTags)); - final String sensorNamePrefix = "internal." + threadId + ".task." + context.taskId().toString(); + final String sensorNamePrefix = "internal." + threadId + ".task." + context.taskInfo().toString(); final Sensor processSensor = metrics.getSensor(sensorNamePrefix + ".node." + context.currentNode().name() + ".s.process"); final SensorAccessor sensorAccessor = new SensorAccessor(processSensor); 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 60a95d4834427..88e3c49a83ace 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 @@ -32,7 +32,6 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.LockException; import org.apache.kafka.streams.errors.ProcessorStateException; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.internals.ThreadCache; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java index b17169c64f143..78f5f536c1791 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java @@ -28,7 +28,6 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.ProcessorStateException; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.streams.state.internals.OffsetCheckpoint; import org.apache.kafka.test.TestUtils; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateManagerUtilTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateManagerUtilTest.java index bc7fb14ba7da7..01bd5ae53ca24 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateManagerUtilTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateManagerUtilTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.streams.errors.LockException; import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.test.MockKeyValueStore; import org.apache.kafka.test.TestUtils; 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 594fc7e842ffc..fb169b41a6494 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 @@ -33,7 +33,6 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.ProcessorStateManager.StateStoreMetadata; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.test.MockStateRestoreListener; 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 5eeaaa24524f1..9cb72d345450f 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 @@ -53,7 +53,6 @@ import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.Task.TaskType; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; 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 71e6af16bf088..381334fd7a440 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 @@ -62,7 +62,6 @@ import org.apache.kafka.streams.kstream.internals.MaterializedInternal; import org.apache.kafka.streams.processor.LogAndSkipOnInvalidTimestamp; import org.apache.kafka.streams.processor.PunctuationType; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.TaskMetadata; import org.apache.kafka.streams.processor.ThreadMetadata; import org.apache.kafka.streams.processor.api.ProcessorSupplier; @@ -1682,7 +1681,7 @@ public void shouldReturnActiveTaskMetadataWhileRunningState() { final ThreadMetadata metadata = thread.threadMetadata(); assertEquals(StreamThread.State.RUNNING.name(), metadata.threadState()); - assertTrue(metadata.activeTasks().contains(new TaskMetadata(task1.toString(), Utils.mkSet(t1p1), new HashMap<>(), new HashMap<>(), Optional.empty()))); + assertTrue(metadata.activeTasks().contains(new TaskMetadata(task1, Utils.mkSet(t1p1), new HashMap<>(), new HashMap<>(), Optional.empty()))); assertTrue(metadata.standbyTasks().isEmpty()); assertTrue("#threadState() was: " + metadata.threadState() + "; expected either RUNNING, STARTING, PARTITIONS_REVOKED, PARTITIONS_ASSIGNED, or CREATED", @@ -1735,7 +1734,7 @@ public void shouldReturnStandbyTaskMetadataWhileRunningState() { final ThreadMetadata threadMetadata = thread.threadMetadata(); assertEquals(StreamThread.State.RUNNING.name(), threadMetadata.threadState()); - assertTrue(threadMetadata.standbyTasks().contains(new TaskMetadata(task1.toString(), Utils.mkSet(t1p1), new HashMap<>(), new HashMap<>(), Optional.empty()))); + assertTrue(threadMetadata.standbyTasks().contains(new TaskMetadata(task1, Utils.mkSet(t1p1), new HashMap<>(), new HashMap<>(), Optional.empty()))); assertTrue(threadMetadata.activeTasks().isEmpty()); thread.taskManager().shutdown(true); 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 3ef892a7c803d..d99b124863a70 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 @@ -55,7 +55,6 @@ import org.apache.kafka.streams.kstream.internals.ConsumedInternal; import org.apache.kafka.streams.kstream.internals.InternalStreamsBuilder; import org.apache.kafka.streams.kstream.internals.MaterializedInternal; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; import org.apache.kafka.streams.processor.internals.assignment.AssignmentInfo; import org.apache.kafka.streams.processor.internals.assignment.AssignorConfiguration; @@ -830,7 +829,7 @@ private static Set tasksForState(final String storeName, if (stateChangelogTopics.contains(changelogTopic)) { for (final TaskId id : tasks) { - if (id.topicGroupId == entry.getKey().nodeGroupId) { + if (id.subtopologyId() == entry.getKey().nodeGroupId) { ids.add(id); } } @@ -2108,8 +2107,7 @@ private static AssignmentInfo checkAssignment(final Set expectedTopics, final TaskId id = entry.getKey(); final Set partitions = entry.getValue(); for (final TopicPartition partition : partitions) { - // since default grouper, taskid.partition == partition.partition() - assertEquals(id.partition, partition.partition()); + assertEquals(id.partition(), partition.partition()); standbyTopics.add(partition.topic()); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsProducerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsProducerTest.java index f4dec89829464..161197d16ca8b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsProducerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsProducerTest.java @@ -37,7 +37,6 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskMigratedException; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.test.MockClientSupplier; import org.junit.Before; import org.junit.Test; 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 07fc378e18514..9c7fcf7cbedd5 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 @@ -42,7 +42,6 @@ import org.apache.kafka.streams.errors.TaskCorruptedException; import org.apache.kafka.streams.errors.TaskMigratedException; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.StreamThread.ProcessingMode; import org.apache.kafka.streams.processor.internals.Task.State; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java index a652759c271bd..36bc838b6acda 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.streams.processor.internals.assignment; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.state.HostInfo; import org.junit.Test; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentTestUtils.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentTestUtils.java index 1afa76f22a6bf..7f5067220de9f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentTestUtils.java @@ -23,7 +23,7 @@ import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.internals.KafkaFutureImpl; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.Task; import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; @@ -375,7 +375,7 @@ static TaskSkewReport analyzeTaskAssignmentBalance(final Map final UUID client = entry.getKey(); final ClientState clientState = entry.getValue(); for (final TaskId task : clientState.activeTasks()) { - final int subtopology = task.topicGroupId; + final int subtopology = task.subtopologyId(); subtopologyToClientsWithPartition .computeIfAbsent(subtopology, initialClientCounts) .get(client) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java index e1853dcf94294..cb50a67d8b22b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.streams.processor.internals.assignment; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.Task; import org.junit.Test; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ConstrainedPrioritySetTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ConstrainedPrioritySetTest.java index d14e279eaa0dc..312f3ec5458f5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ConstrainedPrioritySetTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ConstrainedPrioritySetTest.java @@ -17,7 +17,7 @@ package org.apache.kafka.streams.processor.internals.assignment; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.junit.Test; import java.util.UUID; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/FallbackPriorTaskAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/FallbackPriorTaskAssignorTest.java index 491fff1df63eb..9b9f510b3e62b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/FallbackPriorTaskAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/FallbackPriorTaskAssignorTest.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.junit.Test; import java.util.Collections; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignorTest.java index a2d4716b45870..90a1215b6df60 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/HighAvailabilityTaskAssignorTest.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.assignment.AssignorConfiguration.AssignmentConfigs; import org.junit.Test; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/LegacySubscriptionInfoSerde.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/LegacySubscriptionInfoSerde.java index cdd651f239b47..fef3c560cfa1b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/LegacySubscriptionInfoSerde.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/LegacySubscriptionInfoSerde.java @@ -19,7 +19,7 @@ import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.LATEST_SUPPORTED_VERSION; import org.apache.kafka.streams.errors.TaskAssignmentException; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java index 47ca31922301e..1767c0ef1c59b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.junit.Test; import java.util.ArrayList; @@ -733,7 +733,7 @@ private void assertActiveTaskTopicGroupIdsEvenlyDistributed() { final List topicGroupIds = new ArrayList<>(); final Set activeTasks = clientStateEntry.getValue().activeTasks(); for (final TaskId activeTask : activeTasks) { - topicGroupIds.add(activeTask.topicGroupId); + topicGroupIds.add(activeTask.subtopologyId()); } Collections.sort(topicGroupIds); assertThat(topicGroupIds, equalTo(expectedTopicGroupIds)); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfoTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfoTest.java index f81c0cf2fec11..e647c91a78bac 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfoTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfoTest.java @@ -17,8 +17,9 @@ package org.apache.kafka.streams.processor.internals.assignment; import java.util.Map; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.Task; +import org.apache.kafka.streams.processor.internals.TaskId; + import org.junit.Test; import java.nio.ByteBuffer; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignorConvergenceTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignorConvergenceTest.java index 68c9dfec107de..f9d7cb1161e64 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignorConvergenceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskAssignorConvergenceTest.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.assignment.AssignorConfiguration.AssignmentConfigs; import org.junit.Test; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovementTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovementTest.java index 9b58d18c88f60..c9145cf839f1e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovementTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/TaskMovementTest.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.junit.Test; import java.util.Collection; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java index 38e98603c7be0..31bfde23dae2a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java @@ -30,7 +30,7 @@ import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreContext; import org.apache.kafka.streams.processor.StreamPartitioner; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.apache.kafka.streams.processor.internals.RecordCollector; import org.apache.kafka.streams.processor.internals.RecordCollectorImpl; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStoreTest.java index 8fdbd33cef964..de89603918c91 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStoreTest.java @@ -21,7 +21,7 @@ import org.apache.kafka.streams.kstream.internals.SessionWindow; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.ProcessorContextImpl; import org.apache.kafka.streams.state.SessionStore; import org.apache.kafka.test.MockRecordCollector; @@ -55,7 +55,7 @@ public void setUp() { } private void init() { - EasyMock.expect(context.taskId()).andReturn(taskId); + EasyMock.expect(context.taskInfo()).andReturn(taskId); EasyMock.expect(context.recordCollector()).andReturn(collector); inner.init((StateStoreContext) context, store); EasyMock.expectLastCall(); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStoreTest.java index 50c18fe0573e5..17d19fb234754 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStoreTest.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.ProcessorContextImpl; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.test.MockRecordCollector; @@ -57,7 +57,7 @@ public void setUp() { } private void init() { - EasyMock.expect(context.taskId()).andReturn(taskId); + EasyMock.expect(context.taskInfo()).andReturn(taskId); EasyMock.expect(context.recordCollector()).andReturn(collector); inner.init((StateStoreContext) context, store); EasyMock.expectLastCall(); 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 36e3297b560c1..36a873b622344 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 @@ -20,7 +20,7 @@ import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.ProcessorContextImpl; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.test.MockRecordCollector; @@ -55,7 +55,7 @@ public void setUp() { } private void init() { - EasyMock.expect(context.taskId()).andReturn(taskId); + EasyMock.expect(context.taskInfo()).andReturn(taskId); EasyMock.expect(context.recordCollector()).andReturn(collector); inner.init((StateStoreContext) context, store); EasyMock.expectLastCall(); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProviderTest.java index bd23173b52115..84e337216a477 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProviderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/GlobalStateStoreProviderTest.java @@ -24,7 +24,7 @@ import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.ProcessorContextImpl; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.QueryableStoreTypes; @@ -108,7 +108,7 @@ public void before() { .andStubReturn( new StreamsMetricsImpl(new Metrics(), "threadName", StreamsConfig.METRICS_LATEST, new MockTime()) ); - expect(mockContext.taskId()).andStubReturn(new TaskId(0, 0)); + expect(mockContext.taskInfo()).andStubReturn(new TaskId(0, 0)); expect(mockContext.recordCollector()).andStubReturn(null); expectSerdes(mockContext); replay(mockContext); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentTest.java index 859aea1168a96..3657bd49ef818 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentTest.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder; import org.apache.kafka.test.TestUtils; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java index 095ff87f6f77f..184f3a650320f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java @@ -35,7 +35,7 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; @@ -141,7 +141,7 @@ public void before() { expect(context.metrics()).andStubReturn( new StreamsMetricsImpl(metrics, "test", builtInMetricsVersion, mockTime) ); - expect(context.taskId()).andStubReturn(taskId); + expect(context.taskInfo()).andStubReturn(taskId); expect(context.changelogFor(STORE_NAME)).andStubReturn(CHANGELOG_TOPIC); expect(inner.name()).andStubReturn(STORE_NAME); storeLevelGroup = diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java index 07ffed3b31056..03e805af7430f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java @@ -37,7 +37,7 @@ import org.apache.kafka.streams.kstream.internals.SessionWindow; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; @@ -144,7 +144,7 @@ public void before() { expect(context.applicationId()).andStubReturn(APPLICATION_ID); expect(context.metrics()) .andStubReturn(new StreamsMetricsImpl(metrics, "test", builtInMetricsVersion, mockTime)); - expect(context.taskId()).andStubReturn(taskId); + expect(context.taskInfo()).andStubReturn(taskId); expect(context.changelogFor(STORE_NAME)).andStubReturn(CHANGELOG_TOPIC); expect(innerStore.name()).andStubReturn(STORE_NAME); storeLevelGroup = diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreTest.java index c0ef7c61f627d..e142f77444567 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStoreTest.java @@ -35,7 +35,7 @@ import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; @@ -143,7 +143,7 @@ public void before() { expect(context.applicationId()).andStubReturn(APPLICATION_ID); expect(context.metrics()) .andStubReturn(new StreamsMetricsImpl(metrics, "test", builtInMetricsVersion, mockTime)); - expect(context.taskId()).andStubReturn(taskId); + expect(context.taskInfo()).andStubReturn(taskId); expect(context.changelogFor(STORE_NAME)).andStubReturn(CHANGELOG_TOPIC); expectSerdes(); expect(inner.name()).andStubReturn(STORE_NAME); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java index 538c8d3173a69..be100b165ed1a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java @@ -148,7 +148,7 @@ public void setUp() { StreamsConfig.METRICS_0100_TO_24.equals(builtInMetricsVersion) ? THREAD_ID_TAG_KEY_FROM_0100_TO_24 : THREAD_ID_TAG_KEY; tags = mkMap( mkEntry(threadIdTagKey, threadId), - mkEntry("task-id", context.taskId().toString()), + mkEntry("task-id", context.taskInfo().toString()), mkEntry(STORE_TYPE + "-state-id", STORE_NAME) ); } @@ -249,7 +249,7 @@ public void testMetrics() { storeLevelGroup, threadIdTagKey, threadId, - context.taskId().toString(), + context.taskInfo().toString(), STORE_TYPE, STORE_NAME ))); @@ -259,7 +259,7 @@ public void testMetrics() { storeLevelGroup, threadIdTagKey, threadId, - context.taskId().toString(), + context.taskInfo().toString(), STORE_TYPE, ROLLUP_VALUE ))); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java index c9e9a8d94692c..a33fca6e1190b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java @@ -34,7 +34,7 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.RocksDBConfigSetter; @@ -741,7 +741,7 @@ public void shouldVerifyThatMetricsRecordedFromStatisticsGetMeasurementsFromRock context = EasyMock.niceMock(InternalMockProcessorContext.class); EasyMock.expect(context.metrics()).andStubReturn(streamsMetrics); - EasyMock.expect(context.taskId()).andStubReturn(taskId); + EasyMock.expect(context.taskInfo()).andStubReturn(taskId); EasyMock.expect(context.appConfigs()) .andStubReturn(new StreamsConfig(StreamsTestUtils.getStreamsConfig()).originals()); EasyMock.expect(context.stateDir()).andStubReturn(dir); @@ -773,7 +773,7 @@ public void shouldVerifyThatMetricsRecordedFromPropertiesGetMeasurementsFromRock context = EasyMock.niceMock(InternalMockProcessorContext.class); EasyMock.expect(context.metrics()).andStubReturn(streamsMetrics); - EasyMock.expect(context.taskId()).andStubReturn(taskId); + EasyMock.expect(context.taskInfo()).andStubReturn(taskId); EasyMock.expect(context.appConfigs()) .andStubReturn(new StreamsConfig(StreamsTestUtils.getStreamsConfig()).originals()); EasyMock.expect(context.stateDir()).andStubReturn(dir); @@ -805,7 +805,7 @@ public void shouldVerifyThatPropertyBasedMetricsUseValidPropertyName() { final Properties props = StreamsTestUtils.getStreamsConfig(); context = EasyMock.niceMock(InternalMockProcessorContext.class); EasyMock.expect(context.metrics()).andStubReturn(streamsMetrics); - EasyMock.expect(context.taskId()).andStubReturn(taskId); + EasyMock.expect(context.taskInfo()).andStubReturn(taskId); EasyMock.expect(context.appConfigs()).andStubReturn(new StreamsConfig(props).originals()); EasyMock.expect(context.stateDir()).andStubReturn(dir); EasyMock.replay(context); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java index 5f85803812974..f05c111b6a6ad 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java @@ -28,7 +28,7 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.TopologyWrapper; import org.apache.kafka.streams.errors.InvalidStateStoreException; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; @@ -399,7 +399,7 @@ private StreamTask createStreamsTask(final StreamsConfig streamsConfig, final TaskId taskId) { final Metrics metrics = new Metrics(); final LogContext logContext = new LogContext("test-stream-task "); - final Set partitions = Collections.singleton(new TopicPartition(topicName, taskId.partition)); + final Set partitions = Collections.singleton(new TopicPartition(topicName, taskId.partition())); final ProcessorStateManager stateManager = new ProcessorStateManager( taskId, Task.TaskType.ACTIVE, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java index 7d420c7768f4c..07a57ba6ab140 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java @@ -30,7 +30,7 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.internals.Change; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.processor.internals.RecordBatchingStateRestoreCallback; import org.apache.kafka.streams.state.ValueAndTimestamp; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentTest.java index 9d339b73a4df4..0ec751d6251a6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedSegmentTest.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder; import org.apache.kafka.test.TestUtils; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderGaugesTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderGaugesTest.java index 2695b8654f4c2..ee5203108ac3a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderGaugesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderGaugesTest.java @@ -22,7 +22,7 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.junit.Test; import org.rocksdb.Cache; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderTest.java index dc08f845fe1ab..82ee941e7ed09 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecorderTest.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.internals.metrics.RocksDBMetrics.RocksDBMetricContext; import org.junit.Before; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecordingTriggerTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecordingTriggerTest.java index d7c5c9796f57c..da376b36ac0cd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecordingTriggerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/metrics/RocksDBMetricsRecordingTriggerTest.java @@ -18,7 +18,7 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.junit.Before; import org.junit.Test; diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java index e8ec04c23ca52..06544a2c96769 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -50,7 +50,7 @@ public Processor get() { @Override public void init(final ProcessorContext context) { super.init(context); - System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskId()); + System.out.println("[DEV] initializing processor: topic=" + topic + " taskId=" + context.taskInfo()); System.out.flush(); numRecordsProcessed = 0; smallestOffset = Long.MAX_VALUE; @@ -75,7 +75,7 @@ public void process(final Object key, final Object value) { @Override public void close() { - System.out.printf("Close processor for task %s%n", context().taskId()); + System.out.printf("Close processor for task %s%n", context().taskInfo()); System.out.println("processed " + numRecordsProcessed + " records"); final long processed; if (largestOffset >= smallestOffset) { diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java index 82b6ccce7591f..d24295992b363 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java @@ -33,7 +33,7 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.TaskAssignmentException; import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.DefaultKafkaClientSupplier; import org.apache.kafka.streams.processor.internals.StreamsPartitionAssignor; import org.apache.kafka.streams.processor.internals.TaskManager; 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 55f22a2529e8b..ed819df4dfb65 100644 --- a/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/InternalMockProcessorContext.java @@ -31,7 +31,7 @@ import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.AbstractProcessorContext; @@ -425,7 +425,7 @@ public void logChange(final String storeName, key, value, null, - taskId().partition, + this.taskInfo().partition(), timestamp, BYTES_KEY_SERIALIZER, BYTEARRAY_VALUE_SERIALIZER); diff --git a/streams/src/test/java/org/apache/kafka/test/MockInternalProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockInternalProcessorContext.java index 370dca75ce774..3d7e4230f7fdd 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockInternalProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockInternalProcessorContext.java @@ -20,7 +20,7 @@ import org.apache.kafka.streams.processor.MockProcessorContext; import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.api.RecordMetadata; 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 b596a7ef82c8c..8b4861bf32afd 100644 --- a/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/NoOpProcessorContext.java @@ -24,7 +24,7 @@ import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.AbstractProcessorContext; diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java index 63e6a278da06d..81d97a3bc034b 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java @@ -49,7 +49,7 @@ import org.apache.kafka.streams.processor.StateRestoreListener; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.internals.ChangelogRegister; import org.apache.kafka.streams.processor.internals.ClientUtils; import org.apache.kafka.streams.processor.internals.GlobalProcessorContextImpl; diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java index 50794b2ca7363..a8a07cbb2b2a4 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java @@ -25,6 +25,7 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.TaskInfo; import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.internals.ApiUtils; @@ -59,7 +60,7 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.Supplier { // Immutable fields ================================================ private final StreamsMetricsImpl metrics; - private final TaskId taskId; + private final org.apache.kafka.streams.processor.internals.TaskId taskId; private final StreamsConfig config; private final File stateDir; @@ -194,7 +195,7 @@ public MockProcessorContext() { put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ""); } }, - new TaskId(0, 0), + new org.apache.kafka.streams.processor.internals.TaskId(0, 0), null); } @@ -208,18 +209,18 @@ public MockProcessorContext() { */ @SuppressWarnings({"WeakerAccess", "unused"}) public MockProcessorContext(final Properties config) { - this(config, new TaskId(0, 0), null); + this(config, new org.apache.kafka.streams.processor.internals.TaskId(0, 0), null); } /** * Create a {@link MockProcessorContext} with a specified taskId and null stateDir. * * @param config a {@link Properties} object, used to configure the context and the processor. - * @param taskId a {@link TaskId}, which the context makes available via {@link MockProcessorContext#taskId()}. + * @param taskId a {@link org.apache.kafka.streams.processor.internals.TaskId}, which the context makes available via {@link MockProcessorContext#taskInfo()}. * @param stateDir a {@link File}, which the context makes available viw {@link MockProcessorContext#stateDir()}. */ @SuppressWarnings({"WeakerAccess", "unused"}) - public MockProcessorContext(final Properties config, final TaskId taskId, final File stateDir) { + public MockProcessorContext(final Properties config, final org.apache.kafka.streams.processor.internals.TaskId taskId, final File stateDir) { final Properties configCopy = new Properties(); configCopy.putAll(config); configCopy.putIfAbsent(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy-bootstrap-host:0"); @@ -246,10 +247,16 @@ public String applicationId() { } @Override - public TaskId taskId() { + public TaskInfo taskInfo() { return taskId; } + @Deprecated + @Override + public org.apache.kafka.streams.processor.TaskId taskId() { + return taskId.convertToOldTaskId(); + } + @Override public Map appConfigs() { final Map combined = new HashMap<>(); diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java index 1032ad7a55071..a94859269383d 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java @@ -23,6 +23,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.TaskInfo; import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.kstream.Transformer; @@ -33,7 +34,6 @@ import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.ClientUtils; import org.apache.kafka.streams.processor.internals.RecordCollector; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; @@ -69,7 +69,7 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.Supplier { // Immutable fields ================================================ private final StreamsMetricsImpl metrics; - private final TaskId taskId; + private final org.apache.kafka.streams.processor.internals.TaskId taskId; private final StreamsConfig config; private final File stateDir; @@ -215,7 +215,7 @@ public MockProcessorContext() { mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, ""), mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "") )), - new TaskId(0, 0), + new org.apache.kafka.streams.processor.internals.TaskId(0, 0), null ); } @@ -229,17 +229,17 @@ public MockProcessorContext() { * @param config a Properties object, used to configure the context and the processor. */ public MockProcessorContext(final Properties config) { - this(config, new TaskId(0, 0), null); + this(config, new org.apache.kafka.streams.processor.internals.TaskId(0, 0), null); } /** * Create a {@link MockProcessorContext} with a specified taskId and null stateDir. * * @param config a {@link Properties} object, used to configure the context and the processor. - * @param taskId a {@link TaskId}, which the context makes available via {@link MockProcessorContext#taskId()}. + * @param taskId a {@link org.apache.kafka.streams.processor.internals.TaskId}, which the context makes available via {@link MockProcessorContext#taskInfo()}. * @param stateDir a {@link File}, which the context makes available viw {@link MockProcessorContext#stateDir()}. */ - public MockProcessorContext(final Properties config, final TaskId taskId, final File stateDir) { + public MockProcessorContext(final Properties config, final org.apache.kafka.streams.processor.internals.TaskId taskId, final File stateDir) { final Properties configCopy = new Properties(); configCopy.putAll(config); configCopy.putIfAbsent(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy-bootstrap-host:0"); @@ -266,10 +266,16 @@ public String applicationId() { } @Override - public TaskId taskId() { + public TaskInfo taskInfo() { return taskId; } + @Deprecated + @Override + public org.apache.kafka.streams.processor.TaskId taskId() { + return taskId.convertToOldTaskId(); + } + @Override public Map appConfigs() { final Map combined = new HashMap<>(); @@ -455,7 +461,13 @@ public String applicationId() { } @Override - public TaskId taskId() { + public TaskInfo taskInfo() { + return MockProcessorContext.this.taskInfo(); + } + + @Deprecated + @Override + public org.apache.kafka.streams.processor.TaskId taskId() { return MockProcessorContext.this.taskId(); } 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 ae345ead54eef..6346d7d316fd9 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 @@ -24,7 +24,7 @@ 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.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.state.KeyValueStore; @@ -230,7 +230,7 @@ public void shouldCaptureApplicationAndRecordMetadata() { @Override public void process(final String key, final Object value) { context().forward("appId", context().applicationId()); - context().forward("taskId", context().taskId()); + context().forward("taskId", context().taskInfo()); context().forward("topic", context().topic()); context().forward("partition", context().partition()); @@ -357,7 +357,7 @@ public void fullConstructorShouldSetAllExpectedAttributes() { final MockProcessorContext context = new MockProcessorContext(config, new TaskId(1, 1), dummyFile); assertEquals("testFullConstructor", context.applicationId()); - assertEquals(new TaskId(1, 1), context.taskId()); + assertEquals(new TaskId(1, 1), context.taskInfo()); assertEquals("testFullConstructor", context.appConfigs().get(StreamsConfig.APPLICATION_ID_CONFIG)); assertEquals("testFullConstructor", context.appConfigsWithPrefix("application.").get("id")); assertEquals(Serdes.String().getClass(), context.keySerde().getClass()); 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 ad7bc169d8076..3499909cc4c14 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 @@ -39,7 +39,7 @@ import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.ProcessorSupplier; diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextAPITest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextAPITest.java index 88c67d5692508..b912b782f6d0a 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextAPITest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextAPITest.java @@ -20,7 +20,7 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.api.MockProcessorContext; import org.apache.kafka.streams.processor.api.MockProcessorContext.CapturedForward; import org.apache.kafka.streams.processor.api.Processor; @@ -45,6 +45,7 @@ import static org.apache.kafka.common.utils.Utils.mkProperties; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; public class MockProcessorContextAPITest { @@ -256,7 +257,7 @@ public void init(final ProcessorContext context) { @Override public void process(final Record record) { context.forward(new Record("appId", context.applicationId(), 0L)); - context.forward(new Record("taskId", context.taskId(), 0L)); + context.forward(new Record("taskId", context.taskInfo(), 0L)); if (context.recordMetadata().isPresent()) { final RecordMetadata recordMetadata = context.recordMetadata().get(); @@ -343,7 +344,7 @@ public void fullConstructorShouldSetAllExpectedAttributes() { new MockProcessorContext<>(config, new TaskId(1, 1), dummyFile); assertThat(context.applicationId(), is("testFullConstructor")); - assertThat(context.taskId(), is(new TaskId(1, 1))); + assertThat(context.taskInfo(), equalTo(new TaskId(1, 1))); assertThat(context.appConfigs().get(StreamsConfig.APPLICATION_ID_CONFIG), is("testFullConstructor")); assertThat(context.appConfigsWithPrefix("application.").get("id"), is("testFullConstructor")); assertThat(context.keySerde().getClass(), is(Serdes.String().getClass())); diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextStateStoreTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextStateStoreTest.java index ca6426678bb55..542e7110e1a06 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextStateStoreTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextStateStoreTest.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.api.MockProcessorContext; import org.apache.kafka.streams.state.KeyValueBytesStoreSupplier; import org.apache.kafka.streams.state.KeyValueStore; diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/wordcount/WindowedWordCountProcessorTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/wordcount/WindowedWordCountProcessorTest.java index 203c51dbf90a1..41d300973b3a5 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/wordcount/WindowedWordCountProcessorTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/wordcount/WindowedWordCountProcessorTest.java @@ -18,7 +18,7 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TaskId; import org.apache.kafka.streams.processor.api.MockProcessorContext; import org.apache.kafka.streams.processor.api.MockProcessorContext.CapturedForward; import org.apache.kafka.streams.processor.api.Processor;