vvcephei commented on a change in pull request #9396: URL: https://github.com/apache/kafka/pull/9396#discussion_r504073110
########## File path: streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java ########## @@ -0,0 +1,494 @@ +/* + * 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.api; + +import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.TopologyTestDriver; +import org.apache.kafka.streams.kstream.Transformer; +import org.apache.kafka.streams.kstream.ValueTransformer; +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.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; +import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; +import org.apache.kafka.streams.state.internals.InMemoryKeyValueStore; + +import java.io.File; +import java.time.Duration; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Properties; + +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.common.utils.Utils.mkProperties; + +/** + * {@link MockProcessorContext} is a mock of {@link ProcessorContext} for users to test their {@link Processor}, + * {@link Transformer}, and {@link ValueTransformer} implementations. + * <p> + * The tests for this class (org.apache.kafka.streams.MockProcessorContextTest) include several behavioral + * tests that serve as example usage. + * <p> + * Note that this class does not take any automated actions (such as firing scheduled punctuators). + * It simply captures any data it witnesses. + * If you require more automated tests, we recommend wrapping your {@link Processor} in a minimal source-processor-sink + * {@link Topology} and using the {@link TopologyTestDriver}. + */ +public class MockProcessorContext<KForward, VForward> implements ProcessorContext<KForward, VForward>, RecordCollector.Supplier { Review comment: Thanks. I'll go ahead with the merge, then. It seems like `InternalProcessorContext` is only "public" in the sense that Java 8 is insufficiently expressive to make it private. I guess I should say "internal" and "external". But it is certainly an _internal_ type, therefore, we should not ever return it to the users when they call _external_ APIs (such as the constructor of this class). Once we upgrade to java 9, we should add module definitions so that we stop exporting internal types in the public API completely. As I understand things, it would actually become a compiler error at that point for us to provide an external class that inherits from an internal interface. I guess I don't see this as a big concern anyway. The way I implemented the state stores in the last PR makes them capable of accepting any implementation of `ProcessorContext` or `StateStoreContext`, although they'll disable some features if the context isn't an instance of `InternalProcessorContext`. To me, this seems like a fine situation. We could also note that there is https://cwiki.apache.org/confluence/display/KAFKA/KIP-622%3A+Add+currentSystemTimeMs+and+currentStreamTimeMs+to+ProcessorContext, which would remove the need for casting to InternalProcessorContext to begin with. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org