You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/08/26 02:13:12 UTC

[GitHub] [kafka] vvcephei opened a new pull request #9222: KAFKA-10437: Implement test-utils and StateStore changes for KIP-478

vvcephei opened a new pull request #9222:
URL: https://github.com/apache/kafka/pull/9222


   Propose a new init method for StateStore so that it works with the new ProcessorContext.
   Convert the test-utils MockProcessorContext to the new API.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on a change in pull request #9222: KAFKA-10437: Implement test-utils and StateStore changes for KIP-478

Posted by GitBox <gi...@apache.org>.
vvcephei commented on a change in pull request #9222:
URL: https://github.com/apache/kafka/pull/9222#discussion_r477477190



##########
File path: streams/examples/src/test/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorTest.java
##########
@@ -36,19 +36,19 @@
 public class WordCountProcessorTest {
     @Test
     public void test() {
-        final MockProcessorContext context = new MockProcessorContext();
+        final MockProcessorContext<String, String> context = new MockProcessorContext<>();
 
         // Create, initialize, and register the state store.
         final KeyValueStore<String, Integer> store =
             Stores.keyValueStoreBuilder(Stores.inMemoryKeyValueStore("Counts"), Serdes.String(), Serdes.Integer())
                 .withLoggingDisabled() // Changelog is not supported by MockProcessorContext.
                 // Caching is disabled by default, but FYI: caching is also not supported by MockProcessorContext.
                 .build();
-        store.init(context, store);
+        store.init(context.getStateStoreContext(), store);

Review comment:
       Here's where we're switching contexts to the StateStoreContext to invoke the new API.

##########
File path: streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java
##########
@@ -0,0 +1,602 @@
+/*
+ * 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.header.Headers;
+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.KeyValue;
+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.internals.ApiUtils;
+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.To;
+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.lang.reflect.Field;
+import java.time.Duration;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+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 {
+    // Immutable fields ================================================
+    private final StreamsMetricsImpl metrics;
+    private final TaskId taskId;
+    private final StreamsConfig config;
+    private final File stateDir;
+
+    // settable record metadata ================================================
+    private String topic;
+    private Integer partition;
+    private Long offset;
+    private Headers headers;
+    private Long timestamp;
+
+    // mocks ================================================
+    private final Map<String, StateStore> stateStores = new HashMap<>();
+    private final List<CapturedPunctuator> punctuators = new LinkedList<>();
+    private final List<CapturedForward<KForward, VForward>> capturedForwards = new LinkedList<>();
+    private boolean committed = false;
+
+
+    /**
+     * {@link CapturedPunctuator} holds captured punctuators, along with their scheduling information.
+     */
+    public static final class CapturedPunctuator {
+        private final long intervalMs;
+        private final PunctuationType type;
+        private final Punctuator punctuator;
+        private boolean cancelled = false;
+
+        private CapturedPunctuator(final long intervalMs, final PunctuationType type, final Punctuator punctuator) {
+            this.intervalMs = intervalMs;
+            this.type = type;
+            this.punctuator = punctuator;
+        }
+
+        @SuppressWarnings("unused")
+        public long getIntervalMs() {
+            return intervalMs;
+        }
+
+        @SuppressWarnings("unused")
+        public PunctuationType getType() {
+            return type;
+        }
+
+        @SuppressWarnings("unused")
+        public Punctuator getPunctuator() {
+            return punctuator;
+        }
+
+        @SuppressWarnings({"WeakerAccess", "unused"})
+        public void cancel() {
+            cancelled = true;
+        }
+
+        @SuppressWarnings("unused")
+        public boolean cancelled() {
+            return cancelled;
+        }
+    }
+
+    public static final class CapturedForward<KForward, VForward> {
+        private final String childName;
+        private final long timestamp;
+        private final KeyValue<KForward, VForward> keyValue;
+
+        private CapturedForward(final To to, final KeyValue<KForward, VForward> keyValue) {
+            if (keyValue == null) {
+                throw new IllegalArgumentException("keyValue can't be null");
+            }
+
+            try {
+                final Field field = To.class.getDeclaredField("childName");
+                field.setAccessible(true);
+                childName = (String) field.get(to);
+            } catch (final IllegalAccessException | NoSuchFieldException e) {
+                throw new RuntimeException(e);
+            }
+            timestamp = getTimestamp(to);
+
+            this.keyValue = keyValue;
+        }
+
+        /**
+         * The child this data was forwarded to.
+         *
+         * @return The child name, or {@code null} if it was broadcast.
+         */
+        @SuppressWarnings("unused")
+        public String childName() {
+            return childName;
+        }
+
+        /**
+         * The timestamp attached to the forwarded record.
+         *
+         * @return A timestamp, or {@code -1} if none was forwarded.
+         */
+        @SuppressWarnings("unused")
+        public long timestamp() {
+            return timestamp;
+        }
+
+        /**
+         * The data forwarded.
+         *
+         * @return A key/value pair. Not null.
+         */
+        @SuppressWarnings("unused")
+        public KeyValue<KForward, VForward> keyValue() {
+            return keyValue;
+        }
+
+        @Override
+        public String toString() {
+            return "CapturedForward{" +
+                "childName='" + childName + '\'' +
+                ", timestamp=" + timestamp +
+                ", keyValue=" + keyValue +
+                '}';
+        }
+    }
+
+    // constructors ================================================
+
+    /**
+     * Create a {@link MockProcessorContext} with dummy {@code config} and {@code taskId} and {@code null} {@code stateDir}.
+     * Most unit tests using this mock won't need to know the taskId,
+     * and most unit tests should be able to get by with the
+     * {@link InMemoryKeyValueStore}, so the stateDir won't matter.
+     */
+    @SuppressWarnings("unused")
+    public MockProcessorContext() {
+        this(
+            mkProperties(mkMap(
+                mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, ""),
+                mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "")
+            )),
+            new TaskId(0, 0),
+            null
+        );
+    }
+
+    /**
+     * Create a {@link MockProcessorContext} with dummy {@code taskId} and {@code null} {@code stateDir}.
+     * Most unit tests using this mock won't need to know the taskId,
+     * and most unit tests should be able to get by with the
+     * {@link InMemoryKeyValueStore}, so the stateDir won't matter.
+     *
+     * @param config a Properties object, used to configure the context and the processor.
+     */
+    @SuppressWarnings("unused")
+    public MockProcessorContext(final Properties config) {
+        this(config, new 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 stateDir a {@link File}, which the context makes available viw {@link MockProcessorContext#stateDir()}.
+     */
+    @SuppressWarnings("unused")
+    public MockProcessorContext(final Properties config, final TaskId taskId, final File stateDir) {
+        final StreamsConfig streamsConfig = new ClientUtils.QuietStreamsConfig(config);
+        this.taskId = taskId;
+        this.config = streamsConfig;
+        this.stateDir = stateDir;
+        final MetricConfig metricConfig = new MetricConfig();
+        metricConfig.recordLevel(Sensor.RecordingLevel.DEBUG);
+        final String threadId = Thread.currentThread().getName();
+        metrics = new StreamsMetricsImpl(
+            new Metrics(metricConfig),
+            threadId,
+            streamsConfig.getString(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG),
+            Time.SYSTEM
+        );
+        TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor(threadId, taskId.toString(), metrics);
+    }
+
+    @Override
+    public String applicationId() {
+        return config.getString(StreamsConfig.APPLICATION_ID_CONFIG);
+    }
+
+    @Override
+    public TaskId taskId() {
+        return taskId;
+    }
+
+    @Override
+    public Map<String, Object> appConfigs() {
+        final Map<String, Object> combined = new HashMap<>();
+        combined.putAll(config.originals());
+        combined.putAll(config.values());
+        return combined;
+    }
+
+    @Override
+    public Map<String, Object> appConfigsWithPrefix(final String prefix) {
+        return config.originalsWithPrefix(prefix);
+    }
+
+    @Override
+    public Serde<?> keySerde() {
+        return config.defaultKeySerde();
+    }
+
+    @Override
+    public Serde<?> valueSerde() {
+        return config.defaultValueSerde();
+    }
+
+    @Override
+    public File stateDir() {
+        return stateDir;
+    }
+
+    @Override
+    public StreamsMetrics metrics() {
+        return metrics;
+    }
+
+    // settable record metadata ================================================
+
+    /**
+     * The context exposes these metadata for use in the processor. Normally, they are set by the Kafka Streams framework,
+     * but for the purpose of driving unit tests, you can set them directly.
+     *
+     * @param topic     A topic name
+     * @param partition A partition number
+     * @param offset    A record offset
+     * @param timestamp A record timestamp
+     */
+    @SuppressWarnings("unused")
+    public void setRecordMetadata(final String topic,
+                                  final int partition,
+                                  final long offset,
+                                  final Headers headers,
+                                  final long timestamp) {
+        this.topic = topic;
+        this.partition = partition;
+        this.offset = offset;
+        this.headers = headers;
+        this.timestamp = timestamp;
+    }
+
+    /**
+     * The context exposes this metadata for use in the processor. Normally, they are set by the Kafka Streams framework,
+     * but for the purpose of driving unit tests, you can set it directly. Setting this attribute doesn't affect the others.
+     *
+     * @param topic A topic name
+     */
+    @SuppressWarnings("unused")
+    public void setTopic(final String topic) {
+        this.topic = topic;
+    }
+
+    /**
+     * The context exposes this metadata for use in the processor. Normally, they are set by the Kafka Streams framework,
+     * but for the purpose of driving unit tests, you can set it directly. Setting this attribute doesn't affect the others.
+     *
+     * @param partition A partition number
+     */
+    @SuppressWarnings("unused")
+    public void setPartition(final int partition) {
+        this.partition = partition;
+    }
+
+    /**
+     * The context exposes this metadata for use in the processor. Normally, they are set by the Kafka Streams framework,
+     * but for the purpose of driving unit tests, you can set it directly. Setting this attribute doesn't affect the others.
+     *
+     * @param offset A record offset
+     */
+    @SuppressWarnings("unused")
+    public void setOffset(final long offset) {
+        this.offset = offset;
+    }
+
+    /**
+     * The context exposes this metadata for use in the processor. Normally, they are set by the Kafka Streams framework,
+     * but for the purpose of driving unit tests, you can set it directly. Setting this attribute doesn't affect the others.
+     *
+     * @param headers Record headers
+     */
+    @SuppressWarnings("unused")
+    public void setHeaders(final Headers headers) {
+        this.headers = headers;
+    }
+
+    /**
+     * The context exposes this metadata for use in the processor. Normally, they are set by the Kafka Streams framework,
+     * but for the purpose of driving unit tests, you can set it directly. Setting this attribute doesn't affect the others.
+     *
+     * @param timestamp A record timestamp
+     */
+    @SuppressWarnings("unused")
+    public void setTimestamp(final long timestamp) {
+        this.timestamp = timestamp;
+    }
+
+    @Override
+    public String topic() {
+        if (topic == null) {
+            throw new IllegalStateException("Topic must be set before use via setRecordMetadata() or setTopic().");
+        }
+        return topic;
+    }
+
+    @Override
+    public int partition() {
+        if (partition == null) {
+            throw new IllegalStateException("Partition must be set before use via setRecordMetadata() or setPartition().");
+        }
+        return partition;
+    }
+
+    @Override
+    public long offset() {
+        if (offset == null) {
+            throw new IllegalStateException("Offset must be set before use via setRecordMetadata() or setOffset().");
+        }
+        return offset;
+    }
+
+    @Override
+    public Headers headers() {
+        return headers;
+    }
+
+    @Override
+    public long timestamp() {
+        if (timestamp == null) {
+            throw new IllegalStateException("Timestamp must be set before use via setRecordMetadata() or setTimestamp().");
+        }
+        return timestamp;
+    }
+
+    // mocks ================================================
+
+    @Override
+    public void register(final StateStore store,
+                         final StateRestoreCallback stateRestoreCallbackIsIgnoredInMock) {
+        stateStores.put(store.name(), store);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <S extends StateStore> S getStateStore(final String name) {
+        return (S) stateStores.get(name);
+    }
+
+    @Override
+    public Cancellable schedule(final Duration interval,
+                                final PunctuationType type,
+                                final Punctuator callback) {
+        final CapturedPunctuator capturedPunctuator =
+            new CapturedPunctuator(ApiUtils.validateMillisecondDuration(interval, "interval"), type, callback);
+
+        punctuators.add(capturedPunctuator);
+
+        return capturedPunctuator::cancel;
+    }
+
+    /**
+     * Get the punctuators scheduled so far. The returned list is not affected by subsequent calls to {@code schedule(...)}.
+     *
+     * @return A list of captured punctuators.
+     */
+    @SuppressWarnings("unused")
+    public List<CapturedPunctuator> scheduledPunctuators() {
+        return new LinkedList<>(punctuators);
+    }
+
+    @Override
+    public <K extends KForward, V extends VForward> void forward(final K key, final V value) {
+        forward(key, value, To.all());
+    }
+
+    @Override
+    public <K extends KForward, V extends VForward> void forward(final K key, final V value, final To to) {
+        capturedForwards.add(
+            new CapturedForward<>(
+                (getTimestamp(to)) == -1 ? to.withTimestamp(timestamp == null ? -1 : timestamp) : to,
+                new KeyValue<>(key, value)
+            )
+        );
+    }
+
+    /**
+     * Get all the forwarded data this context has observed. The returned list will not be
+     * affected by subsequent interactions with the context. The data in the list is in the same order as the calls to
+     * {@code forward(...)}.
+     *
+     * @return A list of key/value pairs that were previously passed to the context.
+     */
+    public List<CapturedForward<KForward, VForward>> forwarded() {
+        return new LinkedList<>(capturedForwards);
+    }
+
+    /**
+     * Get all the forwarded data this context has observed for a specific child by name.
+     * The returned list will not be affected by subsequent interactions with the context.
+     * The data in the list is in the same order as the calls to {@code forward(...)}.
+     *
+     * @param childName The child name to retrieve forwards for
+     * @return A list of key/value pairs that were previously passed to the context.
+     */
+    @SuppressWarnings("unused")
+    public List<CapturedForward<KForward, VForward>> forwarded(final String childName) {
+        final LinkedList<CapturedForward<KForward, VForward>> result = new LinkedList<>();
+        for (final CapturedForward<KForward, VForward> capture : capturedForwards) {
+            if (capture.childName() == null || capture.childName().equals(childName)) {
+                result.add(capture);
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Clear the captured forwarded data.
+     */
+    @SuppressWarnings("unused")
+    public void resetForwards() {
+        capturedForwards.clear();
+    }
+
+    @Override
+    public void commit() {
+        committed = true;
+    }
+
+    /**
+     * Whether {@link ProcessorContext#commit()} has been called in this context.
+     *
+     * @return {@code true} iff {@link ProcessorContext#commit()} has been called in this context since construction or reset.
+     */
+    public boolean committed() {
+        return committed;
+    }
+
+    /**
+     * Reset the commit capture to {@code false} (whether or not it was previously {@code true}).
+     */
+    @SuppressWarnings("unused")
+    public void resetCommit() {
+        committed = false;
+    }
+
+    @Override
+    public RecordCollector recordCollector() {
+        // This interface is assumed by state stores that add change-logging.
+        // Rather than risk a mysterious ClassCastException during unit tests, throw an explanatory exception.
+
+        throw new UnsupportedOperationException(
+            "MockProcessorContext does not provide record collection. " +
+                "For processor unit tests, use an in-memory state store with change-logging disabled. " +
+                "Alternatively, use the TopologyTestDriver for testing processor/store/topology integration."
+        );
+    }
+
+    /**
+     * Used to get a {@link StateStoreContext} for use with
+     * {@link StateStore#init(StateStoreContext, StateStore)}
+     * if you need to initialize a store for your tests.
+     * @return a {@link StateStoreContext} that delegates to this ProcessorContext.
+     */
+    public StateStoreContext getStateStoreContext() {

Review comment:
       Added this so that users can still just have one context that they can use for initializing their processors and stores.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreToProcessorContextAdapter.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.common.header.Headers;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.streams.StreamsMetrics;
+import org.apache.kafka.streams.processor.Cancellable;
+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.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.To;
+
+import java.io.File;
+import java.time.Duration;
+import java.util.Map;
+
+public final class StoreToProcessorContextAdapter implements ProcessorContext {
+    private final StateStoreContext delegate;
+
+    public static ProcessorContext adapt(final StateStoreContext delegate) {
+        if (delegate instanceof ProcessorToStoreContextAdapter) {
+            return ((ProcessorToStoreContextAdapter) delegate).delegate();
+        } else {
+            return new StoreToProcessorContextAdapter(delegate);
+        }
+    }

Review comment:
       To let the new method delegate to the old one, we adapt the provided StateStoreContext to ProcessorContext.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/StateStoreContext.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.streams.StreamsMetrics;
+import org.apache.kafka.streams.errors.StreamsException;
+
+import java.io.File;
+import java.util.Map;
+
+/**
+ * Processor context interface.
+ */
+public interface StateStoreContext {

Review comment:
       Here's the proposal for the new context. I've preserved all the "general context" members while dropping all the "record context" ones and anything else that seemed inappropriate for state stores to invoke.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java
##########
@@ -61,7 +84,9 @@
      * @throws IllegalStateException If store gets registered after initialized is already finished
      * @throws StreamsException if the store's change log does not contain the partition
      */
-    void init(ProcessorContext context, StateStore root);
+    default void init(final StateStoreContext context, final StateStore root) {
+        init(StoreToProcessorContextAdapter.adapt(context), root);
+    }

Review comment:
       Proposing to add this new version of `init` since the old `ProcessorContext` is slated to be deprecated once KIP-478 is fully implemented.
   
   Rather than just accepting the whole new ProcessorContext, I'm proposing to add a new StateStoreContext to avoid leaking a bunch of methods that should really only be invoked in Processor implementations.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on a change in pull request #9222: KAFKA-10437: Implement test-utils and StateStore changes for KIP-478

Posted by GitBox <gi...@apache.org>.
vvcephei commented on a change in pull request #9222:
URL: https://github.com/apache/kafka/pull/9222#discussion_r478586348



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/StateStoreContext.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.streams.StreamsMetrics;
+import org.apache.kafka.streams.errors.StreamsException;
+
+import java.io.File;
+import java.util.Map;
+
+/**
+ * Processor context interface.
+ */
+public interface StateStoreContext {
+
+    /**
+     * Returns the application id.
+     *
+     * @return the application id
+     */
+    String applicationId();

Review comment:
       Thanks, @guozhangwang . I think they'd be used in Processors also, and these methods would still be present on ProcessorContext, so they would still be available.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on pull request #9222: KAFKA-10437: Implement test-utils and StateStore changes for KIP-478

Posted by GitBox <gi...@apache.org>.
vvcephei commented on pull request #9222:
URL: https://github.com/apache/kafka/pull/9222#issuecomment-680438318


   Hey @abbccdda , this is a Part 4 PR that I extracted out from Part 3 (#9221 ) when it became too large. If you have a chance, I'd appreciate your review. Especially wrt the parts that require KIP amendments.
   
   Thanks as always for your reviews!


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] guozhangwang commented on a change in pull request #9222: KAFKA-10437: Implement test-utils and StateStore changes for KIP-478

Posted by GitBox <gi...@apache.org>.
guozhangwang commented on a change in pull request #9222:
URL: https://github.com/apache/kafka/pull/9222#discussion_r477676262



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/StateStoreContext.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.streams.StreamsMetrics;
+import org.apache.kafka.streams.errors.StreamsException;
+
+import java.io.File;
+import java.util.Map;
+
+/**
+ * Processor context interface.
+ */
+public interface StateStoreContext {
+
+    /**
+     * Returns the application id.
+     *
+     * @return the application id
+     */
+    String applicationId();

Review comment:
       Are appId / taskId / metrics / appConfigs only needed for state store context? Is it possible that they would also be used within Processor?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on a change in pull request #9222: KAFKA-10437: Implement test-utils and StateStore changes for KIP-478

Posted by GitBox <gi...@apache.org>.
vvcephei commented on a change in pull request #9222:
URL: https://github.com/apache/kafka/pull/9222#discussion_r476988013



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java
##########
@@ -61,7 +84,14 @@
      * @throws IllegalStateException If store gets registered after initialized is already finished
      * @throws StreamsException if the store's change log does not contain the partition
      */
-    void init(ProcessorContext context, StateStore root);
+    default void init(final ProcessorContext<?, ?> context, final StateStore root) {
+        final org.apache.kafka.streams.processor.ProcessorContext adapted =
+            ProcessorContextReverseAdapter.adapt(
+                context,
+                new ProcessorContextReverseAdapter.UnsupportedDeprecatedForwarder()
+            );
+        init(adapted, root);
+    }

Review comment:
       We have to add this so that we can pass in the new ProcessorContext. The default implementation delegates to the old `init` method so that existing store implementations will function with no changes.
   
   If the only callers were internal, we could just adapt at the call site. Unfortunately, users can also call `StateStore#init`, and they would do it if they have their own store implementations or if they use `MockProcessorContext` to test a stateful processor.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalProcessorContextReverseAdapter.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.common.header.Headers;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.utils.Bytes;
+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.TaskId;
+import org.apache.kafka.streams.processor.To;
+import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
+import org.apache.kafka.streams.state.StoreBuilder;
+import org.apache.kafka.streams.state.internals.ThreadCache;
+
+import java.io.File;
+import java.time.Duration;
+import java.util.Map;
+
+public final class InternalProcessorContextReverseAdapter implements InternalProcessorContext {

Review comment:
       I just renamed this from `ProcessorContextReverseAdapter` (which confusingly implemented `InternalProcessorContext`), so that I could add an adapter specifically for non-Internal `ProcessorContext`s.

##########
File path: streams/examples/src/test/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorTest.java
##########
@@ -36,7 +36,7 @@
 public class WordCountProcessorTest {
     @Test
     public void test() {
-        final MockProcessorContext context = new MockProcessorContext();
+        final MockProcessorContext<String, String> context = new MockProcessorContext<>();

Review comment:
       Switching over to the mock of the new API.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java
##########
@@ -45,6 +47,27 @@
      */
     String name();
 
+    /**
+     * Initializes this state store.
+     * <p>
+     * The implementation of this function must register the root store in the context via the
+     * {@link org.apache.kafka.streams.processor.ProcessorContext#register(StateStore, StateRestoreCallback)} function,
+     * where the first {@link StateStore} parameter should always be the passed-in {@code root} object, and
+     * the second parameter should be an object of user's implementation
+     * of the {@link StateRestoreCallback} interface used for restoring the state store from the changelog.
+     * <p>
+     * Note that if the state store engine itself supports bulk writes, users can implement another
+     * interface {@link BatchingStateRestoreCallback} which extends {@link StateRestoreCallback} to
+     * let users implement bulk-load restoration logic instead of restoring one record at a time.
+     * <p>
+     * This method is not called if {@link StateStore#init(ProcessorContext, org.apache.kafka.streams.processor.StateStore)}
+     * is implemented.
+     *
+     * @throws IllegalStateException If store gets registered after initialized is already finished
+     * @throws StreamsException if the store's change log does not contain the partition
+     */
+    void init(org.apache.kafka.streams.processor.ProcessorContext context, StateStore root);

Review comment:
       Not sure why the diff happened this way, but this is the original `init` method. I've just qualified ProcessorContext and added this note to the javadoc: `This method is not called if {@link StateStore#init(ProcessorContext, StateStore)} is implemented.`
   
   Later on, when we deprecate `processor.ProcessorContext`, this method will also become deprecated, pushing store implementers to move to the new API.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextReverseAdapter.java
##########
@@ -18,35 +18,52 @@
 
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.serialization.Serde;
-import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsMetrics;
 import org.apache.kafka.streams.processor.Cancellable;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
 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.TaskId;
 import org.apache.kafka.streams.processor.To;
-import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
-import org.apache.kafka.streams.state.StoreBuilder;
-import org.apache.kafka.streams.state.internals.ThreadCache;
 
 import java.io.File;
 import java.time.Duration;
 import java.util.Map;
 
-public final class ProcessorContextReverseAdapter implements InternalProcessorContext {
-    private final InternalApiProcessorContext<Object, Object> delegate;
+public final class ProcessorContextReverseAdapter implements org.apache.kafka.streams.processor.ProcessorContext {

Review comment:
       Here's the new one, which directly adapts non-Internal `ProcessorContext`. This became necessary, since the `MockProcessorContext` is _not_ an `InternalProcessorContext`, but it still needs to be adapted internally.
   
   Once the KIP is complete, most of these adapters will go away.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextReverseAdapter.java
##########
@@ -18,35 +18,52 @@
 
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.serialization.Serde;
-import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsMetrics;
 import org.apache.kafka.streams.processor.Cancellable;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
 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.TaskId;
 import org.apache.kafka.streams.processor.To;
-import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
-import org.apache.kafka.streams.state.StoreBuilder;
-import org.apache.kafka.streams.state.internals.ThreadCache;
 
 import java.io.File;
 import java.time.Duration;
 import java.util.Map;
 
-public final class ProcessorContextReverseAdapter implements InternalProcessorContext {
-    private final InternalApiProcessorContext<Object, Object> delegate;
+public final class ProcessorContextReverseAdapter implements org.apache.kafka.streams.processor.ProcessorContext {
+    private final ProcessorContext<Object, Object> delegate;
+    private final DeprecatedForwarder deprecatedForwarder;
 
-    static InternalProcessorContext adapt(final InternalApiProcessorContext<Object, Object> delegate) {
+    public interface DeprecatedForwarder {
+        <K, V> void forward(final K key, final V value, final int childIndex);
+    }
+
+    public static final class UnsupportedDeprecatedForwarder implements DeprecatedForwarder {
+        @Override
+        public <K, V> void forward(final K key, final V value, final int childIndex) {
+            throw new UnsupportedOperationException("Forwarding by index was deprecated in 2.0 and is not supported by this ProcessorContext.");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    public static org.apache.kafka.streams.processor.ProcessorContext adapt(final ProcessorContext<?, ?> delegate,
+                                                                            final DeprecatedForwarder deprecatedForwarder) {

Review comment:
       In an attempt to avoid throwing an exception unless there's no alternative, callers of `adapt` have to define what happens if someone calls `forward(final K key, final V value, final int childIndex)`

##########
File path: streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java
##########
@@ -0,0 +1,546 @@
+/*
+ * 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.header.Headers;
+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.KeyValue;
+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.internals.ApiUtils;
+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.TaskId;
+import org.apache.kafka.streams.processor.To;
+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.lang.reflect.Field;
+import java.time.Duration;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+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:
       new MockProcessorContext implementing the new `api.ProcessorContext`. Note the generics.

##########
File path: streams/test-utils/src/test/java/org/apache/kafka/streams/MockApiProcessorContextTest.java
##########
@@ -0,0 +1,405 @@
+/*
+ * 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;
+
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.streams.processor.api.MockProcessorContext;
+import org.apache.kafka.streams.processor.api.MockProcessorContext.CapturedForward;
+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.To;
+import org.apache.kafka.streams.processor.api.Processor;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.processor.internals.ProcessorContextReverseAdapter;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.StoreBuilder;
+import org.apache.kafka.streams.state.Stores;
+import org.junit.Test;
+
+import java.io.File;
+import java.time.Duration;
+import java.util.Iterator;
+import java.util.Properties;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class MockApiProcessorContextTest {

Review comment:
       The test for the new `MockProcessorContext`.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextReverseAdapter.java
##########
@@ -18,35 +18,52 @@
 
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.serialization.Serde;
-import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsMetrics;
 import org.apache.kafka.streams.processor.Cancellable;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
 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.TaskId;
 import org.apache.kafka.streams.processor.To;
-import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
-import org.apache.kafka.streams.state.StoreBuilder;
-import org.apache.kafka.streams.state.internals.ThreadCache;
 
 import java.io.File;
 import java.time.Duration;
 import java.util.Map;
 
-public final class ProcessorContextReverseAdapter implements InternalProcessorContext {
-    private final InternalApiProcessorContext<Object, Object> delegate;
+public final class ProcessorContextReverseAdapter implements org.apache.kafka.streams.processor.ProcessorContext {
+    private final ProcessorContext<Object, Object> delegate;
+    private final DeprecatedForwarder deprecatedForwarder;
 
-    static InternalProcessorContext adapt(final InternalApiProcessorContext<Object, Object> delegate) {
+    public interface DeprecatedForwarder {
+        <K, V> void forward(final K key, final V value, final int childIndex);
+    }
+
+    public static final class UnsupportedDeprecatedForwarder implements DeprecatedForwarder {
+        @Override
+        public <K, V> void forward(final K key, final V value, final int childIndex) {
+            throw new UnsupportedOperationException("Forwarding by index was deprecated in 2.0 and is not supported by this ProcessorContext.");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    public static org.apache.kafka.streams.processor.ProcessorContext adapt(final ProcessorContext<?, ?> delegate,
+                                                                            final DeprecatedForwarder deprecatedForwarder) {
         if (delegate instanceof ProcessorContextAdapter) {
-            return ((ProcessorContextAdapter<Object, Object>) delegate).delegate();
+            return ((ProcessorContextAdapter<?, ?>) delegate).delegate();

Review comment:
       If we're adapting an adapter, just unwrap it.

##########
File path: streams/examples/src/test/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorTest.java
##########
@@ -61,7 +61,7 @@ public void test() {
         context.scheduledPunctuators().get(0).getPunctuator().punctuate(0L);
 
         // finally, we can verify the output.
-        final Iterator<MockProcessorContext.CapturedForward> capturedForwards = context.forwarded().iterator();
+        final Iterator<MockProcessorContext.CapturedForward<String, String>> capturedForwards = context.forwarded().iterator();

Review comment:
       Since the new ProcessorContext bounds the forward calls, I've also added types to the captured forwards.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextReverseAdapter.java
##########
@@ -18,35 +18,52 @@
 
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.serialization.Serde;
-import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsMetrics;
 import org.apache.kafka.streams.processor.Cancellable;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
 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.TaskId;
 import org.apache.kafka.streams.processor.To;
-import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
-import org.apache.kafka.streams.state.StoreBuilder;
-import org.apache.kafka.streams.state.internals.ThreadCache;
 
 import java.io.File;
 import java.time.Duration;
 import java.util.Map;
 
-public final class ProcessorContextReverseAdapter implements InternalProcessorContext {
-    private final InternalApiProcessorContext<Object, Object> delegate;
+public final class ProcessorContextReverseAdapter implements org.apache.kafka.streams.processor.ProcessorContext {
+    private final ProcessorContext<Object, Object> delegate;
+    private final DeprecatedForwarder deprecatedForwarder;
 
-    static InternalProcessorContext adapt(final InternalApiProcessorContext<Object, Object> delegate) {
+    public interface DeprecatedForwarder {
+        <K, V> void forward(final K key, final V value, final int childIndex);
+    }

Review comment:
       Awkwardly, we have no way to adapt the old (deprecated) `forward(final K key, final V value, final int childIndex)`, since it doesn't exist on the new API. If we want to, we could add it, but I don't think we want to.
   
   Another option is to just take this moment to drop it from `processor.ProcessorContext`. It was deprecated in 2.0.0, quite a long time ago.
   
   As it is, I took the most conservative approach I could. We will throw an UnsupportedOperationException only as a last resort.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java
##########
@@ -45,6 +47,27 @@
      */
     String name();
 
+    /**
+     * Initializes this state store.
+     * <p>
+     * The implementation of this function must register the root store in the context via the
+     * {@link org.apache.kafka.streams.processor.ProcessorContext#register(StateStore, StateRestoreCallback)} function,
+     * where the first {@link StateStore} parameter should always be the passed-in {@code root} object, and
+     * the second parameter should be an object of user's implementation
+     * of the {@link StateRestoreCallback} interface used for restoring the state store from the changelog.
+     * <p>
+     * Note that if the state store engine itself supports bulk writes, users can implement another
+     * interface {@link BatchingStateRestoreCallback} which extends {@link StateRestoreCallback} to
+     * let users implement bulk-load restoration logic instead of restoring one record at a time.
+     * <p>
+     * This method is not called if {@link StateStore#init(ProcessorContext, StateStore)}
+     * is implemented.
+     *
+     * @throws IllegalStateException If store gets registered after initialized is already finished
+     * @throws StreamsException if the store's change log does not contain the partition
+     */
+    void init(org.apache.kafka.streams.processor.ProcessorContext context, StateStore root);

Review comment:
       Not sure why the diff happened this way, but this is the existing `init` method. I just qualified the reference to `processor.ProcessorContext` and added the note to the javadoc: `This method is not called if {@link StateStore#init(ProcessorContext, StateStore)} is implemented.`
   
   Once the old `processor.ProcessorContext` is deprecated, this method will also become deprecated, pushing store implementers to use the other one instead.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorAdapter.java
##########
@@ -47,7 +47,7 @@ private ProcessorAdapter(final org.apache.kafka.streams.processor.Processor<KIn,
     @SuppressWarnings("unchecked")
     @Override
     public void init(final ProcessorContext<KOut, VOut> context) {
-        delegate.init(ProcessorContextReverseAdapter.adapt((InternalApiProcessorContext<Object, Object>) context));
+        delegate.init(InternalProcessorContextReverseAdapter.adapt((InternalApiProcessorContext<Object, Object>) context));

Review comment:
       renamed.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextReverseAdapter.java
##########
@@ -18,35 +18,52 @@
 
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.serialization.Serde;
-import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsMetrics;
 import org.apache.kafka.streams.processor.Cancellable;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
 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.TaskId;
 import org.apache.kafka.streams.processor.To;
-import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
-import org.apache.kafka.streams.state.StoreBuilder;
-import org.apache.kafka.streams.state.internals.ThreadCache;
 
 import java.io.File;
 import java.time.Duration;
 import java.util.Map;
 
-public final class ProcessorContextReverseAdapter implements InternalProcessorContext {
-    private final InternalApiProcessorContext<Object, Object> delegate;
+public final class ProcessorContextReverseAdapter implements org.apache.kafka.streams.processor.ProcessorContext {
+    private final ProcessorContext<Object, Object> delegate;
+    private final DeprecatedForwarder deprecatedForwarder;
 
-    static InternalProcessorContext adapt(final InternalApiProcessorContext<Object, Object> delegate) {
+    public interface DeprecatedForwarder {
+        <K, V> void forward(final K key, final V value, final int childIndex);
+    }
+
+    public static final class UnsupportedDeprecatedForwarder implements DeprecatedForwarder {
+        @Override
+        public <K, V> void forward(final K key, final V value, final int childIndex) {
+            throw new UnsupportedOperationException("Forwarding by index was deprecated in 2.0 and is not supported by this ProcessorContext.");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    public static org.apache.kafka.streams.processor.ProcessorContext adapt(final ProcessorContext<?, ?> delegate,
+                                                                            final DeprecatedForwarder deprecatedForwarder) {
         if (delegate instanceof ProcessorContextAdapter) {
-            return ((ProcessorContextAdapter<Object, Object>) delegate).delegate();
+            return ((ProcessorContextAdapter<?, ?>) delegate).delegate();
+        } else if (delegate instanceof InternalApiProcessorContext) {
+            return InternalProcessorContextReverseAdapter.adapt((InternalApiProcessorContext<Object, Object>) delegate);

Review comment:
       If what we're adapting is actually an InternalProcessorContext, good news! We'll just use the other adapter instead.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextReverseAdapter.java
##########
@@ -18,35 +18,52 @@
 
 import org.apache.kafka.common.header.Headers;
 import org.apache.kafka.common.serialization.Serde;
-import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.StreamsMetrics;
 import org.apache.kafka.streams.processor.Cancellable;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
 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.TaskId;
 import org.apache.kafka.streams.processor.To;
-import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
-import org.apache.kafka.streams.state.StoreBuilder;
-import org.apache.kafka.streams.state.internals.ThreadCache;
 
 import java.io.File;
 import java.time.Duration;
 import java.util.Map;
 
-public final class ProcessorContextReverseAdapter implements InternalProcessorContext {
-    private final InternalApiProcessorContext<Object, Object> delegate;
+public final class ProcessorContextReverseAdapter implements org.apache.kafka.streams.processor.ProcessorContext {
+    private final ProcessorContext<Object, Object> delegate;
+    private final DeprecatedForwarder deprecatedForwarder;
 
-    static InternalProcessorContext adapt(final InternalApiProcessorContext<Object, Object> delegate) {
+    public interface DeprecatedForwarder {
+        <K, V> void forward(final K key, final V value, final int childIndex);
+    }
+
+    public static final class UnsupportedDeprecatedForwarder implements DeprecatedForwarder {
+        @Override
+        public <K, V> void forward(final K key, final V value, final int childIndex) {
+            throw new UnsupportedOperationException("Forwarding by index was deprecated in 2.0 and is not supported by this ProcessorContext.");
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    public static org.apache.kafka.streams.processor.ProcessorContext adapt(final ProcessorContext<?, ?> delegate,
+                                                                            final DeprecatedForwarder deprecatedForwarder) {
         if (delegate instanceof ProcessorContextAdapter) {
-            return ((ProcessorContextAdapter<Object, Object>) delegate).delegate();
+            return ((ProcessorContextAdapter<?, ?>) delegate).delegate();
+        } else if (delegate instanceof InternalApiProcessorContext) {
+            return InternalProcessorContextReverseAdapter.adapt((InternalApiProcessorContext<Object, Object>) delegate);
         } else {
-            return new ProcessorContextReverseAdapter(delegate);
+            return new ProcessorContextReverseAdapter(delegate, deprecatedForwarder);

Review comment:
       Last resort: what we're adapting really is a plain `ProcessorContext`.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextReverseAdapter.java
##########
@@ -193,7 +130,7 @@ public Cancellable schedule(final Duration interval, final PunctuationType type,
     @Deprecated
     @Override
     public <K, V> void forward(final K key, final V value, final int childIndex) {
-        delegate.forward(key, value, To.child((currentNode().children()).get(childIndex).name()));
+        deprecatedForwarder.forward(key, value, childIndex);

Review comment:
       Here's where we call the defined behavior instead of delegating to the nonexistent method on the delegate.

##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImplTest.java
##########
@@ -142,7 +143,7 @@ public void shouldNotAllowToSchedulePunctuations() {
     public void shouldNotAllowInitForKeyValueStore() {
         final StateStore store = globalContext.getStateStore(GLOBAL_KEY_VALUE_STORE_NAME);
         try {
-            store.init(null, null);
+            store.init((ProcessorContext) null, null);

Review comment:
       casting `null` to resolve the right overload.

##########
File path: streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java
##########
@@ -0,0 +1,546 @@
+/*
+ * 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.header.Headers;
+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.KeyValue;
+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.internals.ApiUtils;
+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.TaskId;
+import org.apache.kafka.streams.processor.To;
+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.lang.reflect.Field;
+import java.time.Duration;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+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 {
+    // Immutable fields ================================================
+    private final StreamsMetricsImpl metrics;
+    private final TaskId taskId;
+    private final StreamsConfig config;
+    private final File stateDir;
+
+    // settable record metadata ================================================
+    private String topic;
+    private Integer partition;
+    private Long offset;
+    private Headers headers;
+    private Long timestamp;
+
+    // mocks ================================================
+    private final Map<String, StateStore> stateStores = new HashMap<>();
+    private final List<CapturedPunctuator> punctuators = new LinkedList<>();
+    private final List<CapturedForward<KForward, VForward>> capturedForwards = new LinkedList<>();

Review comment:
       Note, we're capturing typed values now.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] vvcephei closed pull request #9222: KAFKA-10437: Implement test-utils and StateStore changes for KIP-478

Posted by GitBox <gi...@apache.org>.
vvcephei closed pull request #9222:
URL: https://github.com/apache/kafka/pull/9222


   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on pull request #9222: KAFKA-10437: Implement test-utils and StateStore changes for KIP-478

Posted by GitBox <gi...@apache.org>.
vvcephei commented on pull request #9222:
URL: https://github.com/apache/kafka/pull/9222#issuecomment-707836238


   This was superceded by other PRs.


----------------------------------------------------------------
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:
users@infra.apache.org