You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by gu...@apache.org on 2017/05/03 23:15:58 UTC

[1/5] kafka git commit: KAFKA-5045: Clarify on KTable APIs for queryable stores

Repository: kafka
Updated Branches:
  refs/heads/trunk a3952aee4 -> ec9e4eafa


http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java
index 7ce0b54..6fc6bd4 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java
@@ -21,6 +21,7 @@ import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.utils.Utils;
 import org.apache.kafka.streams.kstream.internals.KStreamImpl;
 import org.apache.kafka.streams.errors.TopologyBuilderException;
+import org.apache.kafka.streams.kstream.internals.KTableImpl;
 import org.apache.kafka.streams.processor.StateStore;
 import org.apache.kafka.streams.processor.TopologyBuilder;
 import org.apache.kafka.streams.processor.internals.ProcessorTopology;
@@ -42,6 +43,7 @@ import java.util.regex.Pattern;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNull;
 
 public class KStreamBuilderTest {
 
@@ -85,6 +87,19 @@ public class KStreamBuilderTest {
     }
 
     @Test
+    public void testNewStoreName() {
+        assertEquals("X-STATE-STORE-0000000000", builder.newStoreName("X-"));
+        assertEquals("Y-STATE-STORE-0000000001", builder.newStoreName("Y-"));
+        assertEquals("Z-STATE-STORE-0000000002", builder.newStoreName("Z-"));
+
+        KStreamBuilder newBuilder = new KStreamBuilder();
+
+        assertEquals("X-STATE-STORE-0000000000", newBuilder.newStoreName("X-"));
+        assertEquals("Y-STATE-STORE-0000000001", newBuilder.newStoreName("Y-"));
+        assertEquals("Z-STATE-STORE-0000000002", newBuilder.newStoreName("Z-"));
+    }
+
+    @Test
     public void testMerge() {
         String topic1 = "topic-1";
         String topic2 = "topic-2";
@@ -151,16 +166,22 @@ public class KStreamBuilderTest {
     }
 
     @Test
-    public void shouldNotMaterializeSourceKTableIfStateNameNotSpecified() throws Exception {
-        builder.table("topic1", "table1");
-        builder.table("topic2", null);
+    public void shouldStillMaterializeSourceKTableIfStateNameNotSpecified() throws Exception {
+        KTable table1 = builder.table("topic1", "table1");
+        KTable table2 = builder.table("topic2", (String) null);
 
         final ProcessorTopology topology = builder.build(null);
 
-        assertEquals(1, topology.stateStores().size());
+        assertEquals(2, topology.stateStores().size());
         assertEquals("table1", topology.stateStores().get(0).name());
-        assertEquals(1, topology.storeToChangelogTopic().size());
+
+        final String internalStoreName = topology.stateStores().get(1).name();
+        assertTrue(internalStoreName.contains(KTableImpl.STATE_STORE_NAME));
+        assertEquals(2, topology.storeToChangelogTopic().size());
         assertEquals("topic1", topology.storeToChangelogTopic().get("table1"));
+        assertEquals("topic2", topology.storeToChangelogTopic().get(internalStoreName));
+        assertEquals(table1.queryableStoreName(), "table1");
+        assertNull(table2.queryableStoreName());
     }
 
     @Test
@@ -174,11 +195,7 @@ public class KStreamBuilderTest {
         assertEquals("globalTable", stateStores.get(0).name());
     }
 
-    @Test
-    public void shouldBuildGlobalTopologyWithAllGlobalTables() throws Exception {
-        builder.globalTable("table", "globalTable");
-        builder.globalTable("table2", "globalTable2");
-
+    private void doBuildGlobalTopologyWithAllGlobalTables() throws Exception {
         final ProcessorTopology topology = builder.buildGlobalStateTopology();
 
         final List<StateStore> stateStores = topology.globalStateStores();
@@ -189,6 +206,22 @@ public class KStreamBuilderTest {
     }
 
     @Test
+    public void shouldBuildGlobalTopologyWithAllGlobalTables() throws Exception {
+        builder.globalTable("table", "globalTable");
+        builder.globalTable("table2", "globalTable2");
+
+        doBuildGlobalTopologyWithAllGlobalTables();
+    }
+
+    @Test
+    public void shouldBuildGlobalTopologyWithAllGlobalTablesWithInternalStoreName() throws Exception {
+        builder.globalTable("table");
+        builder.globalTable("table2");
+
+        doBuildGlobalTopologyWithAllGlobalTables();
+    }
+
+    @Test
     public void shouldAddGlobalTablesToEachGroup() throws Exception {
         final String one = "globalTable";
         final String two = "globalTable2";
@@ -324,4 +357,4 @@ public class KStreamBuilderTest {
         assertTrue(builder.latestResetTopicsPattern().matcher(topicTwo).matches());
         assertFalse(builder.earliestResetTopicsPattern().matcher(topicTwo).matches());
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java
index 8d1a789..f21c7d3 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java
@@ -27,6 +27,7 @@ import org.apache.kafka.streams.kstream.KStream;
 import org.apache.kafka.streams.kstream.KStreamBuilder;
 import org.apache.kafka.streams.kstream.Reducer;
 import org.apache.kafka.streams.kstream.Merger;
+import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.streams.kstream.SessionWindows;
 import org.apache.kafka.streams.kstream.TimeWindows;
 import org.apache.kafka.streams.processor.StateStoreSupplier;
@@ -34,6 +35,7 @@ import org.apache.kafka.streams.kstream.Windowed;
 import org.apache.kafka.streams.kstream.Windows;
 import org.apache.kafka.streams.state.KeyValueStore;
 import org.apache.kafka.streams.state.SessionStore;
+import org.apache.kafka.streams.state.WindowStore;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockAggregator;
 import org.apache.kafka.test.MockInitializer;
@@ -52,6 +54,7 @@ import java.util.Map;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 
 public class KGroupedStreamImplTest {
 
@@ -80,8 +83,8 @@ public class KGroupedStreamImplTest {
         groupedStream.reduce(null, "store");
     }
 
-    @Test(expected = NullPointerException.class)
-    public void shouldNotHaveNullStoreNameOnReduce() throws Exception {
+    @Test
+    public void shouldAllowNullStoreNameOnReduce() throws Exception {
         groupedStream.reduce(MockReducer.STRING_ADDER, (String) null);
     }
 
@@ -96,6 +99,17 @@ public class KGroupedStreamImplTest {
     }
 
     @Test(expected = NullPointerException.class)
+    public void shouldNotHaveNullStoreSupplierOnCount() throws Exception {
+        groupedStream.count((StateStoreSupplier<KeyValueStore>) null);
+    }
+
+
+    @Test(expected = NullPointerException.class)
+    public void shouldNotHaveNullStoreSupplierOnWindowedCount() throws Exception {
+        groupedStream.count(TimeWindows.of(10), (StateStoreSupplier<WindowStore>) null);
+    }
+
+    @Test(expected = NullPointerException.class)
     public void shouldNotHaveNullReducerWithWindowedReduce() throws Exception {
         groupedStream.reduce(null, TimeWindows.of(10), "store");
     }
@@ -105,8 +119,8 @@ public class KGroupedStreamImplTest {
         groupedStream.reduce(MockReducer.STRING_ADDER, (Windows) null, "store");
     }
 
-    @Test(expected = NullPointerException.class)
-    public void shouldNotHaveNullStoreNameWithWindowedReduce() throws Exception {
+    @Test
+    public void shouldAllowNullStoreNameWithWindowedReduce() throws Exception {
         groupedStream.reduce(MockReducer.STRING_ADDER, TimeWindows.of(10), (String) null);
     }
 
@@ -125,8 +139,8 @@ public class KGroupedStreamImplTest {
         groupedStream.aggregate(MockInitializer.STRING_INIT, null, Serdes.String(), "store");
     }
 
-    @Test(expected = NullPointerException.class)
-    public void shouldNotHaveNullStoreNameOnAggregate() throws Exception {
+    @Test
+    public void shouldAllowNullStoreNameOnAggregate() throws Exception {
         groupedStream.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Serdes.String(), null);
     }
 
@@ -150,8 +164,8 @@ public class KGroupedStreamImplTest {
         groupedStream.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, null, Serdes.String(), "store");
     }
 
-    @Test(expected = NullPointerException.class)
-    public void shouldNotHaveNullStoreNameOnWindowedAggregate() throws Exception {
+    @Test
+    public void shouldAllowNullStoreNameOnWindowedAggregate() throws Exception {
         groupedStream.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, TimeWindows.of(10), Serdes.String(), null);
     }
 
@@ -162,13 +176,33 @@ public class KGroupedStreamImplTest {
 
     @Test(expected = NullPointerException.class)
     public void shouldNotHaveNullStoreSupplierOnWindowedAggregate() throws Exception {
-        groupedStream.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, TimeWindows.of(10), null);
+        groupedStream.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, TimeWindows.of(10), (StateStoreSupplier<WindowStore>) null);
+    }
+
+    private void doAggregateSessionWindows(final Map<Windowed<String>, Integer> results) throws Exception {
+        driver = new KStreamTestDriver(builder, TestUtils.tempDirectory());
+        driver.setTime(10);
+        driver.process(TOPIC, "1", "1");
+        driver.setTime(15);
+        driver.process(TOPIC, "2", "2");
+        driver.setTime(30);
+        driver.process(TOPIC, "1", "1");
+        driver.setTime(70);
+        driver.process(TOPIC, "1", "1");
+        driver.setTime(90);
+        driver.process(TOPIC, "1", "1");
+        driver.setTime(100);
+        driver.process(TOPIC, "1", "1");
+        driver.flushState();
+        assertEquals(Integer.valueOf(2), results.get(new Windowed<>("1", new SessionWindow(10, 30))));
+        assertEquals(Integer.valueOf(1), results.get(new Windowed<>("2", new SessionWindow(15, 15))));
+        assertEquals(Integer.valueOf(3), results.get(new Windowed<>("1", new SessionWindow(70, 100))));
     }
 
     @Test
     public void shouldAggregateSessionWindows() throws Exception {
         final Map<Windowed<String>, Integer> results = new HashMap<>();
-        groupedStream.aggregate(new Initializer<Integer>() {
+        KTable table = groupedStream.aggregate(new Initializer<Integer>() {
             @Override
             public Integer apply() {
                 return 0;
@@ -183,43 +217,49 @@ public class KGroupedStreamImplTest {
             public Integer apply(final String aggKey, final Integer aggOne, final Integer aggTwo) {
                 return aggOne + aggTwo;
             }
-        }, SessionWindows.with(30), Serdes.Integer(), "session-store")
-                .foreach(new ForeachAction<Windowed<String>, Integer>() {
-                    @Override
-                    public void apply(final Windowed<String> key, final Integer value) {
-                        results.put(key, value);
-                    }
-                });
+        }, SessionWindows.with(30), Serdes.Integer(), "session-store");
+        table.foreach(new ForeachAction<Windowed<String>, Integer>() {
+            @Override
+            public void apply(final Windowed<String> key, final Integer value) {
+                results.put(key, value);
+            }
+        });
 
-        driver = new KStreamTestDriver(builder, TestUtils.tempDirectory());
-        driver.setTime(10);
-        driver.process(TOPIC, "1", "1");
-        driver.setTime(15);
-        driver.process(TOPIC, "2", "2");
-        driver.setTime(30);
-        driver.process(TOPIC, "1", "1");
-        driver.setTime(70);
-        driver.process(TOPIC, "1", "1");
-        driver.setTime(90);
-        driver.process(TOPIC, "1", "1");
-        driver.setTime(100);
-        driver.process(TOPIC, "1", "1");
-        driver.flushState();
-        assertEquals(Integer.valueOf(2), results.get(new Windowed<>("1", new SessionWindow(10, 30))));
-        assertEquals(Integer.valueOf(1), results.get(new Windowed<>("2", new SessionWindow(15, 15))));
-        assertEquals(Integer.valueOf(3), results.get(new Windowed<>("1", new SessionWindow(70, 100))));
+        doAggregateSessionWindows(results);
+        assertEquals(table.queryableStoreName(), "session-store");
     }
 
     @Test
-    public void shouldCountSessionWindows() throws Exception {
-        final Map<Windowed<String>, Long> results = new HashMap<>();
-        groupedStream.count(SessionWindows.with(30), "session-store")
-                .foreach(new ForeachAction<Windowed<String>, Long>() {
-                    @Override
-                    public void apply(final Windowed<String> key, final Long value) {
-                        results.put(key, value);
-                    }
-                });
+    public void shouldAggregateSessionWindowsWithInternalStoreName() throws Exception {
+        final Map<Windowed<String>, Integer> results = new HashMap<>();
+        KTable table = groupedStream.aggregate(new Initializer<Integer>() {
+            @Override
+            public Integer apply() {
+                return 0;
+            }
+        }, new Aggregator<String, String, Integer>() {
+            @Override
+            public Integer apply(final String aggKey, final String value, final Integer aggregate) {
+                return aggregate + 1;
+            }
+        }, new Merger<String, Integer>() {
+            @Override
+            public Integer apply(final String aggKey, final Integer aggOne, final Integer aggTwo) {
+                return aggOne + aggTwo;
+            }
+        }, SessionWindows.with(30), Serdes.Integer());
+        table.foreach(new ForeachAction<Windowed<String>, Integer>() {
+            @Override
+            public void apply(final Windowed<String> key, final Integer value) {
+                results.put(key, value);
+            }
+        });
+
+        doAggregateSessionWindows(results);
+        assertNull(table.queryableStoreName());
+    }
+
+    private void doCountSessionWindows(final Map<Windowed<String>, Long> results) throws Exception {
         driver = new KStreamTestDriver(builder, TestUtils.tempDirectory());
         driver.setTime(10);
         driver.process(TOPIC, "1", "1");
@@ -240,22 +280,34 @@ public class KGroupedStreamImplTest {
     }
 
     @Test
-    public void shouldReduceSessionWindows() throws Exception {
-        final Map<Windowed<String>, String> results = new HashMap<>();
-        groupedStream.reduce(
-                new Reducer<String>() {
-                    @Override
-                    public String apply(final String value1, final String value2) {
-                        return value1 + ":" + value2;
-                    }
-                }, SessionWindows.with(30),
-                "session-store")
-                .foreach(new ForeachAction<Windowed<String>, String>() {
-                    @Override
-                    public void apply(final Windowed<String> key, final String value) {
-                        results.put(key, value);
-                    }
-                });
+    public void shouldCountSessionWindows() throws Exception {
+        final Map<Windowed<String>, Long> results = new HashMap<>();
+        KTable table = groupedStream.count(SessionWindows.with(30), "session-store");
+        table.foreach(new ForeachAction<Windowed<String>, Long>() {
+            @Override
+            public void apply(final Windowed<String> key, final Long value) {
+                results.put(key, value);
+            }
+        });
+        doCountSessionWindows(results);
+        assertEquals(table.queryableStoreName(), "session-store");
+    }
+
+    @Test
+    public void shouldCountSessionWindowsWithInternalStoreName() throws Exception {
+        final Map<Windowed<String>, Long> results = new HashMap<>();
+        KTable table = groupedStream.count(SessionWindows.with(30));
+        table.foreach(new ForeachAction<Windowed<String>, Long>() {
+            @Override
+            public void apply(final Windowed<String> key, final Long value) {
+                results.put(key, value);
+            }
+        });
+        doCountSessionWindows(results);
+        assertNull(table.queryableStoreName());
+    }
+
+    private void doReduceSessionWindows(final Map<Windowed<String>, String> results) throws Exception {
         driver = new KStreamTestDriver(builder, TestUtils.tempDirectory());
         driver.setTime(10);
         driver.process(TOPIC, "1", "A");
@@ -275,6 +327,48 @@ public class KGroupedStreamImplTest {
         assertEquals("A:B:C", results.get(new Windowed<>("1", new SessionWindow(70, 100))));
     }
 
+    @Test
+    public void shouldReduceSessionWindows() throws Exception {
+        final Map<Windowed<String>, String> results = new HashMap<>();
+        KTable table = groupedStream.reduce(
+                new Reducer<String>() {
+                    @Override
+                    public String apply(final String value1, final String value2) {
+                        return value1 + ":" + value2;
+                    }
+                }, SessionWindows.with(30),
+                "session-store");
+        table.foreach(new ForeachAction<Windowed<String>, String>() {
+            @Override
+            public void apply(final Windowed<String> key, final String value) {
+                results.put(key, value);
+            }
+        });
+        doReduceSessionWindows(results);
+        assertEquals(table.queryableStoreName(), "session-store");
+    }
+
+    @Test
+    public void shouldReduceSessionWindowsWithInternalStoreName() throws Exception {
+        final Map<Windowed<String>, String> results = new HashMap<>();
+        KTable table = groupedStream.reduce(
+                new Reducer<String>() {
+                    @Override
+                    public String apply(final String value1, final String value2) {
+                        return value1 + ":" + value2;
+                    }
+                }, SessionWindows.with(30));
+        table.foreach(new ForeachAction<Windowed<String>, String>() {
+            @Override
+            public void apply(final Windowed<String> key, final String value) {
+                results.put(key, value);
+            }
+        });
+        doReduceSessionWindows(results);
+        assertNull(table.queryableStoreName());
+    }
+
+
     @Test(expected = NullPointerException.class)
     public void shouldNotAcceptNullReducerWhenReducingSessionWindows() throws Exception {
         groupedStream.reduce(null, SessionWindows.with(10), "store");
@@ -285,8 +379,8 @@ public class KGroupedStreamImplTest {
         groupedStream.reduce(MockReducer.STRING_ADDER, (SessionWindows) null, "store");
     }
 
-    @Test(expected = NullPointerException.class)
-    public void shouldNotAcceptNullStoreNameWhenReducingSessionWindows() throws Exception {
+    @Test
+    public void shouldAcceptNullStoreNameWhenReducingSessionWindows() throws Exception {
         groupedStream.reduce(MockReducer.STRING_ADDER, SessionWindows.with(10), (String) null);
     }
 
@@ -323,11 +417,11 @@ public class KGroupedStreamImplTest {
     @Test(expected = NullPointerException.class)
     public void shouldNotAcceptNullSessionMergerWhenAggregatingSessionWindows() throws Exception {
         groupedStream.aggregate(MockInitializer.STRING_INIT,
-                                MockAggregator.TOSTRING_ADDER,
-                                null,
-                                SessionWindows.with(10),
-                                Serdes.String(),
-                                "storeName");
+                MockAggregator.TOSTRING_ADDER,
+                null,
+                SessionWindows.with(10),
+                Serdes.String(),
+                "storeName");
     }
 
     @Test(expected = NullPointerException.class)
@@ -340,8 +434,8 @@ public class KGroupedStreamImplTest {
         }, null, Serdes.String(), "storeName");
     }
 
-    @Test(expected = NullPointerException.class)
-    public void shouldNotAcceptNullStoreNameWhenAggregatingSessionWindows() throws Exception {
+    @Test
+    public void shouldAcceptNullStoreNameWhenAggregatingSessionWindows() throws Exception {
         groupedStream.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, new Merger<String, String>() {
             @Override
             public String apply(final String aggKey, final String aggOne, final String aggTwo) {
@@ -375,8 +469,8 @@ public class KGroupedStreamImplTest {
         groupedStream.count((SessionWindows) null, "store");
     }
 
-    @Test(expected = NullPointerException.class)
-    public void shouldNotAcceptNullStoreNameWhenCountingSessionWindows() throws Exception {
+    @Test
+    public void shouldAcceptNullStoreNameWhenCountingSessionWindows() throws Exception {
         groupedStream.count(SessionWindows.with(90), (String) null);
     }
 
@@ -390,19 +484,7 @@ public class KGroupedStreamImplTest {
         groupedStream.count(SessionWindows.with(90), (StateStoreSupplier<SessionStore>) null);
     }
 
-    @Test
-    public void shouldCountWindowed() throws Exception {
-        final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>();
-        groupedStream.count(
-                TimeWindows.of(500L),
-                "aggregate-by-key-windowed")
-                .foreach(new ForeachAction<Windowed<String>, Long>() {
-                    @Override
-                    public void apply(final Windowed<String> key, final Long value) {
-                        results.add(KeyValue.pair(key, value));
-                    }
-                });
-
+    private void doCountWindowed(final List<KeyValue<Windowed<String>, Long>> results) throws Exception {
         driver = new KStreamTestDriver(builder, TestUtils.tempDirectory(), 0);
         driver.setTime(0);
         driver.process(TOPIC, "1", "A");
@@ -423,4 +505,35 @@ public class KGroupedStreamImplTest {
                 KeyValue.pair(new Windowed<>("2", new TimeWindow(500, 1000)), 2L)
         )));
     }
+
+    @Test
+    public void shouldCountWindowed() throws Exception {
+        final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>();
+        groupedStream.count(
+                TimeWindows.of(500L),
+                "aggregate-by-key-windowed")
+                .foreach(new ForeachAction<Windowed<String>, Long>() {
+                    @Override
+                    public void apply(final Windowed<String> key, final Long value) {
+                        results.add(KeyValue.pair(key, value));
+                    }
+                });
+
+        doCountWindowed(results);
+    }
+
+    @Test
+    public void shouldCountWindowedWithInternalStoreName() throws Exception {
+        final List<KeyValue<Windowed<String>, Long>> results = new ArrayList<>();
+        groupedStream.count(
+                TimeWindows.of(500L))
+                .foreach(new ForeachAction<Windowed<String>, Long>() {
+                    @Override
+                    public void apply(final Windowed<String> key, final Long value) {
+                        results.add(KeyValue.pair(key, value));
+                    }
+                });
+
+        doCountWindowed(results);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java
index 4934204..1e49b22 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java
@@ -24,6 +24,8 @@ import org.apache.kafka.streams.kstream.KGroupedTable;
 import org.apache.kafka.streams.kstream.KStreamBuilder;
 import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.streams.kstream.KeyValueMapper;
+import org.apache.kafka.streams.processor.StateStoreSupplier;
+import org.apache.kafka.streams.state.KeyValueStore;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockAggregator;
 import org.apache.kafka.test.MockInitializer;
@@ -38,6 +40,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 
 
 public class KGroupedTableImplTest {
@@ -61,8 +64,13 @@ public class KGroupedTableImplTest {
         driver = null;
     }
 
-    @Test(expected = NullPointerException.class)
-    public void shouldNotAllowNullStoreNameOnAggregate() throws Exception {
+    @Test
+    public void shouldAllowNullStoreNameOnCount()  {
+        groupedTable.count((String) null);
+    }
+
+    @Test
+    public void shouldAllowNullStoreNameOnAggregate() throws Exception {
         groupedTable.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, MockAggregator.TOSTRING_REMOVER, (String) null);
     }
 
@@ -96,8 +104,8 @@ public class KGroupedTableImplTest {
         groupedTable.reduce(MockReducer.STRING_ADDER, null, "store");
     }
 
-    @Test(expected = NullPointerException.class)
-    public void shouldNotAllowNullStoreNameOnReduce() throws Exception {
+    @Test
+    public void shouldAllowNullStoreNameOnReduce() throws Exception {
         groupedTable.reduce(MockReducer.STRING_ADDER, MockReducer.STRING_REMOVER, (String) null);
     }
 
@@ -108,24 +116,10 @@ public class KGroupedTableImplTest {
 
     @Test(expected = NullPointerException.class)
     public void shouldNotAllowNullStoreSupplierOnReduce() throws Exception {
-        groupedTable.reduce(MockReducer.STRING_ADDER, MockReducer.STRING_REMOVER, (String) null);
+        groupedTable.reduce(MockReducer.STRING_ADDER, MockReducer.STRING_REMOVER, (StateStoreSupplier<KeyValueStore>) null);
     }
 
-    @Test
-    public void shouldReduce() throws Exception {
-        final String topic = "input";
-        final KeyValueMapper<String, Number, KeyValue<String, Integer>> intProjection =
-            new KeyValueMapper<String, Number, KeyValue<String, Integer>>() {
-                @Override
-                public KeyValue<String, Integer> apply(String key, Number value) {
-                    return KeyValue.pair(key, value.intValue());
-                }
-            };
-
-        final KTable<String, Integer> reduced = builder.table(Serdes.String(), Serdes.Double(), topic, "store")
-            .groupBy(intProjection)
-            .reduce(MockReducer.INTEGER_ADDER, MockReducer.INTEGER_SUBTRACTOR, "reduced");
-
+    private void doShouldReduce(final KTable<String, Integer> reduced, final String topic) throws Exception {
         final Map<String, Integer> results = new HashMap<>();
         reduced.foreach(new ForeachAction<String, Integer>() {
             @Override
@@ -152,4 +146,42 @@ public class KGroupedTableImplTest {
         assertEquals(Integer.valueOf(5), results.get("A"));
         assertEquals(Integer.valueOf(6), results.get("B"));
     }
+
+    @Test
+    public void shouldReduce() throws Exception {
+        final String topic = "input";
+        final KeyValueMapper<String, Number, KeyValue<String, Integer>> intProjection =
+            new KeyValueMapper<String, Number, KeyValue<String, Integer>>() {
+                @Override
+                public KeyValue<String, Integer> apply(String key, Number value) {
+                    return KeyValue.pair(key, value.intValue());
+                }
+            };
+
+        final KTable<String, Integer> reduced = builder.table(Serdes.String(), Serdes.Double(), topic, "store")
+            .groupBy(intProjection)
+            .reduce(MockReducer.INTEGER_ADDER, MockReducer.INTEGER_SUBTRACTOR, "reduced");
+
+        doShouldReduce(reduced, topic);
+        assertEquals(reduced.queryableStoreName(), "reduced");
+    }
+
+    @Test
+    public void shouldReduceWithInternalStoreName() throws Exception {
+        final String topic = "input";
+        final KeyValueMapper<String, Number, KeyValue<String, Integer>> intProjection =
+            new KeyValueMapper<String, Number, KeyValue<String, Integer>>() {
+                @Override
+                public KeyValue<String, Integer> apply(String key, Number value) {
+                    return KeyValue.pair(key, value.intValue());
+                }
+            };
+
+        final KTable<String, Integer> reduced = builder.table(Serdes.String(), Serdes.Double(), topic, "store")
+            .groupBy(intProjection)
+            .reduce(MockReducer.INTEGER_ADDER, MockReducer.INTEGER_SUBTRACTOR);
+
+        doShouldReduce(reduced, topic);
+        assertNull(reduced.queryableStoreName());
+    }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java
index 23325c4..2c37230 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java
@@ -212,18 +212,7 @@ public class KTableAggregateTest {
                 ), proc.processed);
     }
 
-    @Test
-    public void testCount() throws IOException {
-        final KStreamBuilder builder = new KStreamBuilder();
-        final String input = "count-test-input";
-        final MockProcessorSupplier<String, Long> proc = new MockProcessorSupplier<>();
-
-        builder.table(Serdes.String(), Serdes.String(), input, "anyStoreName")
-                .groupBy(MockKeyValueMapper.<String, String>SelectValueKeyValueMapper(), stringSerde, stringSerde)
-                .count("count")
-                .toStream()
-                .process(proc);
-
+    private void testCountHelper(final KStreamBuilder builder, final String input, final MockProcessorSupplier<String, Long> proc) throws IOException {
         driver = new KStreamTestDriver(builder, stateDir);
 
         driver.process(input, "A", "green");
@@ -240,12 +229,42 @@ public class KTableAggregateTest {
 
 
         assertEquals(Utils.mkList(
-                 "green:1",
-                 "green:2",
-                 "green:1", "blue:1",
-                 "yellow:1",
-                 "green:2"
-                 ), proc.processed);
+            "green:1",
+            "green:2",
+            "green:1", "blue:1",
+            "yellow:1",
+            "green:2"
+        ), proc.processed);
+    }
+
+    @Test
+    public void testCount() throws IOException {
+        final KStreamBuilder builder = new KStreamBuilder();
+        final String input = "count-test-input";
+        final MockProcessorSupplier<String, Long> proc = new MockProcessorSupplier<>();
+
+        builder.table(Serdes.String(), Serdes.String(), input, "anyStoreName")
+                .groupBy(MockKeyValueMapper.<String, String>SelectValueKeyValueMapper(), stringSerde, stringSerde)
+                .count("count")
+                .toStream()
+                .process(proc);
+
+        testCountHelper(builder, input, proc);
+    }
+
+    @Test
+    public void testCountWithInternalStore() throws IOException {
+        final KStreamBuilder builder = new KStreamBuilder();
+        final String input = "count-test-input";
+        final MockProcessorSupplier<String, Long> proc = new MockProcessorSupplier<>();
+
+        builder.table(Serdes.String(), Serdes.String(), input, "anyStoreName")
+            .groupBy(MockKeyValueMapper.<String, String>SelectValueKeyValueMapper(), stringSerde, stringSerde)
+            .count()
+            .toStream()
+            .process(proc);
+
+        testCountHelper(builder, input, proc);
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java
index 3ab4300..c6721f7 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java
@@ -57,6 +57,28 @@ public class KTableFilterTest {
         stateDir = TestUtils.tempDirectory("kafka-test");
     }
 
+    private void doTestKTable(final KStreamBuilder builder, final KTable<String, Integer> table2,
+                              final KTable<String, Integer> table3, final String topic1) {
+        MockProcessorSupplier<String, Integer> proc2 = new MockProcessorSupplier<>();
+        MockProcessorSupplier<String, Integer> proc3 = new MockProcessorSupplier<>();
+        table2.toStream().process(proc2);
+        table3.toStream().process(proc3);
+
+        driver = new KStreamTestDriver(builder, stateDir, Serdes.String(), Serdes.Integer());
+
+        driver.process(topic1, "A", 1);
+        driver.process(topic1, "B", 2);
+        driver.process(topic1, "C", 3);
+        driver.process(topic1, "D", 4);
+        driver.flushState();
+        driver.process(topic1, "A", null);
+        driver.process(topic1, "B", null);
+        driver.flushState();
+
+        proc2.checkAndClearProcessResult("A:null", "B:2", "C:null", "D:4", "A:null", "B:null");
+        proc3.checkAndClearProcessResult("A:1", "B:null", "C:3", "D:null", "A:null", "B:null");
+    }
+
     @Test
     public void testKTable() {
         final KStreamBuilder builder = new KStreamBuilder();
@@ -78,53 +100,41 @@ public class KTableFilterTest {
             }
         });
 
-        MockProcessorSupplier<String, Integer> proc2 = new MockProcessorSupplier<>();
-        MockProcessorSupplier<String, Integer> proc3 = new MockProcessorSupplier<>();
-        table2.toStream().process(proc2);
-        table3.toStream().process(proc3);
-
-        driver = new KStreamTestDriver(builder, stateDir);
-
-        driver.process(topic1, "A", 1);
-        driver.process(topic1, "B", 2);
-        driver.process(topic1, "C", 3);
-        driver.process(topic1, "D", 4);
-        driver.flushState();
-        driver.process(topic1, "A", null);
-        driver.process(topic1, "B", null);
-        driver.flushState();
-
-        proc2.checkAndClearProcessResult("A:null", "B:2", "C:null", "D:4", "A:null", "B:null");
-        proc3.checkAndClearProcessResult("A:1", "B:null", "C:3", "D:null", "A:null", "B:null");
+        doTestKTable(builder, table2, table3, topic1);
     }
 
     @Test
-    public void testValueGetter() throws IOException {
-        KStreamBuilder builder = new KStreamBuilder();
+    public void testQueryableKTable() {
+        final KStreamBuilder builder = new KStreamBuilder();
 
         String topic1 = "topic1";
 
-        KTableImpl<String, Integer, Integer> table1 =
-                (KTableImpl<String, Integer, Integer>) builder.table(stringSerde, intSerde, topic1, "anyStoreName");
-        KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
-                new Predicate<String, Integer>() {
-                    @Override
-                    public boolean test(String key, Integer value) {
-                        return (value % 2) == 0;
-                    }
-                });
-        KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table1.filterNot(
-                new Predicate<String, Integer>() {
-                    @Override
-                    public boolean test(String key, Integer value) {
-                        return (value % 2) == 0;
-                    }
-                });
+        KTable<String, Integer> table1 = builder.table(stringSerde, intSerde, topic1, "anyStoreName");
+
+        KTable<String, Integer> table2 = table1.filter(new Predicate<String, Integer>() {
+            @Override
+            public boolean test(String key, Integer value) {
+                return (value % 2) == 0;
+            }
+        }, "anyStoreNameFilter");
+        KTable<String, Integer> table3 = table1.filterNot(new Predicate<String, Integer>() {
+            @Override
+            public boolean test(String key, Integer value) {
+                return (value % 2) == 0;
+            }
+        });
+
+        doTestKTable(builder, table2, table3, topic1);
+    }
 
+    private void doTestValueGetter(final KStreamBuilder builder,
+                                   final KTableImpl<String, Integer, Integer> table2,
+                                   final KTableImpl<String, Integer, Integer> table3,
+                                   final String topic1) throws IOException {
         KTableValueGetterSupplier<String, Integer> getterSupplier2 = table2.valueGetterSupplier();
         KTableValueGetterSupplier<String, Integer> getterSupplier3 = table3.valueGetterSupplier();
 
-        driver = new KStreamTestDriver(builder, stateDir, null, null);
+        driver = new KStreamTestDriver(builder, stateDir, Serdes.String(), Serdes.Integer());
 
         KTableValueGetter<String, Integer> getter2 = getterSupplier2.get();
         KTableValueGetter<String, Integer> getter3 = getterSupplier3.get();
@@ -178,7 +188,7 @@ public class KTableFilterTest {
     }
 
     @Test
-    public void testNotSendingOldValue() throws IOException {
+    public void testValueGetter() throws IOException {
         KStreamBuilder builder = new KStreamBuilder();
 
         String topic1 = "topic1";
@@ -192,14 +202,54 @@ public class KTableFilterTest {
                         return (value % 2) == 0;
                     }
                 });
+        KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table1.filterNot(
+                new Predicate<String, Integer>() {
+                    @Override
+                    public boolean test(String key, Integer value) {
+                        return (value % 2) == 0;
+                    }
+                });
+
+        doTestValueGetter(builder, table2, table3, topic1);
+    }
 
+    @Test
+    public void testQueryableValueGetter() throws IOException {
+        KStreamBuilder builder = new KStreamBuilder();
+
+        String topic1 = "topic1";
+
+        KTableImpl<String, Integer, Integer> table1 =
+            (KTableImpl<String, Integer, Integer>) builder.table(stringSerde, intSerde, topic1, "anyStoreName");
+        KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
+            new Predicate<String, Integer>() {
+                @Override
+                public boolean test(String key, Integer value) {
+                    return (value % 2) == 0;
+                }
+            }, "anyStoreNameFilter");
+        KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table1.filterNot(
+            new Predicate<String, Integer>() {
+                @Override
+                public boolean test(String key, Integer value) {
+                    return (value % 2) == 0;
+                }
+            });
+
+        doTestValueGetter(builder, table2, table3, topic1);
+    }
+
+    private void doTestNotSendingOldValue(final KStreamBuilder builder,
+                                          final KTableImpl<String, Integer, Integer> table1,
+                                          final KTableImpl<String, Integer, Integer> table2,
+                                          final String topic1) throws IOException {
         MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>();
         MockProcessorSupplier<String, Integer> proc2 = new MockProcessorSupplier<>();
 
         builder.addProcessor("proc1", proc1, table1.name);
         builder.addProcessor("proc2", proc2, table2.name);
 
-        driver = new KStreamTestDriver(builder, stateDir, null, null);
+        driver = new KStreamTestDriver(builder, stateDir, Serdes.String(), Serdes.Integer());
 
         driver.process(topic1, "A", 1);
         driver.process(topic1, "B", 1);
@@ -227,8 +277,9 @@ public class KTableFilterTest {
         proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)");
     }
 
+
     @Test
-    public void testSendingOldValue() throws IOException {
+    public void testNotSendingOldValue() throws IOException {
         KStreamBuilder builder = new KStreamBuilder();
 
         String topic1 = "topic1";
@@ -243,6 +294,32 @@ public class KTableFilterTest {
                     }
                 });
 
+        doTestNotSendingOldValue(builder, table1, table2, topic1);
+    }
+
+    @Test
+    public void testQueryableNotSendingOldValue() throws IOException {
+        KStreamBuilder builder = new KStreamBuilder();
+
+        String topic1 = "topic1";
+
+        KTableImpl<String, Integer, Integer> table1 =
+            (KTableImpl<String, Integer, Integer>) builder.table(stringSerde, intSerde, topic1, "anyStoreName");
+        KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
+            new Predicate<String, Integer>() {
+                @Override
+                public boolean test(String key, Integer value) {
+                    return (value % 2) == 0;
+                }
+            }, "anyStoreNameFilter");
+
+        doTestNotSendingOldValue(builder, table1, table2, topic1);
+    }
+
+    private void doTestSendingOldValue(final KStreamBuilder builder,
+                                       final KTableImpl<String, Integer, Integer> table1,
+                                       final KTableImpl<String, Integer, Integer> table2,
+                                       final String topic1) throws IOException {
         table2.enableSendingOldValues();
 
         MockProcessorSupplier<String, Integer> proc1 = new MockProcessorSupplier<>();
@@ -251,7 +328,7 @@ public class KTableFilterTest {
         builder.addProcessor("proc1", proc1, table1.name);
         builder.addProcessor("proc2", proc2, table2.name);
 
-        driver = new KStreamTestDriver(builder, stateDir, null, null);
+        driver = new KStreamTestDriver(builder, stateDir, Serdes.String(), Serdes.Integer());
 
         driver.process(topic1, "A", 1);
         driver.process(topic1, "B", 1);
@@ -280,23 +357,47 @@ public class KTableFilterTest {
     }
 
     @Test
-    public void testSkipNullOnMaterialization() throws IOException {
-        // Do not explicitly set enableSendingOldValues. Let a further downstream stateful operator trigger it instead.
+    public void testSendingOldValue() throws IOException {
         KStreamBuilder builder = new KStreamBuilder();
 
         String topic1 = "topic1";
 
-        KTableImpl<String, String, String> table1 =
-            (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1, "anyStoreName");
-        KTableImpl<String, String, String> table2 = (KTableImpl<String, String, String>) table1.filter(
-            new Predicate<String, String>() {
+        KTableImpl<String, Integer, Integer> table1 =
+                (KTableImpl<String, Integer, Integer>) builder.table(stringSerde, intSerde, topic1, "anyStoreName");
+        KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
+                new Predicate<String, Integer>() {
+                    @Override
+                    public boolean test(String key, Integer value) {
+                        return (value % 2) == 0;
+                    }
+                });
+
+        doTestSendingOldValue(builder, table1, table2, topic1);
+    }
+
+    @Test
+    public void testQueryableSendingOldValue() throws IOException {
+        KStreamBuilder builder = new KStreamBuilder();
+
+        String topic1 = "topic1";
+
+        KTableImpl<String, Integer, Integer> table1 =
+            (KTableImpl<String, Integer, Integer>) builder.table(stringSerde, intSerde, topic1, "anyStoreName");
+        KTableImpl<String, Integer, Integer> table2 = (KTableImpl<String, Integer, Integer>) table1.filter(
+            new Predicate<String, Integer>() {
                 @Override
-                public boolean test(String key, String value) {
-                    return value.equalsIgnoreCase("accept");
+                public boolean test(String key, Integer value) {
+                    return (value % 2) == 0;
                 }
-            }).groupBy(MockKeyValueMapper.<String, String>NoOpKeyValueMapper())
-            .reduce(MockReducer.STRING_ADDER, MockReducer.STRING_REMOVER, "mock-result");
+            }, "anyStoreNameFilter");
+
+        doTestSendingOldValue(builder, table1, table2, topic1);
+    }
 
+    private void doTestSkipNullOnMaterialization(final KStreamBuilder builder,
+                                                 final KTableImpl<String, String, String> table1,
+                                                 final KTableImpl<String, String, String> table2,
+                                                 final String topic1) throws IOException {
         MockProcessorSupplier<String, String> proc1 = new MockProcessorSupplier<>();
         MockProcessorSupplier<String, String> proc2 = new MockProcessorSupplier<>();
 
@@ -314,6 +415,48 @@ public class KTableFilterTest {
     }
 
     @Test
+    public void testSkipNullOnMaterialization() throws IOException {
+        // Do not explicitly set enableSendingOldValues. Let a further downstream stateful operator trigger it instead.
+        KStreamBuilder builder = new KStreamBuilder();
+
+        String topic1 = "topic1";
+
+        KTableImpl<String, String, String> table1 =
+            (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1, "anyStoreName");
+        KTableImpl<String, String, String> table2 = (KTableImpl<String, String, String>) table1.filter(
+            new Predicate<String, String>() {
+                @Override
+                public boolean test(String key, String value) {
+                    return value.equalsIgnoreCase("accept");
+                }
+            }).groupBy(MockKeyValueMapper.<String, String>NoOpKeyValueMapper())
+            .reduce(MockReducer.STRING_ADDER, MockReducer.STRING_REMOVER, "mock-result");
+
+        doTestSkipNullOnMaterialization(builder, table1, table2, topic1);
+    }
+
+    @Test
+    public void testQueryableSkipNullOnMaterialization() throws IOException {
+        // Do not explicitly set enableSendingOldValues. Let a further downstream stateful operator trigger it instead.
+        KStreamBuilder builder = new KStreamBuilder();
+
+        String topic1 = "topic1";
+
+        KTableImpl<String, String, String> table1 =
+            (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1, "anyStoreName");
+        KTableImpl<String, String, String> table2 = (KTableImpl<String, String, String>) table1.filter(
+            new Predicate<String, String>() {
+                @Override
+                public boolean test(String key, String value) {
+                    return value.equalsIgnoreCase("accept");
+                }
+            }, "anyStoreNameFilter").groupBy(MockKeyValueMapper.<String, String>NoOpKeyValueMapper())
+            .reduce(MockReducer.STRING_ADDER, MockReducer.STRING_REMOVER, "mock-result");
+
+        doTestSkipNullOnMaterialization(builder, table1, table2, topic1);
+    }
+
+    @Test
     public void testTypeVariance() throws Exception {
         Predicate<Number, Object> numberKeyPredicate = new Predicate<Number, Object>() {
             @Override

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java
index 7aef28a..1fbaebf 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java
@@ -25,8 +25,10 @@ import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.streams.kstream.Predicate;
 import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.apache.kafka.streams.kstream.ValueMapper;
+import org.apache.kafka.streams.processor.StateStoreSupplier;
 import org.apache.kafka.streams.processor.internals.SinkNode;
 import org.apache.kafka.streams.processor.internals.SourceNode;
+import org.apache.kafka.streams.state.KeyValueStore;
 import org.apache.kafka.test.KStreamTestDriver;
 import org.apache.kafka.test.MockAggregator;
 import org.apache.kafka.test.MockInitializer;
@@ -419,13 +421,13 @@ public class KTableImplTest {
     }
 
     @Test(expected = NullPointerException.class)
-    public void shouldNotAllowNullTopicInThrough() throws Exception {
-        table.through(null, "store");
+    public void shouldAllowNullTopicInThrough() throws Exception {
+        table.through((String) null, "store");
     }
 
-    @Test(expected = NullPointerException.class)
-    public void shouldNotAllowNullStoreInThrough() throws Exception {
-        table.through("topic", null);
+    @Test
+    public void shouldAllowNullStoreInThrough() throws Exception {
+        table.through("topic", (String) null);
     }
 
     @Test(expected = NullPointerException.class)
@@ -438,6 +440,26 @@ public class KTableImplTest {
         table.join(null, MockValueJoiner.TOSTRING_JOINER);
     }
 
+    @Test
+    public void shouldAllowNullStoreInJoin() throws Exception {
+        table.join(table, MockValueJoiner.TOSTRING_JOINER, null, (String) null);
+    }
+
+    @Test(expected = NullPointerException.class)
+    public void shouldNotAllowNullStoreSupplierInJoin() throws Exception {
+        table.join(table, MockValueJoiner.TOSTRING_JOINER, (StateStoreSupplier<KeyValueStore>) null);
+    }
+
+    @Test(expected = NullPointerException.class)
+    public void shouldNotAllowNullStoreSupplierInLeftJoin() throws Exception {
+        table.leftJoin(table, MockValueJoiner.TOSTRING_JOINER, (StateStoreSupplier<KeyValueStore>) null);
+    }
+
+    @Test(expected = NullPointerException.class)
+    public void shouldNotAllowNullStoreSupplierInOuterJoin() throws Exception {
+        table.outerJoin(table, MockValueJoiner.TOSTRING_JOINER, (StateStoreSupplier<KeyValueStore>) null);
+    }
+
     @Test(expected = NullPointerException.class)
     public void shouldNotAllowNullJoinerJoin() throws Exception {
         table.join(table, null);

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java
index e3d3e95..fdc07f3 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java
@@ -67,23 +67,10 @@ public class KTableKTableJoinTest {
         stateDir = TestUtils.tempDirectory("kafka-test");
     }
 
-    @Test
-    public void testJoin() throws Exception {
-        final KStreamBuilder builder = new KStreamBuilder();
-
-        final int[] expectedKeys = new int[]{0, 1, 2, 3};
-
-        final KTable<Integer, String> table1;
-        final KTable<Integer, String> table2;
-        final KTable<Integer, String> joined;
-        final MockProcessorSupplier<Integer, String> processor;
-
-        processor = new MockProcessorSupplier<>();
-        table1 = builder.table(intSerde, stringSerde, topic1, storeName1);
-        table2 = builder.table(intSerde, stringSerde, topic2, storeName2);
-        joined = table1.join(table2, MockValueJoiner.TOSTRING_JOINER);
-        joined.toStream().process(processor);
-
+    private void doTestJoin(final KStreamBuilder builder,
+                            final int[] expectedKeys,
+                            final MockProcessorSupplier<Integer, String> processor,
+                            final KTable<Integer, String> joined) {
         final Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
 
         assertEquals(1, copartitionGroups.size());
@@ -91,7 +78,7 @@ public class KTableKTableJoinTest {
 
         final KTableValueGetterSupplier<Integer, String> getterSupplier = ((KTableImpl<Integer, String, String>) joined).valueGetterSupplier();
 
-        driver = new KStreamTestDriver(builder, stateDir);
+        driver = new KStreamTestDriver(builder, stateDir, Serdes.Integer(), Serdes.String());
         driver.setTime(0L);
 
         final KTableValueGetter<Integer, String> getter = getterSupplier.get();
@@ -175,7 +162,7 @@ public class KTableKTableJoinTest {
     }
 
     @Test
-    public void testNotSendingOldValues() throws Exception {
+    public void testJoin() throws Exception {
         final KStreamBuilder builder = new KStreamBuilder();
 
         final int[] expectedKeys = new int[]{0, 1, 2, 3};
@@ -183,22 +170,60 @@ public class KTableKTableJoinTest {
         final KTable<Integer, String> table1;
         final KTable<Integer, String> table2;
         final KTable<Integer, String> joined;
-        final MockProcessorSupplier<Integer, String> proc;
+        final MockProcessorSupplier<Integer, String> processor;
 
+        processor = new MockProcessorSupplier<>();
         table1 = builder.table(intSerde, stringSerde, topic1, storeName1);
         table2 = builder.table(intSerde, stringSerde, topic2, storeName2);
         joined = table1.join(table2, MockValueJoiner.TOSTRING_JOINER);
+        joined.toStream().process(processor);
 
-        proc = new MockProcessorSupplier<>();
-        builder.addProcessor("proc", proc, ((KTableImpl<?, ?, ?>) joined).name);
+        doTestJoin(builder, expectedKeys, processor, joined);
+    }
 
-        driver = new KStreamTestDriver(builder, stateDir);
-        driver.setTime(0L);
 
-        assertFalse(((KTableImpl<?, ?, ?>) table1).sendingOldValueEnabled());
-        assertFalse(((KTableImpl<?, ?, ?>) table2).sendingOldValueEnabled());
-        assertFalse(((KTableImpl<?, ?, ?>) joined).sendingOldValueEnabled());
+    @Test
+    public void testQueryableJoin() throws Exception {
+        final KStreamBuilder builder = new KStreamBuilder();
+
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
+
+        final KTable<Integer, String> table1;
+        final KTable<Integer, String> table2;
+        final KTable<Integer, String> joined;
+        final MockProcessorSupplier<Integer, String> processor;
+
+        processor = new MockProcessorSupplier<>();
+        table1 = builder.table(intSerde, stringSerde, topic1, storeName1);
+        table2 = builder.table(intSerde, stringSerde, topic2, storeName2);
+        joined = table1.join(table2, MockValueJoiner.TOSTRING_JOINER, Serdes.String(), "anyQueryableName");
+        joined.toStream().process(processor);
+
+        doTestJoin(builder, expectedKeys, processor, joined);
+    }
+
+    private void doTestSendingOldValues(final KStreamBuilder builder,
+                                        final int[] expectedKeys,
+                                        final KTable<Integer, String> table1,
+                                        final KTable<Integer, String> table2,
+                                        final MockProcessorSupplier<Integer, String> proc,
+                                        final KTable<Integer, String> joined,
+                                        final boolean sendOldValues) {
 
+
+        driver = new KStreamTestDriver(builder, stateDir, Serdes.Integer(), Serdes.String());
+        driver.setTime(0L);
+
+        if (!sendOldValues) {
+            assertFalse(((KTableImpl<?, ?, ?>) table1).sendingOldValueEnabled());
+            assertFalse(((KTableImpl<?, ?, ?>) table2).sendingOldValueEnabled());
+            assertFalse(((KTableImpl<?, ?, ?>) joined).sendingOldValueEnabled());
+        } else {
+            ((KTableImpl<?, ?, ?>) joined).enableSendingOldValues();
+            assertTrue(((KTableImpl<?, ?, ?>) table1).sendingOldValueEnabled());
+            assertTrue(((KTableImpl<?, ?, ?>) table2).sendingOldValueEnabled());
+            assertTrue(((KTableImpl<?, ?, ?>) joined).sendingOldValueEnabled());
+        }
         // push two items to the primary stream. the other table is empty
 
         for (int i = 0; i < 2; i++) {
@@ -259,7 +284,7 @@ public class KTableKTableJoinTest {
     }
 
     @Test
-    public void testSendingOldValues() throws Exception {
+    public void testNotSendingOldValues() throws Exception {
         final KStreamBuilder builder = new KStreamBuilder();
 
         final int[] expectedKeys = new int[]{0, 1, 2, 3};
@@ -272,73 +297,54 @@ public class KTableKTableJoinTest {
         table1 = builder.table(intSerde, stringSerde, topic1, storeName1);
         table2 = builder.table(intSerde, stringSerde, topic2, storeName2);
         joined = table1.join(table2, MockValueJoiner.TOSTRING_JOINER);
-
-        ((KTableImpl<?, ?, ?>) joined).enableSendingOldValues();
-
         proc = new MockProcessorSupplier<>();
         builder.addProcessor("proc", proc, ((KTableImpl<?, ?, ?>) joined).name);
 
-        driver = new KStreamTestDriver(builder, stateDir);
-        driver.setTime(0L);
+        doTestSendingOldValues(builder, expectedKeys, table1, table2, proc, joined, false);
 
-        assertTrue(((KTableImpl<?, ?, ?>) table1).sendingOldValueEnabled());
-        assertTrue(((KTableImpl<?, ?, ?>) table2).sendingOldValueEnabled());
-        assertTrue(((KTableImpl<?, ?, ?>) joined).sendingOldValueEnabled());
+    }
 
-        // push two items to the primary stream. the other table is empty
+    @Test
+    public void testQueryableNotSendingOldValues() throws Exception {
+        final KStreamBuilder builder = new KStreamBuilder();
 
-        for (int i = 0; i < 2; i++) {
-            driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]);
-        }
-        driver.flushState();
-        proc.checkAndClearProcessResult();
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-        // push two items to the other stream. this should produce two items.
+        final KTable<Integer, String> table1;
+        final KTable<Integer, String> table2;
+        final KTable<Integer, String> joined;
+        final MockProcessorSupplier<Integer, String> proc;
 
-        for (int i = 0; i < 2; i++) {
-            driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]);
-        }
-        driver.flushState();
-        proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)");
+        table1 = builder.table(intSerde, stringSerde, topic1, storeName1);
+        table2 = builder.table(intSerde, stringSerde, topic2, storeName2);
+        joined = table1.join(table2, MockValueJoiner.TOSTRING_JOINER, Serdes.String(), "anyQueryableName");
+        proc = new MockProcessorSupplier<>();
+        builder.addProcessor("proc", proc, ((KTableImpl<?, ?, ?>) joined).name);
 
-        // push all four items to the primary stream. this should produce two items.
+        doTestSendingOldValues(builder, expectedKeys, table1, table2, proc, joined, false);
 
-        for (int expectedKey : expectedKeys) {
-            driver.process(topic1, expectedKey, "XX" + expectedKey);
-        }
-        driver.flushState();
-        proc.checkAndClearProcessResult("0:(XX0+Y0<-X0+Y0)", "1:(XX1+Y1<-X1+Y1)");
+    }
 
-        // push all items to the other stream. this should produce four items.
-        for (int expectedKey : expectedKeys) {
-            driver.process(topic2, expectedKey, "YY" + expectedKey);
-        }
-        driver.flushState();
-        proc.checkAndClearProcessResult("0:(XX0+YY0<-XX0+Y0)", "1:(XX1+YY1<-XX1+Y1)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)");
+    @Test
+    public void testSendingOldValues() throws Exception {
+        final KStreamBuilder builder = new KStreamBuilder();
 
-        // push all four items to the primary stream. this should produce four items.
+        final int[] expectedKeys = new int[]{0, 1, 2, 3};
 
-        for (int expectedKey : expectedKeys) {
-            driver.process(topic1, expectedKey, "X" + expectedKey);
-        }
-        driver.flushState();
-        proc.checkAndClearProcessResult("0:(X0+YY0<-XX0+YY0)", "1:(X1+YY1<-XX1+YY1)", "2:(X2+YY2<-XX2+YY2)", "3:(X3+YY3<-XX3+YY3)");
+        final KTable<Integer, String> table1;
+        final KTable<Integer, String> table2;
+        final KTable<Integer, String> joined;
+        final MockProcessorSupplier<Integer, String> proc;
 
-        // push two items with null to the other stream as deletes. this should produce two item.
+        table1 = builder.table(intSerde, stringSerde, topic1, storeName1);
+        table2 = builder.table(intSerde, stringSerde, topic2, storeName2);
+        joined = table1.join(table2, MockValueJoiner.TOSTRING_JOINER);
 
-        for (int i = 0; i < 2; i++) {
-            driver.process(topic2, expectedKeys[i], null);
-        }
-        driver.flushState();
-        proc.checkAndClearProcessResult("0:(null<-X0+YY0)", "1:(null<-X1+YY1)");
+        proc = new MockProcessorSupplier<>();
+        builder.addProcessor("proc", proc, ((KTableImpl<?, ?, ?>) joined).name);
 
-        // push all four items to the primary stream. this should produce two items.
+        doTestSendingOldValues(builder, expectedKeys, table1, table2, proc, joined, true);
 
-        for (int expectedKey : expectedKeys) {
-            driver.process(topic1, expectedKey, "XX" + expectedKey);
-        }
-        driver.flushState();
-        proc.checkAndClearProcessResult("2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)");
     }
 
     private KeyValue<Integer, String> kv(Integer key, String value) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java
index 3b590c6..70b1a55 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java
@@ -58,6 +58,17 @@ public class KTableMapValuesTest {
         stateDir = TestUtils.tempDirectory("kafka-test");
     }
 
+    private void doTestKTable(final KStreamBuilder builder, final String topic1, final MockProcessorSupplier<String, Integer> proc2) {
+        driver = new KStreamTestDriver(builder, stateDir, Serdes.String(), Serdes.String());
+
+        driver.process(topic1, "A", "1");
+        driver.process(topic1, "B", "2");
+        driver.process(topic1, "C", "3");
+        driver.process(topic1, "D", "4");
+        driver.flushState();
+        assertEquals(Utils.mkList("A:1", "B:2", "C:3", "D:4"), proc2.processed);
+    }
+
     @Test
     public void testKTable() {
         final KStreamBuilder builder = new KStreamBuilder();
@@ -75,50 +86,41 @@ public class KTableMapValuesTest {
         MockProcessorSupplier<String, Integer> proc2 = new MockProcessorSupplier<>();
         table2.toStream().process(proc2);
 
-        driver = new KStreamTestDriver(builder, stateDir);
-
-        driver.process(topic1, "A", "1");
-        driver.process(topic1, "B", "2");
-        driver.process(topic1, "C", "3");
-        driver.process(topic1, "D", "4");
-        driver.flushState();
-        assertEquals(Utils.mkList("A:1", "B:2", "C:3", "D:4"), proc2.processed);
+        doTestKTable(builder, topic1, proc2);
     }
 
     @Test
-    public void testValueGetter() throws IOException {
-        KStreamBuilder builder = new KStreamBuilder();
+    public void testQueryableKTable() {
+        final KStreamBuilder builder = new KStreamBuilder();
 
         String topic1 = "topic1";
-        String topic2 = "topic2";
-        String storeName1 = "storeName1";
-        String storeName2 = "storeName2";
 
-        KTableImpl<String, String, String> table1 =
-                (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1, storeName1);
-        KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
-                new ValueMapper<String, Integer>() {
-                    @Override
-                    public Integer apply(String value) {
-                        return new Integer(value);
-                    }
-                });
-        KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table2.filter(
-                new Predicate<String, Integer>() {
-                    @Override
-                    public boolean test(String key, Integer value) {
-                        return (value % 2) == 0;
-                    }
-                });
-        KTableImpl<String, String, String> table4 = (KTableImpl<String, String, String>)
-                table1.through(stringSerde, stringSerde, topic2, storeName2);
+        KTable<String, String> table1 = builder.table(stringSerde, stringSerde, topic1, "anyStoreName");
+        KTable<String, Integer> table2 = table1.mapValues(new ValueMapper<CharSequence, Integer>() {
+            @Override
+            public Integer apply(CharSequence value) {
+                return value.charAt(0) - 48;
+            }
+        }, Serdes.Integer(), "anyName");
+
+        MockProcessorSupplier<String, Integer> proc2 = new MockProcessorSupplier<>();
+        table2.toStream().process(proc2);
 
+        doTestKTable(builder, topic1, proc2);
+    }
+
+    private void doTestValueGetter(final KStreamBuilder builder,
+                                   final String topic1,
+                                   final KTableImpl<String, String, String> table1,
+                                   final KTableImpl<String, String, Integer> table2,
+                                   final KTableImpl<String, Integer, Integer> table3,
+                                   final KTableImpl<String, String, String> table4) {
         KTableValueGetterSupplier<String, String> getterSupplier1 = table1.valueGetterSupplier();
         KTableValueGetterSupplier<String, Integer> getterSupplier2 = table2.valueGetterSupplier();
         KTableValueGetterSupplier<String, Integer> getterSupplier3 = table3.valueGetterSupplier();
         KTableValueGetterSupplier<String, String> getterSupplier4 = table4.valueGetterSupplier();
 
-        driver = new KStreamTestDriver(builder, stateDir, null, null);
+        driver = new KStreamTestDriver(builder, stateDir, Serdes.String(), Serdes.String());
         KTableValueGetter<String, String> getter1 = getterSupplier1.get();
         getter1.init(driver.context());
         KTableValueGetter<String, Integer> getter2 = getterSupplier2.get();
@@ -209,6 +211,68 @@ public class KTableMapValuesTest {
     }
 
     @Test
+    public void testValueGetter() throws IOException {
+        KStreamBuilder builder = new KStreamBuilder();
+
+        String topic1 = "topic1";
+        String topic2 = "topic2";
+        String storeName1 = "storeName1";
+        String storeName2 = "storeName2";
+
+        KTableImpl<String, String, String> table1 =
+                (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1, storeName1);
+        KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
+                new ValueMapper<String, Integer>() {
+                    @Override
+                    public Integer apply(String value) {
+                        return new Integer(value);
+                    }
+                });
+        KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table2.filter(
+                new Predicate<String, Integer>() {
+                    @Override
+                    public boolean test(String key, Integer value) {
+                        return (value % 2) == 0;
+                    }
+                });
+        KTableImpl<String, String, String> table4 = (KTableImpl<String, String, String>)
+                table1.through(stringSerde, stringSerde, topic2, storeName2);
+
+        doTestValueGetter(builder, topic1, table1, table2, table3, table4);
+    }
+
+    @Test
+    public void testQueryableValueGetter() throws IOException {
+        KStreamBuilder builder = new KStreamBuilder();
+
+        String topic1 = "topic1";
+        String topic2 = "topic2";
+        String storeName1 = "storeName1";
+        String storeName2 = "storeName2";
+
+        KTableImpl<String, String, String> table1 =
+            (KTableImpl<String, String, String>) builder.table(stringSerde, stringSerde, topic1, storeName1);
+        KTableImpl<String, String, Integer> table2 = (KTableImpl<String, String, Integer>) table1.mapValues(
+            new ValueMapper<String, Integer>() {
+                @Override
+                public Integer apply(String value) {
+                    return new Integer(value);
+                }
+            }, Serdes.Integer(), "anyMapName");
+        KTableImpl<String, Integer, Integer> table3 = (KTableImpl<String, Integer, Integer>) table2.filter(
+            new Predicate<String, Integer>() {
+                @Override
+                public boolean test(String key, Integer value) {
+                    return (value % 2) == 0;
+                }
+            }, "anyFilterName");
+        KTableImpl<String, String, String> table4 = (KTableImpl<String, String, String>)
+            table1.through(stringSerde, stringSerde, topic2, storeName2);
+
+        doTestValueGetter(builder, topic1, table1, table2, table3, table4);
+    }
+
+    @Test
     public void testNotSendingOldValue() throws IOException {
         KStreamBuilder builder = new KStreamBuilder();
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java
index be9f7fb..41ab803 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java
@@ -735,7 +735,7 @@ public class TopologyBuilderTest {
     public void shouldNotAllowToAddGlobalStoreWithSourceNameEqualsProcessorName() {
         final String sameNameForSourceAndProcessor = "sameName";
         final TopologyBuilder topologyBuilder = new TopologyBuilder()
-            .addGlobalStore(new MockStateStoreSupplier.MockStateStore("anyName", false),
+            .addGlobalStore(new MockStateStoreSupplier("anyName", false, false),
                 sameNameForSourceAndProcessor,
                 null,
                 null,

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java
index 65b3e2f..d19c91a 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java
@@ -210,11 +210,11 @@ public class ProcessorTopologyTest {
                 .withStringKeys().withStringValues().inMemory().disableLogging().build();
         final String global = "global";
         final String topic = "topic";
-        final KeyValueStore<String, String> globalStore = (KeyValueStore<String, String>) storeSupplier.get();
         final TopologyBuilder topologyBuilder = this.builder
-                .addGlobalStore(globalStore, global, STRING_DESERIALIZER, STRING_DESERIALIZER, topic, "processor", define(new StatefulProcessor("my-store")));
+                .addGlobalStore(storeSupplier, global, STRING_DESERIALIZER, STRING_DESERIALIZER, topic, "processor", define(new StatefulProcessor("my-store")));
 
         driver = new ProcessorTopologyTestDriver(config, topologyBuilder);
+        final KeyValueStore<String, String> globalStore = (KeyValueStore<String, String>) topologyBuilder.globalStateStores().get("my-store");
         driver.process(topic, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER);
         driver.process(topic, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER);
         assertEquals("value1", globalStore.get("key1"));


[3/5] kafka git commit: KAFKA-5045: Clarify on KTable APIs for queryable stores

Posted by gu...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java
----------------------------------------------------------------------
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java
index 290142b..e6219c2 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java
@@ -24,7 +24,10 @@ import org.apache.kafka.streams.KeyValue;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.kstream.internals.WindowedSerializer;
 import org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner;
+import org.apache.kafka.streams.processor.StateStoreSupplier;
 import org.apache.kafka.streams.processor.StreamPartitioner;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.QueryableStoreType;
 import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
 
 /**
@@ -47,7 +50,7 @@ import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
  *     final KafkaStreams streams = ...;
  *     streams.start()
  *     ...
- *     final String queryableStoreName = table.getStoreName(); // returns null if KTable is not queryable
+ *     final String queryableStoreName = table.queryableStoreName(); // returns null if KTable is not queryable
  *     ReadOnlyKeyValueStore view = streams.store(queryableStoreName, QueryableStoreTypes.keyValueStore());
  *     view.get(key);
  *}</pre>
@@ -87,6 +90,79 @@ public interface KTable<K, V> {
     KTable<K, V> filter(final Predicate<? super K, ? super V> predicate);
 
     /**
+     * Create a new {@code KTable} that consists of all records of this {@code KTable} which satisfy the given
+     * predicate.
+     * All records that do not satisfy the predicate are dropped.
+     * For each {@code KTable} update the filter is evaluated on the update record to produce an update record for the
+     * result {@code KTable}.
+     * This is a stateless record-by-record operation.
+     * <p>
+     * Note that {@code filter} for a <i>changelog stream</i> works different to {@link KStream#filter(Predicate)
+     * record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
+     * have delete semantics.
+     * Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded
+     * directly if required (i.e., if there is anything to be deleted).
+     * Furthermore, for each record that gets dropped (i.e., dot not satisfied the given predicate) a tombstone record
+     * is forwarded.
+     * <p>
+     * To query the local {@link KeyValueStore} it must be obtained via
+     * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
+     * <pre>{@code
+     * KafkaStreams streams = ... // filtering words
+     * ReadOnlyKeyValueStore<K,V> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, V>keyValueStore());
+     * K key = "some-word";
+     * V valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
+     * }</pre>
+     * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to
+     * query the value of the key on a parallel running instance of your Kafka Streams application.
+     * <p>
+     *
+     * @param predicate a filter {@link Predicate} that is applied to each record
+     * @param queryableStoreName a user-provided name of the underlying {@link KTable} that can be
+     *                          used to subsequently query the operation results; valid characters are ASCII
+     *                          alphanumerics, '.', '_' and '-'. If {@code null} then the results cannot be queried
+     *                          (i.e., that would be equivalent to calling {@link KTable#filter(Predicate)}.
+     * @return a {@code KTable} that contains only those records that satisfy the given predicate
+     * @see #filterNot(Predicate)
+     */
+    KTable<K, V> filter(final Predicate<? super K, ? super V> predicate, final String queryableStoreName);
+
+    /**
+     * Create a new {@code KTable} that consists of all records of this {@code KTable} which satisfy the given
+     * predicate.
+     * All records that do not satisfy the predicate are dropped.
+     * For each {@code KTable} update the filter is evaluated on the update record to produce an update record for the
+     * result {@code KTable}.
+     * This is a stateless record-by-record operation.
+     * <p>
+     * Note that {@code filter} for a <i>changelog stream</i> works different to {@link KStream#filter(Predicate)
+     * record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
+     * have delete semantics.
+     * Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded
+     * directly if required (i.e., if there is anything to be deleted).
+     * Furthermore, for each record that gets dropped (i.e., dot not satisfied the given predicate) a tombstone record
+     * is forwarded.
+     * <p>
+     * To query the local {@link KeyValueStore} it must be obtained via
+     * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
+     * <pre>{@code
+     * KafkaStreams streams = ... // filtering words
+     * ReadOnlyKeyValueStore<K,V> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, V>keyValueStore());
+     * K key = "some-word";
+     * V valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
+     * }</pre>
+     * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to
+     * query the value of the key on a parallel running instance of your Kafka Streams application.
+     * <p>
+     *
+     * @param predicate a filter {@link Predicate} that is applied to each record
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
+     * @return a {@code KTable} that contains only those records that satisfy the given predicate
+     * @see #filterNot(Predicate)
+     */
+    KTable<K, V> filter(final Predicate<? super K, ? super V> predicate, final StateStoreSupplier<KeyValueStore> storeSupplier);
+
+    /**
      * Create a new {@code KTable} that consists all records of this {@code KTable} which do <em>not</em> satisfy the
      * given predicate.
      * All records that <em>do</em> satisfy the predicate are dropped.
@@ -109,6 +185,78 @@ public interface KTable<K, V> {
     KTable<K, V> filterNot(final Predicate<? super K, ? super V> predicate);
 
     /**
+     * Create a new {@code KTable} that consists all records of this {@code KTable} which do <em>not</em> satisfy the
+     * given predicate.
+     * All records that <em>do</em> satisfy the predicate are dropped.
+     * For each {@code KTable} update the filter is evaluated on the update record to produce an update record for the
+     * result {@code KTable}.
+     * This is a stateless record-by-record operation.
+     * <p>
+     * Note that {@code filterNot} for a <i>changelog stream</i> works different to {@link KStream#filterNot(Predicate)
+     * record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
+     * have delete semantics.
+     * Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded
+     * directly if required (i.e., if there is anything to be deleted).
+     * Furthermore, for each record that gets dropped (i.e., does satisfy the given predicate) a tombstone record is
+     * forwarded.
+     * <p>
+     * To query the local {@link KeyValueStore} it must be obtained via
+     * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
+     * <pre>{@code
+     * KafkaStreams streams = ... // filtering words
+     * ReadOnlyKeyValueStore<K,V> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, V>keyValueStore());
+     * K key = "some-word";
+     * V valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
+     * }</pre>
+     * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to
+     * query the value of the key on a parallel running instance of your Kafka Streams application.
+     * <p>
+     * @param predicate a filter {@link Predicate} that is applied to each record
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
+     * @return a {@code KTable} that contains only those records that do <em>not</em> satisfy the given predicate
+     * @see #filter(Predicate)
+     */
+    KTable<K, V> filterNot(final Predicate<? super K, ? super V> predicate, final StateStoreSupplier<KeyValueStore> storeSupplier);
+
+    /**
+     * Create a new {@code KTable} that consists all records of this {@code KTable} which do <em>not</em> satisfy the
+     * given predicate.
+     * All records that <em>do</em> satisfy the predicate are dropped.
+     * For each {@code KTable} update the filter is evaluated on the update record to produce an update record for the
+     * result {@code KTable}.
+     * This is a stateless record-by-record operation.
+     * <p>
+     * Note that {@code filterNot} for a <i>changelog stream</i> works different to {@link KStream#filterNot(Predicate)
+     * record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
+     * have delete semantics.
+     * Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded
+     * directly if required (i.e., if there is anything to be deleted).
+     * Furthermore, for each record that gets dropped (i.e., does satisfy the given predicate) a tombstone record is
+     * forwarded.
+     * <p>
+     * To query the local {@link KeyValueStore} it must be obtained via
+     * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
+     * <pre>{@code
+     * KafkaStreams streams = ... // filtering words
+     * ReadOnlyKeyValueStore<K,V> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, V>keyValueStore());
+     * K key = "some-word";
+     * V valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
+     * }</pre>
+     * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to
+     * query the value of the key on a parallel running instance of your Kafka Streams application.
+     * <p>
+     * @param predicate a filter {@link Predicate} that is applied to each record
+     * @param queryableStoreName a user-provided name of the underlying {@link KTable} that can be
+     * used to subsequently query the operation results; valid characters are ASCII
+     * alphanumerics, '.', '_' and '-'. If {@code null} then the results cannot be queried
+     * (i.e., that would be equivalent to calling {@link KTable#filterNot(Predicate)}.
+     * @return a {@code KTable} that contains only those records that do <em>not</em> satisfy the given predicate
+     * @see #filter(Predicate)
+     */
+    KTable<K, V> filterNot(final Predicate<? super K, ? super V> predicate, final String queryableStoreName);
+
+
+    /**
      * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
      * (with possible new type)in the new {@code KTable}.
      * For each {@code KTable} update the provided {@link ValueMapper} is applied to the value of the update record and
@@ -144,6 +292,97 @@ public interface KTable<K, V> {
 
 
     /**
+     * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
+     * (with possible new type)in the new {@code KTable}.
+     * For each {@code KTable} update the provided {@link ValueMapper} is applied to the value of the update record and
+     * computes a new value for it, resulting in an update record for the result {@code KTable}.
+     * Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K:V'>}.
+     * This is a stateless record-by-record operation.
+     * <p>
+     * The example below counts the number of token of the value string.
+     * <pre>{@code
+     * KTable<String, String> inputTable = builder.table("topic");
+     * KTable<String, Integer> outputTable = inputTable.mapValue(new ValueMapper<String, Integer> {
+     *     Integer apply(String value) {
+     *         return value.split(" ").length;
+     *     }
+     * });
+     * }</pre>
+     * <p>
+     * To query the local {@link KeyValueStore} representing outputTable above it must be obtained via
+     * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
+     * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to
+     * query the value of the key on a parallel running instance of your Kafka Streams application.
+     * <p>
+     * <p>
+     * This operation preserves data co-location with respect to the key.
+     * Thus, <em>no</em> internal data redistribution is required if a key based operator (like a join) is applied to
+     * the result {@code KTable}.
+     * <p>
+     * Note that {@code mapValues} for a <i>changelog stream</i> works different to {@link KStream#mapValues(ValueMapper)
+     * record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
+     * have delete semantics.
+     * Thus, for tombstones the provided value-mapper is not evaluated but the tombstone record is forwarded directly to
+     * delete the corresponding record in the result {@code KTable}.
+     *
+     * @param mapper a {@link ValueMapper} that computes a new output value
+     * @param queryableStoreName a user-provided name of the underlying {@link KTable} that can be
+     * used to subsequently query the operation results; valid characters are ASCII
+     * alphanumerics, '.', '_' and '-'. If {@code null} then the results cannot be queried
+     * (i.e., that would be equivalent to calling {@link KTable#mapValues(ValueMapper)}.
+     * @param valueSerde serializer for new value type
+     * @param <VR>   the value type of the result {@code KTable}
+     *
+     * @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
+     */
+    <VR> KTable<K, VR> mapValues(final ValueMapper<? super V, ? extends VR> mapper, final Serde<VR> valueSerde, final String queryableStoreName);
+
+    /**
+     * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value
+     * (with possible new type)in the new {@code KTable}.
+     * For each {@code KTable} update the provided {@link ValueMapper} is applied to the value of the update record and
+     * computes a new value for it, resulting in an update record for the result {@code KTable}.
+     * Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K:V'>}.
+     * This is a stateless record-by-record operation.
+     * <p>
+     * The example below counts the number of token of the value string.
+     * <pre>{@code
+     * KTable<String, String> inputTable = builder.table("topic");
+     * KTable<String, Integer> outputTable = inputTable.mapValue(new ValueMapper<String, Integer> {
+     *     Integer apply(String value) {
+     *         return value.split(" ").length;
+     *     }
+     * });
+     * }</pre>
+     * <p>
+     * To query the local {@link KeyValueStore} representing outputTable above it must be obtained via
+     * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
+     * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to
+     * query the value of the key on a parallel running instance of your Kafka Streams application.
+     * <p>
+     * <p>
+     * This operation preserves data co-location with respect to the key.
+     * Thus, <em>no</em> internal data redistribution is required if a key based operator (like a join) is applied to
+     * the result {@code KTable}.
+     * <p>
+     * Note that {@code mapValues} for a <i>changelog stream</i> works different to {@link KStream#mapValues(ValueMapper)
+     * record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records)
+     * have delete semantics.
+     * Thus, for tombstones the provided value-mapper is not evaluated but the tombstone record is forwarded directly to
+     * delete the corresponding record in the result {@code KTable}.
+     *
+     * @param mapper a {@link ValueMapper} that computes a new output value
+     * @param valueSerde serializer for new value type
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
+     * @param <VR>   the value type of the result {@code KTable}
+     * @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type)
+     */
+    <VR> KTable<K, VR> mapValues(final ValueMapper<? super V, ? extends VR> mapper,
+                                 final Serde<VR> valueSerde,
+                                 final StateStoreSupplier<KeyValueStore> storeSupplier);
+
+
+    /**
      * Print the update records of this {@code KTable} to {@code System.out}.
      * This function will use the generated name of the parent processor node to label the key/value pairs printed to
      * the console.
@@ -156,7 +395,11 @@ public interface KTable<K, V> {
      * <p>
      * Note that {@code print()} is not applied to the internal state store and only called for each new {@code KTable}
      * update record.
+     * @deprecated Use the Interactive Queries APIs (e.g., {@link KafkaStreams#store(String, QueryableStoreType) }
+     * followed by {@link ReadOnlyKeyValueStore#all()}) to iterate over the keys of a KTable. Alternatively
+     * convert to a KStream using {@code toStream()} and then use {@link KStream#print()} on the result.
      */
+    @Deprecated
     void print();
 
     /**
@@ -173,7 +416,11 @@ public interface KTable<K, V> {
      * update record.
      *
      * @param streamName the name used to label the key/value pairs printed to the console
+     * @deprecated Use the Interactive Queries APIs (e.g., {@link KafkaStreams#store(String, QueryableStoreType) }
+     * followed by {@link ReadOnlyKeyValueStore#all()}) to iterate over the keys of a KTable. Alternatively
+     * convert to a KStream using {@code toStream()} and then use {@link KStream#print(String)} on the result.
      */
+    @Deprecated
     void print(final String streamName);
 
     /**
@@ -191,8 +438,12 @@ public interface KTable<K, V> {
      * update record.
      *
      * @param keySerde key serde used to deserialize key if type is {@code byte[]},
-     * @param valSerde value serde used to deserialize value if type is {@code byte[]},
+     * @param valSerde value serde used to deserialize value if type is {@code byte[]}
+     * @deprecated Use the Interactive Queries APIs (e.g., {@link KafkaStreams#store(String, QueryableStoreType) }
+     * followed by {@link ReadOnlyKeyValueStore#all()}) to iterate over the keys of a KTable. Alternatively
+     * convert to a KStream using {@code toStream()} and then use {@link KStream#print(Serde, Serde)} on the result.
      */
+    @Deprecated
     void print(final Serde<K> keySerde,
                final Serde<V> valSerde);
 
@@ -212,7 +463,11 @@ public interface KTable<K, V> {
      * @param keySerde   key serde used to deserialize key if type is {@code byte[]},
      * @param valSerde   value serde used to deserialize value if type is {@code byte[]},
      * @param streamName the name used to label the key/value pairs printed to the console
+     * @deprecated Use the Interactive Queries APIs (e.g., {@link KafkaStreams#store(String, QueryableStoreType) }
+     * followed by {@link ReadOnlyKeyValueStore#all()}) to iterate over the keys of a KTable. Alternatively
+     * convert to a KStream using {@code toStream()} and then use {@link KStream#print(Serde, Serde, String)} on the result.
      */
+    @Deprecated
     void print(final Serde<K> keySerde,
                final Serde<V> valSerde,
                final String streamName);
@@ -232,7 +487,11 @@ public interface KTable<K, V> {
      * {@code KTable} update record.
      *
      * @param filePath name of file to write to
+     * @deprecated Use the Interactive Queries APIs (e.g., {@link KafkaStreams#store(String, QueryableStoreType) }
+     * followed by {@link ReadOnlyKeyValueStore#all()}) to iterate over the keys of a KTable. Alternatively
+     * convert to a KStream using {@code toStream()} and then use {@link KStream#writeAsText(String)}} on the result.
      */
+    @Deprecated
     void writeAsText(final String filePath);
 
     /**
@@ -250,7 +509,11 @@ public interface KTable<K, V> {
      *
      * @param filePath   name of file to write to
      * @param streamName the name used to label the key/value pairs printed out to the console
+     * @deprecated Use the Interactive Queries APIs (e.g., {@link KafkaStreams#store(String, QueryableStoreType) }
+     * followed by {@link ReadOnlyKeyValueStore#all()}) to iterate over the keys of a KTable. Alternatively
+     * convert to a KStream using {@code toStream()} and then use {@link KStream#writeAsText(String, String)}} on the result.
      */
+    @Deprecated
     void writeAsText(final String filePath,
                      final String streamName);
 
@@ -270,8 +533,12 @@ public interface KTable<K, V> {
      *
      * @param filePath name of file to write to
      * @param keySerde key serde used to deserialize key if type is {@code byte[]},
-     * @param valSerde value serde used to deserialize value if type is {@code byte[]},
+     * @param valSerde value serde used to deserialize value if type is {@code byte[]}
+     * @deprecated Use the Interactive Queries APIs (e.g., {@link KafkaStreams#store(String, QueryableStoreType) }
+     * followed by {@link ReadOnlyKeyValueStore#all()}) to iterate over the keys of a KTable. Alternatively
+     * convert to a KStream using {@code toStream()} and then use {@link KStream#writeAsText(String, Serde, Serde)}} on the result.
      */
+    @Deprecated
     void  writeAsText(final String filePath,
                       final Serde<K> keySerde,
                       final Serde<V> valSerde);
@@ -292,8 +559,13 @@ public interface KTable<K, V> {
      * @param filePath name of file to write to
      * @param streamName the name used to label the key/value pairs printed to the console
      * @param keySerde key serde used to deserialize key if type is {@code byte[]},
-     * @param valSerde value serde used to deserialize value if type is {@code byte[]},
+     * @param valSerde value serde used to deserialize value if type is {@code byte[]}
+     * @deprecated Use the Interactive Queries APIs (e.g., {@link KafkaStreams#store(String, QueryableStoreType) }
+     * followed by {@link ReadOnlyKeyValueStore#all()}) to iterate over the keys of a KTable. Alternatively
+     * convert to a KStream using {@code toStream()} and then use {@link KStream#writeAsText(String, String, Serde, Serde)}} on the result.
+
      */
+    @Deprecated
     void writeAsText(final String filePath,
                      final String streamName,
                      final Serde<K> keySerde,
@@ -307,7 +579,11 @@ public interface KTable<K, V> {
      * {@code KTable} update record.
      *
      * @param action an action to perform on each record
+     * @deprecated Use the Interactive Queries APIs (e.g., {@link KafkaStreams#store(String, QueryableStoreType) }
+     * followed by {@link ReadOnlyKeyValueStore#all()}) to iterate over the keys of a KTable. Alternatively
+     * convert to a KStream using {@code toStream()} and then use {@link KStream#foreach(ForeachAction)}} on the result.
      */
+    @Deprecated
     void foreach(final ForeachAction<? super K, ? super V> action);
 
     /**
@@ -361,12 +637,94 @@ public interface KTable<K, V> {
      * The store name must be a valid Kafka topic name and cannot contain characters other than ASCII alphanumerics, '.', '_' and '-'.
      *
      * @param topic     the topic name
-     * @param storeName the state store name used for the result {@code KTable}; valid characters are ASCII
-     *                  alphanumerics, '.', '_' and '-'
+     * @param queryableStoreName the state store name used for the result {@code KTable}; valid characters are ASCII
+     *                  alphanumerics, '.', '_' and '-'. If {@code null} this is the equivalent of {@link KTable#through(String)()}
+     * @return a {@code KTable} that contains the exact same (and potentially repartitioned) records as this {@code KTable}
+     */
+    KTable<K, V> through(final String topic,
+                         final String queryableStoreName);
+
+    /**
+     * Materialize this changelog stream to a topic and creates a new {@code KTable} from the topic using default
+     * serializers and deserializers and producer's {@link DefaultPartitioner}.
+     * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is
+     * started).
+     * <p>
+     * This is equivalent to calling {@link #to(String) #to(someTopicName)} and
+     * {@link KStreamBuilder#table(String, String) KStreamBuilder#table(someTopicName, queryableStoreName)}.
+     * <p>
+     * The resulting {@code KTable} will be materialized in a local state store with the given store name (cf.
+     * {@link KStreamBuilder#table(String, String)})
+     * The store name must be a valid Kafka topic name and cannot contain characters other than ASCII alphanumerics, '.', '_' and '-'.
+     *
+     * @param topic     the topic name
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
      * @return a {@code KTable} that contains the exact same (and potentially repartitioned) records as this {@code KTable}
      */
     KTable<K, V> through(final String topic,
-                         final String storeName);
+                         final StateStoreSupplier<KeyValueStore> storeSupplier);
+
+    /**
+     * Materialize this changelog stream to a topic and creates a new {@code KTable} from the topic using default
+     * serializers and deserializers and producer's {@link DefaultPartitioner}.
+     * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is
+     * started).
+     * <p>
+     * This is equivalent to calling {@link #to(String) #to(someTopicName)} and
+     * {@link KStreamBuilder#table(String, String) KStreamBuilder#table(someTopicName)}.
+     * <p>
+     * The resulting {@code KTable} will be materialized in a local state store with an internal store name (cf.
+     * {@link KStreamBuilder#table(String)})
+     *
+     * @param topic     the topic name
+     * @return a {@code KTable} that contains the exact same (and potentially repartitioned) records as this {@code KTable}
+     */
+    KTable<K, V> through(final String topic);
+
+    /**
+     * Materialize this changelog stream to a topic and creates a new {@code KTable} from the topic using default
+     * serializers and deserializers and a customizable {@link StreamPartitioner} to determine the distribution of
+     * records to partitions.
+     * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is
+     * started).
+     * <p>
+     * This is equivalent to calling {@link #to(StreamPartitioner, String) #to(partitioner, someTopicName)} and
+     * {@link KStreamBuilder#table(String, String) KStreamBuilder#table(someTopicName)}.
+     * <p>
+     * The resulting {@code KTable} will be materialized in a local state store with an internal store name (cf.
+     * {@link KStreamBuilder#table(String)})
+     *
+     * @param partitioner the function used to determine how records are distributed among partitions of the topic,
+     *                    if not specified producer's {@link DefaultPartitioner} will be used
+     * @param topic       the topic name
+     * @return a {@code KTable} that contains the exact same (and potentially repartitioned) records as this {@code KTable}
+     */
+    KTable<K, V> through(final StreamPartitioner<? super K, ? super V> partitioner,
+                         final String topic);
+
+    /**
+     * Materialize this changelog stream to a topic and creates a new {@code KTable} from the topic using default
+     * serializers and deserializers and a customizable {@link StreamPartitioner} to determine the distribution of
+     * records to partitions.
+     * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is
+     * started).
+     * <p>
+     * This is equivalent to calling {@link #to(StreamPartitioner, String) #to(partitioner, someTopicName)} and
+     * {@link KStreamBuilder#table(String, String) KStreamBuilder#table(someTopicName, queryableStoreName)}.
+     * <p>
+     * The resulting {@code KTable} will be materialized in a local state store with the given store name (cf.
+     * {@link KStreamBuilder#table(String, String)})
+     *
+     * @param partitioner the function used to determine how records are distributed among partitions of the topic,
+     *                    if not specified producer's {@link DefaultPartitioner} will be used
+     * @param topic       the topic name
+     * @param queryableStoreName   the state store name used for the result {@code KTable}.
+     *                             If {@code null} this is the equivalent of {@link KTable#through(StreamPartitioner, String)}
+     * @return a {@code KTable} that contains the exact same (and potentially repartitioned) records as this {@code KTable}
+     */
+    KTable<K, V> through(final StreamPartitioner<? super K, ? super V> partitioner,
+                         final String topic,
+                         final String queryableStoreName);
 
     /**
      * Materialize this changelog stream to a topic and creates a new {@code KTable} from the topic using default
@@ -384,12 +742,12 @@ public interface KTable<K, V> {
      * @param partitioner the function used to determine how records are distributed among partitions of the topic,
      *                    if not specified producer's {@link DefaultPartitioner} will be used
      * @param topic       the topic name
-     * @param storeName   the state store name used for the result {@code KTable}
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
      * @return a {@code KTable} that contains the exact same (and potentially repartitioned) records as this {@code KTable}
      */
     KTable<K, V> through(final StreamPartitioner<? super K, ? super V> partitioner,
                          final String topic,
-                         final String storeName);
+                         final StateStoreSupplier<KeyValueStore> storeSupplier);
 
     /**
      * Materialize this changelog stream to a topic and creates a new {@code KTable} from the topic.
@@ -410,43 +768,156 @@ public interface KTable<K, V> {
      * @param valSerde  value serde used to send key-value pairs,
      *                  if not specified the default value serde defined in the configuration will be used
      * @param topic     the topic name
-     * @param storeName the state store name used for the result {@code KTable}
+     * @param queryableStoreName the state store name used for the result {@code KTable}.
+     *                           If {@code null} this is the equivalent of {@link KTable#through(Serde, Serde, String)()}
      * @return a {@code KTable} that contains the exact same (and potentially repartitioned) records as this {@code KTable}
      */
     KTable<K, V> through(final Serde<K> keySerde, Serde<V> valSerde,
                          final String topic,
-                         final String storeName);
+                         final String queryableStoreName);
 
     /**
-     * Materialize this changelog stream to a topic and creates a new {@code KTable} from the topic using a customizable
-     * {@link StreamPartitioner} to determine the distribution of records to partitions.
+     * Materialize this changelog stream to a topic and creates a new {@code KTable} from the topic.
      * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is
      * started).
      * <p>
-     * This is equivalent to calling {@link #to(Serde, Serde, StreamPartitioner, String)
-     * #to(keySerde, valueSerde, partitioner, someTopicName)} and
+     * If {@code keySerde} provides a {@link WindowedSerializer} for the key {@link WindowedStreamPartitioner} is
+     * used&mdash;otherwise producer's {@link DefaultPartitioner} is used.
+     * <p>
+     * This is equivalent to calling {@link #to(Serde, Serde, String) #to(keySerde, valueSerde, someTopicName)} and
      * {@link KStreamBuilder#table(String, String) KStreamBuilder#table(someTopicName, queryableStoreName)}.
      * <p>
      * The resulting {@code KTable} will be materialized in a local state store with the given store name (cf.
      * {@link KStreamBuilder#table(String, String)})
      *
-     * @param keySerde    key serde used to send key-value pairs,
-     *                    if not specified the default key serde defined in the configuration will be used
-     * @param valSerde    value serde used to send key-value pairs,
-     *                    if not specified the default value serde defined in the configuration will be used
-     * @param partitioner the function used to determine how records are distributed among partitions of the topic,
-     *                    if not specified and {@code keySerde} provides a {@link WindowedSerializer} for the key
+     * @param keySerde  key serde used to send key-value pairs,
+     *                  if not specified the default key serde defined in the configuration will be used
+     * @param valSerde  value serde used to send key-value pairs,
+     *                  if not specified the default value serde defined in the configuration will be used
+     * @param topic     the topic name
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
+     * @return a {@code KTable} that contains the exact same (and potentially repartitioned) records as this {@code KTable}
+     */
+    KTable<K, V> through(final Serde<K> keySerde, Serde<V> valSerde,
+                         final String topic,
+                         final StateStoreSupplier<KeyValueStore> storeSupplier);
+
+    /**
+     * Materialize this changelog stream to a topic and creates a new {@code KTable} from the topic.
+     * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is
+     * started).
+     * <p>
+     * If {@code keySerde} provides a {@link WindowedSerializer} for the key {@link WindowedStreamPartitioner} is
+     * used&mdash;otherwise producer's {@link DefaultPartitioner} is used.
+     * <p>
+     * This is equivalent to calling {@link #to(Serde, Serde, String) #to(keySerde, valueSerde, someTopicName)} and
+     * {@link KStreamBuilder#table(String) KStreamBuilder#table(someTopicName)}.
+     * <p>
+     * The resulting {@code KTable} will be materialized in a local state store with an interna; store name (cf.
+     * {@link KStreamBuilder#table(String)})
+     *
+     * @param keySerde  key serde used to send key-value pairs,
+     *                  if not specified the default key serde defined in the configuration will be used
+     * @param valSerde  value serde used to send key-value pairs,
+     *                  if not specified the default value serde defined in the configuration will be used
+     * @param topic     the topic name
+     * @return a {@code KTable} that contains the exact same (and potentially repartitioned) records as this {@code KTable}
+     */
+    KTable<K, V> through(final Serde<K> keySerde, Serde<V> valSerde,
+                         final String topic);
+
+    /**
+     * Materialize this changelog stream to a topic and creates a new {@code KTable} from the topic using a customizable
+     * {@link StreamPartitioner} to determine the distribution of records to partitions.
+     * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is
+     * started).
+     * <p>
+     * This is equivalent to calling {@link #to(Serde, Serde, StreamPartitioner, String)
+     * #to(keySerde, valueSerde, partitioner, someTopicName)} and
+     * {@link KStreamBuilder#table(String, String) KStreamBuilder#table(someTopicName, queryableStoreName)}.
+     * <p>
+     * The resulting {@code KTable} will be materialized in a local state store with the given store name (cf.
+     * {@link KStreamBuilder#table(String, String)})
+     *
+     * @param keySerde    key serde used to send key-value pairs,
+     *                    if not specified the default key serde defined in the configuration will be used
+     * @param valSerde    value serde used to send key-value pairs,
+     *                    if not specified the default value serde defined in the configuration will be used
+     * @param partitioner the function used to determine how records are distributed among partitions of the topic,
+     *                    if not specified and {@code keySerde} provides a {@link WindowedSerializer} for the key
+     *                    {@link WindowedStreamPartitioner} will be used&mdash;otherwise {@link DefaultPartitioner} will
+     *                    be used
+     * @param topic      the topic name
+     * @param queryableStoreName  the state store name used for the result {@code KTable}.
+     *                            If {@code null} this is the equivalent of {@link KTable#through(Serde, Serde, StreamPartitioner, String)()}
+     * @return a {@code KTable} that contains the exact same (and potentially repartitioned) records as this {@code KTable}
+     */
+    KTable<K, V> through(final Serde<K> keySerde,
+                         final Serde<V> valSerde,
+                         final StreamPartitioner<? super K, ? super V> partitioner,
+                         final String topic,
+                         final String queryableStoreName);
+
+    /**
+     * Materialize this changelog stream to a topic and creates a new {@code KTable} from the topic using a customizable
+     * {@link StreamPartitioner} to determine the distribution of records to partitions.
+     * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is
+     * started).
+     * <p>
+     * This is equivalent to calling {@link #to(Serde, Serde, StreamPartitioner, String)
+     * #to(keySerde, valueSerde, partitioner, someTopicName)} and
+     * {@link KStreamBuilder#table(String, String) KStreamBuilder#table(someTopicName, queryableStoreName)}.
+     * <p>
+     * The resulting {@code KTable} will be materialized in a local state store with the given store name (cf.
+     * {@link KStreamBuilder#table(String, String)})
+     *
+     * @param keySerde    key serde used to send key-value pairs,
+     *                    if not specified the default key serde defined in the configuration will be used
+     * @param valSerde    value serde used to send key-value pairs,
+     *                    if not specified the default value serde defined in the configuration will be used
+     * @param partitioner the function used to determine how records are distributed among partitions of the topic,
+     *                    if not specified and {@code keySerde} provides a {@link WindowedSerializer} for the key
      *                    {@link WindowedStreamPartitioner} will be used&mdash;otherwise {@link DefaultPartitioner} will
      *                    be used
      * @param topic      the topic name
-     * @param storeName  the state store name used for the result {@code KTable}
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
      * @return a {@code KTable} that contains the exact same (and potentially repartitioned) records as this {@code KTable}
      */
     KTable<K, V> through(final Serde<K> keySerde,
                          final Serde<V> valSerde,
                          final StreamPartitioner<? super K, ? super V> partitioner,
                          final String topic,
-                         final String storeName);
+                         final StateStoreSupplier<KeyValueStore> storeSupplier);
+
+    /**
+     * Materialize this changelog stream to a topic and creates a new {@code KTable} from the topic using a customizable
+     * {@link StreamPartitioner} to determine the distribution of records to partitions.
+     * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is
+     * started).
+     * <p>
+     * This is equivalent to calling {@link #to(Serde, Serde, StreamPartitioner, String)
+     * #to(keySerde, valueSerde, partitioner, someTopicName)} and
+     * {@link KStreamBuilder#table(String) KStreamBuilder#table(someTopicName)}.
+     * <p>
+     * The resulting {@code KTable} will be materialized in a local state store with an internal store name (cf.
+     * {@link KStreamBuilder#table(String)})
+     *
+     * @param keySerde    key serde used to send key-value pairs,
+     *                    if not specified the default key serde defined in the configuration will be used
+     * @param valSerde    value serde used to send key-value pairs,
+     *                    if not specified the default value serde defined in the configuration will be used
+     * @param partitioner the function used to determine how records are distributed among partitions of the topic,
+     *                    if not specified and {@code keySerde} provides a {@link WindowedSerializer} for the key
+     *                    {@link WindowedStreamPartitioner} will be used&mdash;otherwise {@link DefaultPartitioner} will
+     *                    be used
+     * @param topic      the topic name
+     * @return a {@code KTable} that contains the exact same (and potentially repartitioned) records as this {@code KTable}
+     */
+    KTable<K, V> through(final Serde<K> keySerde,
+                         final Serde<V> valSerde,
+                         final StreamPartitioner<? super K, ? super V> partitioner,
+                         final String topic);
+
 
     /**
      * Materialize this changelog stream to a topic using default serializers and deserializers and producer's
@@ -647,11 +1118,8 @@ public interface KTable<K, V> {
                                 final ValueJoiner<? super V, ? super VO, ? extends VR> joiner);
 
     /**
-     * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using
-     * non-windowed left equi join.
+     * Join records of this {@code KTable} with another {@code KTable}'s records using non-windowed inner equi join.
      * The join is a primary key join with join attribute {@code thisKTable.key == otherKTable.key}.
-     * In contrast to {@link #join(KTable, ValueJoiner) inner-join}, all records from left {@code KTable} will produce
-     * an output record (cf. below).
      * The result is an ever updating {@code KTable} that represents the <em>current</em> (i.e., processing time) result
      * of the join.
      * <p>
@@ -660,17 +1128,13 @@ public interface KTable<K, V> {
      * This happens in a symmetric way, i.e., for each update of either {@code this} or the {@code other} input
      * {@code KTable} the result gets updated.
      * <p>
-     * For each {@code KTable} record that finds a corresponding record in the other {@code KTable}'s state the
-     * provided {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record.
-     * Additionally, for each record of left {@code KTable} that does not find a corresponding record in the
-     * right {@code KTable}'s state the provided {@link ValueJoiner} will be called with {@code rightValue =
-     * null} to compute a value (with arbitrary type) for the result record.
+     * For each {@code KTable} record that finds a corresponding record in the other {@code KTable} the provided
+     * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record.
      * The key of the result record is the same as for both joining input records.
      * <p>
      * Note that {@link KeyValue records} with {@code null} values (so-called tombstone records) have delete semantics.
-     * For example, for left input tombstones the provided value-joiner is not called but a tombstone record is
-     * forwarded directly to delete a record in the result {@code KTable} if required (i.e., if there is anything to be
-     * deleted).
+     * Thus, for input tombstones the provided value-joiner is not called but a tombstone record is forwarded
+     * directly to delete a record in the result {@code KTable} if required (i.e., if there is anything to be deleted).
      * <p>
      * Input records with {@code null} key will be dropped and no join computation is performed.
      * <p>
@@ -688,7 +1152,7 @@ public interface KTable<K, V> {
      * <td>&lt;K1:A&gt;</td>
      * <td></td>
      * <td></td>
-     * <td>&lt;K1:ValueJoiner(A,null)&gt;</td>
+     * <td></td>
      * </tr>
      * <tr>
      * <td></td>
@@ -698,18 +1162,99 @@ public interface KTable<K, V> {
      * <td>&lt;K1:ValueJoiner(A,b)&gt;</td>
      * </tr>
      * <tr>
+     * <td>&lt;K1:C&gt;</td>
+     * <td>&lt;K1:C&gt;</td>
+     * <td></td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:ValueJoiner(C,b)&gt;</td>
+     * </tr>
+     * <tr>
+     * <td></td>
+     * <td>&lt;K1:C&gt;</td>
      * <td>&lt;K1:null&gt;</td>
      * <td></td>
+     * <td>&lt;K1:null&gt;</td>
+     * </tr>
+     * </table>
+     * Both input streams (or to be more precise, their underlying source topics) need to have the same number of
+     * partitions.
+     *
+     * @param other  the other {@code KTable} to be joined with this {@code KTable}
+     * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records
+     * @param <VO>   the value type of the other {@code KTable}
+     * @param <VR>   the value type of the result {@code KTable}
+     * @param joinSerde serializer for join result value type
+     * @param queryableStoreName a user-provided name of the underlying {@link KTable} that can be
+     * used to subsequently query the operation results; valid characters are ASCII
+     * alphanumerics, '.', '_' and '-'. If {@code null} then the results cannot be queried
+     * (i.e., that would be equivalent to calling {@link KTable#join(KTable, ValueJoiner)}.
+     * @return a {@code KTable} that contains join-records for each key and values computed by the given
+     * {@link ValueJoiner}, one for each matched record-pair with the same key
+     * @see #leftJoin(KTable, ValueJoiner)
+     * @see #outerJoin(KTable, ValueJoiner)
+     */
+    <VO, VR> KTable<K, VR> join(final KTable<K, VO> other,
+                                final ValueJoiner<? super V, ? super VO, ? extends VR> joiner,
+                                final Serde<VR> joinSerde,
+                                final String queryableStoreName);
+
+    /**
+     * Join records of this {@code KTable} with another {@code KTable}'s records using non-windowed inner equi join.
+     * The join is a primary key join with join attribute {@code thisKTable.key == otherKTable.key}.
+     * The result is an ever updating {@code KTable} that represents the <em>current</em> (i.e., processing time) result
+     * of the join.
+     * <p>
+     * The join is computed by (1) updating the internal state of one {@code KTable} and (2) performing a lookup for a
+     * matching record in the <em>current</em> (i.e., processing time) internal state of the other {@code KTable}.
+     * This happens in a symmetric way, i.e., for each update of either {@code this} or the {@code other} input
+     * {@code KTable} the result gets updated.
+     * <p>
+     * For each {@code KTable} record that finds a corresponding record in the other {@code KTable} the provided
+     * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record.
+     * The key of the result record is the same as for both joining input records.
+     * <p>
+     * Note that {@link KeyValue records} with {@code null} values (so-called tombstone records) have delete semantics.
+     * Thus, for input tombstones the provided value-joiner is not called but a tombstone record is forwarded
+     * directly to delete a record in the result {@code KTable} if required (i.e., if there is anything to be deleted).
+     * <p>
+     * Input records with {@code null} key will be dropped and no join computation is performed.
+     * <p>
+     * Example:
+     * <table border='1'>
+     * <tr>
+     * <th>thisKTable</th>
+     * <th>thisState</th>
+     * <th>otherKTable</th>
+     * <th>otherState</th>
+     * <th>result update record</th>
+     * </tr>
+     * <tr>
+     * <td>&lt;K1:A&gt;</td>
+     * <td>&lt;K1:A&gt;</td>
+     * <td></td>
+     * <td></td>
      * <td></td>
+     * </tr>
+     * <tr>
+     * <td></td>
+     * <td>&lt;K1:A&gt;</td>
      * <td>&lt;K1:b&gt;</td>
-     * <td>&lt;K1:null&gt;</td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:ValueJoiner(A,b)&gt;</td>
      * </tr>
      * <tr>
+     * <td>&lt;K1:C&gt;</td>
+     * <td>&lt;K1:C&gt;</td>
      * <td></td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:ValueJoiner(C,b)&gt;</td>
+     * </tr>
+     * <tr>
      * <td></td>
+     * <td>&lt;K1:C&gt;</td>
      * <td>&lt;K1:null&gt;</td>
      * <td></td>
-     * <td></td>
+     * <td>&lt;K1:null&gt;</td>
      * </tr>
      * </table>
      * Both input streams (or to be more precise, their underlying source topics) need to have the same number of
@@ -719,21 +1264,23 @@ public interface KTable<K, V> {
      * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records
      * @param <VO>   the value type of the other {@code KTable}
      * @param <VR>   the value type of the result {@code KTable}
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
      * @return a {@code KTable} that contains join-records for each key and values computed by the given
-     * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of
-     * left {@code KTable}
-     * @see #join(KTable, ValueJoiner)
+     * {@link ValueJoiner}, one for each matched record-pair with the same key
+     * @see #leftJoin(KTable, ValueJoiner)
      * @see #outerJoin(KTable, ValueJoiner)
      */
-    <VO, VR> KTable<K, VR> leftJoin(final KTable<K, VO> other,
-                                    final ValueJoiner<? super V, ? super VO, ? extends VR> joiner);
+    <VO, VR> KTable<K, VR> join(final KTable<K, VO> other,
+                                final ValueJoiner<? super V, ? super VO, ? extends VR> joiner,
+                                final StateStoreSupplier<KeyValueStore> storeSupplier);
+
 
     /**
      * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using
-     * non-windowed outer equi join.
+     * non-windowed left equi join.
      * The join is a primary key join with join attribute {@code thisKTable.key == otherKTable.key}.
-     * In contrast to {@link #join(KTable, ValueJoiner) inner-join} or {@link #leftJoin(KTable, ValueJoiner) left-join},
-     * all records from both input {@code KTable}s will produce an output record (cf. below).
+     * In contrast to {@link #join(KTable, ValueJoiner) inner-join}, all records from left {@code KTable} will produce
+     * an output record (cf. below).
      * The result is an ever updating {@code KTable} that represents the <em>current</em> (i.e., processing time) result
      * of the join.
      * <p>
@@ -744,14 +1291,15 @@ public interface KTable<K, V> {
      * <p>
      * For each {@code KTable} record that finds a corresponding record in the other {@code KTable}'s state the
      * provided {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record.
-     * Additionally, for each record that does not find a corresponding record in the corresponding other
-     * {@code KTable}'s state the provided {@link ValueJoiner} will be called with {@code null} value for the
-     * corresponding other value to compute a value (with arbitrary type) for the result record.
+     * Additionally, for each record of left {@code KTable} that does not find a corresponding record in the
+     * right {@code KTable}'s state the provided {@link ValueJoiner} will be called with {@code rightValue =
+     * null} to compute a value (with arbitrary type) for the result record.
      * The key of the result record is the same as for both joining input records.
      * <p>
      * Note that {@link KeyValue records} with {@code null} values (so-called tombstone records) have delete semantics.
-     * Thus, for input tombstones the provided value-joiner is not called but a tombstone record is forwarded directly
-     * to delete a record in the result {@code KTable} if required (i.e., if there is anything to be deleted).
+     * For example, for left input tombstones the provided value-joiner is not called but a tombstone record is
+     * forwarded directly to delete a record in the result {@code KTable} if required (i.e., if there is anything to be
+     * deleted).
      * <p>
      * Input records with {@code null} key will be dropped and no join computation is performed.
      * <p>
@@ -783,14 +1331,14 @@ public interface KTable<K, V> {
      * <td></td>
      * <td></td>
      * <td>&lt;K1:b&gt;</td>
-     * <td>&lt;K1:ValueJoiner(null,b)&gt;</td>
+     * <td>&lt;K1:null&gt;</td>
      * </tr>
      * <tr>
      * <td></td>
      * <td></td>
      * <td>&lt;K1:null&gt;</td>
      * <td></td>
-     * <td>&lt;K1:null&gt;</td>
+     * <td></td>
      * </tr>
      * </table>
      * Both input streams (or to be more precise, their underlying source topics) need to have the same number of
@@ -802,17 +1350,443 @@ public interface KTable<K, V> {
      * @param <VR>   the value type of the result {@code KTable}
      * @return a {@code KTable} that contains join-records for each key and values computed by the given
      * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of
-     * both {@code KTable}s
+     * left {@code KTable}
      * @see #join(KTable, ValueJoiner)
-     * @see #leftJoin(KTable, ValueJoiner)
+     * @see #outerJoin(KTable, ValueJoiner)
      */
-    <VO, VR> KTable<K, VR> outerJoin(final KTable<K, VO> other,
-                                     final ValueJoiner<? super V, ? super VO, ? extends VR> joiner);
+    <VO, VR> KTable<K, VR> leftJoin(final KTable<K, VO> other,
+                                    final ValueJoiner<? super V, ? super VO, ? extends VR> joiner);
+
+    /**
+     * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using
+     * non-windowed left equi join.
+     * The join is a primary key join with join attribute {@code thisKTable.key == otherKTable.key}.
+     * In contrast to {@link #join(KTable, ValueJoiner) inner-join}, all records from left {@code KTable} will produce
+     * an output record (cf. below).
+     * The result is an ever updating {@code KTable} that represents the <em>current</em> (i.e., processing time) result
+     * of the join.
+     * <p>
+     * The join is computed by (1) updating the internal state of one {@code KTable} and (2) performing a lookup for a
+     * matching record in the <em>current</em> (i.e., processing time) internal state of the other {@code KTable}.
+     * This happens in a symmetric way, i.e., for each update of either {@code this} or the {@code other} input
+     * {@code KTable} the result gets updated.
+     * <p>
+     * For each {@code KTable} record that finds a corresponding record in the other {@code KTable}'s state the
+     * provided {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record.
+     * Additionally, for each record of left {@code KTable} that does not find a corresponding record in the
+     * right {@code KTable}'s state the provided {@link ValueJoiner} will be called with {@code rightValue =
+     * null} to compute a value (with arbitrary type) for the result record.
+     * The key of the result record is the same as for both joining input records.
+     * <p>
+     * Note that {@link KeyValue records} with {@code null} values (so-called tombstone records) have delete semantics.
+     * For example, for left input tombstones the provided value-joiner is not called but a tombstone record is
+     * forwarded directly to delete a record in the result {@code KTable} if required (i.e., if there is anything to be
+     * deleted).
+     * <p>
+     * Input records with {@code null} key will be dropped and no join computation is performed.
+     * <p>
+     * Example:
+     * <table border='1'>
+     * <tr>
+     * <th>thisKTable</th>
+     * <th>thisState</th>
+     * <th>otherKTable</th>
+     * <th>otherState</th>
+     * <th>result update record</th>
+     * </tr>
+     * <tr>
+     * <td>&lt;K1:A&gt;</td>
+     * <td>&lt;K1:A&gt;</td>
+     * <td></td>
+     * <td></td>
+     * <td>&lt;K1:ValueJoiner(A,null)&gt;</td>
+     * </tr>
+     * <tr>
+     * <td></td>
+     * <td>&lt;K1:A&gt;</td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:ValueJoiner(A,b)&gt;</td>
+     * </tr>
+     * <tr>
+     * <td>&lt;K1:null&gt;</td>
+     * <td></td>
+     * <td></td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:null&gt;</td>
+     * </tr>
+     * <tr>
+     * <td></td>
+     * <td></td>
+     * <td>&lt;K1:null&gt;</td>
+     * <td></td>
+     * <td></td>
+     * </tr>
+     * </table>
+     * Both input streams (or to be more precise, their underlying source topics) need to have the same number of
+     * partitions.
+     *
+     * @param other  the other {@code KTable} to be joined with this {@code KTable}
+     * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records
+     * @param <VO>   the value type of the other {@code KTable}
+     * @param <VR>   the value type of the result {@code KTable}
+     * @param joinSerde serializer for join result value type
+     * @param queryableStoreName a user-provided name of the underlying {@link KTable} that can be
+     * used to subsequently query the operation results; valid characters are ASCII
+     * alphanumerics, '.', '_' and '-'. If {@code null} then the results cannot be queried
+     * (i.e., that would be equivalent to calling {@link KTable#leftJoin(KTable, ValueJoiner)}.
+     * @return a {@code KTable} that contains join-records for each key and values computed by the given
+     * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of
+     * left {@code KTable}
+     * @see #join(KTable, ValueJoiner)
+     * @see #outerJoin(KTable, ValueJoiner)
+     */
+    <VO, VR> KTable<K, VR> leftJoin(final KTable<K, VO> other,
+                                    final ValueJoiner<? super V, ? super VO, ? extends VR> joiner,
+                                    final Serde<VR> joinSerde,
+                                    final String queryableStoreName);
+
+    /**
+     * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using
+     * non-windowed left equi join.
+     * The join is a primary key join with join attribute {@code thisKTable.key == otherKTable.key}.
+     * In contrast to {@link #join(KTable, ValueJoiner) inner-join}, all records from left {@code KTable} will produce
+     * an output record (cf. below).
+     * The result is an ever updating {@code KTable} that represents the <em>current</em> (i.e., processing time) result
+     * of the join.
+     * <p>
+     * The join is computed by (1) updating the internal state of one {@code KTable} and (2) performing a lookup for a
+     * matching record in the <em>current</em> (i.e., processing time) internal state of the other {@code KTable}.
+     * This happens in a symmetric way, i.e., for each update of either {@code this} or the {@code other} input
+     * {@code KTable} the result gets updated.
+     * <p>
+     * For each {@code KTable} record that finds a corresponding record in the other {@code KTable}'s state the
+     * provided {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record.
+     * Additionally, for each record of left {@code KTable} that does not find a corresponding record in the
+     * right {@code KTable}'s state the provided {@link ValueJoiner} will be called with {@code rightValue =
+     * null} to compute a value (with arbitrary type) for the result record.
+     * The key of the result record is the same as for both joining input records.
+     * <p>
+     * Note that {@link KeyValue records} with {@code null} values (so-called tombstone records) have delete semantics.
+     * For example, for left input tombstones the provided value-joiner is not called but a tombstone record is
+     * forwarded directly to delete a record in the result {@code KTable} if required (i.e., if there is anything to be
+     * deleted).
+     * <p>
+     * Input records with {@code null} key will be dropped and no join computation is performed.
+     * <p>
+     * Example:
+     * <table border='1'>
+     * <tr>
+     * <th>thisKTable</th>
+     * <th>thisState</th>
+     * <th>otherKTable</th>
+     * <th>otherState</th>
+     * <th>result update record</th>
+     * </tr>
+     * <tr>
+     * <td>&lt;K1:A&gt;</td>
+     * <td>&lt;K1:A&gt;</td>
+     * <td></td>
+     * <td></td>
+     * <td>&lt;K1:ValueJoiner(A,null)&gt;</td>
+     * </tr>
+     * <tr>
+     * <td></td>
+     * <td>&lt;K1:A&gt;</td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:ValueJoiner(A,b)&gt;</td>
+     * </tr>
+     * <tr>
+     * <td>&lt;K1:null&gt;</td>
+     * <td></td>
+     * <td></td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:null&gt;</td>
+     * </tr>
+     * <tr>
+     * <td></td>
+     * <td></td>
+     * <td>&lt;K1:null&gt;</td>
+     * <td></td>
+     * <td></td>
+     * </tr>
+     * </table>
+     * Both input streams (or to be more precise, their underlying source topics) need to have the same number of
+     * partitions.
+     *
+     * @param other  the other {@code KTable} to be joined with this {@code KTable}
+     * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records
+     * @param <VO>   the value type of the other {@code KTable}
+     * @param <VR>   the value type of the result {@code KTable}
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
+     * @return a {@code KTable} that contains join-records for each key and values computed by the given
+     * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of
+     * left {@code KTable}
+     * @see #join(KTable, ValueJoiner)
+     * @see #outerJoin(KTable, ValueJoiner)
+     */
+    <VO, VR> KTable<K, VR> leftJoin(final KTable<K, VO> other,
+                                    final ValueJoiner<? super V, ? super VO, ? extends VR> joiner,
+                                    final StateStoreSupplier<KeyValueStore> storeSupplier);
+
+
+    /**
+     * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using
+     * non-windowed outer equi join.
+     * The join is a primary key join with join attribute {@code thisKTable.key == otherKTable.key}.
+     * In contrast to {@link #join(KTable, ValueJoiner) inner-join} or {@link #leftJoin(KTable, ValueJoiner) left-join},
+     * all records from both input {@code KTable}s will produce an output record (cf. below).
+     * The result is an ever updating {@code KTable} that represents the <em>current</em> (i.e., processing time) result
+     * of the join.
+     * <p>
+     * The join is computed by (1) updating the internal state of one {@code KTable} and (2) performing a lookup for a
+     * matching record in the <em>current</em> (i.e., processing time) internal state of the other {@code KTable}.
+     * This happens in a symmetric way, i.e., for each update of either {@code this} or the {@code other} input
+     * {@code KTable} the result gets updated.
+     * <p>
+     * For each {@code KTable} record that finds a corresponding record in the other {@code KTable}'s state the
+     * provided {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record.
+     * Additionally, for each record that does not find a corresponding record in the corresponding other
+     * {@code KTable}'s state the provided {@link ValueJoiner} will be called with {@code null} value for the
+     * corresponding other value to compute a value (with arbitrary type) for the result record.
+     * The key of the result record is the same as for both joining input records.
+     * <p>
+     * Note that {@link KeyValue records} with {@code null} values (so-called tombstone records) have delete semantics.
+     * Thus, for input tombstones the provided value-joiner is not called but a tombstone record is forwarded directly
+     * to delete a record in the result {@code KTable} if required (i.e., if there is anything to be deleted).
+     * <p>
+     * Input records with {@code null} key will be dropped and no join computation is performed.
+     * <p>
+     * Example:
+     * <table border='1'>
+     * <tr>
+     * <th>thisKTable</th>
+     * <th>thisState</th>
+     * <th>otherKTable</th>
+     * <th>otherState</th>
+     * <th>result update record</th>
+     * </tr>
+     * <tr>
+     * <td>&lt;K1:A&gt;</td>
+     * <td>&lt;K1:A&gt;</td>
+     * <td></td>
+     * <td></td>
+     * <td>&lt;K1:ValueJoiner(A,null)&gt;</td>
+     * </tr>
+     * <tr>
+     * <td></td>
+     * <td>&lt;K1:A&gt;</td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:ValueJoiner(A,b)&gt;</td>
+     * </tr>
+     * <tr>
+     * <td>&lt;K1:null&gt;</td>
+     * <td></td>
+     * <td></td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:ValueJoiner(null,b)&gt;</td>
+     * </tr>
+     * <tr>
+     * <td></td>
+     * <td></td>
+     * <td>&lt;K1:null&gt;</td>
+     * <td></td>
+     * <td>&lt;K1:null&gt;</td>
+     * </tr>
+     * </table>
+     * Both input streams (or to be more precise, their underlying source topics) need to have the same number of
+     * partitions.
+     *
+     * @param other  the other {@code KTable} to be joined with this {@code KTable}
+     * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records
+     * @param <VO>   the value type of the other {@code KTable}
+     * @param <VR>   the value type of the result {@code KTable}
+     * @return a {@code KTable} that contains join-records for each key and values computed by the given
+     * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of
+     * both {@code KTable}s
+     * @see #join(KTable, ValueJoiner)
+     * @see #leftJoin(KTable, ValueJoiner)
+     */
+    <VO, VR> KTable<K, VR> outerJoin(final KTable<K, VO> other,
+                                     final ValueJoiner<? super V, ? super VO, ? extends VR> joiner);
+
+    /**
+     * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using
+     * non-windowed outer equi join.
+     * The join is a primary key join with join attribute {@code thisKTable.key == otherKTable.key}.
+     * In contrast to {@link #join(KTable, ValueJoiner) inner-join} or {@link #leftJoin(KTable, ValueJoiner) left-join},
+     * all records from both input {@code KTable}s will produce an output record (cf. below).
+     * The result is an ever updating {@code KTable} that represents the <em>current</em> (i.e., processing time) result
+     * of the join.
+     * <p>
+     * The join is computed by (1) updating the internal state of one {@code KTable} and (2) performing a lookup for a
+     * matching record in the <em>current</em> (i.e., processing time) internal state of the other {@code KTable}.
+     * This happens in a symmetric way, i.e., for each update of either {@code this} or the {@code other} input
+     * {@code KTable} the result gets updated.
+     * <p>
+     * For each {@code KTable} record that finds a corresponding record in the other {@code KTable}'s state the
+     * provided {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record.
+     * Additionally, for each record that does not find a corresponding record in the corresponding other
+     * {@code KTable}'s state the provided {@link ValueJoiner} will be called with {@code null} value for the
+     * corresponding other value to compute a value (with arbitrary type) for the result record.
+     * The key of the result record is the same as for both joining input records.
+     * <p>
+     * Note that {@link KeyValue records} with {@code null} values (so-called tombstone records) have delete semantics.
+     * Thus, for input tombstones the provided value-joiner is not called but a tombstone record is forwarded directly
+     * to delete a record in the result {@code KTable} if required (i.e., if there is anything to be deleted).
+     * <p>
+     * Input records with {@code null} key will be dropped and no join computation is performed.
+     * <p>
+     * Example:
+     * <table border='1'>
+     * <tr>
+     * <th>thisKTable</th>
+     * <th>thisState</th>
+     * <th>otherKTable</th>
+     * <th>otherState</th>
+     * <th>result update record</th>
+     * </tr>
+     * <tr>
+     * <td>&lt;K1:A&gt;</td>
+     * <td>&lt;K1:A&gt;</td>
+     * <td></td>
+     * <td></td>
+     * <td>&lt;K1:ValueJoiner(A,null)&gt;</td>
+     * </tr>
+     * <tr>
+     * <td></td>
+     * <td>&lt;K1:A&gt;</td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:ValueJoiner(A,b)&gt;</td>
+     * </tr>
+     * <tr>
+     * <td>&lt;K1:null&gt;</td>
+     * <td></td>
+     * <td></td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:ValueJoiner(null,b)&gt;</td>
+     * </tr>
+     * <tr>
+     * <td></td>
+     * <td></td>
+     * <td>&lt;K1:null&gt;</td>
+     * <td></td>
+     * <td>&lt;K1:null&gt;</td>
+     * </tr>
+     * </table>
+     * Both input streams (or to be more precise, their underlying source topics) need to have the same number of
+     * partitions.
+     *
+     * @param other  the other {@code KTable} to be joined with this {@code KTable}
+     * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records
+     * @param <VO>   the value type of the other {@code KTable}
+     * @param <VR>   the value type of the result {@code KTable}
+     * @param joinSerde serializer for join result value type
+     * @param queryableStoreName a user-provided name of the underlying {@link KTable} that can be
+     * used to subsequently query the operation results; valid characters are ASCII
+     * alphanumerics, '.', '_' and '-'. If {@code null} then the results cannot be queried
+     * (i.e., that would be equivalent to calling {@link KTable#outerJoin(KTable, ValueJoiner)}.
+     * @return a {@code KTable} that contains join-records for each key and values computed by the given
+     * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of
+     * both {@code KTable}s
+     * @see #join(KTable, ValueJoiner)
+     * @see #leftJoin(KTable, ValueJoiner)
+     */
+    <VO, VR> KTable<K, VR> outerJoin(final KTable<K, VO> other,
+                                     final ValueJoiner<? super V, ? super VO, ? extends VR> joiner,
+                                     final Serde<VR> joinSerde,
+                                     final String queryableStoreName);
+
+    /**
+     * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using
+     * non-windowed outer equi join.
+     * The join is a primary key join with join attribute {@code thisKTable.key == otherKTable.key}.
+     * In contrast to {@link #join(KTable, ValueJoiner) inner-join} or {@link #leftJoin(KTable, ValueJoiner) left-join},
+     * all records from both input {@code KTable}s will produce an output record (cf. below).
+     * The result is an ever updating {@code KTable} that represents the <em>current</em> (i.e., processing time) result
+     * of the join.
+     * <p>
+     * The join is computed by (1) updating the internal state of one {@code KTable} and (2) performing a lookup for a
+     * matching record in the <em>current</em> (i.e., processing time) internal state of the other {@code KTable}.
+     * This happens in a symmetric way, i.e., for each update of either {@code this} or the {@code other} input
+     * {@code KTable} the result gets updated.
+     * <p>
+     * For each {@code KTable} record that finds a corresponding record in the other {@code KTable}'s state the
+     * provided {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record.
+     * Additionally, for each record that does not find a corresponding record in the corresponding other
+     * {@code KTable}'s state the provided {@link ValueJoiner} will be called with {@code null} value for the
+     * corresponding other value to compute a value (with arbitrary type) for the result record.
+     * The key of the result record is the same as for both joining input records.
+     * <p>
+     * Note that {@link KeyValue records} with {@code null} values (so-called tombstone records) have delete semantics.
+     * Thus, for input tombstones the provided value-joiner is not called but a tombstone record is forwarded directly
+     * to delete a record in the result {@code KTable} if required (i.e., if there is anything to be deleted).
+     * <p>
+     * Input records with {@code null} key will be dropped and no join computation is performed.
+     * <p>
+     * Example:
+     * <table border='1'>
+     * <tr>
+     * <th>thisKTable</th>
+     * <th>thisState</th>
+     * <th>otherKTable</th>
+     * <th>otherState</th>
+     * <th>result update record</th>
+     * </tr>
+     * <tr>
+     * <td>&lt;K1:A&gt;</td>
+     * <td>&lt;K1:A&gt;</td>
+     * <td></td>
+     * <td></td>
+     * <td>&lt;K1:ValueJoiner(A,null)&gt;</td>
+     * </tr>
+     * <tr>
+     * <td></td>
+     * <td>&lt;K1:A&gt;</td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:ValueJoiner(A,b)&gt;</td>
+     * </tr>
+     * <tr>
+     * <td>&lt;K1:null&gt;</td>
+     * <td></td>
+     * <td></td>
+     * <td>&lt;K1:b&gt;</td>
+     * <td>&lt;K1:ValueJoiner(null,b)&gt;</td>
+     * </tr>
+     * <tr>
+     * <td></td>
+     * <td></td>
+     * <td>&lt;K1:null&gt;</td>
+     * <td></td>
+     * <td>&lt;K1:null&gt;</td>
+     * </tr>
+     * </table>
+     * Both input streams (or to be more precise, their underlying source topics) need to have the same number of
+     * partitions.
+     *
+     * @param other  the other {@code KTable} to be joined with this {@code KTable}
+     * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records
+     * @param <VO>   the value type of the other {@code KTable}
+     * @param <VR>   the value type of the result {@code KTable}
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
+     * @return a {@code KTable} that contains join-records for each key and values computed by the given
+     * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of
+     * both {@code KTable}s
+     * @see #join(KTable, ValueJoiner)
+     * @see #leftJoin(KTable, ValueJoiner)
+     */
+    <VO, VR> KTable<K, VR> outerJoin(final KTable<K, VO> other,
+                                     final ValueJoiner<? super V, ? super VO, ? extends VR> joiner,
+                                     final StateStoreSupplier<KeyValueStore> storeSupplier);
 
     /**
-     * Get the name of the local state store used for materializing this {@code KTable}.
+     * Get the name of the local state store used that can be used to query this {@code KTable}.
      *
-     * @return the underlying state store name, or {@code null} if this {@code KTable} is not materialized
+     * @return the underlying state store name, or {@code null} if this {@code KTable} cannot be queried.
      */
-    String getStoreName();
+    String queryableStoreName();
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java
----------------------------------------------------------------------
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java
index dce5d12..8aea44d 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java
@@ -47,6 +47,7 @@ public abstract class AbstractStream<K> {
         this.sourceNodes = sourceNodes;
     }
 
+
     Set<String> ensureJoinableWith(final AbstractStream<K> other) {
         Set<String> allSourceNodes = new HashSet<>();
         allSourceNodes.addAll(sourceNodes);
@@ -57,6 +58,12 @@ public abstract class AbstractStream<K> {
         return allSourceNodes;
     }
 
+    String getOrCreateName(final String queryableStoreName, final String prefix) {
+        final String returnName = queryableStoreName != null ? queryableStoreName : topology.newStoreName(prefix);
+        Topic.validate(returnName);
+        return returnName;
+    }
+
     static <T2, T1, R> ValueJoiner<T2, T1, R> reverseJoiner(final ValueJoiner<T1, T2, R> joiner) {
         return new ValueJoiner<T2, T1, R>() {
             @Override


[4/5] kafka git commit: KAFKA-5045: Clarify on KTable APIs for queryable stores

Posted by gu...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java
----------------------------------------------------------------------
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java
index 8685e8b..d14e600 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java
@@ -46,7 +46,7 @@ public interface KGroupedTable<K, V> {
      * the same key into a new instance of {@link KTable}.
      * Records with {@code null} key are ignored.
      * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view)
-     * that can be queried using the provided {@code storeName}.
+     * that can be queried using the provided {@code queryableStoreName}.
      * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream.
      * <p>
      * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
@@ -60,7 +60,7 @@ public interface KGroupedTable<K, V> {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ... // counting words
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-word";
      * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -68,20 +68,48 @@ public interface KGroupedTable<K, V> {
      * query the value of the key on a parallel running instance of your Kafka Streams application.
      * <p>
      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
-     * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
+     * The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
      * user-specified in {@link StreamsConfig} via parameter
-     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the
-     * provide {@code storeName}, and "-changelog" is a fixed suffix.
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "queryableStoreName" is the
+     * provide {@code queryableStoreName}, and "-changelog" is a fixed suffix.
      * The store name must be a valid Kafka topic name and cannot contain characters other than ASCII alphanumerics,
      * '.', '_' and '-'.
      * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
      *
-     * @param storeName     the name of the underlying {@link KTable} state store; valid characters are ASCII
-     *                      alphanumerics, '.', '_' and '-'
+     * @param queryableStoreName     the name of the underlying {@link KTable} state store; valid characters are ASCII
+     * alphanumerics, '.', '_' and '-'. If {@code null} this is the equivalent of {@link KGroupedTable#count()}.
      * @return a {@link KTable} that contains "update" records with unmodified keys and {@link Long} values that
      * represent the latest (rolling) count (i.e., number of records) for each key
      */
-    KTable<K, Long> count(final String storeName);
+    KTable<K, Long> count(final String queryableStoreName);
+
+    /**
+     * Count number of records of the original {@link KTable} that got {@link KTable#groupBy(KeyValueMapper) mapped} to
+     * the same key into a new instance of {@link KTable}.
+     * Records with {@code null} key are ignored.
+     * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view)
+     * that can be queried using the provided {@code queryableStoreName}.
+     * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream.
+     * <p>
+     * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
+     * the same key.
+     * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
+     * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
+     * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
+     * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
+     * <p>
+     * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
+     * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
+     * user-specified in {@link StreamsConfig} via parameter
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "internalStoreName" is an internal name
+     * and "-changelog" is a fixed suffix.
+     * Note that the internal store name may not be queriable through Interactive Queries.
+     * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
+     *
+     * @return a {@link KTable} that contains "update" records with unmodified keys and {@link Long} values that
+     * represent the latest (rolling) count (i.e., number of records) for each key
+     */
+    KTable<K, Long> count();
 
     /**
      * Count number of records of the original {@link KTable} that got {@link KTable#groupBy(KeyValueMapper) mapped} to
@@ -102,8 +130,8 @@ public interface KGroupedTable<K, V> {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ... // counting words
-     * String storeName = storeSupplier.name();
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * String queryableStoreName = storeSupplier.name();
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-word";
      * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -111,13 +139,13 @@ public interface KGroupedTable<K, V> {
      * query the value of the key on a parallel running instance of your Kafka Streams application.
      * <p>
      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
-     * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
+     * The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
      * user-specified in {@link StreamsConfig} via parameter
-     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the
-     * provide {@code storeName}, and "-changelog" is a fixed suffix.
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "queryableStoreName" is the
+     * provide {@code queryableStoreName}, and "-changelog" is a fixed suffix.
      * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
      *
-     * @param storeSupplier user defined state store supplier
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
      * @return a {@link KTable} that contains "update" records with unmodified keys and {@link Long} values that
      * represent the latest (rolling) count (i.e., number of records) for each key
      */
@@ -130,7 +158,7 @@ public interface KGroupedTable<K, V> {
      * Combining implies that the type of the aggregate result is the same as the type of the input value
      * (c.f. {@link #aggregate(Initializer, Aggregator, Aggregator, Serde, String)}).
      * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view)
-     * that can be queried using the provided {@code storeName}.
+     * that can be queried using the provided {@code queryableStoreName}.
      * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream.
      * <p>
      * Each update to the original {@link KTable} results in a two step update of the result {@link KTable}.
@@ -167,7 +195,7 @@ public interface KGroupedTable<K, V> {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ... // counting words
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-word";
      * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -175,24 +203,80 @@ public interface KGroupedTable<K, V> {
      * query the value of the key on a parallel running instance of your Kafka Streams application.
      * <p>
      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
-     * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
+     * The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
      * user-specified in {@link StreamsConfig} via parameter
-     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the
-     * provide {@code storeName}, and "-changelog" is a fixed suffix.
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "queryableStoreName" is the
+     * provide {@code queryableStoreName}, and "-changelog" is a fixed suffix.
      * The store name must be a valid Kafka topic name and cannot contain characters other than ASCII alphanumerics,
      * '.', '_' and '-'.
      * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
      *
      * @param adder      a {@link Reducer} that adds a new value to the aggregate result
      * @param subtractor a {@link Reducer} that removed an old value from the aggregate result
-     * @param storeName     the name of the underlying {@link KTable} state store; valid characters are ASCII alphanumerics,
-     *                      '.', '_' and '-'
+     * @param queryableStoreName     the name of the underlying {@link KTable} state store; valid characters are ASCII alphanumerics,
+     * '.', '_' and '-'. If {@code null} this is the equivalent of {@link KGroupedTable#reduce(Reducer, Reducer)} ()}.
      * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the
      * latest (rolling) aggregate for each key
      */
     KTable<K, V> reduce(final Reducer<V> adder,
                         final Reducer<V> subtractor,
-                        final String storeName);
+                        final String queryableStoreName);
+
+    /**
+     * Combine the value of records of the original {@link KTable} that got {@link KTable#groupBy(KeyValueMapper)
+     * mapped} to the same key into a new instance of {@link KTable}.
+     * Records with {@code null} key are ignored.
+     * Combining implies that the type of the aggregate result is the same as the type of the input value
+     * (c.f. {@link #aggregate(Initializer, Aggregator, Aggregator, Serde, String)}).
+     * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view)
+     * that can be queried using the provided {@code queryableStoreName}.
+     * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream.
+     * <p>
+     * Each update to the original {@link KTable} results in a two step update of the result {@link KTable}.
+     * The specified {@link Reducer adder} is applied for each update record and computes a new aggregate using the
+     * current aggregate and the record's value by adding the new record to the aggregate.
+     * The specified {@link Reducer substractor} is applied for each "replaced" record of the original {@link KTable}
+     * and computes a new aggregate using the current aggregate and the record's value by "removing" the "replaced"
+     * record from the aggregate.
+     * If there is no current aggregate the {@link Reducer} is not applied and the new aggregate will be the record's
+     * value as-is.
+     * Thus, {@code reduce(Reducer, Reducer, String)} can be used to compute aggregate functions like sum.
+     * For sum, the adder and substractor would work as follows:
+     * <pre>{@code
+     * public class SumAdder implements Reducer<Integer> {
+     *   public Integer apply(Integer currentAgg, Integer newValue) {
+     *     return currentAgg + newValue;
+     *   }
+     * }
+     *
+     * public class SumSubtractor implements Reducer<Integer> {
+     *   public Integer apply(Integer currentAgg, Integer oldValue) {
+     *     return currentAgg - oldValue;
+     *   }
+     * }
+     * }</pre>
+     * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
+     * the same key.
+     * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
+     * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
+     * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
+     * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
+     * <p>
+     * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
+     * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
+     * user-specified in {@link StreamsConfig} via parameter
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "internalStoreName" is an internal name
+     * and "-changelog" is a fixed suffix.
+     * Note that the internal store name may not be queriable through Interactive Queries.
+     * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
+     *
+     * @param adder      a {@link Reducer} that adds a new value to the aggregate result
+     * @param subtractor a {@link Reducer} that removed an old value from the aggregate result
+     * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the
+     * latest (rolling) aggregate for each key
+     */
+    KTable<K, V> reduce(final Reducer<V> adder,
+                        final Reducer<V> subtractor);
 
     /**
      * Combine the value of records of the original {@link KTable} that got {@link KTable#groupBy(KeyValueMapper)
@@ -238,8 +322,8 @@ public interface KGroupedTable<K, V> {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ... // counting words
-     * String storeName = storeSupplier.name();
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * String queryableStoreName = storeSupplier.name();
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-word";
      * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -247,15 +331,15 @@ public interface KGroupedTable<K, V> {
      * query the value of the key on a parallel running instance of your Kafka Streams application.
      * <p>
      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
-     * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
+     * The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
      * user-specified in {@link StreamsConfig} via parameter
-     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the
-     * provide {@code storeName}, and "-changelog" is a fixed suffix.
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "queryableStoreName" is the
+     * provide {@code queryableStoreName}, and "-changelog" is a fixed suffix.
      * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
      *
      * @param adder         a {@link Reducer} that adds a new value to the aggregate result
      * @param subtractor    a {@link Reducer} that removed an old value from the aggregate result
-     * @param storeSupplier user defined state store supplier
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
      * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the
      * latest (rolling) aggregate for each key
      */
@@ -319,7 +403,7 @@ public interface KGroupedTable<K, V> {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ... // counting words
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-word";
      * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -327,16 +411,17 @@ public interface KGroupedTable<K, V> {
      * query the value of the key on a parallel running instance of your Kafka Streams application.
      * <p>
      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
-     * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
+     * The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
      * user-specified in {@link StreamsConfig} via parameter
-     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the
-     * provide {@code storeName}, and "-changelog" is a fixed suffix.
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "queryableStoreName" is the
+     * provide {@code queryableStoreName}, and "-changelog" is a fixed suffix.
      * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
      *
      * @param initializer a {@link Initializer} that provides an initial aggregate result value
      * @param adder       a {@link Aggregator} that adds a new record to the aggregate result
      * @param subtractor  a {@link Aggregator} that removed an old record from the aggregate result
-     * @param storeName   the name of the underlying {@link KTable} state store
+     * @param queryableStoreName   the name of the underlying {@link KTable} state store.
+     *                             If {@code null} this is the equivalent of {@link KGroupedTable#aggregate(Initializer, Aggregator, Aggregator)} ()}.
      * @param <VR>        the value type of the aggregated {@link KTable}
      * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the
      * latest (rolling) aggregate for each key
@@ -344,7 +429,7 @@ public interface KGroupedTable<K, V> {
     <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
                                  final Aggregator<? super K, ? super V, VR> adder,
                                  final Aggregator<? super K, ? super V, VR> subtractor,
-                                 final String storeName);
+                                 final String queryableStoreName);
 
     /**
      * Aggregate the value of records of the original {@link KTable} that got {@link KTable#groupBy(KeyValueMapper)
@@ -352,8 +437,79 @@ public interface KGroupedTable<K, V> {
      * Records with {@code null} key are ignored.
      * Aggregating is a generalization of {@link #reduce(Reducer, Reducer, String) combining via reduce(...)} as it,
      * for example, allows the result to have a different type than the input values.
+     * If the result value type does not match the {@link StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value
+     * serde} you should use {@link KGroupedTable#aggregate(Initializer, Aggregator, Aggregator, Serde, String)
+     * aggregate(Initializer, Aggregator, Aggregator, Serde, String)}.
      * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view)
-     * that can be queried using the provided {@code storeName}.
+     * provided by the given {@code storeSupplier}.
+     * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream.
+     * <p>
+     * The specified {@link Initializer} is applied once directly before the first input record is processed to
+     * provide an initial intermediate aggregation result that is used to process the first record.
+     * Each update to the original {@link KTable} results in a two step update of the result {@link KTable}.
+     * The specified {@link Aggregator adder} is applied for each update record and computes a new aggregate using the
+     * current aggregate (or for the very first record using the intermediate aggregation result provided via the
+     * {@link Initializer}) and the record's value by adding the new record to the aggregate.
+     * The specified {@link Aggregator substractor} is applied for each "replaced" record of the original {@link KTable}
+     * and computes a new aggregate using the current aggregate and the record's value by "removing" the "replaced"
+     * record from the aggregate.
+     * Thus, {@code aggregate(Initializer, Aggregator, Aggregator, String)} can be used to compute aggregate functions
+     * like sum.
+     * For sum, the initializer, adder, and substractor would work as follows:
+     * <pre>{@code
+     * // in this example, LongSerde.class must be set as default value serde in StreamsConfig
+     * public class SumInitializer implements Initializer<Long> {
+     *   public Long apply() {
+     *     return 0L;
+     *   }
+     * }
+     *
+     * public class SumAdder implements Aggregator<String, Integer, Long> {
+     *   public Long apply(String key, Integer newValue, Long aggregate) {
+     *     return aggregate + newValue;
+     *   }
+     * }
+     *
+     * public class SumSubstractor implements Aggregator<String, Integer, Long> {
+     *   public Long apply(String key, Integer oldValue, Long aggregate) {
+     *     return aggregate - oldValue;
+     *   }
+     * }
+     * }</pre>
+     * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
+     * the same key.
+     * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
+     * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
+     * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
+     * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
+     * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
+     * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
+     * user-specified in {@link StreamsConfig} via parameter
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "internalStoreName" is an internal name
+     * and "-changelog" is a fixed suffix.
+     * Note that the internal store name may not be queriable through Interactive Queries.
+     * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
+     *
+     * @param initializer a {@link Initializer} that provides an initial aggregate result value
+     * @param adder       a {@link Aggregator} that adds a new record to the aggregate result
+     * @param subtractor  a {@link Aggregator} that removed an old record from the aggregate result
+     * @param <VR>        the value type of the aggregated {@link KTable}
+     * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the
+     * latest (rolling) aggregate for each key
+     */
+    <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
+                                 final Aggregator<? super K, ? super V, VR> adder,
+                                 final Aggregator<? super K, ? super V, VR> subtractor);
+
+
+    /**
+     * Aggregate the value of records of the original {@link KTable} that got {@link KTable#groupBy(KeyValueMapper)
+     * mapped} to the same key into a new instance of {@link KTable} using default serializers and deserializers.
+     * Records with {@code null} key are ignored.
+     * Aggregating is a generalization of {@link #reduce(Reducer, Reducer, String) combining via reduce(...)} as it,
+     * for example, allows the result to have a different type than the input values.
+     * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view)
+     * that can be queried using the provided {@code queryableStoreName}.
      * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream.
      * <p>
      * The specified {@link Initializer} is applied once directly before the first input record is processed to
@@ -398,7 +554,7 @@ public interface KGroupedTable<K, V> {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ... // counting words
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-word";
      * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -406,10 +562,10 @@ public interface KGroupedTable<K, V> {
      * query the value of the key on a parallel running instance of your Kafka Streams application.
      * <p>
      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
-     * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
+     * The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
      * user-specified in {@link StreamsConfig} via parameter
-     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the
-     * provide {@code storeName}, and "-changelog" is a fixed suffix.
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "queryableStoreName" is the
+     * provide {@code queryableStoreName}, and "-changelog" is a fixed suffix.
      * The store name must be a valid Kafka topic name and cannot contain characters other than ASCII alphanumerics,
      * '.', '_' and '-'.
      * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
@@ -419,8 +575,8 @@ public interface KGroupedTable<K, V> {
      * @param subtractor    a {@link Aggregator} that removed an old record from the aggregate result
      * @param aggValueSerde aggregate value serdes for materializing the aggregated table,
      *                      if not specified the default serdes defined in the configs will be used
-     * @param storeName     the name of the underlying {@link KTable} state store; valid characters are ASCII
-     *                      alphanumerics, '.', '_' and '-'
+     * @param queryableStoreName     the name of the underlying {@link KTable} state store; valid characters are ASCII
+     * alphanumerics, '.', '_' and '-'. If {@code null} this is the equivalent of {@link KGroupedTable#aggregate(Initializer, Aggregator, Aggregator, Serde)} ()}.
      * @param <VR>          the value type of the aggregated {@link KTable}
      * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the
      * latest (rolling) aggregate for each key
@@ -429,7 +585,78 @@ public interface KGroupedTable<K, V> {
                                  final Aggregator<? super K, ? super V, VR> adder,
                                  final Aggregator<? super K, ? super V, VR> subtractor,
                                  final Serde<VR> aggValueSerde,
-                                 final String storeName);
+                                 final String queryableStoreName);
+
+    /**
+     * Aggregate the value of records of the original {@link KTable} that got {@link KTable#groupBy(KeyValueMapper)
+     * mapped} to the same key into a new instance of {@link KTable} using default serializers and deserializers.
+     * Records with {@code null} key are ignored.
+     * Aggregating is a generalization of {@link #reduce(Reducer, Reducer, String) combining via reduce(...)} as it,
+     * for example, allows the result to have a different type than the input values.
+     * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view)
+     * that can be queried using the provided {@code queryableStoreName}.
+     * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream.
+     * <p>
+     * The specified {@link Initializer} is applied once directly before the first input record is processed to
+     * provide an initial intermediate aggregation result that is used to process the first record.
+     * Each update to the original {@link KTable} results in a two step update of the result {@link KTable}.
+     * The specified {@link Aggregator adder} is applied for each update record and computes a new aggregate using the
+     * current aggregate (or for the very first record using the intermediate aggregation result provided via the
+     * {@link Initializer}) and the record's value by adding the new record to the aggregate.
+     * The specified {@link Aggregator substractor} is applied for each "replaced" record of the original {@link KTable}
+     * and computes a new aggregate using the current aggregate and the record's value by "removing" the "replaced"
+     * record from the aggregate.
+     * Thus, {@code aggregate(Initializer, Aggregator, Aggregator, String)} can be used to compute aggregate functions
+     * like sum.
+     * For sum, the initializer, adder, and substractor would work as follows:
+     * <pre>{@code
+     * public class SumInitializer implements Initializer<Long> {
+     *   public Long apply() {
+     *     return 0L;
+     *   }
+     * }
+     *
+     * public class SumAdder implements Aggregator<String, Integer, Long> {
+     *   public Long apply(String key, Integer newValue, Long aggregate) {
+     *     return aggregate + newValue;
+     *   }
+     * }
+     *
+     * public class SumSubstractor implements Aggregator<String, Integer, Long> {
+     *   public Long apply(String key, Integer oldValue, Long aggregate) {
+     *     return aggregate - oldValue;
+     *   }
+     * }
+     * }</pre>
+     * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
+     * the same key.
+     * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
+     * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
+     * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
+     * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
+     * <p>
+     * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
+     * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
+     * user-specified in {@link StreamsConfig} via parameter
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "internalStoreName" is an internal name
+     * and "-changelog" is a fixed suffix.
+     * Note that the internal store name may not be queriable through Interactive Queries.
+     * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
+     *
+     * @param initializer   a {@link Initializer} that provides an initial aggregate result value
+     * @param adder         a {@link Aggregator} that adds a new record to the aggregate result
+     * @param subtractor    a {@link Aggregator} that removed an old record from the aggregate result
+     * @param aggValueSerde aggregate value serdes for materializing the aggregated table,
+     *                      if not specified the default serdes defined in the configs will be used
+     * @param <VR>          the value type of the aggregated {@link KTable}
+     * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the
+     * latest (rolling) aggregate for each key
+     */
+    <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
+                                 final Aggregator<? super K, ? super V, VR> adder,
+                                 final Aggregator<? super K, ? super V, VR> subtractor,
+                                 final Serde<VR> aggValueSerde);
+
 
     /**
      * Aggregate the value of records of the original {@link KTable} that got {@link KTable#groupBy(KeyValueMapper)
@@ -483,8 +710,8 @@ public interface KGroupedTable<K, V> {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ... // counting words
-     * String storeName = storeSupplier.name();
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * String queryableStoreName = storeSupplier.name();
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-word";
      * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -492,16 +719,16 @@ public interface KGroupedTable<K, V> {
      * query the value of the key on a parallel running instance of your Kafka Streams application.
      * <p>
      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
-     * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
+     * The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
      * user-specified in {@link StreamsConfig} via parameter
-     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the
-     * provide {@code storeName}, and "-changelog" is a fixed suffix.
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "queryableStoreName" is the
+     * provide {@code queryableStoreName}, and "-changelog" is a fixed suffix.
      * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
      *
      * @param initializer   a {@link Initializer} that provides an initial aggregate result value
      * @param adder         a {@link Aggregator} that adds a new record to the aggregate result
      * @param subtractor    a {@link Aggregator} that removed an old record from the aggregate result
-     * @param storeSupplier user defined state store supplier
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
      * @param <VR>          the value type of the aggregated {@link KTable}
      * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the
      * latest (rolling) aggregate for each key

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java
----------------------------------------------------------------------
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java
index c361cad..0e02c8f 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java
@@ -27,7 +27,6 @@ import org.apache.kafka.streams.kstream.internals.KTableImpl;
 import org.apache.kafka.streams.kstream.internals.KTableSource;
 import org.apache.kafka.streams.kstream.internals.KTableSourceValueGetterSupplier;
 import org.apache.kafka.streams.processor.ProcessorSupplier;
-import org.apache.kafka.streams.processor.StateStore;
 import org.apache.kafka.streams.processor.StateStoreSupplier;
 import org.apache.kafka.streams.processor.TopologyBuilder;
 import org.apache.kafka.streams.state.KeyValueStore;
@@ -35,6 +34,7 @@ import org.apache.kafka.streams.state.QueryableStoreType;
 import org.apache.kafka.streams.state.internals.RocksDBKeyValueStoreSupplier;
 
 import java.util.Collections;
+import java.util.Objects;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.regex.Pattern;
 
@@ -243,7 +243,7 @@ public class KStreamBuilder extends TopologyBuilder {
      * If this is not the case the returned {@link KTable} will be corrupted.
      * <p>
      * The resulting {@link KTable} will be materialized in a local {@link KeyValueStore} with the given
-     * {@code storeName}.
+     * {@code queryableStoreName}.
      * However, no internal changelog topic is created since the original input topic can be used for recovery (cf.
      * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}).
      * <p>
@@ -251,7 +251,7 @@ public class KStreamBuilder extends TopologyBuilder {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ...
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-key";
      * Long valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -259,12 +259,103 @@ public class KStreamBuilder extends TopologyBuilder {
      * query the value of the key on a parallel running instance of your Kafka Streams application.
      *
      * @param topic     the topic name; cannot be {@code null}
-     * @param storeName the state store name; cannot be {@code null}
+     * @param queryableStoreName the state store name; If {@code null} this is the equivalent of {@link KStreamBuilder#table(String)} ()}.
      * @return a {@link KTable} for the specified topic
      */
     public <K, V> KTable<K, V> table(final String topic,
-                                     final String storeName) {
-        return table(null, null, null, topic, storeName);
+                                     final String queryableStoreName) {
+        return table(null, null, null, topic, queryableStoreName);
+    }
+
+    /**
+     * Create a {@link KTable} for the specified topic.
+     * The default {@code "auto.offset.reset"} strategy and default key and value deserializers as specified in the
+     * {@link StreamsConfig config} are used.
+     * Input {@link KeyValue records} with {@code null} key will be dropped.
+     * <p>
+     * Note that the specified input topics must be partitioned by key.
+     * If this is not the case the returned {@link KTable} will be corrupted.
+     * <p>
+     * The resulting {@link KTable} will be materialized in a local {@link KeyValueStore} with the given
+     * {@code queryableStoreName}.
+     * However, no internal changelog topic is created since the original input topic can be used for recovery (cf.
+     * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}).
+     * <p>
+     * To query the local {@link KeyValueStore} it must be obtained via
+     * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
+     * <pre>{@code
+     * KafkaStreams streams = ...
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * String key = "some-key";
+     * Long valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
+     * }</pre>
+     * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to
+     * query the value of the key on a parallel running instance of your Kafka Streams application.
+     *
+     * @param topic     the topic name; cannot be {@code null}
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
+     * @return a {@link KTable} for the specified topic
+     */
+    public <K, V> KTable<K, V> table(final String topic,
+                                     final StateStoreSupplier<KeyValueStore> storeSupplier) {
+        return table(null, null, null, topic, storeSupplier);
+    }
+
+
+    /**
+     * Create a {@link KTable} for the specified topic.
+     * The default {@code "auto.offset.reset"} strategy and default key and value deserializers as specified in the
+     * {@link StreamsConfig config} are used.
+     * Input {@link KeyValue records} with {@code null} key will be dropped.
+     * <p>
+     * Note that the specified input topics must be partitioned by key.
+     * If this is not the case the returned {@link KTable} will be corrupted.
+     * <p>
+     * The resulting {@link KTable} will be materialized in a local {@link KeyValueStore} with an internal
+     * store name. Note that that store name may not be queriable through Interactive Queries.
+     * No internal changelog topic is created since the original input topic can be used for recovery (cf.
+     * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}).
+     * @param topic     the topic name; cannot be {@code null}
+     * @return a {@link KTable} for the specified topic
+     */
+    public <K, V> KTable<K, V> table(final String topic) {
+        return table(null, null, null, topic, (String) null);
+    }
+
+    /**
+     * Create a {@link KTable} for the specified topic.
+     * The default key and value deserializers as specified in the {@link StreamsConfig config} are used.
+     * Input {@link KeyValue records} with {@code null} key will be dropped.
+     * <p>
+     * Note that the specified input topics must be partitioned by key.
+     * If this is not the case the returned {@link KTable} will be corrupted.
+     * <p>
+     * The resulting {@link KTable} will be materialized in a local {@link KeyValueStore} with the given
+     * {@code queryableStoreName}.
+     * However, no internal changelog topic is created since the original input topic can be used for recovery (cf.
+     * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}).
+     * <p>
+     * To query the local {@link KeyValueStore} it must be obtained via
+     * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
+     * <pre>{@code
+     * KafkaStreams streams = ...
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * String key = "some-key";
+     * Long valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
+     * }</pre>
+     * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to
+     * query the value of the key on a parallel running instance of your Kafka Streams application.
+     *
+     * @param offsetReset the {@code "auto.offset.reset"} policy to use for the specified topic if no valid committed
+     *                    offsets are available
+     * @param topic       the topic name; cannot be {@code null}
+     * @param queryableStoreName   the state store name; If {@code null} this is the equivalent of {@link KStreamBuilder#table(AutoOffsetReset, String)} ()}.
+     * @return a {@link KTable} for the specified topic
+     */
+    public <K, V> KTable<K, V> table(final AutoOffsetReset offsetReset,
+                                     final String topic,
+                                     final String queryableStoreName) {
+        return table(offsetReset, null, null, topic, queryableStoreName);
     }
 
     /**
@@ -276,7 +367,7 @@ public class KStreamBuilder extends TopologyBuilder {
      * If this is not the case the returned {@link KTable} will be corrupted.
      * <p>
      * The resulting {@link KTable} will be materialized in a local {@link KeyValueStore} with the given
-     * {@code storeName}.
+     * {@code queryableStoreName}.
      * However, no internal changelog topic is created since the original input topic can be used for recovery (cf.
      * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}).
      * <p>
@@ -284,7 +375,7 @@ public class KStreamBuilder extends TopologyBuilder {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ...
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-key";
      * Long valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -294,13 +385,36 @@ public class KStreamBuilder extends TopologyBuilder {
      * @param offsetReset the {@code "auto.offset.reset"} policy to use for the specified topic if no valid committed
      *                    offsets are available
      * @param topic       the topic name; cannot be {@code null}
-     * @param storeName   the state store name; cannot be {@code null}
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
      * @return a {@link KTable} for the specified topic
      */
     public <K, V> KTable<K, V> table(final AutoOffsetReset offsetReset,
                                      final String topic,
-                                     final String storeName) {
-        return table(offsetReset, null, null, topic, storeName);
+                                     final StateStoreSupplier<KeyValueStore> storeSupplier) {
+        return table(offsetReset, null, null, topic, storeSupplier);
+    }
+
+    /**
+     * Create a {@link KTable} for the specified topic.
+     * The default key and value deserializers as specified in the {@link StreamsConfig config} are used.
+     * Input {@link KeyValue records} with {@code null} key will be dropped.
+     * <p>
+     * Note that the specified input topics must be partitioned by key.
+     * If this is not the case the returned {@link KTable} will be corrupted.
+     * <p>
+     * The resulting {@link KTable} will be materialized in a local {@link KeyValueStore} with an internal
+     * store name. Note that that store name may not be queriable through Interactive Queries.
+     * No internal changelog topic is created since the original input topic can be used for recovery (cf.
+     * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}).
+     * <p>
+     * @param offsetReset the {@code "auto.offset.reset"} policy to use for the specified topic if no valid committed
+     *                    offsets are available
+     * @param topic       the topic name; if {@code null} an internal store name will be automatically given.
+     * @return a {@link KTable} for the specified topic
+     */
+    public <K, V> KTable<K, V> table(final AutoOffsetReset offsetReset,
+                                     final String topic) {
+        return table(offsetReset, null, null, topic, (String) null);
     }
 
     /**
@@ -312,7 +426,7 @@ public class KStreamBuilder extends TopologyBuilder {
      * If this is not the case the returned {@link KTable} will be corrupted.
      * <p>
      * The resulting {@link KTable} will be materialized in a local {@link KeyValueStore} with the given
-     * {@code storeName}.
+     * {@code queryableStoreName}.
      * However, no internal changelog topic is created since the original input topic can be used for recovery (cf.
      * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}).
      * <p>
@@ -320,7 +434,7 @@ public class KStreamBuilder extends TopologyBuilder {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ...
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-key";
      * Long valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -332,25 +446,26 @@ public class KStreamBuilder extends TopologyBuilder {
      * @param valSerde  value serde used to send key-value pairs,
      *                  if not specified the default value serde defined in the configuration will be used
      * @param topic     the topic name; cannot be {@code null}
-     * @param storeName the state store name; cannot be {@code null}
+     * @param queryableStoreName the state store name; If {@code null} this is the equivalent of {@link KStreamBuilder#table(Serde, Serde, String)} ()}.
      * @return a {@link KTable} for the specified topic
      */
     public <K, V> KTable<K, V> table(final Serde<K> keySerde,
                                      final Serde<V> valSerde,
                                      final String topic,
-                                     final String storeName) {
-        return table(null, keySerde, valSerde, topic, storeName);
+                                     final String queryableStoreName) {
+        return table(null, keySerde, valSerde, topic, queryableStoreName);
     }
 
     /**
      * Create a {@link KTable} for the specified topic.
+     * The default {@code "auto.offset.reset"} strategy as specified in the {@link StreamsConfig config} is used.
      * Input {@link KeyValue records} with {@code null} key will be dropped.
      * <p>
      * Note that the specified input topics must be partitioned by key.
      * If this is not the case the returned {@link KTable} will be corrupted.
      * <p>
      * The resulting {@link KTable} will be materialized in a local {@link KeyValueStore} with the given
-     * {@code storeName}.
+     * {@code queryableStoreName}.
      * However, no internal changelog topic is created since the original input topic can be used for recovery (cf.
      * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}).
      * <p>
@@ -358,7 +473,94 @@ public class KStreamBuilder extends TopologyBuilder {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ...
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * String key = "some-key";
+     * Long valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
+     * }</pre>
+     * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to
+     * query the value of the key on a parallel running instance of your Kafka Streams application.
+     *
+     * @param keySerde  key serde used to send key-value pairs,
+     *                  if not specified the default key serde defined in the configuration will be used
+     * @param valSerde  value serde used to send key-value pairs,
+     *                  if not specified the default value serde defined in the configuration will be used
+     * @param topic     the topic name; cannot be {@code null}
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
+     * @return a {@link KTable} for the specified topic
+     */
+    public <K, V> KTable<K, V> table(final Serde<K> keySerde,
+                                     final Serde<V> valSerde,
+                                     final String topic,
+                                     final StateStoreSupplier<KeyValueStore> storeSupplier) {
+        return table(null, keySerde, valSerde, topic, storeSupplier);
+    }
+
+    /**
+     * Create a {@link KTable} for the specified topic.
+     * The default {@code "auto.offset.reset"} strategy as specified in the {@link StreamsConfig config} is used.
+     * Input {@link KeyValue records} with {@code null} key will be dropped.
+     * <p>
+     * Note that the specified input topics must be partitioned by key.
+     * If this is not the case the returned {@link KTable} will be corrupted.
+     * <p>
+     * The resulting {@link KTable} will be materialized in a local {@link KeyValueStore} with an internal
+     * store name. Note that that store name may not be queriable through Interactive Queries.
+     * No internal changelog topic is created since the original input topic can be used for recovery (cf.
+     * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}).
+     * <p>
+     * @param keySerde  key serde used to send key-value pairs,
+     *                  if not specified the default key serde defined in the configuration will be used
+     * @param valSerde  value serde used to send key-value pairs,
+     *                  if not specified the default value serde defined in the configuration will be used
+     * @param topic     the topic name; cannot be {@code null}
+     * @return a {@link KTable} for the specified topic
+     */
+    public <K, V> KTable<K, V> table(final Serde<K> keySerde,
+                                     final Serde<V> valSerde,
+                                     final String topic) {
+        return table(null, keySerde, valSerde, topic, (String) null);
+    }
+
+    private <K, V> KTable<K, V> doTable(final AutoOffsetReset offsetReset,
+                                        final Serde<K> keySerde,
+                                        final Serde<V> valSerde,
+                                        final String topic,
+                                        final StateStoreSupplier<KeyValueStore> storeSupplier,
+                                        final boolean isQueryable) {
+        final String source = newName(KStreamImpl.SOURCE_NAME);
+        final String name = newName(KTableImpl.SOURCE_NAME);
+        final ProcessorSupplier<K, V> processorSupplier = new KTableSource<>(storeSupplier.name());
+
+        addSource(offsetReset, source, keySerde == null ? null : keySerde.deserializer(),
+                valSerde == null ? null : valSerde.deserializer(),
+                topic);
+        addProcessor(name, processorSupplier, source);
+
+        final KTableImpl<K, ?, V> kTable = new KTableImpl<>(this, name, processorSupplier,
+                keySerde, valSerde, Collections.singleton(source), storeSupplier.name(), isQueryable);
+
+        addStateStore(storeSupplier, name);
+        connectSourceStoreAndTopic(storeSupplier.name(), topic);
+
+        return kTable;
+    }
+    /**
+     * Create a {@link KTable} for the specified topic.
+     * Input {@link KeyValue records} with {@code null} key will be dropped.
+     * <p>
+     * Note that the specified input topics must be partitioned by key.
+     * If this is not the case the returned {@link KTable} will be corrupted.
+     * <p>
+     * The resulting {@link KTable} will be materialized in a local {@link KeyValueStore} with the given
+     * {@code queryableStoreName}.
+     * However, no internal changelog topic is created since the original input topic can be used for recovery (cf.
+     * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}).
+     * <p>
+     * To query the local {@link KeyValueStore} it must be obtained via
+     * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
+     * <pre>{@code
+     * KafkaStreams streams = ...
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-key";
      * Long valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -372,37 +574,142 @@ public class KStreamBuilder extends TopologyBuilder {
      * @param valSerde    value serde used to send key-value pairs,
      *                    if not specified the default value serde defined in the configuration will be used
      * @param topic       the topic name; cannot be {@code null}
-     * @param storeName   the state store name; cannot be {@code null}
+     * @param queryableStoreName   the state store name; If {@code null} this is the equivalent of {@link KStreamBuilder#table(AutoOffsetReset, Serde, Serde, String)} ()} ()}.
      * @return a {@link KTable} for the specified topic
      */
     public <K, V> KTable<K, V> table(final AutoOffsetReset offsetReset,
                                      final Serde<K> keySerde,
                                      final Serde<V> valSerde,
                                      final String topic,
-                                     final String storeName) {
-        final String source = newName(KStreamImpl.SOURCE_NAME);
-        final String name = newName(KTableImpl.SOURCE_NAME);
-        final ProcessorSupplier<K, V> processorSupplier = new KTableSource<>(storeName);
-
-        addSource(offsetReset, source, keySerde == null ? null : keySerde.deserializer(), valSerde == null ? null : valSerde.deserializer(), topic);
-        addProcessor(name, processorSupplier, source);
-
-        final KTableImpl<K, ?, V> kTable = new KTableImpl<>(this, name, processorSupplier, Collections.singleton(source), storeName);
+                                     final String queryableStoreName) {
+        final String internalStoreName = queryableStoreName != null ? queryableStoreName : newStoreName(KTableImpl.SOURCE_NAME);
+        final StateStoreSupplier storeSupplier = new RocksDBKeyValueStoreSupplier<>(internalStoreName,
+                keySerde,
+                valSerde,
+                false,
+                Collections.<String, String>emptyMap(),
+                true);
+        return doTable(offsetReset, keySerde, valSerde, topic, storeSupplier, queryableStoreName != null);
+    }
 
-        // only materialize the KTable into a state store if the storeName is not null
-        if (storeName != null) {
-            final StateStoreSupplier storeSupplier = new RocksDBKeyValueStoreSupplier<>(storeName,
-                    keySerde,
-                    valSerde,
-                    false,
-                    Collections.<String, String>emptyMap(),
-                    true);
+    /**
+     * Create a {@link KTable} for the specified topic.
+     * The default {@code "auto.offset.reset"} strategy as specified in the {@link StreamsConfig config} is used.
+     * Input {@link KeyValue records} with {@code null} key will be dropped.
+     * <p>
+     * Note that the specified input topics must be partitioned by key.
+     * If this is not the case the returned {@link KTable} will be corrupted.
+     * <p>
+     * The resulting {@link KTable} will be materialized in a local {@link KeyValueStore} with an internal
+     * store name. Note that that store name may not be queriable through Interactive Queries.
+     * No internal changelog topic is created since the original input topic can be used for recovery (cf.
+     * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}).
+     * <p>
+     * @param offsetReset the {@code "auto.offset.reset"} policy to use for the specified topic if no valid committed
+     *                    offsets are available
+     * @param keySerde  key serde used to send key-value pairs,
+     *                  if not specified the default key serde defined in the configuration will be used
+     * @param valSerde  value serde used to send key-value pairs,
+     *                  if not specified the default value serde defined in the configuration will be used
+     * @param topic     the topic name; cannot be {@code null}
+     * @return a {@link KTable} for the specified topic
+     */
+    public <K, V> KTable<K, V> table(final AutoOffsetReset offsetReset,
+                                     final Serde<K> keySerde,
+                                     final Serde<V> valSerde,
+                                     final String topic) {
+        return table(offsetReset, keySerde, valSerde, topic, (String) null);
+    }
+    /**
+     * Create a {@link KTable} for the specified topic.
+     * Input {@link KeyValue records} with {@code null} key will be dropped.
+     * <p>
+     * Note that the specified input topics must be partitioned by key.
+     * If this is not the case the returned {@link KTable} will be corrupted.
+     * <p>
+     * The resulting {@link KTable} will be materialized in a local {@link KeyValueStore} with the given
+     * {@code queryableStoreName}.
+     * However, no internal changelog topic is created since the original input topic can be used for recovery (cf.
+     * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}).
+     * <p>
+     * To query the local {@link KeyValueStore} it must be obtained via
+     * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
+     * <pre>{@code
+     * KafkaStreams streams = ...
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * String key = "some-key";
+     * Long valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
+     * }</pre>
+     * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to
+     * query the value of the key on a parallel running instance of your Kafka Streams application.
+     *
+     * @param offsetReset the {@code "auto.offset.reset"} policy to use for the specified topic if no valid committed
+     *                    offsets are available
+     * @param keySerde    key serde used to send key-value pairs,
+     *                    if not specified the default key serde defined in the configuration will be used
+     * @param valSerde    value serde used to send key-value pairs,
+     *                    if not specified the default value serde defined in the configuration will be used
+     * @param topic       the topic name; cannot be {@code null}
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
+     * @return a {@link KTable} for the specified topic
+     */
+    public <K, V> KTable<K, V> table(final AutoOffsetReset offsetReset,
+                                     final Serde<K> keySerde,
+                                     final Serde<V> valSerde,
+                                     final String topic,
+                                     final StateStoreSupplier<KeyValueStore> storeSupplier) {
+        Objects.requireNonNull(storeSupplier, "storeSupplier can't be null");
+        return doTable(offsetReset, keySerde, valSerde, topic, storeSupplier, true);
+    }
 
-            addStateStore(storeSupplier, name);
-            connectSourceStoreAndTopic(storeName, topic);
-        }
+    /**
+     * Create a {@link GlobalKTable} for the specified topic.
+     * The default key and value deserializers as specified in the {@link StreamsConfig config} are used.
+     * Input {@link KeyValue records} with {@code null} key will be dropped.
+     * <p>
+     * The resulting {@link GlobalKTable} will be materialized in a local {@link KeyValueStore} with the given
+     * {@code queryableStoreName}.
+     * However, no internal changelog topic is created since the original input topic can be used for recovery (cf.
+     * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}).
+     * <p>
+     * To query the local {@link KeyValueStore} it must be obtained via
+     * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
+     * <pre>{@code
+     * KafkaStreams streams = ...
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * String key = "some-key";
+     * Long valueForKey = localStore.get(key);
+     * }</pre>
+     * Note that {@link GlobalKTable} always applies {@code "auto.offset.reset"} strategy {@code "earliest"}
+     * regardless of the specified value in {@link StreamsConfig}.
+     *
+     * @param topic     the topic name; cannot be {@code null}
+     * @param queryableStoreName the state store name; If {@code null} this is the equivalent of {@link KStreamBuilder#globalTable(String)}
+     * @return a {@link GlobalKTable} for the specified topic
+     */
+    public <K, V> GlobalKTable<K, V> globalTable(final String topic,
+                                                 final String queryableStoreName) {
+        return globalTable(null, null, topic, queryableStoreName);
+    }
 
-        return kTable;
+    /**
+     * Create a {@link GlobalKTable} for the specified topic.
+     * The default key and value deserializers as specified in the {@link StreamsConfig config} are used.
+     * Input {@link KeyValue records} with {@code null} key will be dropped.
+     * <p>
+     * The resulting {@link GlobalKTable} will be materialized in a local {@link KeyValueStore} with an internal
+     * store name. Note that that store name may not be queriable through Interactive Queries.
+     * No internal changelog topic is created since the original input topic can be used for recovery (cf.
+     * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}).
+     * <p>
+     * Note that {@link GlobalKTable} always applies {@code "auto.offset.reset"} strategy {@code "earliest"}
+     * regardless of the specified value in {@link StreamsConfig}.
+     *
+     * @param topic     the topic name; cannot be {@code null}
+     * @return a {@link GlobalKTable} for the specified topic
+     */
+    public <K, V> GlobalKTable<K, V> globalTable(final String topic) {
+        return globalTable(null, null, topic, (String) null);
     }
 
     /**
@@ -411,7 +718,7 @@ public class KStreamBuilder extends TopologyBuilder {
      * Input {@link KeyValue records} with {@code null} key will be dropped.
      * <p>
      * The resulting {@link GlobalKTable} will be materialized in a local {@link KeyValueStore} with the given
-     * {@code storeName}.
+     * {@code queryableStoreName}.
      * However, no internal changelog topic is created since the original input topic can be used for recovery (cf.
      * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}).
      * <p>
@@ -419,20 +726,33 @@ public class KStreamBuilder extends TopologyBuilder {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ...
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-key";
      * Long valueForKey = localStore.get(key);
      * }</pre>
      * Note that {@link GlobalKTable} always applies {@code "auto.offset.reset"} strategy {@code "earliest"}
      * regardless of the specified value in {@link StreamsConfig}.
      *
+     * @param keySerde  key serde used to send key-value pairs,
+     *                  if not specified the default key serde defined in the configuration will be used
+     * @param valSerde  value serde used to send key-value pairs,
+     *                  if not specified the default value serde defined in the configuration will be used
      * @param topic     the topic name; cannot be {@code null}
-     * @param storeName the state store name; cannot be {@code null}
+     * @param queryableStoreName the state store name; If {@code null} this is the equivalent of {@link KStreamBuilder#globalTable(Serde, Serde, String)} ()}
      * @return a {@link GlobalKTable} for the specified topic
      */
-    public <K, V> GlobalKTable<K, V> globalTable(final String topic,
-                                                 final String storeName) {
-        return globalTable(null, null, topic, storeName);
+    @SuppressWarnings("unchecked")
+    public <K, V> GlobalKTable<K, V> globalTable(final Serde<K> keySerde,
+                                                 final Serde<V> valSerde,
+                                                 final String topic,
+                                                 final String queryableStoreName) {
+        final String internalStoreName = queryableStoreName != null ? queryableStoreName : newStoreName(KTableImpl.SOURCE_NAME);
+        return doGlobalTable(keySerde, valSerde, topic, new RocksDBKeyValueStoreSupplier<>(internalStoreName,
+                            keySerde,
+                            valSerde,
+                    false,
+                            Collections.<String, String>emptyMap(),
+                    true));
     }
 
     /**
@@ -441,7 +761,7 @@ public class KStreamBuilder extends TopologyBuilder {
      * Input {@link KeyValue records} with {@code null} key will be dropped.
      * <p>
      * The resulting {@link GlobalKTable} will be materialized in a local {@link KeyValueStore} with the given
-     * {@code storeName}.
+     * {@code queryableStoreName}.
      * However, no internal changelog topic is created since the original input topic can be used for recovery (cf.
      * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}).
      * <p>
@@ -449,7 +769,7 @@ public class KStreamBuilder extends TopologyBuilder {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ...
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-key";
      * Long valueForKey = localStore.get(key);
      * }</pre>
@@ -461,31 +781,60 @@ public class KStreamBuilder extends TopologyBuilder {
      * @param valSerde  value serde used to send key-value pairs,
      *                  if not specified the default value serde defined in the configuration will be used
      * @param topic     the topic name; cannot be {@code null}
-     * @param storeName the state store name; cannot be {@code null}
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
      * @return a {@link GlobalKTable} for the specified topic
      */
     @SuppressWarnings("unchecked")
     public <K, V> GlobalKTable<K, V> globalTable(final Serde<K> keySerde,
                                                  final Serde<V> valSerde,
                                                  final String topic,
-                                                 final String storeName) {
+                                                 final StateStoreSupplier<KeyValueStore> storeSupplier) {
+        return doGlobalTable(keySerde, valSerde, topic, storeSupplier);
+    }
+
+    private <K, V> GlobalKTable<K, V> doGlobalTable(final Serde<K> keySerde,
+                                                    final Serde<V> valSerde,
+                                                    final String topic,
+                                                    final StateStoreSupplier<KeyValueStore> storeSupplier) {
+        Objects.requireNonNull(storeSupplier, "storeSupplier can't be null");
         final String sourceName = newName(KStreamImpl.SOURCE_NAME);
         final String processorName = newName(KTableImpl.SOURCE_NAME);
-        final KTableSource<K, V> tableSource = new KTableSource<>(storeName);
+        final KTableSource<K, V> tableSource = new KTableSource<>(storeSupplier.name());
 
 
         final Deserializer<K> keyDeserializer = keySerde == null ? null : keySerde.deserializer();
         final Deserializer<V> valueDeserializer = valSerde == null ? null : valSerde.deserializer();
 
-        final StateStore store = new RocksDBKeyValueStoreSupplier<>(storeName,
-                                                                    keySerde,
-                                                                    valSerde,
-                                                                    false,
-                                                                    Collections.<String, String>emptyMap(),
-                                                                    true).get();
+        addGlobalStore(storeSupplier, sourceName, keyDeserializer, valueDeserializer, topic, processorName, tableSource);
+        return new GlobalKTableImpl(new KTableSourceValueGetterSupplier<>(storeSupplier.name()));
+    }
 
-        addGlobalStore(store, sourceName, keyDeserializer, valueDeserializer, topic, processorName, tableSource);
-        return new GlobalKTableImpl(new KTableSourceValueGetterSupplier<>(storeName));
+    /**
+     * Create a {@link GlobalKTable} for the specified topic.
+     * The default key and value deserializers as specified in the {@link StreamsConfig config} are used.
+     * Input {@link KeyValue records} with {@code null} key will be dropped.
+     * <p>
+     * The resulting {@link GlobalKTable} will be materialized in a local {@link KeyValueStore} with an internal
+     * store name. Note that that store name may not be queriable through Interactive Queries.
+     * No internal changelog topic is created since the original input topic can be used for recovery (cf.
+     * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}).
+     * <p>
+     * Note that {@link GlobalKTable} always applies {@code "auto.offset.reset"} strategy {@code "earliest"}
+     * regardless of the specified value in {@link StreamsConfig}.
+     *
+     * @param keySerde  key serde used to send key-value pairs,
+     *                  if not specified the default key serde defined in the configuration will be used
+     * @param valSerde  value serde used to send key-value pairs,
+     *                  if not specified the default value serde defined in the configuration will be used
+     * @param topic     the topic name; cannot be {@code null}
+     * @return a {@link GlobalKTable} for the specified topic
+     */
+    @SuppressWarnings("unchecked")
+    public <K, V> GlobalKTable<K, V> globalTable(final Serde<K> keySerde,
+                                                 final Serde<V> valSerde,
+                                                 final String topic) {
+
+        return globalTable(keySerde, valSerde, topic, (String) null);
     }
 
     /**
@@ -512,4 +861,16 @@ public class KStreamBuilder extends TopologyBuilder {
         return prefix + String.format("%010d", index.getAndIncrement());
     }
 
+    /**
+     * <strong>This function is only for internal usage only and should not be called.</strong>
+     * <p>
+     * Create a unique state store name.
+     *
+     * @param prefix processor name prefix
+     * @return a new unique name
+     */
+    public String newStoreName(final String prefix) {
+        return prefix + String.format(KTableImpl.STATE_STORE_NAME + "%010d", index.getAndIncrement());
+    }
+
 }


[5/5] kafka git commit: KAFKA-5045: Clarify on KTable APIs for queryable stores

Posted by gu...@apache.org.
KAFKA-5045: Clarify on KTable APIs for queryable stores

This is the implementation of KIP-114: KTable state stores and improved semantics:
- Allow for decoupling between querying and materialisation
- consistent APIs, overloads with queryableName and without
- depreciated several KTable calls
- new unit and integration tests

In this implementation, state stores are materialized if the user desires them to be queryable. In subsequent versions we can offer a second option, to have a view-like state store. The tradeoff then would be between storage space (materialize) and re-computation (view). That tradeoff can be exploited by later query optimizers.

Author: Eno Thereska <en...@gmail.com>

Reviewers: Damian Guy, Matthias J. Sax, Guozhang Wang

Closes #2832 from enothereska/KAFKA-5045-ktable


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/ec9e4eaf
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/ec9e4eaf
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/ec9e4eaf

Branch: refs/heads/trunk
Commit: ec9e4eafa406fec897713310bafdedf6bbb3c0c5
Parents: a3952ae
Author: Eno Thereska <en...@gmail.com>
Authored: Wed May 3 16:15:54 2017 -0700
Committer: Guozhang Wang <wa...@gmail.com>
Committed: Wed May 3 16:15:54 2017 -0700

----------------------------------------------------------------------
 .../kafka/streams/kstream/KGroupedStream.java   |  552 +++++++--
 .../kafka/streams/kstream/KGroupedTable.java    |  323 +++++-
 .../kafka/streams/kstream/KStreamBuilder.java   |  479 +++++++-
 .../apache/kafka/streams/kstream/KTable.java    | 1094 +++++++++++++++++-
 .../kstream/internals/AbstractStream.java       |    7 +
 .../kstream/internals/KGroupedStreamImpl.java   |  128 +-
 .../kstream/internals/KGroupedTableImpl.java    |  113 +-
 .../streams/kstream/internals/KStreamImpl.java  |    2 +-
 .../streams/kstream/internals/KTableFilter.java |   26 +-
 .../streams/kstream/internals/KTableImpl.java   |  326 +++++-
 .../internals/KTableKTableJoinMerger.java       |   35 +-
 .../kstream/internals/KTableMapValues.java      |   27 +-
 .../streams/processor/TopologyBuilder.java      |   20 +-
 .../KStreamAggregationIntegrationTest.java      |   28 +-
 .../KTableKTableJoinIntegrationTest.java        |  155 ++-
 .../QueryableStateIntegrationTest.java          |  178 +++
 .../streams/kstream/KStreamBuilderTest.java     |   55 +-
 .../internals/KGroupedStreamImplTest.java       |  279 +++--
 .../internals/KGroupedTableImplTest.java        |   72 +-
 .../kstream/internals/KTableAggregateTest.java  |   55 +-
 .../kstream/internals/KTableFilterTest.java     |  245 +++-
 .../kstream/internals/KTableImplTest.java       |   32 +-
 .../kstream/internals/KTableKTableJoinTest.java |  162 +--
 .../kstream/internals/KTableMapValuesTest.java  |  128 +-
 .../streams/processor/TopologyBuilderTest.java  |    2 +-
 .../internals/ProcessorTopologyTest.java        |    4 +-
 26 files changed, 3793 insertions(+), 734 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java
----------------------------------------------------------------------
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java
index c961c7e..2cdf047 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java
@@ -49,7 +49,7 @@ public interface KGroupedStream<K, V> {
      * Count the number of records in this stream by the grouped key.
      * Records with {@code null} key or value are ignored.
      * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view)
-     * that can be queried using the provided {@code storeName}.
+     * that can be queried using the provided {@code queryableStoreName}.
      * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream.
      * <p>
      * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
@@ -63,7 +63,7 @@ public interface KGroupedStream<K, V> {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ... // counting words
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-word";
      * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -73,18 +73,44 @@ public interface KGroupedStream<K, V> {
      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
      * Therefore, the store name must be a valid Kafka topic name and cannot contain characters other than ASCII
      * alphanumerics, '.', '_' and '-'.
-     * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
+     * The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
      * user-specified in {@link StreamsConfig} via parameter
-     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the
-     * provide {@code storeName}, and "-changelog" is a fixed suffix.
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "queryableStoreName" is the
+     * provide {@code queryableStoreName}, and "-changelog" is a fixed suffix.
      * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
      *
-     * @param storeName the name of the underlying {@link KTable} state store; valid characters are ASCII
-     *                  alphanumerics, '.', '_' and '-'
+     * @param queryableStoreName the name of the underlying {@link KTable} state store; valid characters are ASCII
+     * alphanumerics, '.', '_' and '-'. If {@code null} then this will be equivalent to {@link KGroupedStream#count()}.
      * @return a {@link KTable} that contains "update" records with unmodified keys and {@link Long} values that
      * represent the latest (rolling) count (i.e., number of records) for each key
      */
-    KTable<K, Long> count(final String storeName);
+    KTable<K, Long> count(final String queryableStoreName);
+
+    /**
+     * Count the number of records in this stream by the grouped key.
+     * Records with {@code null} key or value are ignored.
+     * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view).
+     * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream.
+     * <p>
+     * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
+     * the same key.
+     * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
+     * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
+     * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
+     * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
+     * <p>
+     * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
+     * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
+     * user-specified in {@link StreamsConfig} via parameter
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "internalStoreName" is an internal name
+     * and "-changelog" is a fixed suffix.
+     * Note that the internal store name may not be queriable through Interactive Queries.
+     * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
+     *
+     * @return a {@link KTable} that contains "update" records with unmodified keys and {@link Long} values that
+     * represent the latest (rolling) count (i.e., number of records) for each key
+     */
+    KTable<K, Long> count();
 
     /**
      * Count the number of records in this stream by the grouped key.
@@ -105,15 +131,15 @@ public interface KGroupedStream<K, V> {
      * Use {@link StateStoreSupplier#name()} to get the store name:
      * <pre>{@code
      * KafkaStreams streams = ... // counting words
-     * String storeName = storeSupplier.name();
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * String queryableStoreName = storeSupplier.name();
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-word";
      * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
      * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to
      * query the value of the key on a parallel running instance of your Kafka Streams application.
      *
-     * @param storeSupplier user defined state store supplier
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
      * @return a {@link KTable} that contains "update" records with unmodified keys and {@link Long} values that
      * represent the latest (rolling) count (i.e., number of records) for each key
      */
@@ -125,7 +151,7 @@ public interface KGroupedStream<K, V> {
      * The specified {@code windows} define either hopping time windows that can be overlapping or tumbling (c.f.
      * {@link TimeWindows}) or they define landmark windows (c.f. {@link UnlimitedWindows}).
      * The result is written into a local windowed {@link KeyValueStore} (which is basically an ever-updating
-     * materialized view) that can be queried using the provided {@code storeName}.
+     * materialized view) that can be queried using the provided {@code queryableStoreName}.
      * Windows are retained until their retention time expires (c.f. {@link Windows#until(long)}).
      * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
      * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
@@ -141,7 +167,7 @@ public interface KGroupedStream<K, V> {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ... // counting words
-     * ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(storeName, QueryableStoreTypes.<String, Long>windowStore());
+     * ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore());
      * String key = "some-word";
      * long fromTime = ...;
      * long toTime = ...;
@@ -153,20 +179,52 @@ public interface KGroupedStream<K, V> {
      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
      * Therefore, the store name must be a valid Kafka topic name and cannot contain characters other than ASCII
      * alphanumerics, '.', '_' and '-'.
-     * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
+     * The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
      * user-specified in {@link StreamsConfig StreamsConfig} via parameter
-     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the
-     * provide {@code storeName}, and "-changelog" is a fixed suffix.
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "queryableStoreName" is the
+     * provide {@code queryableStoreName}, and "-changelog" is a fixed suffix.
      * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
      *
      * @param windows   the specification of the aggregation {@link Windows}
-     * @param storeName the name of the underlying {@link KTable} state store; valid characters are ASCII
-     *                  alphanumerics, '.', '_' and '-'
+     * @param queryableStoreName the name of the underlying {@link KTable} state store; valid characters are ASCII
+     * alphanumerics, '.', '_' and '-'. If {@code null} then this will be equivalent to {@link KGroupedStream#count(Windows)}.
      * @return a windowed {@link KTable} that contains "update" records with unmodified keys and {@link Long} values
      * that represent the latest (rolling) count (i.e., number of records) for each key within a window
      */
     <W extends Window> KTable<Windowed<K>, Long> count(final Windows<W> windows,
-                                                       final String storeName);
+                                                       final String queryableStoreName);
+
+    /**
+     * Count the number of records in this stream by the grouped key and the defined windows.
+     * Records with {@code null} key or value are ignored.
+     * The specified {@code windows} define either hopping time windows that can be overlapping or tumbling (c.f.
+     * {@link TimeWindows}) or they define landmark windows (c.f. {@link UnlimitedWindows}).
+     * The result is written into a local windowed {@link KeyValueStore} (which is basically an ever-updating
+     * materialized view) that can be queried using the provided {@code queryableName}.
+     * Windows are retained until their retention time expires (c.f. {@link Windows#until(long)}).
+     * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
+     * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
+     * <p>
+     * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
+     * the same window and key.
+     * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
+     * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
+     * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
+     * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
+     * <p>
+     * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
+     * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
+     * user-specified in {@link StreamsConfig StreamsConfig} via parameter
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "internalStoreName" is an internal name
+     * and "-changelog" is a fixed suffix.
+     * Note that the internal store name may not be queriable through Interactive Queries.
+     * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
+     *
+     * @param windows   the specification of the aggregation {@link Windows}
+     * @return a windowed {@link KTable} that contains "update" records with unmodified keys and {@link Long} values
+     * that represent the latest (rolling) count (i.e., number of records) for each key within a window
+     */
+    <W extends Window> KTable<Windowed<K>, Long> count(final Windows<W> windows);
 
     /**
      * Count the number of records in this stream by the grouped key and the defined windows.
@@ -191,8 +249,8 @@ public interface KGroupedStream<K, V> {
      * Use {@link StateStoreSupplier#name()} to get the store name:
      * <pre>{@code
      * KafkaStreams streams = ... // counting words
-     * String storeName = storeSupplier.name();
-     * ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(storeName, QueryableStoreTypes.<String, Long>windowStore());
+     * String queryableStoreName = storeSupplier.name();
+     * ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableName, QueryableStoreTypes.<String, Long>windowStore());
      * String key = "some-word";
      * long fromTime = ...;
      * long toTime = ...;
@@ -202,7 +260,7 @@ public interface KGroupedStream<K, V> {
      * query the value of the key on a parallel running instance of your Kafka Streams application.
      *
      * @param windows       the specification of the aggregation {@link Windows}
-     * @param storeSupplier user defined state store supplier
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
      * @return a windowed {@link KTable} that contains "update" records with unmodified keys and {@link Long} values
      * that represent the latest (rolling) count (i.e., number of records) for each key within a window
      */
@@ -214,7 +272,7 @@ public interface KGroupedStream<K, V> {
      * Count the number of records in this stream by the grouped key into {@link SessionWindows}.
      * Records with {@code null} key or value are ignored.
      * The result is written into a local {@link SessionStore} (which is basically an ever-updating
-     * materialized view) that can be queried using the provided {@code storeName}.
+     * materialized view) that can be queried using the provided {@code queryableStoreName}.
      * SessionWindows are retained until their retention time expires (c.f. {@link SessionWindows#until(long)}).
      * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
      * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
@@ -226,27 +284,48 @@ public interface KGroupedStream<K, V> {
      * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
      * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
      * <p>
-     * To query the local {@link SessionStore} it must be obtained via
+     * To query the local windowed {@link KeyValueStore} it must be obtained via
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}.
      * Use {@link StateStoreSupplier#name()} to get the store name:
      * <pre>{@code
-     * KafkaStreams streams = ... // counting words
-     * String storeName = storeSupplier.name();
-     * ReadOnlySessionStore<String,Long> sessionStore = streams.store(storeName, QueryableStoreTypes.<String, Long>sessionStore());
-     * String key = "some-word";
-     * KeyValueIterator<Windowed<String>, Long> iterator = sessionStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
+     * KafkaStreams streams = ... // compute sum
+     * Sting queryableStoreName = storeSupplier.name();
+     * ReadOnlySessionStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>ReadOnlySessionStore<String, Long>);
+     * String key = "some-key";
+     * KeyValueIterator<Windowed<String>, Long> sumForKeyForWindows = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
      * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to
      * query the value of the key on a parallel running instance of your Kafka Streams application.
      *
+     * @param sessionWindows the specification of the aggregation {@link SessionWindows}
+     * @param queryableStoreName  the name of the state store created from this operation; valid characters are ASCII
+     * alphanumerics, '.', '_' and '-. If {@code null} then this will be equivalent to {@link KGroupedStream#count(SessionWindows)} ()}.
+     * @return a windowed {@link KTable} that contains "update" records with unmodified keys and {@link Long} values
+     * that represent the latest (rolling) count (i.e., number of records) for each key within a window
+     */
+    KTable<Windowed<K>, Long> count(final SessionWindows sessionWindows, final String queryableStoreName);
+
+    /**
+     * Count the number of records in this stream by the grouped key into {@link SessionWindows}.
+     * Records with {@code null} key or value are ignored.
+     * The result is written into a local {@link SessionStore} (which is basically an ever-updating
+     * materialized view) that can be queried using the provided {@code queryableStoreName}.
+     * SessionWindows are retained until their retention time expires (c.f. {@link SessionWindows#until(long)}).
+     * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
+     * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
+     * <p>
+     * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
+     * the same window and key.
+     * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
+     * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
+     * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
+     * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
      *
      * @param sessionWindows the specification of the aggregation {@link SessionWindows}
-     * @param storeName      the name of the state store created from this operation; valid characters are ASCII
-     *                       alphanumerics, '.', '_' and '-
      * @return a windowed {@link KTable} that contains "update" records with unmodified keys and {@link Long} values
      * that represent the latest (rolling) count (i.e., number of records) for each key within a window
      */
-    KTable<Windowed<K>, Long> count(final SessionWindows sessionWindows, final String storeName);
+    KTable<Windowed<K>, Long> count(final SessionWindows sessionWindows);
 
     /**
      * Count the number of records in this stream by the grouped key into {@link SessionWindows}.
@@ -264,22 +343,21 @@ public interface KGroupedStream<K, V> {
      * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
      * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
      * <p>
-     * To query the local {@link SessionStore} it must be obtained via
+     * To query the local windowed {@link KeyValueStore} it must be obtained via
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}.
      * Use {@link StateStoreSupplier#name()} to get the store name:
      * <pre>{@code
-     * KafkaStreams streams = ... // counting words
-     * String storeName = storeSupplier.name();
-     * ReadOnlySessionStore<String,Long> sessionStore = streams.store(storeName, QueryableStoreTypes.<String, Long>sessionStore());
-     * String key = "some-word";
-     * KeyValueIterator<Windowed<String>, Long> iterator = sessionStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
+     * KafkaStreams streams = ... // compute sum
+     * Sting queryableStoreName = storeSupplier.name();
+     * ReadOnlySessionStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>ReadOnlySessionStore<String, Long>);
+     * String key = "some-key";
+     * KeyValueIterator<Windowed<String>, Long> sumForKeyForWindows = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
      * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to
      * query the value of the key on a parallel running instance of your Kafka Streams application.
      *
-     *
      * @param sessionWindows the specification of the aggregation {@link SessionWindows}
-     * @param storeSupplier  user defined state store supplier
+     * @param storeSupplier  user defined state store supplier. Cannot be {@code null}.
      * @return a windowed {@link KTable} that contains "update" records with unmodified keys and {@link Long} values
      * that represent the latest (rolling) count (i.e., number of records) for each key within a window
      */
@@ -292,7 +370,42 @@ public interface KGroupedStream<K, V> {
      * Combining implies that the type of the aggregate result is the same as the type of the input value
      * (c.f. {@link #aggregate(Initializer, Aggregator, Serde, String)}).
      * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view)
-     * that can be queried using the provided {@code storeName}.
+     * that can be queried using the provided {@code queryableStoreName}.
+     * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream.
+     * <p>
+     * The specified {@link Reducer} is applied for each input record and computes a new aggregate using the current
+     * aggregate and the record's value.
+     * If there is no current aggregate the {@link Reducer} is not applied and the new aggregate will be the record's
+     * value as-is.
+     * Thus, {@code reduce(Reducer, String)} can be used to compute aggregate functions like sum, min, or max.
+     * <p>
+     * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
+     * the same key.
+     * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
+     * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
+     * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
+     * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
+     * <p>
+     * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
+     * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
+     * user-specified in {@link StreamsConfig} via parameter
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "internalStoreName" is an internal name
+     * and "-changelog" is a fixed suffix.
+     * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
+     *
+     * @param reducer   a {@link Reducer} that computes a new aggregate result
+     * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the
+     * latest (rolling) aggregate for each key
+     */
+    KTable<K, V> reduce(final Reducer<V> reducer);
+
+    /**
+     * Combine the values of records in this stream by the grouped key.
+     * Records with {@code null} key or value are ignored.
+     * Combining implies that the type of the aggregate result is the same as the type of the input value
+     * (c.f. {@link #aggregate(Initializer, Aggregator, Serde, String)}).
+     * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view)
+     * that can be queried using the provided {@code queryableStoreName}.
      * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream.
      * <p>
      * The specified {@link Reducer} is applied for each input record and computes a new aggregate using the current
@@ -312,7 +425,7 @@ public interface KGroupedStream<K, V> {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ... // compute sum
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-key";
      * Long sumForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -322,20 +435,21 @@ public interface KGroupedStream<K, V> {
      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
      * Therefore, the store name must be a valid Kafka topic name and cannot contain characters other than ASCII
      * alphanumerics, '.', '_' and '-'.
-     * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
+     * The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
      * user-specified in {@link StreamsConfig} via parameter
-     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the
-     * provide {@code storeName}, and "-changelog" is a fixed suffix.
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "queryableStoreName" is the
+     * provide {@code queryableStoreName}, and "-changelog" is a fixed suffix.
      * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
      *
      * @param reducer   a {@link Reducer} that computes a new aggregate result
-     * @param storeName the name of the underlying {@link KTable} state store; valid characters are ASCII
-     *                  alphanumerics, '.', '_' and '-'
+     * @param queryableStoreName the name of the underlying {@link KTable} state store; valid characters are ASCII
+     * alphanumerics, '.', '_' and '-'. If {@code null} then this will be equivalent to {@link KGroupedStream#reduce(Reducer)} ()}.
      * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the
      * latest (rolling) aggregate for each key
      */
     KTable<K, V> reduce(final Reducer<V> reducer,
-                        final String storeName);
+                        final String queryableStoreName);
+
 
     /**
      * Combine the value of records in this stream by the grouped key.
@@ -365,8 +479,8 @@ public interface KGroupedStream<K, V> {
      * Use {@link StateStoreSupplier#name()} to get the store name:
      * <pre>{@code
      * KafkaStreams streams = ... // compute sum
-     * String storeName = storeSupplier.name();
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * String queryableStoreName = storeSupplier.name();
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-key";
      * Long sumForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -374,7 +488,7 @@ public interface KGroupedStream<K, V> {
      * query the value of the key on a parallel running instance of your Kafka Streams application.
      *
      * @param reducer   a {@link Reducer} that computes a new aggregate result
-     * @param storeSupplier user defined state store supplier
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
      * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the
      * latest (rolling) aggregate for each key
      */
@@ -389,7 +503,7 @@ public interface KGroupedStream<K, V> {
      * The specified {@code windows} define either hopping time windows that can be overlapping or tumbling (c.f.
      * {@link TimeWindows}) or they define landmark windows (c.f. {@link UnlimitedWindows}).
      * The result is written into a local windowed {@link KeyValueStore} (which is basically an ever-updating
-     * materialized view) that can be queried using the provided {@code storeName}.
+     * materialized view) that can be queried using the provided {@code queryableStoreName}.
      * Windows are retained until their retention time expires (c.f. {@link Windows#until(long)}).
      * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
      * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
@@ -411,7 +525,7 @@ public interface KGroupedStream<K, V> {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ... // compute sum
-     * ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(storeName, QueryableStoreTypes.<String, Long>windowStore());
+     * ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore());
      * String key = "some-key";
      * long fromTime = ...;
      * long toTime = ...;
@@ -423,22 +537,64 @@ public interface KGroupedStream<K, V> {
      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
      * Therefore, the store name must be a valid Kafka topic name and cannot contain characters other than ASCII
      * alphanumerics, '.', '_' and '-'.
-     * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
+     * The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
      * user-specified in {@link StreamsConfig} via parameter
-     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the
-     * provide {@code storeName}, and "-changelog" is a fixed suffix.
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "queryableStoreName" is the
+     * provide {@code queryableStoreName}, and "-changelog" is a fixed suffix.
      * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
      *
      * @param reducer   a {@link Reducer} that computes a new aggregate result
      * @param windows   the specification of the aggregation {@link Windows}
-     * @param storeName the name of the state store created from this operation; valid characters are ASCII
-     *                  alphanumerics, '.', '_' and '-'
+     * @param queryableStoreName the name of the state store created from this operation; valid characters are ASCII
+     * alphanumerics, '.', '_' and '-'. If {@code null} then this will be equivalent to {@link KGroupedStream#reduce(Reducer, Windows)} ()}.
      * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent
      * the latest (rolling) aggregate for each key within a window
      */
     <W extends Window> KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
                                                      final Windows<W> windows,
-                                                     final String storeName);
+                                                     final String queryableStoreName);
+
+    /**
+     * Combine the number of records in this stream by the grouped key and the defined windows.
+     * Records with {@code null} key or value are ignored.
+     * Combining implies that the type of the aggregate result is the same as the type of the input value
+     * (c.f. {@link #aggregate(Initializer, Aggregator, Windows, Serde, String)}).
+     * The specified {@code windows} define either hopping time windows that can be overlapping or tumbling (c.f.
+     * {@link TimeWindows}) or they define landmark windows (c.f. {@link UnlimitedWindows}).
+     * The result is written into a local windowed {@link KeyValueStore} (which is basically an ever-updating
+     * materialized view) that can be queried using the provided {@code queryableStoreName}.
+     * Windows are retained until their retention time expires (c.f. {@link Windows#until(long)}).
+     * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
+     * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
+     * <p>
+     * The specified {@link Reducer} is applied for each input record and computes a new aggregate using the current
+     * aggregate and the record's value.
+     * If there is no current aggregate the {@link Reducer} is not applied and the new aggregate will be the record's
+     * value as-is.
+     * Thus, {@code reduce(Reducer, Windows, String)} can be used to compute aggregate functions like sum, min, or max.
+     * <p>
+     * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
+     * the same window and key.
+     * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
+     * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
+     * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
+     * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
+     * <p>
+     * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
+     * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
+     * user-specified in {@link StreamsConfig} via parameter
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "internalStoreName" is an internal name
+     * and "-changelog" is a fixed suffix.
+     * Note that the internal store name may not be queriable through Interactive Queries.
+     * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
+     *
+     * @param reducer   a {@link Reducer} that computes a new aggregate result
+     * @param windows   the specification of the aggregation {@link Windows}
+     * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent
+     * the latest (rolling) aggregate for each key within a window
+     */
+    <W extends Window> KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
+                                                     final Windows<W> windows);
 
     /**
      * Combine the values of records in this stream by the grouped key and the defined windows.
@@ -472,8 +628,8 @@ public interface KGroupedStream<K, V> {
      * Use {@link StateStoreSupplier#name()} to get the store name:
      * <pre>{@code
      * KafkaStreams streams = ... // compute sum
-     * Sting storeName = storeSupplier.name();
-     * ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(storeName, QueryableStoreTypes.<String, Long>windowStore());
+     * Sting queryableStoreName = storeSupplier.name();
+     * ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore());
      * String key = "some-key";
      * long fromTime = ...;
      * long toTime = ...;
@@ -484,7 +640,7 @@ public interface KGroupedStream<K, V> {
      *
      * @param reducer       a {@link Reducer} that computes a new aggregate result
      * @param windows       the specification of the aggregation {@link Windows}
-     * @param storeSupplier user defined state store supplier
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
      * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent
      * the latest (rolling) aggregate for each key within a window
      */
@@ -498,7 +654,7 @@ public interface KGroupedStream<K, V> {
      * Combining implies that the type of the aggregate result is the same as the type of the input value
      * (c.f. {@link #aggregate(Initializer, Aggregator, Merger, SessionWindows, Serde, String)}).
      * The result is written into a local {@link SessionStore} (which is basically an ever-updating
-     * materialized view) that can be queried using the provided {@code storeName}.
+     * materialized view) that can be queried using the provided {@code queryableStoreName}.
      * SessionWindows are retained until their retention time expires (c.f. {@link SessionWindows#until(long)}).
      * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
      * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
@@ -517,13 +673,15 @@ public interface KGroupedStream<K, V> {
      * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
      * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
      * <p>
-     * To query the local {@link SessionStore} it must be obtained via
-     * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
+     * To query the local windowed {@link KeyValueStore} it must be obtained via
+     * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}.
+     * Use {@link StateStoreSupplier#name()} to get the store name:
      * <pre>{@code
      * KafkaStreams streams = ... // compute sum
-     * ReadOnlySessionStore<String,Long> sessionStore = streams.store(storeName, QueryableStoreTypes.<String, Long>sessionStore());
+     * Sting queryableStoreName = storeSupplier.name();
+     * ReadOnlySessionStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>ReadOnlySessionStore<String, Long>);
      * String key = "some-key";
-     * KeyValueIterator<Windowed<String>, Long> sumForKeyForSession = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
+     * KeyValueIterator<Windowed<String>, Long> sumForKeyForWindows = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
      * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to
      * query the value of the key on a parallel running instance of your Kafka Streams application.
@@ -531,21 +689,53 @@ public interface KGroupedStream<K, V> {
      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
      * Therefore, the store name must be a valid Kafka topic name and cannot contain characters other than ASCII
      * alphanumerics, '.', '_' and '-'.
-     * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
+     * The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
      * user-specified in {@link StreamsConfig} via parameter
-     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the
-     * provide {@code storeName}, and "-changelog" is a fixed suffix.
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "queryableStoreName" is the
+     * provide {@code queryableStoreName}, and "-changelog" is a fixed suffix.
      * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
      * @param reducer           the instance of {@link Reducer}
      * @param sessionWindows    the specification of the aggregation {@link SessionWindows}
-     * @param storeName         the name of the state store created from this operation; valid characters are ASCII
-     *                          alphanumerics, '.', '_' and '-'
+     * @param queryableStoreName     the name of the state store created from this operation; valid characters are ASCII
+     * alphanumerics, '.', '_' and '-'. If {@code null} then this will be equivalent to {@link KGroupedStream#reduce(Reducer, SessionWindows)} ()}.
      * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent
      * the latest (rolling) aggregate for each key within a window
      */
     KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
                                   final SessionWindows sessionWindows,
-                                  final String storeName);
+                                  final String queryableStoreName);
+
+    /**
+     * Combine values of this stream by the grouped key into {@link SessionWindows}.
+     * Records with {@code null} key or value are ignored.
+     * Combining implies that the type of the aggregate result is the same as the type of the input value
+     * (c.f. {@link #aggregate(Initializer, Aggregator, Merger, SessionWindows, Serde, String)}).
+     * The result is written into a local {@link SessionStore} (which is basically an ever-updating
+     * materialized view) that can be queried using the provided {@code queryableStoreName}.
+     * SessionWindows are retained until their retention time expires (c.f. {@link SessionWindows#until(long)}).
+     * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
+     * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
+     * <p>
+     * The specified {@link Reducer} is applied for each input record and computes a new aggregate using the current
+     * aggregate and the record's value.
+     * If there is no current aggregate the {@link Reducer} is not applied and the new aggregate will be the record's
+     * value as-is.
+     * Thus, {@code reduce(Reducer, SessionWindows, String)} can be used to compute aggregate functions like sum, min,
+     * or max.
+     * <p>
+     * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
+     * the same window and key.
+     * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
+     * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
+     * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
+     * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
+     * @param reducer           the instance of {@link Reducer}
+     * @param sessionWindows    the specification of the aggregation {@link SessionWindows}
+     * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent
+     * the latest (rolling) aggregate for each key within a window
+     */
+    KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
+                                  final SessionWindows sessionWindows);
 
     /**
      * Combine values of this stream by the grouped key into {@link SessionWindows}.
@@ -572,15 +762,15 @@ public interface KGroupedStream<K, V> {
      * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
      * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
      * <p>
-     * To query the local {@link SessionStore} it must be obtained via
+     * To query the local windowed {@link KeyValueStore} it must be obtained via
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}.
      * Use {@link StateStoreSupplier#name()} to get the store name:
      * <pre>{@code
      * KafkaStreams streams = ... // compute sum
-     * Sting storeName = storeSupplier.name();
-     * ReadOnlySessionStore<String,Long> sessionStore = streams.store(storeName, QueryableStoreTypes.<String, Long>sessionStore());
+     * Sting queryableStoreName = storeSupplier.name();
+     * ReadOnlySessionStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>ReadOnlySessionStore<String, Long>);
      * String key = "some-key";
-     * KeyValueIterator<Windowed<String>, Long> sumForKeyForSession = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
+     * KeyValueIterator<Windowed<String>, Long> sumForKeyForWindows = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
      * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to
      * query the value of the key on a parallel running instance of your Kafka Streams application.
@@ -588,14 +778,14 @@ public interface KGroupedStream<K, V> {
      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
      * Therefore, the store name must be a valid Kafka topic name and cannot contain characters other than ASCII
      * alphanumerics, '.', '_' and '-'.
-     * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
+     * The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
      * user-specified in {@link StreamsConfig} via parameter
-     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the
-     * provide {@code storeName}, and "-changelog" is a fixed suffix.
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "queryableStoreName" is the
+     * provide {@code queryableStoreName}, and "-changelog" is a fixed suffix.
      * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
      * @param reducer           the instance of {@link Reducer}
      * @param sessionWindows    the specification of the aggregation {@link SessionWindows}
-     * @param storeSupplier     user defined state store supplier
+     * @param storeSupplier     user defined state store supplier. Cannot be {@code null}.
      * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent
      * the latest (rolling) aggregate for each key within a window
      */
@@ -610,7 +800,7 @@ public interface KGroupedStream<K, V> {
      * Aggregating is a generalization of {@link #reduce(Reducer, String) combining via reduce(...)} as it, for example,
      * allows the result to have a different type than the input values.
      * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view)
-     * that can be queried using the provided {@code storeName}.
+     * that can be queried using the provided {@code queryableStoreName}.
      * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream.
      * <p>
      * The specified {@link Initializer} is applied once directly before the first input record is processed to
@@ -632,7 +822,7 @@ public interface KGroupedStream<K, V> {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ... // some aggregation on value type double
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-key";
      * Long aggForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -642,18 +832,18 @@ public interface KGroupedStream<K, V> {
      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
      * Therefore, the store name must be a valid Kafka topic name and cannot contain characters other than ASCII
      * alphanumerics, '.', '_' and '-'.
-     * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
+     * The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
      * user-specified in {@link StreamsConfig} via parameter
-     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the
-     * provide {@code storeName}, and "-changelog" is a fixed suffix.
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "queryableStoreName" is the
+     * provide {@code queryableStoreName}, and "-changelog" is a fixed suffix.
      * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
      *
      * @param initializer   an {@link Initializer} that computes an initial intermediate aggregation result
      * @param aggregator    an {@link Aggregator} that computes a new aggregate result
      * @param aggValueSerde aggregate value serdes for materializing the aggregated table,
      *                      if not specified the default serdes defined in the configs will be used
-     * @param storeName     the name of the state store created from this operation; valid characters are ASCII
-     *                      alphanumerics, '.', '_' and '-'
+     * @param queryableStoreName the name of the state store created from this operation; valid characters are ASCII
+     * alphanumerics, '.', '_' and '-'. If {@code null} then this will be equivalent to {@link KGroupedStream#aggregate(Initializer, Aggregator, Serde)} ()} ()}.
      * @param <VR>          the value type of the resulting {@link KTable}
      * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the
      * latest (rolling) aggregate for each key
@@ -661,7 +851,52 @@ public interface KGroupedStream<K, V> {
     <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
                                  final Aggregator<? super K, ? super V, VR> aggregator,
                                  final Serde<VR> aggValueSerde,
-                                 final String storeName);
+                                 final String queryableStoreName);
+
+    /**
+     * Aggregate the values of records in this stream by the grouped key.
+     * Records with {@code null} key or value are ignored.
+     * Aggregating is a generalization of {@link #reduce(Reducer, String) combining via reduce(...)} as it, for example,
+     * allows the result to have a different type than the input values.
+     * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view)
+     * that can be queried using the provided {@code queryableStoreName}.
+     * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream.
+     * <p>
+     * The specified {@link Initializer} is applied once directly before the first input record is processed to
+     * provide an initial intermediate aggregation result that is used to process the first record.
+     * The specified {@link Aggregator} is applied for each input record and computes a new aggregate using the current
+     * aggregate (or for the very first record using the intermediate aggregation result provided via the
+     * {@link Initializer}) and the record's value.
+     * Thus, {@code aggregate(Initializer, Aggregator, Serde, String)} can be used to compute aggregate functions like
+     * count (c.f. {@link #count(String)}).
+     * <p>
+     * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
+     * the same key.
+     * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
+     * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
+     * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
+     * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
+     * <p>
+     * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
+     * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
+     * user-specified in {@link StreamsConfig} via parameter
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "internalStoreName" is an internal name
+     * and "-changelog" is a fixed suffix.
+     * Note that the internal store name may not be queriable through Interactive Queries.
+     * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
+     *
+     * @param initializer   an {@link Initializer} that computes an initial intermediate aggregation result
+     * @param aggregator    an {@link Aggregator} that computes a new aggregate result
+     * @param aggValueSerde aggregate value serdes for materializing the aggregated table,
+     *                      if not specified the default serdes defined in the configs will be used
+     * @param <VR>          the value type of the resulting {@link KTable}
+     * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the
+     * latest (rolling) aggregate for each key
+     */
+    <VR> KTable<K, VR> aggregate(final Initializer<VR> initializer,
+                                 final Aggregator<? super K, ? super V, VR> aggregator,
+                                 final Serde<VR> aggValueSerde);
+
 
     /**
      * Aggregate the values of records in this stream by the grouped key.
@@ -692,8 +927,8 @@ public interface KGroupedStream<K, V> {
      * Use {@link StateStoreSupplier#name()} to get the store name:
      * <pre>{@code
      * KafkaStreams streams = ... // some aggregation on value type double
-     * Sting storeName = storeSupplier.name();
-     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, QueryableStoreTypes.<String, Long>keyValueStore());
+     * Sting queryableStoreName = storeSupplier.name();
+     * ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
      * String key = "some-key";
      * Long aggForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -702,7 +937,7 @@ public interface KGroupedStream<K, V> {
      *
      * @param initializer   an {@link Initializer} that computes an initial intermediate aggregation result
      * @param aggregator    an {@link Aggregator} that computes a new aggregate result
-     * @param storeSupplier user defined state store supplier
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
      * @param <VR>          the value type of the resulting {@link KTable}
      * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the
      * latest (rolling) aggregate for each key
@@ -719,7 +954,7 @@ public interface KGroupedStream<K, V> {
      * The specified {@code windows} define either hopping time windows that can be overlapping or tumbling (c.f.
      * {@link TimeWindows}) or they define landmark windows (c.f. {@link UnlimitedWindows}).
      * The result is written into a local windowed {@link KeyValueStore} (which is basically an ever-updating
-     * materialized view) that can be queried using the provided {@code storeName}.
+     * materialized view) that can be queried using the provided {@code queryableStoreName}.
      * Windows are retained until their retention time expires (c.f. {@link Windows#until(long)}).
      * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
      * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
@@ -743,7 +978,7 @@ public interface KGroupedStream<K, V> {
      * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}:
      * <pre>{@code
      * KafkaStreams streams = ... // some windowed aggregation on value type double
-     * ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(storeName, QueryableStoreTypes.<String, Long>windowStore());
+     * ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore());
      * String key = "some-key";
      * long fromTime = ...;
      * long toTime = ...;
@@ -755,10 +990,10 @@ public interface KGroupedStream<K, V> {
      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
      * Therefore, the store name must be a valid Kafka topic name and cannot contain characters other than ASCII
      * alphanumerics, '.', '_' and '-'.
-     * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
+     * The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
      * user-specified in {@link StreamsConfig} via parameter
-     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the
-     * provide {@code storeName}, and "-changelog" is a fixed suffix.
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "queryableStoreName" is the
+     * provide {@code queryableStoreName}, and "-changelog" is a fixed suffix.
      * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
      *
      *
@@ -768,8 +1003,8 @@ public interface KGroupedStream<K, V> {
      * @param aggValueSerde aggregate value serdes for materializing the aggregated table,
      *                      if not specified the default serdes defined in the configs will be used
      * @param <VR>          the value type of the resulting {@link KTable}
-     * @param storeName     the name of the state store created from this operation; valid characters are ASCII
-     *                      alphanumerics, '.', '_' and '-'
+     * @param queryableStoreName the name of the state store created from this operation; valid characters are ASCII
+     * alphanumerics, '.', '_' and '-'. If {@code null} then this will be equivalent to {@link KGroupedStream#aggregate(Initializer, Aggregator, Windows, Serde)} ()} ()}.
      * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent
      * the latest (rolling) aggregate for each key within a window
      */
@@ -777,7 +1012,58 @@ public interface KGroupedStream<K, V> {
                                                              final Aggregator<? super K, ? super V, VR> aggregator,
                                                              final Windows<W> windows,
                                                              final Serde<VR> aggValueSerde,
-                                                             final String storeName);
+                                                             final String queryableStoreName);
+
+    /**
+     * Aggregate the values of records in this stream by the grouped key and defined windows.
+     * Records with {@code null} key or value are ignored.
+     * Aggregating is a generalization of {@link #reduce(Reducer, Windows, String) combining via reduce(...)} as it,
+     * for example, allows the result to have a different type than the input values.
+     * The specified {@code windows} define either hopping time windows that can be overlapping or tumbling (c.f.
+     * {@link TimeWindows}) or they define landmark windows (c.f. {@link UnlimitedWindows}).
+     * The result is written into a local windowed {@link KeyValueStore} (which is basically an ever-updating
+     * materialized view) that can be queried using the provided {@code queryableStoreName}.
+     * Windows are retained until their retention time expires (c.f. {@link Windows#until(long)}).
+     * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
+     * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
+     * <p>
+     * The specified {@link Initializer} is applied once per window directly before the first input record is
+     * processed to provide an initial intermediate aggregation result that is used to process the first record.
+     * The specified {@link Aggregator} is applied for each input record and computes a new aggregate using the current
+     * aggregate (or for the very first record using the intermediate aggregation result provided via the
+     * {@link Initializer}) and the record's value.
+     * Thus, {@code aggregate(Initializer, Aggregator, Windows, Serde, String)} can be used to compute aggregate
+     * functions like count (c.f. {@link #count(String)}).
+     * <p>
+     * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
+     * the same window and key.
+     * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
+     * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
+     * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
+     * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
+     * <p>
+     * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
+     * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
+     * user-specified in {@link StreamsConfig} via parameter
+     * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "internalStoreName" is an internal name
+     * and "-changelog" is a fixed suffix.
+     * Note that the internal store name may not be queriable through Interactive Queries.
+     * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}.
+     *
+     * @param initializer   an {@link Initializer} that computes an initial intermediate aggregation result
+     * @param aggregator    an {@link Aggregator} that computes a new aggregate result
+     * @param windows       the specification of the aggregation {@link Windows}
+     * @param aggValueSerde aggregate value serdes for materializing the aggregated table,
+     *                      if not specified the default serdes defined in the configs will be used
+     * @param <VR>          the value type of the resulting {@link KTable}
+     * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent
+     * the latest (rolling) aggregate for each key within a window
+     */
+    <W extends Window, VR> KTable<Windowed<K>, VR> aggregate(final Initializer<VR> initializer,
+                                                             final Aggregator<? super K, ? super V, VR> aggregator,
+                                                             final Windows<W> windows,
+                                                             final Serde<VR> aggValueSerde);
+
 
     /**
      * Aggregate the values of records in this stream by the grouped key and defined windows.
@@ -812,8 +1098,8 @@ public interface KGroupedStream<K, V> {
      * Use {@link StateStoreSupplier#name()} to get the store name:
      * <pre>{@code
      * KafkaStreams streams = ... // some windowed aggregation on value type Long
-     * Sting storeName = storeSupplier.name();
-     * ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(storeName, QueryableStoreTypes.<String, Long>windowStore());
+     * Sting queryableStoreName = storeSupplier.name();
+     * ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore());
      * String key = "some-key";
      * long fromTime = ...;
      * long toTime = ...;
@@ -826,7 +1112,7 @@ public interface KGroupedStream<K, V> {
      * @param aggregator    an {@link Aggregator} that computes a new aggregate result
      * @param windows       the specification of the aggregation {@link Windows}
      * @param <VR>          the value type of the resulting {@link KTable}
-     * @param storeSupplier user defined state store supplier
+     * @param storeSupplier user defined state store supplier. Cannot be {@code null}.
      * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent
      * the latest (rolling) aggregate for each key within a window
      */
@@ -841,7 +1127,7 @@ public interface KGroupedStream<K, V> {
      * Aggregating is a generalization of {@link #reduce(Reducer, SessionWindows, String) combining via
      * reduce(...)} as it, for example, allows the result to have a different type than the input values.
      * The result is written into a local {@link SessionStore} (which is basically an ever-updating
-     * materialized view) that can be queried using the provided {@code storeName}.
+     * materialized view) that can be queried using the provided {@code queryableStoreName}.
      * SessionWindows are retained until their retention time expires (c.f. {@link SessionWindows#until(long)}).
      * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
      * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
@@ -866,8 +1152,8 @@ public interface KGroupedStream<K, V> {
      * Use {@link StateStoreSupplier#name()} to get the store name:
      * <pre>{@code
      * KafkaStreams streams = ... // some windowed aggregation on value type double
-     * Sting storeName = storeSupplier.name();
-     * ReadOnlySessionStore<String, Long> sessionStore = streams.store(storeName, QueryableStoreTypes.<String, Long>sessionStore());
+     * Sting queryableStoreName = storeSupplier.name();
+     * ReadOnlySessionStore<String, Long> sessionStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>sessionStore());
      * String key = "some-key";
      * KeyValueIterator<Windowed<String>, Long> aggForKeyForSession = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -881,8 +1167,8 @@ public interface KGroupedStream<K, V> {
      * @param aggValueSerde aggregate value serdes for materializing the aggregated table,
      *                      if not specified the default serdes defined in the configs will be used
      * @param <T>           the value type of the resulting {@link KTable}
-     * @param storeName     the name of the state store created from this operation; valid characters are ASCII
-     *                      alphanumerics, '.', '_' and '-'
+     * @param queryableStoreName the name of the state store created from this operation; valid characters are ASCII
+     * alphanumerics, '.', '_' and '-'. If {@code null} then this will be equivalent to {@link KGroupedStream#aggregate(Initializer, Aggregator, Merger, SessionWindows, Serde)} ()} ()}.
      * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent
      * the latest (rolling) aggregate for each key within a window
      */
@@ -891,7 +1177,49 @@ public interface KGroupedStream<K, V> {
                                          final Merger<? super K, T> sessionMerger,
                                          final SessionWindows sessionWindows,
                                          final Serde<T> aggValueSerde,
-                                         final String storeName);
+                                         final String queryableStoreName);
+
+    /**
+     * Aggregate the values of records in this stream by the grouped key and defined {@link SessionWindows}.
+     * Records with {@code null} key or value are ignored.
+     * Aggregating is a generalization of {@link #reduce(Reducer, SessionWindows, String) combining via
+     * reduce(...)} as it, for example, allows the result to have a different type than the input values.
+     * The result is written into a local {@link SessionStore} (which is basically an ever-updating
+     * materialized view) that can be queried using the provided {@code queryableStoreName}.
+     * SessionWindows are retained until their retention time expires (c.f. {@link SessionWindows#until(long)}).
+     * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
+     * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
+     * <p>
+     * The specified {@link Initializer} is applied once per session directly before the first input record is
+     * processed to provide an initial intermediate aggregation result that is used to process the first record.
+     * The specified {@link Aggregator} is applied for each input record and computes a new aggregate using the current
+     * aggregate (or for the very first record using the intermediate aggregation result provided via the
+     * {@link Initializer}) and the record's value.
+     * Thus, {@code aggregate(Initializer, Aggregator, Merger, SessionWindows, Serde, String)} can be used to compute
+     * aggregate functions like count (c.f. {@link #count(String)})
+     * <p>
+     * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
+     * the same window and key.
+     * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
+     * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
+     * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
+     * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
+     * <p>
+     * @param initializer    the instance of {@link Initializer}
+     * @param aggregator     the instance of {@link Aggregator}
+     * @param sessionMerger  the instance of {@link Merger}
+     * @param sessionWindows the specification of the aggregation {@link SessionWindows}
+     * @param aggValueSerde aggregate value serdes for materializing the aggregated table,
+     *                      if not specified the default serdes defined in the configs will be used
+     * @param <T>           the value type of the resulting {@link KTable}
+     * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent
+     * the latest (rolling) aggregate for each key within a window
+     */
+    <T> KTable<Windowed<K>, T> aggregate(final Initializer<T> initializer,
+                                         final Aggregator<? super K, ? super V, T> aggregator,
+                                         final Merger<? super K, T> sessionMerger,
+                                         final SessionWindows sessionWindows,
+                                         final Serde<T> aggValueSerde);
 
     /**
      * Aggregate the values of records in this stream by the grouped key and defined {@link SessionWindows}.
@@ -924,8 +1252,8 @@ public interface KGroupedStream<K, V> {
      * Use {@link StateStoreSupplier#name()} to get the store name:
      * <pre>{@code
      * KafkaStreams streams = ... // some windowed aggregation on value type double
-     * Sting storeName = storeSupplier.name();
-     * ReadOnlySessionStore<String, Long> sessionStore = streams.store(storeName, QueryableStoreTypes.<String, Long>sessionStore());
+     * Sting queryableStoreName = storeSupplier.name();
+     * ReadOnlySessionStore<String, Long> sessionStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>sessionStore());
      * String key = "some-key";
      * KeyValueIterator<Windowed<String>, Long> aggForKeyForSession = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
      * }</pre>
@@ -939,7 +1267,7 @@ public interface KGroupedStream<K, V> {
      * @param sessionWindows the specification of the aggregation {@link SessionWindows}
      * @param aggValueSerde  aggregate value serdes for materializing the aggregated table,
      *                       if not specified the default serdes defined in the configs will be used
-     * @param storeSupplier  user defined state store supplier
+     * @param storeSupplier  user defined state store supplier. Cannot be {@code null}.
      * @param <T>           the value type of the resulting {@link KTable}
      * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent
      * the latest (rolling) aggregate for each key within a window


[2/5] kafka git commit: KAFKA-5045: Clarify on KTable APIs for queryable stores

Posted by gu...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java
----------------------------------------------------------------------
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java
index cc6a126..6ed3e84 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImpl.java
@@ -16,7 +16,6 @@
  */
 package org.apache.kafka.streams.kstream.internals;
 
-import org.apache.kafka.common.internals.Topic;
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.streams.kstream.Aggregator;
@@ -47,6 +46,7 @@ class KGroupedStreamImpl<K, V> extends AbstractStream<K> implements KGroupedStre
     private final Serde<K> keySerde;
     private final Serde<V> valSerde;
     private final boolean repartitionRequired;
+    private boolean isQueryable = true;
 
     KGroupedStreamImpl(final KStreamBuilder topology,
                        final String name,
@@ -58,12 +58,26 @@ class KGroupedStreamImpl<K, V> extends AbstractStream<K> implements KGroupedStre
         this.keySerde = keySerde;
         this.valSerde = valSerde;
         this.repartitionRequired = repartitionRequired;
+        this.isQueryable = true;
+    }
+
+    private void determineIsQueryable(final String queryableStoreName) {
+        if (queryableStoreName == null) {
+            isQueryable = false;
+        } // no need for else {} since isQueryable is true by default
     }
 
     @Override
     public KTable<K, V> reduce(final Reducer<V> reducer,
-                               final String storeName) {
-        return reduce(reducer, keyValueStore(keySerde, valSerde, storeName));
+                               final String queryableStoreName) {
+        determineIsQueryable(queryableStoreName);
+        return reduce(reducer, keyValueStore(keySerde, valSerde, getOrCreateName(queryableStoreName, REDUCE_NAME)));
+    }
+
+    @Override
+    public KTable<K, V> reduce(final Reducer<V> reducer) {
+        determineIsQueryable(null);
+        return reduce(reducer, (String) null);
     }
 
     @Override
@@ -82,8 +96,16 @@ class KGroupedStreamImpl<K, V> extends AbstractStream<K> implements KGroupedStre
     @Override
     public <W extends Window> KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
                                                             final Windows<W> windows,
-                                                            final String storeName) {
-        return reduce(reducer, windows, windowedStore(keySerde, valSerde, windows, storeName));
+                                                            final String queryableStoreName) {
+        determineIsQueryable(queryableStoreName);
+        return reduce(reducer, windows, windowedStore(keySerde, valSerde, windows, getOrCreateName(queryableStoreName, REDUCE_NAME)));
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <W extends Window> KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
+                                                            final Windows<W> windows) {
+        return reduce(reducer, windows, (String) null);
     }
 
     @SuppressWarnings("unchecked")
@@ -105,8 +127,16 @@ class KGroupedStreamImpl<K, V> extends AbstractStream<K> implements KGroupedStre
     public <T> KTable<K, T> aggregate(final Initializer<T> initializer,
                                       final Aggregator<? super K, ? super V, T> aggregator,
                                       final Serde<T> aggValueSerde,
-                                      final String storeName) {
-        return aggregate(initializer, aggregator, keyValueStore(keySerde, aggValueSerde, storeName));
+                                      final String queryableStoreName) {
+        determineIsQueryable(queryableStoreName);
+        return aggregate(initializer, aggregator, keyValueStore(keySerde, aggValueSerde, getOrCreateName(queryableStoreName, AGGREGATE_NAME)));
+    }
+
+    @Override
+    public <T> KTable<K, T> aggregate(final Initializer<T> initializer,
+                                      final Aggregator<? super K, ? super V, T> aggregator,
+                                      final Serde<T> aggValueSerde) {
+        return aggregate(initializer, aggregator, aggValueSerde, null);
     }
 
     @Override
@@ -128,8 +158,18 @@ class KGroupedStreamImpl<K, V> extends AbstractStream<K> implements KGroupedStre
                                                                   final Aggregator<? super K, ? super V, T> aggregator,
                                                                   final Windows<W> windows,
                                                                   final Serde<T> aggValueSerde,
-                                                                  final String storeName) {
-        return aggregate(initializer, aggregator, windows, windowedStore(keySerde, aggValueSerde, windows, storeName));
+                                                                  final String queryableStoreName) {
+        determineIsQueryable(queryableStoreName);
+        return aggregate(initializer, aggregator, windows, windowedStore(keySerde, aggValueSerde, windows, getOrCreateName(queryableStoreName, AGGREGATE_NAME)));
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <W extends Window, T> KTable<Windowed<K>, T> aggregate(final Initializer<T> initializer,
+                                                                  final Aggregator<? super K, ? super V, T> aggregator,
+                                                                  final Windows<W> windows,
+                                                                  final Serde<T> aggValueSerde) {
+        return aggregate(initializer, aggregator, windows, aggValueSerde, null);
     }
 
     @SuppressWarnings("unchecked")
@@ -150,8 +190,14 @@ class KGroupedStreamImpl<K, V> extends AbstractStream<K> implements KGroupedStre
     }
 
     @Override
-    public KTable<K, Long> count(final String storeName) {
-        return count(keyValueStore(keySerde, Serdes.Long(), storeName));
+    public KTable<K, Long> count(final String queryableStoreName) {
+        determineIsQueryable(queryableStoreName);
+        return count(keyValueStore(keySerde, Serdes.Long(), getOrCreateName(queryableStoreName, AGGREGATE_NAME)));
+    }
+
+    @Override
+    public KTable<K, Long> count() {
+        return count((String) null);
     }
 
     @Override
@@ -171,8 +217,14 @@ class KGroupedStreamImpl<K, V> extends AbstractStream<K> implements KGroupedStre
 
     @Override
     public <W extends Window> KTable<Windowed<K>, Long> count(final Windows<W> windows,
-                                                              final String storeName) {
-        return count(windows, windowedStore(keySerde, Serdes.Long(), windows, storeName));
+                                                              final String queryableStoreName) {
+        determineIsQueryable(queryableStoreName);
+        return count(windows, windowedStore(keySerde, Serdes.Long(), windows, getOrCreateName(queryableStoreName, AGGREGATE_NAME)));
+    }
+
+    @Override
+    public <W extends Window> KTable<Windowed<K>, Long> count(final Windows<W> windows) {
+        return count(windows, (String) null);
     }
 
     @Override
@@ -201,15 +253,14 @@ class KGroupedStreamImpl<K, V> extends AbstractStream<K> implements KGroupedStre
                                                 final Merger<? super K, T> sessionMerger,
                                                 final SessionWindows sessionWindows,
                                                 final Serde<T> aggValueSerde,
-                                                final String storeName) {
-        Objects.requireNonNull(storeName, "storeName can't be null");
-        Topic.validate(storeName);
+                                                final String queryableStoreName) {
+        determineIsQueryable(queryableStoreName);
         return aggregate(initializer,
                          aggregator,
                          sessionMerger,
                          sessionWindows,
                          aggValueSerde,
-                         storeFactory(keySerde, aggValueSerde, storeName)
+                         storeFactory(keySerde, aggValueSerde, getOrCreateName(queryableStoreName, AGGREGATE_NAME))
                           .sessionWindowed(sessionWindows.maintainMs()).build());
 
 
@@ -221,6 +272,16 @@ class KGroupedStreamImpl<K, V> extends AbstractStream<K> implements KGroupedStre
                                                 final Aggregator<? super K, ? super V, T> aggregator,
                                                 final Merger<? super K, T> sessionMerger,
                                                 final SessionWindows sessionWindows,
+                                                final Serde<T> aggValueSerde) {
+        return aggregate(initializer, aggregator, sessionMerger, sessionWindows, aggValueSerde, (String) null);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <T> KTable<Windowed<K>, T> aggregate(final Initializer<T> initializer,
+                                                final Aggregator<? super K, ? super V, T> aggregator,
+                                                final Merger<? super K, T> sessionMerger,
+                                                final SessionWindows sessionWindows,
                                                 final Serde<T> aggValueSerde,
                                                 final StateStoreSupplier<SessionStore> storeSupplier) {
         Objects.requireNonNull(initializer, "initializer can't be null");
@@ -237,14 +298,17 @@ class KGroupedStreamImpl<K, V> extends AbstractStream<K> implements KGroupedStre
     }
 
     @SuppressWarnings("unchecked")
-    public KTable<Windowed<K>, Long> count(final SessionWindows sessionWindows, final String storeName) {
-        Objects.requireNonNull(storeName, "storeName can't be null");
-        Topic.validate(storeName);
+    public KTable<Windowed<K>, Long> count(final SessionWindows sessionWindows, final String queryableStoreName) {
+        determineIsQueryable(queryableStoreName);
         return count(sessionWindows,
-                     storeFactory(keySerde, Serdes.Long(), storeName)
+                     storeFactory(keySerde, Serdes.Long(), getOrCreateName(queryableStoreName, AGGREGATE_NAME))
                              .sessionWindowed(sessionWindows.maintainMs()).build());
     }
 
+    public KTable<Windowed<K>, Long> count(final SessionWindows sessionWindows) {
+        return count(sessionWindows, (String) null);
+    }
+
     @SuppressWarnings("unchecked")
     @Override
     public KTable<Windowed<K>, Long> count(final SessionWindows sessionWindows,
@@ -278,15 +342,22 @@ class KGroupedStreamImpl<K, V> extends AbstractStream<K> implements KGroupedStre
     @Override
     public KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
                                          final SessionWindows sessionWindows,
-                                         final String storeName) {
+                                         final String queryableStoreName) {
+        determineIsQueryable(queryableStoreName);
 
-        Objects.requireNonNull(storeName, "storeName can't be null");
-        Topic.validate(storeName);
         return reduce(reducer, sessionWindows,
-                      storeFactory(keySerde, valSerde, storeName)
+                      storeFactory(keySerde, valSerde, getOrCreateName(queryableStoreName, AGGREGATE_NAME))
                               .sessionWindowed(sessionWindows.maintainMs()).build());
     }
 
+    @SuppressWarnings("unchecked")
+    @Override
+    public KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
+                                         final SessionWindows sessionWindows) {
+
+        return reduce(reducer, sessionWindows, (String) null);
+    }
+
     @Override
     public KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
                                          final SessionWindows sessionWindows,
@@ -339,16 +410,17 @@ class KGroupedStreamImpl<K, V> extends AbstractStream<K> implements KGroupedStre
                 aggregateSupplier,
                 sourceName.equals(this.name) ? sourceNodes
                         : Collections.singleton(sourceName),
-                storeSupplier.name());
+                storeSupplier.name(),
+                isQueryable);
     }
 
     /**
      * @return the new sourceName if repartitioned. Otherwise the name of this stream
      */
-    private String repartitionIfRequired(final String storeName) {
+    private String repartitionIfRequired(final String queryableStoreName) {
         if (!repartitionRequired) {
             return this.name;
         }
-        return KStreamImpl.createReparitionedSource(this, keySerde, valSerde, storeName);
+        return KStreamImpl.createReparitionedSource(this, keySerde, valSerde, queryableStoreName);
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java
----------------------------------------------------------------------
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java
index 08a4c5d..7e62727 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java
@@ -16,7 +16,6 @@
  */
 package org.apache.kafka.streams.kstream.internals;
 
-import org.apache.kafka.common.internals.Topic;
 import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
@@ -48,41 +47,64 @@ public class KGroupedTableImpl<K, V> extends AbstractStream<K> implements KGroup
 
     protected final Serde<? extends K> keySerde;
     protected final Serde<? extends V> valSerde;
+    private boolean isQueryable = true;
 
-    public KGroupedTableImpl(KStreamBuilder topology,
-                             String name,
-                             String sourceName,
-                             Serde<? extends K> keySerde,
-                             Serde<? extends V> valSerde) {
+    public KGroupedTableImpl(final KStreamBuilder topology,
+                             final String name,
+                             final String sourceName,
+                             final Serde<? extends K> keySerde,
+                             final Serde<? extends V> valSerde) {
         super(topology, name, Collections.singleton(sourceName));
         this.keySerde = keySerde;
         this.valSerde = valSerde;
+        this.isQueryable = true;
+    }
+
+    private void determineIsQueryable(final String queryableStoreName) {
+        if (queryableStoreName == null) {
+            isQueryable = false;
+        } // no need for else {} since isQueryable is true by default
+    }
+
+    @Override
+    public <T> KTable<K, T> aggregate(final Initializer<T> initializer,
+                                      final Aggregator<? super K, ? super V, T> adder,
+                                      final Aggregator<? super K, ? super V, T> subtractor,
+                                      final Serde<T> aggValueSerde,
+                                      final String queryableStoreName) {
+        determineIsQueryable(queryableStoreName);
+        return aggregate(initializer, adder, subtractor, keyValueStore(keySerde, aggValueSerde, getOrCreateName(queryableStoreName, AGGREGATE_NAME)));
+    }
+
+    @Override
+    public <T> KTable<K, T> aggregate(final Initializer<T> initializer,
+                                      final Aggregator<? super K, ? super V, T> adder,
+                                      final Aggregator<? super K, ? super V, T> subtractor,
+                                      final Serde<T> aggValueSerde) {
+        return aggregate(initializer, adder, subtractor, aggValueSerde, (String) null);
     }
 
     @Override
-    public <T> KTable<K, T> aggregate(Initializer<T> initializer,
-                                      Aggregator<? super K, ? super V, T> adder,
-                                      Aggregator<? super K, ? super V, T> subtractor,
-                                      Serde<T> aggValueSerde,
-                                      String storeName) {
-        return aggregate(initializer, adder, subtractor, keyValueStore(keySerde, aggValueSerde, storeName));
+    public <T> KTable<K, T> aggregate(final Initializer<T> initializer,
+                                      final Aggregator<? super K, ? super V, T> adder,
+                                      final Aggregator<? super K, ? super V, T> subtractor,
+                                      final String queryableStoreName) {
+        determineIsQueryable(queryableStoreName);
+        return aggregate(initializer, adder, subtractor, null, getOrCreateName(queryableStoreName, AGGREGATE_NAME));
     }
 
     @Override
-    public <T> KTable<K, T> aggregate(Initializer<T> initializer,
-                                      Aggregator<? super K, ? super V, T> adder,
-                                      Aggregator<? super K, ? super V, T> subtractor,
-                                      String storeName) {
-        Objects.requireNonNull(storeName, "storeName can't be null");
-        Topic.validate(storeName);
-        return aggregate(initializer, adder, subtractor, null, storeName);
+    public <T> KTable<K, T> aggregate(final Initializer<T> initializer,
+                                      final Aggregator<? super K, ? super V, T> adder,
+                                      final Aggregator<? super K, ? super V, T> subtractor) {
+        return aggregate(initializer, adder, subtractor, (String) null);
     }
 
     @Override
-    public <T> KTable<K, T> aggregate(Initializer<T> initializer,
-                                      Aggregator<? super K, ? super V, T> adder,
-                                      Aggregator<? super K, ? super V, T> subtractor,
-                                      StateStoreSupplier<KeyValueStore> storeSupplier) {
+    public <T> KTable<K, T> aggregate(final Initializer<T> initializer,
+                                      final Aggregator<? super K, ? super V, T> adder,
+                                      final Aggregator<? super K, ? super V, T> subtractor,
+                                      final StateStoreSupplier<KeyValueStore> storeSupplier) {
         Objects.requireNonNull(initializer, "initializer can't be null");
         Objects.requireNonNull(adder, "adder can't be null");
         Objects.requireNonNull(subtractor, "subtractor can't be null");
@@ -91,9 +113,9 @@ public class KGroupedTableImpl<K, V> extends AbstractStream<K> implements KGroup
         return doAggregate(aggregateSupplier, AGGREGATE_NAME, storeSupplier);
     }
 
-    private <T> KTable<K, T> doAggregate(ProcessorSupplier<K, Change<V>> aggregateSupplier,
-                                         String functionName,
-                                         StateStoreSupplier<KeyValueStore> storeSupplier) {
+    private <T> KTable<K, T> doAggregate(final ProcessorSupplier<K, Change<V>> aggregateSupplier,
+                                         final String functionName,
+                                         final StateStoreSupplier<KeyValueStore> storeSupplier) {
         String sinkName = topology.newName(KStreamImpl.SINK_NAME);
         String sourceName = topology.newName(KStreamImpl.SOURCE_NAME);
         String funcName = topology.newName(functionName);
@@ -120,22 +142,27 @@ public class KGroupedTableImpl<K, V> extends AbstractStream<K> implements KGroup
         topology.addStateStore(storeSupplier, funcName);
 
         // return the KTable representation with the intermediate topic as the sources
-        return new KTableImpl<>(topology, funcName, aggregateSupplier, Collections.singleton(sourceName), storeSupplier.name());
+        return new KTableImpl<>(topology, funcName, aggregateSupplier, Collections.singleton(sourceName), storeSupplier.name(), isQueryable);
+    }
+
+    @Override
+    public KTable<K, V> reduce(final Reducer<V> adder,
+                               final Reducer<V> subtractor,
+                               final String queryableStoreName) {
+        determineIsQueryable(queryableStoreName);
+        return reduce(adder, subtractor, keyValueStore(keySerde, valSerde, getOrCreateName(queryableStoreName, REDUCE_NAME)));
     }
 
     @Override
-    public KTable<K, V> reduce(Reducer<V> adder,
-                               Reducer<V> subtractor,
-                               String storeName) {
-        Objects.requireNonNull(storeName, "storeName can't be null");
-        Topic.validate(storeName);
-        return reduce(adder, subtractor, keyValueStore(keySerde, valSerde, storeName));
+    public KTable<K, V> reduce(final Reducer<V> adder,
+                               final Reducer<V> subtractor) {
+        return reduce(adder, subtractor, (String) null);
     }
 
     @Override
-    public KTable<K, V> reduce(Reducer<V> adder,
-                               Reducer<V> subtractor,
-                               StateStoreSupplier<KeyValueStore> storeSupplier) {
+    public KTable<K, V> reduce(final Reducer<V> adder,
+                               final Reducer<V> subtractor,
+                               final StateStoreSupplier<KeyValueStore> storeSupplier) {
         Objects.requireNonNull(adder, "adder can't be null");
         Objects.requireNonNull(subtractor, "subtractor can't be null");
         Objects.requireNonNull(storeSupplier, "storeSupplier can't be null");
@@ -144,14 +171,18 @@ public class KGroupedTableImpl<K, V> extends AbstractStream<K> implements KGroup
     }
 
     @Override
-    public KTable<K, Long> count(String storeName) {
-        Objects.requireNonNull(storeName, "storeName can't be null");
-        Topic.validate(storeName);
-        return count(keyValueStore(keySerde, Serdes.Long(), storeName));
+    public KTable<K, Long> count(final String queryableStoreName) {
+        determineIsQueryable(queryableStoreName);
+        return count(keyValueStore(keySerde, Serdes.Long(), getOrCreateName(queryableStoreName, AGGREGATE_NAME)));
+    }
+
+    @Override
+    public KTable<K, Long> count() {
+        return count((String) null);
     }
 
     @Override
-    public KTable<K, Long> count(StateStoreSupplier<KeyValueStore> storeSupplier) {
+    public KTable<K, Long> count(final StateStoreSupplier<KeyValueStore> storeSupplier) {
         return this.aggregate(
                 new Initializer<Long>() {
                     @Override

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
----------------------------------------------------------------------
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
index bbd4ac4..b751294 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
@@ -617,7 +617,7 @@ public class KStreamImpl<K, V> extends AbstractStream<K> implements KStream<K, V
 
         final String name = topology.newName(leftJoin ? LEFTJOIN_NAME : JOIN_NAME);
         topology.addProcessor(name, new KStreamKTableJoin<>(((KTableImpl<K, ?, V1>) other).valueGetterSupplier(), joiner, leftJoin), this.name);
-        topology.connectProcessorAndStateStores(name, other.getStoreName());
+        topology.connectProcessorAndStateStores(name, ((KTableImpl<K, ?, V1>) other).internalStoreName());
         topology.connectProcessors(this.name, ((KTableImpl<K, ?, V1>) other).name);
 
         return new KStreamImpl<>(topology, name, allSourceNodes, false);

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java
----------------------------------------------------------------------
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java
index 774f235..af8c906 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java
@@ -20,19 +20,22 @@ import org.apache.kafka.streams.kstream.Predicate;
 import org.apache.kafka.streams.processor.AbstractProcessor;
 import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.state.KeyValueStore;
 
 class KTableFilter<K, V> implements KTableProcessorSupplier<K, V, V> {
 
     private final KTableImpl<K, ?, V> parent;
     private final Predicate<? super K, ? super V> predicate;
     private final boolean filterNot;
-
+    private final String queryableName;
     private boolean sendOldValues = false;
 
-    public KTableFilter(KTableImpl<K, ?, V> parent, Predicate<? super K, ? super V> predicate, boolean filterNot) {
+    public KTableFilter(final KTableImpl<K, ?, V> parent, final Predicate<? super K, ? super V> predicate,
+                        final boolean filterNot, final String queryableName) {
         this.parent = parent;
         this.predicate = predicate;
         this.filterNot = filterNot;
+        this.queryableName = queryableName;
     }
 
     @Override
@@ -74,6 +77,18 @@ class KTableFilter<K, V> implements KTableProcessorSupplier<K, V, V> {
     }
 
     private class KTableFilterProcessor extends AbstractProcessor<K, Change<V>> {
+        private KeyValueStore<K, V> store;
+        private TupleForwarder<K, V> tupleForwarder;
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public void init(ProcessorContext context) {
+            super.init(context);
+            if (queryableName != null) {
+                store = (KeyValueStore<K, V>) context.getStateStore(queryableName);
+                tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener<K, V>(context, sendOldValues), sendOldValues);
+            }
+        }
 
         @Override
         public void process(K key, Change<V> change) {
@@ -83,7 +98,12 @@ class KTableFilter<K, V> implements KTableProcessorSupplier<K, V, V> {
             if (sendOldValues && oldValue == null && newValue == null)
                 return; // unnecessary to forward here.
 
-            context().forward(key, new Change<>(newValue, oldValue));
+            if (queryableName != null) {
+                store.put(key, newValue);
+                tupleForwarder.maybeForward(key, newValue, oldValue);
+            } else {
+                context().forward(key, new Change<>(newValue, oldValue));
+            }
         }
 
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
----------------------------------------------------------------------
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
index 6120f91..96a0b2c 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
@@ -29,7 +29,9 @@ import org.apache.kafka.streams.kstream.Predicate;
 import org.apache.kafka.streams.kstream.ValueJoiner;
 import org.apache.kafka.streams.kstream.ValueMapper;
 import org.apache.kafka.streams.processor.ProcessorSupplier;
+import org.apache.kafka.streams.processor.StateStoreSupplier;
 import org.apache.kafka.streams.processor.StreamPartitioner;
+import org.apache.kafka.streams.state.KeyValueStore;
 
 import java.io.FileNotFoundException;
 import java.io.PrintWriter;
@@ -67,58 +69,157 @@ public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K,
 
     private static final String TOSTREAM_NAME = "KTABLE-TOSTREAM-";
 
+    public static final String STATE_STORE_NAME = "STATE-STORE-";
+
     private final ProcessorSupplier<?, ?> processorSupplier;
 
-    private final String storeName;
+    private final String queryableStoreName;
+    private final boolean isQueryable;
 
     private boolean sendOldValues = false;
+    private final Serde<K> keySerde;
+    private final Serde<V> valSerde;
 
+    public KTableImpl(KStreamBuilder topology,
+                      String name,
+                      ProcessorSupplier<?, ?> processorSupplier,
+                      Set<String> sourceNodes,
+                      final String queryableStoreName,
+                      boolean isQueryable) {
+        super(topology, name, sourceNodes);
+        this.processorSupplier = processorSupplier;
+        this.queryableStoreName = queryableStoreName;
+        this.keySerde = null;
+        this.valSerde = null;
+        this.isQueryable = isQueryable;
+    }
 
     public KTableImpl(KStreamBuilder topology,
                       String name,
                       ProcessorSupplier<?, ?> processorSupplier,
+                      final Serde<K> keySerde,
+                      final Serde<V> valSerde,
                       Set<String> sourceNodes,
-                      final String storeName) {
+                      final String queryableStoreName,
+                      boolean isQueryable) {
         super(topology, name, sourceNodes);
         this.processorSupplier = processorSupplier;
-        this.storeName = storeName;
+        this.queryableStoreName = queryableStoreName;
+        this.keySerde = keySerde;
+        this.valSerde = valSerde;
+        this.isQueryable = isQueryable;
     }
 
     @Override
-    public String getStoreName() {
-        return this.storeName;
+    public String queryableStoreName() {
+        if (!isQueryable) {
+            return null;
+        }
+        return this.queryableStoreName;
     }
 
-    @Override
-    public KTable<K, V> filter(Predicate<? super K, ? super V> predicate) {
+    String internalStoreName() {
+        return this.queryableStoreName;
+    }
+
+    private KTable<K, V> doFilter(final Predicate<? super K, ? super V> predicate,
+                                  final StateStoreSupplier<KeyValueStore> storeSupplier,
+                                  boolean isFilterNot) {
         Objects.requireNonNull(predicate, "predicate can't be null");
         String name = topology.newName(FILTER_NAME);
-        KTableProcessorSupplier<K, V, V> processorSupplier = new KTableFilter<>(this, predicate, false);
+        String internalStoreName = null;
+        if (storeSupplier != null) {
+            internalStoreName = storeSupplier.name();
+        }
+        KTableProcessorSupplier<K, V, V> processorSupplier = new KTableFilter<>(this, predicate, isFilterNot, internalStoreName);
         topology.addProcessor(name, processorSupplier, this.name);
+        if (storeSupplier != null) {
+            topology.addStateStore(storeSupplier, name);
+        }
+        return new KTableImpl<>(topology, name, processorSupplier, this.keySerde, this.valSerde, sourceNodes, internalStoreName, internalStoreName != null);
+    }
 
-        return new KTableImpl<>(topology, name, processorSupplier, sourceNodes, this.storeName);
+    @Override
+    public KTable<K, V> filter(final Predicate<? super K, ? super V> predicate) {
+        return filter(predicate, (String) null);
     }
 
     @Override
-    public KTable<K, V> filterNot(final Predicate<? super K, ? super V> predicate) {
-        Objects.requireNonNull(predicate, "predicate can't be null");
-        String name = topology.newName(FILTER_NAME);
-        KTableProcessorSupplier<K, V, V> processorSupplier = new KTableFilter<>(this, predicate, true);
+    public KTable<K, V> filter(final Predicate<? super K, ? super V> predicate, final String queryableStoreName) {
+        StateStoreSupplier<KeyValueStore> storeSupplier = null;
+        if (queryableStoreName != null) {
+            storeSupplier = keyValueStore(this.keySerde, this.valSerde, queryableStoreName);
+        }
+        return doFilter(predicate, storeSupplier, false);
+    }
 
-        topology.addProcessor(name, processorSupplier, this.name);
+    @Override
+    public KTable<K, V> filter(final Predicate<? super K, ? super V> predicate, final StateStoreSupplier<KeyValueStore> storeSupplier) {
+        Objects.requireNonNull(storeSupplier, "storeSupplier can't be null");
+        return doFilter(predicate, storeSupplier, false);
+    }
 
-        return new KTableImpl<>(topology, name, processorSupplier, sourceNodes, this.storeName);
+    @Override
+    public KTable<K, V> filterNot(final Predicate<? super K, ? super V> predicate) {
+        return filterNot(predicate, (String) null);
+    }
+
+    @Override
+    public KTable<K, V> filterNot(final Predicate<? super K, ? super V> predicate, final String queryableStoreName) {
+        StateStoreSupplier<KeyValueStore> storeSupplier = null;
+        if (queryableStoreName != null) {
+            storeSupplier = keyValueStore(this.keySerde, this.valSerde, queryableStoreName);
+        }
+        return doFilter(predicate, storeSupplier, true);
     }
 
     @Override
-    public <V1> KTable<K, V1> mapValues(ValueMapper<? super V, ? extends V1> mapper) {
+    public KTable<K, V> filterNot(final Predicate<? super K, ? super V> predicate, final StateStoreSupplier<KeyValueStore> storeSupplier) {
+        Objects.requireNonNull(storeSupplier, "storeSupplier can't be null");
+        return doFilter(predicate, storeSupplier, true);
+    }
+
+    private <V1> KTable<K, V1> doMapValues(final ValueMapper<? super V, ? extends V1> mapper,
+                                           final Serde<V1> valueSerde,
+                                           final StateStoreSupplier<KeyValueStore> storeSupplier) {
         Objects.requireNonNull(mapper);
         String name = topology.newName(MAPVALUES_NAME);
-        KTableProcessorSupplier<K, V, V1> processorSupplier = new KTableMapValues<>(this, mapper);
-
+        String internalStoreName = null;
+        if (storeSupplier != null) {
+            internalStoreName = storeSupplier.name();
+        }
+        KTableProcessorSupplier<K, V, V1> processorSupplier = new KTableMapValues<>(this, mapper, internalStoreName);
         topology.addProcessor(name, processorSupplier, this.name);
+        if (storeSupplier != null) {
+            topology.addStateStore(storeSupplier, name);
+            return new KTableImpl<>(topology, name, processorSupplier, this.keySerde, valueSerde, sourceNodes, internalStoreName, true);
+        } else {
+            return new KTableImpl<>(topology, name, processorSupplier, sourceNodes, this.queryableStoreName, false);
+        }
+    }
+
+    @Override
+    public <V1> KTable<K, V1> mapValues(final ValueMapper<? super V, ? extends V1> mapper) {
+        return mapValues(mapper, null, (String) null);
+    }
 
-        return new KTableImpl<>(topology, name, processorSupplier, sourceNodes, this.storeName);
+    @Override
+    public <V1> KTable<K, V1> mapValues(final ValueMapper<? super V, ? extends V1> mapper,
+                                        final Serde<V1> valueSerde,
+                                        final String queryableStoreName) {
+        StateStoreSupplier<KeyValueStore> storeSupplier = null;
+        if (queryableStoreName != null) {
+            storeSupplier = keyValueStore(this.keySerde, valueSerde, queryableStoreName);
+        }
+        return doMapValues(mapper, valueSerde, storeSupplier);
+    }
+
+    @Override
+    public  <V1> KTable<K, V1> mapValues(final ValueMapper<? super V, ? extends V1> mapper,
+                                         final Serde<V1> valueSerde,
+                                         final StateStoreSupplier<KeyValueStore> storeSupplier) {
+        Objects.requireNonNull(storeSupplier, "storeSupplier can't be null");
+        return doMapValues(mapper, valueSerde, storeSupplier);
     }
 
     @Override
@@ -193,30 +294,98 @@ public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K,
     }
 
     @Override
-    public KTable<K, V> through(Serde<K> keySerde,
-                                Serde<V> valSerde,
-                                StreamPartitioner<? super K, ? super V> partitioner,
-                                String topic,
-                                final String storeName) {
-        Objects.requireNonNull(storeName, "storeName can't be null");
+    public KTable<K, V> through(final Serde<K> keySerde,
+                                final Serde<V> valSerde,
+                                final StreamPartitioner<? super K, ? super V> partitioner,
+                                final String topic,
+                                final String queryableStoreName) {
+        final String internalStoreName = queryableStoreName != null ? queryableStoreName : topology.newStoreName(KTableImpl.TOSTREAM_NAME);
+
+        to(keySerde, valSerde, partitioner, topic);
+
+        return topology.table(keySerde, valSerde, topic, internalStoreName);
+    }
+
+    @Override
+    public KTable<K, V> through(final Serde<K> keySerde,
+                                final Serde<V> valSerde,
+                                final StreamPartitioner<? super K, ? super V> partitioner,
+                                final String topic,
+                                final StateStoreSupplier<KeyValueStore> storeSupplier) {
+        Objects.requireNonNull(storeSupplier, "storeSupplier can't be null");
         to(keySerde, valSerde, partitioner, topic);
 
-        return topology.table(keySerde, valSerde, topic, storeName);
+        return topology.table(keySerde, valSerde, topic, storeSupplier);
+    }
+
+    @Override
+    public KTable<K, V> through(final Serde<K> keySerde,
+                                final Serde<V> valSerde,
+                                final StreamPartitioner<? super K, ? super V> partitioner,
+                                final String topic) {
+        return through(keySerde, valSerde, partitioner, topic, (String) null);
+    }
+    @Override
+    public KTable<K, V> through(final Serde<K> keySerde,
+                                final Serde<V> valSerde,
+                                final String topic,
+                                final String queryableStoreName) {
+        return through(keySerde, valSerde, null, topic, queryableStoreName);
+    }
+
+    @Override
+    public KTable<K, V> through(final Serde<K> keySerde,
+                                final Serde<V> valSerde,
+                                final String topic,
+                                final StateStoreSupplier<KeyValueStore> storeSupplier) {
+        Objects.requireNonNull(storeSupplier, "storeSupplier can't be null");
+        return through(keySerde, valSerde, null, topic, storeSupplier);
+    }
+
+    @Override
+    public KTable<K, V> through(final Serde<K> keySerde,
+                                final Serde<V> valSerde,
+                                final String topic) {
+        return through(keySerde, valSerde, null, topic, (String) null);
+    }
+
+    @Override
+    public KTable<K, V> through(final StreamPartitioner<? super K, ? super V> partitioner,
+                                final String topic,
+                                final String queryableStoreName) {
+        return through(null, null, partitioner, topic, queryableStoreName);
+    }
+
+    @Override
+    public KTable<K, V> through(final StreamPartitioner<? super K, ? super V> partitioner,
+                                final String topic,
+                                final StateStoreSupplier<KeyValueStore> storeSupplier) {
+        Objects.requireNonNull(storeSupplier, "storeSupplier can't be null");
+        return through(null, null, partitioner, topic, storeSupplier);
+    }
+
+    @Override
+    public KTable<K, V> through(final StreamPartitioner<? super K, ? super V> partitioner,
+                                final String topic) {
+        return through(null, null, partitioner, topic, (String) null);
     }
 
     @Override
-    public KTable<K, V> through(Serde<K> keySerde, Serde<V> valSerde, String topic, final String storeName) {
-        return through(keySerde, valSerde, null, topic, storeName);
+    public KTable<K, V> through(final String topic,
+                                final String queryableStoreName) {
+        return through(null, null, null, topic, queryableStoreName);
     }
 
     @Override
-    public KTable<K, V> through(StreamPartitioner<? super K, ? super V> partitioner, String topic, final String storeName) {
-        return through(null, null, partitioner, topic, storeName);
+    public KTable<K, V> through(final String topic,
+                                final StateStoreSupplier<KeyValueStore> storeSupplier) {
+        Objects.requireNonNull(storeSupplier, "storeSupplier can't be null");
+        return through(null, null, null, topic, storeSupplier);
     }
 
     @Override
-    public KTable<K, V> through(String topic, final String storeName) {
-        return through(null, null, null, topic, storeName);
+    public KTable<K, V> through(final String topic) {
+        return through(null, null, null, topic, (String) null);
     }
 
     @Override
@@ -259,27 +428,94 @@ public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K,
     }
 
     @Override
-    public <V1, R> KTable<K, R> join(final KTable<K, V1> other, final ValueJoiner<? super V, ? super V1, ? extends R> joiner) {
-        return doJoin(other, joiner, false, false);
+    public <V1, R> KTable<K, R> join(final KTable<K, V1> other,
+                                     final ValueJoiner<? super V, ? super V1, ? extends R> joiner) {
+        return doJoin(other, joiner, false, false, null, (String) null);
     }
 
+    @Override
+    public <V1, R> KTable<K, R> join(final KTable<K, V1> other,
+                                     final ValueJoiner<? super V, ? super V1, ? extends R> joiner,
+                                     final Serde<R> joinSerde,
+                                     final String queryableStoreName) {
+        return doJoin(other, joiner, false, false, joinSerde, queryableStoreName);
+    }
+
+    @Override
+    public <V1, R> KTable<K, R> join(final KTable<K, V1> other,
+                                     final ValueJoiner<? super V, ? super V1, ? extends R> joiner,
+                                     final StateStoreSupplier<KeyValueStore> storeSupplier) {
+        Objects.requireNonNull(storeSupplier, "storeSupplier can't be null");
+        return doJoin(other, joiner, false, false, storeSupplier);
+    }
+
+    @Override
+    public <V1, R> KTable<K, R> outerJoin(final KTable<K, V1> other,
+                                          final ValueJoiner<? super V, ? super V1, ? extends R> joiner) {
+        return doJoin(other, joiner, true, true, null, (String) null);
+    }
 
+    @Override
+    public <V1, R> KTable<K, R> outerJoin(final KTable<K, V1> other,
+                                          final ValueJoiner<? super V, ? super V1, ? extends R> joiner,
+                                          final Serde<R> joinSerde,
+                                          final String queryableStoreName) {
+        return doJoin(other, joiner, true, true, joinSerde, queryableStoreName);
+    }
 
     @Override
-    public <V1, R> KTable<K, R> outerJoin(final KTable<K, V1> other, final ValueJoiner<? super V, ? super V1, ? extends R> joiner) {
-        return doJoin(other, joiner, true, true);
+    public <V1, R> KTable<K, R> outerJoin(final KTable<K, V1> other,
+                                          final ValueJoiner<? super V, ? super V1, ? extends R> joiner,
+                                          final StateStoreSupplier<KeyValueStore> storeSupplier) {
+        Objects.requireNonNull(storeSupplier, "storeSupplier can't be null");
+        return doJoin(other, joiner, true, true, storeSupplier);
     }
 
     @Override
-    public <V1, R> KTable<K, R> leftJoin(final KTable<K, V1> other, final ValueJoiner<? super V, ? super V1, ? extends R> joiner) {
-        return doJoin(other, joiner, true, false);
+    public <V1, R> KTable<K, R> leftJoin(final KTable<K, V1> other,
+                                         final ValueJoiner<? super V, ? super V1, ? extends R> joiner) {
+        return doJoin(other, joiner, true, false, null, (String) null);
+    }
+
+    @Override
+    public <V1, R> KTable<K, R> leftJoin(final KTable<K, V1> other,
+                                         final ValueJoiner<? super V, ? super V1, ? extends R> joiner,
+                                         final Serde<R> joinSerde,
+                                         final String queryableStoreName) {
+        return doJoin(other, joiner, true, false, joinSerde, queryableStoreName);
+    }
+
+    @Override
+    public <V1, R> KTable<K, R> leftJoin(final KTable<K, V1> other,
+                                         final ValueJoiner<? super V, ? super V1, ? extends R> joiner,
+                                         final StateStoreSupplier<KeyValueStore> storeSupplier) {
+        Objects.requireNonNull(storeSupplier, "storeSupplier can't be null");
+        return doJoin(other, joiner, true, false, storeSupplier);
     }
 
     @SuppressWarnings("unchecked")
-    private <V1, R> KTable<K, R> doJoin(final KTable<K, V1> other, ValueJoiner<? super V, ? super V1, ? extends R> joiner, final boolean leftOuter, final boolean rightOuter) {
+    private <V1, R> KTable<K, R> doJoin(final KTable<K, V1> other,
+                                        ValueJoiner<? super V, ? super V1, ? extends R> joiner,
+                                        final boolean leftOuter,
+                                        final boolean rightOuter,
+                                        final Serde<R> joinSerde,
+                                        final String queryableStoreName) {
         Objects.requireNonNull(other, "other can't be null");
         Objects.requireNonNull(joiner, "joiner can't be null");
 
+        final StateStoreSupplier storeSupplier = queryableStoreName == null ? null : keyValueStore(this.keySerde, joinSerde, queryableStoreName);
+
+        return doJoin(other, joiner, leftOuter, rightOuter, storeSupplier);
+    }
+
+    private <V1, R> KTable<K, R> doJoin(final KTable<K, V1> other,
+                                        ValueJoiner<? super V, ? super V1, ? extends R> joiner,
+                                        final boolean leftOuter,
+                                        final boolean rightOuter,
+                                        final StateStoreSupplier<KeyValueStore> storeSupplier) {
+        Objects.requireNonNull(other, "other can't be null");
+        Objects.requireNonNull(joiner, "joiner can't be null");
+        final String internalQueryableName = storeSupplier == null ? null : storeSupplier.name();
         final Set<String> allSourceNodes = ensureJoinableWith((AbstractStream<K>) other);
 
         if (leftOuter) {
@@ -308,8 +544,10 @@ public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K,
         }
 
         final KTableKTableJoinMerger<K, R> joinMerge = new KTableKTableJoinMerger<>(
-            new KTableImpl<K, V, R>(topology, joinThisName, joinThis, sourceNodes, storeName),
-                new KTableImpl<K, V1, R>(topology, joinOtherName, joinOther, ((KTableImpl<K, ?, ?>) other).sourceNodes, other.getStoreName())
+                new KTableImpl<K, V, R>(topology, joinThisName, joinThis, sourceNodes, this.internalStoreName(), false),
+                new KTableImpl<K, V1, R>(topology, joinOtherName, joinOther, ((KTableImpl<K, ?, ?>) other).sourceNodes,
+                        ((KTableImpl<K, ?, ?>) other).internalStoreName(), false),
+                internalQueryableName
         );
 
         topology.addProcessor(joinThisName, joinThis, this.name);
@@ -318,7 +556,11 @@ public class KTableImpl<K, S, V> extends AbstractStream<K> implements KTable<K,
         topology.connectProcessorAndStateStores(joinThisName, ((KTableImpl) other).valueGetterSupplier().storeNames());
         topology.connectProcessorAndStateStores(joinOtherName, valueGetterSupplier().storeNames());
 
-        return new KTableImpl<>(topology, joinMergeName, joinMerge, allSourceNodes, null);
+        if (internalQueryableName != null) {
+            topology.addStateStore(storeSupplier, joinMergeName);
+        }
+
+        return new KTableImpl<>(topology, joinMergeName, joinMerge, allSourceNodes, internalQueryableName, internalQueryableName != null);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java
----------------------------------------------------------------------
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java
index ee7a064..82d9c26 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinMerger.java
@@ -18,15 +18,22 @@ package org.apache.kafka.streams.kstream.internals;
 
 import org.apache.kafka.streams.processor.AbstractProcessor;
 import org.apache.kafka.streams.processor.Processor;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.state.KeyValueStore;
 
 class KTableKTableJoinMerger<K, V> implements KTableProcessorSupplier<K, V, V> {
 
     private final KTableImpl<K, ?, V> parent1;
     private final KTableImpl<K, ?, V> parent2;
+    private final String queryableName;
+    private boolean sendOldValues = false;
 
-    public KTableKTableJoinMerger(KTableImpl<K, ?, V> parent1, KTableImpl<K, ?, V> parent2) {
+    public KTableKTableJoinMerger(final KTableImpl<K, ?, V> parent1,
+                                  final KTableImpl<K, ?, V> parent2,
+                                  final String queryableName) {
         this.parent1 = parent1;
         this.parent2 = parent2;
+        this.queryableName = queryableName;
     }
 
     @Override
@@ -43,13 +50,35 @@ class KTableKTableJoinMerger<K, V> implements KTableProcessorSupplier<K, V, V> {
     public void enableSendingOldValues() {
         parent1.enableSendingOldValues();
         parent2.enableSendingOldValues();
+        sendOldValues = true;
     }
 
-    private static final class KTableKTableJoinMergeProcessor<K, V>
+    private class KTableKTableJoinMergeProcessor<K, V>
         extends AbstractProcessor<K, Change<V>> {
+        private KeyValueStore<K, V> store;
+        private TupleForwarder<K, V> tupleForwarder;
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public void init(ProcessorContext context) {
+            super.init(context);
+            if (queryableName != null) {
+                store = (KeyValueStore<K, V>) context.getStateStore(queryableName);
+                tupleForwarder = new TupleForwarder<>(store, context,
+                    new ForwardingCacheFlushListener<K, V>(context, sendOldValues),
+                    sendOldValues);
+            }
+        }
+
         @Override
         public void process(K key, Change<V> value) {
-            context().forward(key, value);
+
+            if (queryableName != null) {
+                store.put(key, value.newValue);
+                tupleForwarder.maybeForward(key, value.newValue, value.oldValue);
+            } else {
+                context().forward(key, value);
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java
----------------------------------------------------------------------
diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java
index 90610de..41dd7cd 100644
--- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java
+++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableMapValues.java
@@ -20,18 +20,21 @@ import org.apache.kafka.streams.kstream.ValueMapper;
 import org.apache.kafka.streams.processor.AbstractProcessor;
 import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.state.KeyValueStore;
 
 
 class KTableMapValues<K, V, V1> implements KTableProcessorSupplier<K, V, V1> {
 
     private final KTableImpl<K, ?, V> parent;
     private final ValueMapper<? super V, ? extends V1> mapper;
-
+    private final String queryableName;
     private boolean sendOldValues = false;
 
-    public KTableMapValues(KTableImpl<K, ?, V> parent, ValueMapper<? super V, ? extends V1> mapper) {
+    public KTableMapValues(final KTableImpl<K, ?, V> parent, final ValueMapper<? super V, ? extends V1> mapper,
+                           final String queryableName) {
         this.parent = parent;
         this.mapper = mapper;
+        this.queryableName = queryableName;
     }
 
     @Override
@@ -73,12 +76,30 @@ class KTableMapValues<K, V, V1> implements KTableProcessorSupplier<K, V, V1> {
 
     private class KTableMapValuesProcessor extends AbstractProcessor<K, Change<V>> {
 
+        private KeyValueStore<K, V1> store;
+        private TupleForwarder<K, V1> tupleForwarder;
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public void init(ProcessorContext context) {
+            super.init(context);
+            if (queryableName != null) {
+                store = (KeyValueStore<K, V1>) context.getStateStore(queryableName);
+                tupleForwarder = new TupleForwarder<>(store, context, new ForwardingCacheFlushListener<K, V1>(context, sendOldValues), sendOldValues);
+            }
+        }
+
         @Override
         public void process(K key, Change<V> change) {
             V1 newValue = computeValue(change.newValue);
             V1 oldValue = sendOldValues ? computeValue(change.oldValue) : null;
 
-            context().forward(key, new Change<>(newValue, oldValue));
+            if (queryableName != null) {
+                store.put(key, newValue);
+                tupleForwarder.maybeForward(key, newValue, oldValue);
+            } else {
+                context().forward(key, new Change<>(newValue, oldValue));
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java
----------------------------------------------------------------------
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java
index a7bc5bd..81f0ff9 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java
@@ -30,6 +30,7 @@ import org.apache.kafka.streams.processor.internals.QuickUnion;
 import org.apache.kafka.streams.processor.internals.SinkNode;
 import org.apache.kafka.streams.processor.internals.SourceNode;
 import org.apache.kafka.streams.processor.internals.StreamPartitionAssignor.SubscriptionUpdates;
+import org.apache.kafka.streams.state.KeyValueStore;
 import org.apache.kafka.streams.state.internals.WindowStoreSupplier;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -460,7 +461,7 @@ public class TopologyBuilder {
      * receive all records forwarded from the {@link SourceNode}. This
      * {@link ProcessorNode} should be used to keep the {@link StateStore} up-to-date.
      *
-     * @param store                 the instance of {@link StateStore}
+     * @param storeSupplier         user defined state store supplier
      * @param sourceName            name of the {@link SourceNode} that will be automatically added
      * @param keyDeserializer       the {@link Deserializer} to deserialize keys with
      * @param valueDeserializer     the {@link Deserializer} to deserialize values with
@@ -469,14 +470,14 @@ public class TopologyBuilder {
      * @param stateUpdateSupplier   the instance of {@link ProcessorSupplier}
      * @return this builder instance so methods can be chained together; never null
      */
-    public synchronized TopologyBuilder addGlobalStore(final StateStore store,
+    public synchronized TopologyBuilder addGlobalStore(final StateStoreSupplier<KeyValueStore> storeSupplier,
                                                        final String sourceName,
                                                        final Deserializer keyDeserializer,
                                                        final Deserializer valueDeserializer,
                                                        final String topic,
                                                        final String processorName,
                                                        final ProcessorSupplier stateUpdateSupplier) {
-        Objects.requireNonNull(store, "store must not be null");
+        Objects.requireNonNull(storeSupplier, "store supplier must not be null");
         Objects.requireNonNull(sourceName, "sourceName must not be null");
         Objects.requireNonNull(topic, "topic must not be null");
         Objects.requireNonNull(stateUpdateSupplier, "supplier must not be null");
@@ -487,8 +488,11 @@ public class TopologyBuilder {
         if (nodeFactories.containsKey(processorName)) {
             throw new TopologyBuilderException("Processor " + processorName + " is already added.");
         }
-        if (stateFactories.containsKey(store.name()) || globalStateStores.containsKey(store.name())) {
-            throw new TopologyBuilderException("StateStore " + store.name() + " is already added.");
+        if (stateFactories.containsKey(storeSupplier.name()) || globalStateStores.containsKey(storeSupplier.name())) {
+            throw new TopologyBuilderException("StateStore " + storeSupplier.name() + " is already added.");
+        }
+        if (storeSupplier.loggingEnabled()) {
+            throw new TopologyBuilderException("StateStore " + storeSupplier.name() + " for global table must not have logging enabled.");
         }
         if (sourceName.equals(processorName)) {
             throw new TopologyBuilderException("sourceName and processorName must be different.");
@@ -504,13 +508,13 @@ public class TopologyBuilder {
 
         final String[] parents = {sourceName};
         final ProcessorNodeFactory nodeFactory = new ProcessorNodeFactory(processorName, parents, stateUpdateSupplier);
-        nodeFactory.addStateStore(store.name());
+        nodeFactory.addStateStore(storeSupplier.name());
         nodeFactories.put(processorName, nodeFactory);
         nodeGrouper.add(processorName);
         nodeGrouper.unite(processorName, parents);
 
-        globalStateStores.put(store.name(), store);
-        connectSourceStoreAndTopic(store.name(), topic);
+        globalStateStores.put(storeSupplier.name(), storeSupplier.get());
+        connectSourceStoreAndTopic(storeSupplier.name(), topic);
         return this;
 
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
index e81c7d3..0722210 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java
@@ -355,13 +355,7 @@ public class KStreamAggregationIntegrationTest {
             )));
     }
 
-    @Test
-    public void shouldCount() throws Exception {
-        produceMessages(mockTime.milliseconds());
-
-        groupedStream.count("count-by-key")
-            .to(Serdes.String(), Serdes.Long(), outputTopic);
-
+    private void shouldCountHelper() throws Exception {
         startStreams();
 
         produceMessages(mockTime.milliseconds());
@@ -392,6 +386,26 @@ public class KStreamAggregationIntegrationTest {
     }
 
     @Test
+    public void shouldCount() throws Exception {
+        produceMessages(mockTime.milliseconds());
+
+        groupedStream.count("count-by-key")
+            .to(Serdes.String(), Serdes.Long(), outputTopic);
+
+        shouldCountHelper();
+    }
+
+    @Test
+    public void shouldCountWithInternalStore() throws Exception {
+        produceMessages(mockTime.milliseconds());
+
+        groupedStream.count()
+            .to(Serdes.String(), Serdes.Long(), outputTopic);
+
+        shouldCountHelper();
+    }
+
+    @Test
     public void shouldGroupByKey() throws Exception {
         final long timestamp = mockTime.milliseconds();
         produceMessages(timestamp);

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableJoinIntegrationTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableJoinIntegrationTest.java
index 26deb92..5fa7e49 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableJoinIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableJoinIntegrationTest.java
@@ -30,6 +30,9 @@ import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
 import org.apache.kafka.streams.kstream.KStreamBuilder;
 import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.streams.kstream.ValueJoiner;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.QueryableStoreTypes;
+import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
 import org.apache.kafka.test.IntegrationTest;
 import org.apache.kafka.test.TestUtils;
 import org.junit.After;
@@ -41,11 +44,15 @@ import org.junit.experimental.categories.Category;
 
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.equalTo;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
 
 @Category({IntegrationTest.class})
 public class KTableKTableJoinIntegrationTest {
@@ -134,12 +141,22 @@ public class KTableKTableJoinIntegrationTest {
 
     @Test
     public void shouldInnerInnerJoin() throws Exception {
-        verifyKTableKTableJoin(JoinType.INNER, JoinType.INNER, Collections.singletonList(new KeyValue<>("b", "B1-B2-B3")));
+        verifyKTableKTableJoin(JoinType.INNER, JoinType.INNER, Collections.singletonList(new KeyValue<>("b", "B1-B2-B3")), false);
+    }
+
+    @Test
+    public void shouldInnerInnerJoinQueryable() throws Exception {
+        verifyKTableKTableJoin(JoinType.INNER, JoinType.INNER, Collections.singletonList(new KeyValue<>("b", "B1-B2-B3")), true);
     }
 
     @Test
     public void shouldInnerLeftJoin() throws Exception {
-        verifyKTableKTableJoin(JoinType.INNER, JoinType.LEFT, Collections.singletonList(new KeyValue<>("b", "B1-B2-B3")));
+        verifyKTableKTableJoin(JoinType.INNER, JoinType.LEFT, Collections.singletonList(new KeyValue<>("b", "B1-B2-B3")), false);
+    }
+
+    @Test
+    public void shouldInnerLeftJoinQueryable() throws Exception {
+        verifyKTableKTableJoin(JoinType.INNER, JoinType.LEFT, Collections.singletonList(new KeyValue<>("b", "B1-B2-B3")), true);
     }
 
     @Test
@@ -148,7 +165,16 @@ public class KTableKTableJoinIntegrationTest {
                 new KeyValue<>("a", "null-A3"),
                 new KeyValue<>("b", "null-B3"),
                 new KeyValue<>("c", "null-C3"),
-                new KeyValue<>("b", "B1-B2-B3")));
+                new KeyValue<>("b", "B1-B2-B3")), false);
+    }
+
+    @Test
+    public void shouldInnerOuterJoinQueryable() throws Exception {
+        verifyKTableKTableJoin(JoinType.INNER, JoinType.OUTER, Arrays.asList(
+            new KeyValue<>("a", "null-A3"),
+            new KeyValue<>("b", "null-B3"),
+            new KeyValue<>("c", "null-C3"),
+            new KeyValue<>("b", "B1-B2-B3")), true);
     }
 
     @Test
@@ -156,7 +182,15 @@ public class KTableKTableJoinIntegrationTest {
         verifyKTableKTableJoin(JoinType.LEFT, JoinType.INNER, Arrays.asList(
                 new KeyValue<>("a", "A1-null-A3"),
                 new KeyValue<>("b", "B1-null-B3"),
-                new KeyValue<>("b", "B1-B2-B3")));
+                new KeyValue<>("b", "B1-B2-B3")), false);
+    }
+
+    @Test
+    public void shouldLeftInnerJoinQueryable() throws Exception {
+        verifyKTableKTableJoin(JoinType.LEFT, JoinType.INNER, Arrays.asList(
+            new KeyValue<>("a", "A1-null-A3"),
+            new KeyValue<>("b", "B1-null-B3"),
+            new KeyValue<>("b", "B1-B2-B3")), true);
     }
 
     @Test
@@ -164,7 +198,15 @@ public class KTableKTableJoinIntegrationTest {
         verifyKTableKTableJoin(JoinType.LEFT, JoinType.LEFT, Arrays.asList(
                 new KeyValue<>("a", "A1-null-A3"),
                 new KeyValue<>("b", "B1-null-B3"),
-                new KeyValue<>("b", "B1-B2-B3")));
+                new KeyValue<>("b", "B1-B2-B3")), false);
+    }
+
+    @Test
+    public void shouldLeftLeftJoinQueryable() throws Exception {
+        verifyKTableKTableJoin(JoinType.LEFT, JoinType.LEFT, Arrays.asList(
+            new KeyValue<>("a", "A1-null-A3"),
+            new KeyValue<>("b", "B1-null-B3"),
+            new KeyValue<>("b", "B1-B2-B3")), true);
     }
 
     @Test
@@ -175,7 +217,18 @@ public class KTableKTableJoinIntegrationTest {
                 new KeyValue<>("c", "null-C3"),
                 new KeyValue<>("a", "A1-null-A3"),
                 new KeyValue<>("b", "B1-null-B3"),
-                new KeyValue<>("b", "B1-B2-B3")));
+                new KeyValue<>("b", "B1-B2-B3")), false);
+    }
+
+    @Test
+    public void shouldLeftOuterJoinQueryable() throws Exception {
+        verifyKTableKTableJoin(JoinType.LEFT, JoinType.OUTER, Arrays.asList(
+            new KeyValue<>("a", "null-A3"),
+            new KeyValue<>("b", "null-B3"),
+            new KeyValue<>("c", "null-C3"),
+            new KeyValue<>("a", "A1-null-A3"),
+            new KeyValue<>("b", "B1-null-B3"),
+            new KeyValue<>("b", "B1-B2-B3")), true);
     }
 
     @Test
@@ -184,7 +237,16 @@ public class KTableKTableJoinIntegrationTest {
                 new KeyValue<>("a", "A1-null-A3"),
                 new KeyValue<>("b", "B1-null-B3"),
                 new KeyValue<>("b", "B1-B2-B3"),
-                new KeyValue<>("c", "null-C2-C3")));
+                new KeyValue<>("c", "null-C2-C3")), false);
+    }
+
+    @Test
+    public void shouldOuterInnerJoinQueryable() throws Exception {
+        verifyKTableKTableJoin(JoinType.OUTER, JoinType.INNER, Arrays.asList(
+            new KeyValue<>("a", "A1-null-A3"),
+            new KeyValue<>("b", "B1-null-B3"),
+            new KeyValue<>("b", "B1-B2-B3"),
+            new KeyValue<>("c", "null-C2-C3")), true);
     }
 
     @Test
@@ -193,7 +255,16 @@ public class KTableKTableJoinIntegrationTest {
                 new KeyValue<>("a", "A1-null-A3"),
                 new KeyValue<>("b", "B1-null-B3"),
                 new KeyValue<>("b", "B1-B2-B3"),
-                new KeyValue<>("c", "null-C2-C3")));
+                new KeyValue<>("c", "null-C2-C3")), false);
+    }
+
+    @Test
+    public void shouldOuterLeftJoinQueryable() throws Exception {
+        verifyKTableKTableJoin(JoinType.OUTER, JoinType.LEFT,  Arrays.asList(
+            new KeyValue<>("a", "A1-null-A3"),
+            new KeyValue<>("b", "B1-null-B3"),
+            new KeyValue<>("b", "B1-B2-B3"),
+            new KeyValue<>("c", "null-C2-C3")), true);
     }
 
     @Test
@@ -205,16 +276,30 @@ public class KTableKTableJoinIntegrationTest {
                 new KeyValue<>("a", "A1-null-A3"),
                 new KeyValue<>("b", "B1-null-B3"),
                 new KeyValue<>("b", "B1-B2-B3"),
-                new KeyValue<>("c", "null-C2-C3")));
+                new KeyValue<>("c", "null-C2-C3")), false);
+    }
+
+    @Test
+    public void shouldOuterOuterJoinQueryable() throws Exception {
+        verifyKTableKTableJoin(JoinType.OUTER, JoinType.OUTER, Arrays.asList(
+            new KeyValue<>("a", "null-A3"),
+            new KeyValue<>("b", "null-B3"),
+            new KeyValue<>("c", "null-C3"),
+            new KeyValue<>("a", "A1-null-A3"),
+            new KeyValue<>("b", "B1-null-B3"),
+            new KeyValue<>("b", "B1-B2-B3"),
+            new KeyValue<>("c", "null-C2-C3")), true);
     }
 
 
     private void verifyKTableKTableJoin(final JoinType joinType1,
                                         final JoinType joinType2,
-                                        final List<KeyValue<String, String>> expectedResult) throws Exception {
-        streamsConfig.put(StreamsConfig.APPLICATION_ID_CONFIG, joinType1 + "-" + joinType2 + "-ktable-ktable-join");
+                                        final List<KeyValue<String, String>> expectedResult,
+                                        boolean verifyQueryableState) throws Exception {
+        final String queryableName = verifyQueryableState ? joinType1 + "-" + joinType2 + "-ktable-ktable-join-query" : null;
+        streamsConfig.put(StreamsConfig.APPLICATION_ID_CONFIG, joinType1 + "-" + joinType2 + "-ktable-ktable-join" + queryableName);
 
-        streams = prepareTopology(joinType1, joinType2);
+        streams = prepareTopology(joinType1, joinType2, queryableName);
         streams.start();
 
         final List<KeyValue<String, String>> result = IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(
@@ -223,20 +308,54 @@ public class KTableKTableJoinIntegrationTest {
                 expectedResult.size());
 
         assertThat(result, equalTo(expectedResult));
+
+        if (verifyQueryableState) {
+            verifyKTableKTableJoinQueryableState(joinType1, joinType2, expectedResult);
+        }
     }
-    private KafkaStreams prepareTopology(final JoinType joinType1, final JoinType joinType2) {
+
+    private void verifyKTableKTableJoinQueryableState(final JoinType joinType1,
+                                                      final JoinType joinType2,
+                                                      final List<KeyValue<String, String>> expectedResult) {
+        final String queryableName = joinType1 + "-" + joinType2 + "-ktable-ktable-join-query";
+        final ReadOnlyKeyValueStore<String, String> myJoinStore = streams.store(queryableName,
+            QueryableStoreTypes.<String, String>keyValueStore());
+
+        // store only keeps last set of values, not entire stream of value changes
+        final Map<String, String> expectedInStore = new HashMap<>();
+        for (KeyValue<String, String> expected : expectedResult) {
+            expectedInStore.put(expected.key, expected.value);
+        }
+
+        for (Map.Entry<String, String> expected : expectedInStore.entrySet()) {
+            assertEquals(expected.getValue(), myJoinStore.get(expected.getKey()));
+        }
+        final KeyValueIterator<String, String> all = myJoinStore.all();
+        while (all.hasNext()) {
+            KeyValue<String, String> storeEntry = all.next();
+            assertTrue(expectedResult.contains(storeEntry));
+        }
+        all.close();
+
+    }
+
+    private KafkaStreams prepareTopology(final JoinType joinType1, final JoinType joinType2, final String queryableName) {
         final KStreamBuilder builder = new KStreamBuilder();
 
         final KTable<String, String> table1 = builder.table(TABLE_1, TABLE_1);
         final KTable<String, String> table2 = builder.table(TABLE_2, TABLE_2);
         final KTable<String, String> table3 = builder.table(TABLE_3, TABLE_3);
 
-        join(join(table1, table2, joinType1), table3, joinType2).to(OUTPUT);
+        join(join(table1, table2, joinType1, null /* no need to query intermediate result */), table3,
+            joinType2, queryableName).to(OUTPUT);
 
         return new KafkaStreams(builder, new StreamsConfig(streamsConfig));
     }
 
-    private KTable<String, String> join(KTable<String, String> first, KTable<String, String> second, JoinType joinType) {
+    private KTable<String, String> join(final KTable<String, String> first,
+                                        final KTable<String, String> second,
+                                        final JoinType joinType,
+                                        final String queryableName) {
         final ValueJoiner<String, String, String> joiner = new ValueJoiner<String, String, String>() {
             @Override
             public String apply(final String value1, final String value2) {
@@ -246,11 +365,11 @@ public class KTableKTableJoinIntegrationTest {
 
         switch (joinType) {
             case INNER:
-                return first.join(second, joiner);
+                return first.join(second, joiner, Serdes.String(), queryableName);
             case LEFT:
-                return first.leftJoin(second, joiner);
+                return first.leftJoin(second, joiner, Serdes.String(), queryableName);
             case OUTER:
-                return first.outerJoin(second, joiner);
+                return first.outerJoin(second, joiner, Serdes.String(), queryableName);
         }
 
         throw new RuntimeException("Unknown join type.");

http://git-wip-us.apache.org/repos/asf/kafka/blob/ec9e4eaf/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java
index 314079f..b435ceb 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java
@@ -22,6 +22,7 @@ import org.apache.kafka.clients.producer.KafkaProducer;
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.serialization.LongDeserializer;
+import org.apache.kafka.common.serialization.LongSerializer;
 import org.apache.kafka.common.serialization.Serde;
 import org.apache.kafka.common.serialization.Serdes;
 import org.apache.kafka.common.serialization.StringDeserializer;
@@ -36,6 +37,8 @@ import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
 import org.apache.kafka.streams.kstream.KGroupedStream;
 import org.apache.kafka.streams.kstream.KStream;
 import org.apache.kafka.streams.kstream.KStreamBuilder;
+import org.apache.kafka.streams.kstream.KTable;
+import org.apache.kafka.streams.kstream.Predicate;
 import org.apache.kafka.streams.kstream.TimeWindows;
 import org.apache.kafka.streams.kstream.ValueMapper;
 import org.apache.kafka.streams.state.KeyValueIterator;
@@ -71,6 +74,7 @@ import java.util.concurrent.TimeUnit;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.core.IsEqual.equalTo;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -425,6 +429,180 @@ public class QueryableStateIntegrationTest {
         verifyCanQueryState(10 * 1024 * 1024);
     }
 
+    @Test
+    public void shouldBeAbleToQueryFilterState() throws Exception {
+        streamsConfiguration.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
+        streamsConfiguration.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.Long().getClass());
+        final KStreamBuilder builder = new KStreamBuilder();
+        final String[] keys = {"hello", "goodbye", "welcome", "go", "kafka"};
+        final Set<KeyValue<String, Long>> batch1 = new HashSet<>();
+        batch1.addAll(Arrays.asList(
+            new KeyValue<>(keys[0], 1L),
+            new KeyValue<>(keys[1], 1L),
+            new KeyValue<>(keys[2], 3L),
+            new KeyValue<>(keys[3], 5L),
+            new KeyValue<>(keys[4], 2L)));
+        final Set<KeyValue<String, Long>> expectedBatch1 = new HashSet<>();
+        expectedBatch1.addAll(Arrays.asList(
+            new KeyValue<>(keys[4], 2L)));
+
+        IntegrationTestUtils.produceKeyValuesSynchronously(
+            streamOne,
+            batch1,
+            TestUtils.producerConfig(
+                CLUSTER.bootstrapServers(),
+                StringSerializer.class,
+                LongSerializer.class,
+                new Properties()),
+            mockTime);
+        final Predicate<String, Long> filterPredicate = new Predicate<String, Long>() {
+            @Override
+            public boolean test(String key, Long value) {
+                return key.contains("kafka");
+            }
+        };
+        final KTable<String, Long> t1 = builder.table(streamOne);
+        final KTable<String, Long> t2 = t1.filter(filterPredicate, "queryFilter");
+        t1.filterNot(filterPredicate, "queryFilterNot");
+        t2.to(outputTopic);
+
+        kafkaStreams = new KafkaStreams(builder, streamsConfiguration);
+        kafkaStreams.start();
+
+        waitUntilAtLeastNumRecordProcessed(outputTopic, 2);
+
+        final ReadOnlyKeyValueStore<String, Long>
+            myFilterStore = kafkaStreams.store("queryFilter", QueryableStoreTypes.<String, Long>keyValueStore());
+        final ReadOnlyKeyValueStore<String, Long>
+            myFilterNotStore = kafkaStreams.store("queryFilterNot", QueryableStoreTypes.<String, Long>keyValueStore());
+
+        for (final KeyValue<String, Long> expectedEntry : expectedBatch1) {
+            assertEquals(myFilterStore.get(expectedEntry.key), expectedEntry.value);
+        }
+        for (final KeyValue<String, Long> batchEntry : batch1) {
+            if (!expectedBatch1.contains(batchEntry)) {
+                assertNull(myFilterStore.get(batchEntry.key));
+            }
+        }
+
+        for (final KeyValue<String, Long> expectedEntry : expectedBatch1) {
+            assertNull(myFilterNotStore.get(expectedEntry.key));
+        }
+        for (final KeyValue<String, Long> batchEntry : batch1) {
+            if (!expectedBatch1.contains(batchEntry)) {
+                assertEquals(myFilterNotStore.get(batchEntry.key), batchEntry.value);
+            }
+        }
+    }
+
+    @Test
+    public void shouldBeAbleToQueryMapValuesState() throws Exception {
+        streamsConfiguration.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
+        streamsConfiguration.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
+        final KStreamBuilder builder = new KStreamBuilder();
+        final String[] keys = {"hello", "goodbye", "welcome", "go", "kafka"};
+        final Set<KeyValue<String, String>> batch1 = new HashSet<>();
+        batch1.addAll(Arrays.asList(
+            new KeyValue<>(keys[0], "1"),
+            new KeyValue<>(keys[1], "1"),
+            new KeyValue<>(keys[2], "3"),
+            new KeyValue<>(keys[3], "5"),
+            new KeyValue<>(keys[4], "2")));
+
+        IntegrationTestUtils.produceKeyValuesSynchronously(
+            streamOne,
+            batch1,
+            TestUtils.producerConfig(
+                CLUSTER.bootstrapServers(),
+                StringSerializer.class,
+                StringSerializer.class,
+                new Properties()),
+            mockTime);
+
+        final KTable<String, String> t1 = builder.table(streamOne);
+        final KTable<String, Long> t2 = t1.mapValues(new ValueMapper<String, Long>() {
+            @Override
+            public Long apply(String value) {
+                return Long.valueOf(value);
+            }
+        }, Serdes.Long(), "queryMapValues");
+        t2.to(Serdes.String(), Serdes.Long(), outputTopic);
+
+        kafkaStreams = new KafkaStreams(builder, streamsConfiguration);
+        kafkaStreams.start();
+
+        waitUntilAtLeastNumRecordProcessed(outputTopic, 1);
+
+        final ReadOnlyKeyValueStore<String, Long>
+            myMapStore = kafkaStreams.store("queryMapValues",
+            QueryableStoreTypes.<String, Long>keyValueStore());
+        for (final KeyValue<String, String> batchEntry : batch1) {
+            assertEquals(myMapStore.get(batchEntry.key), Long.valueOf(batchEntry.value));
+        }
+    }
+
+    @Test
+    public void shouldBeAbleToQueryMapValuesAfterFilterState() throws Exception {
+        streamsConfiguration.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
+        streamsConfiguration.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
+        final KStreamBuilder builder = new KStreamBuilder();
+        final String[] keys = {"hello", "goodbye", "welcome", "go", "kafka"};
+        final Set<KeyValue<String, String>> batch1 = new HashSet<>();
+        batch1.addAll(Arrays.asList(
+            new KeyValue<>(keys[0], "1"),
+            new KeyValue<>(keys[1], "1"),
+            new KeyValue<>(keys[2], "3"),
+            new KeyValue<>(keys[3], "5"),
+            new KeyValue<>(keys[4], "2")));
+        final Set<KeyValue<String, Long>> expectedBatch1 = new HashSet<>();
+        expectedBatch1.addAll(Arrays.asList(
+            new KeyValue<>(keys[4], 2L)));
+
+        IntegrationTestUtils.produceKeyValuesSynchronously(
+            streamOne,
+            batch1,
+            TestUtils.producerConfig(
+                CLUSTER.bootstrapServers(),
+                StringSerializer.class,
+                StringSerializer.class,
+                new Properties()),
+            mockTime);
+
+        final Predicate<String, String> filterPredicate = new Predicate<String, String>() {
+            @Override
+            public boolean test(String key, String value) {
+                return key.contains("kafka");
+            }
+        };
+        final KTable<String, String> t1 = builder.table(streamOne);
+        final KTable<String, String> t2 = t1.filter(filterPredicate, "queryFilter");
+        final KTable<String, Long> t3 = t2.mapValues(new ValueMapper<String, Long>() {
+            @Override
+            public Long apply(String value) {
+                return Long.valueOf(value);
+            }
+        }, Serdes.Long(), "queryMapValues");
+        t3.to(Serdes.String(), Serdes.Long(), outputTopic);
+
+        kafkaStreams = new KafkaStreams(builder, streamsConfiguration);
+        kafkaStreams.start();
+
+        waitUntilAtLeastNumRecordProcessed(outputTopic, 1);
+
+        final ReadOnlyKeyValueStore<String, Long>
+            myMapStore = kafkaStreams.store("queryMapValues",
+            QueryableStoreTypes.<String, Long>keyValueStore());
+        for (final KeyValue<String, Long> expectedEntry : expectedBatch1) {
+            assertEquals(myMapStore.get(expectedEntry.key), expectedEntry.value);
+        }
+        for (final KeyValue<String, String> batchEntry : batch1) {
+            final KeyValue<String, Long> batchEntryMapValue = new KeyValue<>(batchEntry.key, Long.valueOf(batchEntry.value));
+            if (!expectedBatch1.contains(batchEntryMapValue)) {
+                assertNull(myMapStore.get(batchEntry.key));
+            }
+        }
+    }
+
     private void verifyCanQueryState(int cacheSizeBytes) throws java.util.concurrent.ExecutionException, InterruptedException {
         streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, cacheSizeBytes);
         final KStreamBuilder builder = new KStreamBuilder();