wcarlson5 commented on code in PR #12742:
URL: https://github.com/apache/kafka/pull/12742#discussion_r1515301279


##########
streams/src/test/java/org/apache/kafka/streams/processor/ReadOnlyStoreTest.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.IntegerDeserializer;
+import org.apache.kafka.common.serialization.IntegerSerializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.TestInputTopic;
+import org.apache.kafka.streams.TestOutputTopic;
+import org.apache.kafka.streams.Topology;
+import org.apache.kafka.streams.TopologyTestDriver;
+import org.apache.kafka.streams.processor.api.Processor;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.processor.api.Record;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.Stores;
+import org.junit.Test;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.equalTo;
+
+public class ReadOnlyStoreTest {
+
+    @Test
+    public void shouldLoadDataIntoReadOnlyStoreAndAllowAccessFromProcessor() {

Review Comment:
   This kind of sounds like its testing the error path if the processor 
shouldn't have access to it tries to write/access that. Do we have a test for 
that? It seems that's what's special about a read only store



##########
streams/src/test/java/org/apache/kafka/streams/processor/ReadOnlyStoreTest.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.IntegerDeserializer;
+import org.apache.kafka.common.serialization.IntegerSerializer;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.TestInputTopic;
+import org.apache.kafka.streams.TestOutputTopic;
+import org.apache.kafka.streams.Topology;
+import org.apache.kafka.streams.TopologyTestDriver;
+import org.apache.kafka.streams.processor.api.Processor;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.processor.api.Record;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.Stores;
+import org.junit.Test;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.equalTo;
+
+public class ReadOnlyStoreTest {
+
+    @Test
+    public void shouldLoadDataIntoReadOnlyStoreAndAllowAccessFromProcessor() {
+        final Topology topology = new Topology();
+        topology.addReadOnlyStateStore(
+            Stores.keyValueStoreBuilder(
+                Stores.inMemoryKeyValueStore("readOnlyStore"),
+                new Serdes.IntegerSerde(),
+                new Serdes.StringSerde()
+            ),
+            "readOnlySource",
+            new IntegerDeserializer(),
+            new StringDeserializer(),
+            "storeTopic",
+            "readOnlyProcessor",
+            () -> new Processor<Integer, String, Void, Void>() {
+                KeyValueStore<Integer, String> store;
+
+                @Override
+                public void init(final ProcessorContext<Void, Void> context) {
+                    store = context.getStateStore("readOnlyStore");
+                }
+                @Override
+                public void process(final Record<Integer, String> record) {
+                    store.put(record.key(), record.value());
+                }
+            }
+        );
+
+        topology.addSource("source", new IntegerDeserializer(), new 
StringDeserializer(), "inputTopic");
+        topology.addProcessor(
+            "processor",
+            () -> new Processor<Integer, String, Integer, String>() {
+                ProcessorContext<Integer, String> context;
+                KeyValueStore<Integer, String> store;
+
+                @Override
+                public void init(final ProcessorContext<Integer, String> 
context) {
+                    this.context = context;
+                    store = context.getStateStore("readOnlyStore");
+                }
+
+                @Override
+                public void process(final Record<Integer, String> record) {
+                    context.forward(record.withValue(
+                        record.value() + " -- " + store.get(record.key())
+                    ));
+                }
+            },
+            "source"
+        );
+        topology.connectProcessorAndStateStores("processor", "readOnlyStore");
+        topology.addSink("sink", "outputTopic", new IntegerSerializer(), new 
StringSerializer(), "processor");
+
+        try (final TopologyTestDriver driver = new 
TopologyTestDriver(topology)) {
+            final TestInputTopic<Integer, String> changelog =

Review Comment:
   Naming this changelog and then writing to it was about 5% confusing. 
   
   super nit but maybe 'readOnlyStoreTopicChangelog'. Idk jut something that 
isn't an already used term



##########
streams/src/test/java/org/apache/kafka/test/MockProcessor.java:
##########
@@ -43,12 +42,12 @@ public MockProcessor() {
     }
 
     @Override
-    public void init(ProcessorContext<KOut, VOut> context) {
+    public void init(final ProcessorContext<KOut, VOut> context) {

Review Comment:
   Thanks for cleaning up



-- 
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.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to