vvcephei commented on a change in pull request #11541:
URL: https://github.com/apache/kafka/pull/11541#discussion_r759857291



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java
##########
@@ -131,6 +149,11 @@ public void put(final Bytes key, final byte[] value, final 
long windowStartTimes
                 });
             }
         }
+
+        if (stateStoreContext != null && 
stateStoreContext.recordMetadata().isPresent()) {

Review comment:
       I think we'll still want to advance the position, even if we otherwise 
ignore the record.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/Position.java
##########
@@ -44,10 +44,12 @@ private Position(final Map<String, Map<Integer, Long>> 
other) {
     }
 
     public Position update(final String topic, final int partition, final long 
offset) {
-        position.computeIfAbsent(topic, k -> new ConcurrentHashMap<>());
-        final ConcurrentMap<Integer, AtomicLong> topicMap = 
position.get(topic);
-        topicMap.computeIfAbsent(partition, k -> new AtomicLong(0));
-        topicMap.get(partition).getAndAccumulate(offset, Math::max);
+        if (topic != null) {
+            position.computeIfAbsent(topic, k -> new ConcurrentHashMap<>());
+            final ConcurrentMap<Integer, AtomicLong> topicMap = 
position.get(topic);
+            topicMap.computeIfAbsent(partition, k -> new AtomicLong(0));
+            topicMap.get(partition).getAndAccumulate(offset, Math::max);

Review comment:
       Computing the max is something I didn't add in the framework PR. I left 
it up to the caller, but it looks like pretty much every caller would have to 
ensure it all over the place, so I'll go ahead and update 
https://github.com/apache/kafka/pull/11557 to preserve these semantics.

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/state/internals/InMemorySessionStoreTest.java
##########
@@ -83,4 +88,31 @@ public void shouldNotExpireFromOpenIterator() {
         assertFalse(sessionStore.findSessions("a", "b", 0L, 20L).hasNext());
     }
 
+    @Test
+    public void shouldMatchPositionAfterPut() {
+
+        final List<KeyValue<Windowed<String>, Long>> entries = new 
ArrayList<>();
+        entries.add(new KeyValue<>(new Windowed<String>("a", new 
SessionWindow(0, 0)), 1L));
+        entries.add(new KeyValue<>(new Windowed<String>("aa", new 
SessionWindow(0, 10)), 2L));
+        entries.add(new KeyValue<>(new Windowed<String>("a", new 
SessionWindow(10, 20)), 3L));
+
+        final MonotonicProcessorRecordContext recordContext = new 
MonotonicProcessorRecordContext("input", 0);
+        context.setRecordContext(recordContext);
+
+        final Position expected = Position.emptyPosition();
+        long offset = 0;
+        for (final KeyValue<Windowed<String>, Long> k : entries) {
+            sessionStore.put(k.key, k.value);
+            expected.update("input", 0, offset);
+            offset++;
+        }
+
+        final MeteredSessionStore<String, Long> meteredSessionStore = 
(MeteredSessionStore<String, Long>) sessionStore;

Review comment:
       Thanks. I had this on my mind when I was working on the tests for the 
framework PR. It occurred to me that another approach would be to simply run a 
dummy query and check the returned position.
   
   But I guess that doesn't really count as a "unit" test, since the 
intervening store layers could in theory be changing the position as well, so 
that approach might not be testing what it thinks it's testing.
   
   This is more direct, and to be honest, I don't think it's that bad in the 
context of a unit test.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


Reply via email to