[GitHub] [kafka] mjsax commented on a diff in pull request #14157: KAFKA-15030: Avoid unnecessary re-serialization in FK-join

2023-08-06 Thread via GitHub


mjsax commented on code in PR #14157:
URL: https://github.com/apache/kafka/pull/14157#discussion_r1285343576


##
streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueRawStoreWrapper.java:
##
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.state.internals;
+
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.query.PositionBound;
+import org.apache.kafka.streams.query.Query;
+import org.apache.kafka.streams.query.QueryConfig;
+import org.apache.kafka.streams.query.QueryResult;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.TimestampedKeyValueStore;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+
+/**
+ * A wrapper class for non-windowed key-value stores used within the DSL. All 
such stores are
+ * instances of either {@link TimestampedKeyValueStore} or {@link 
VersionedKeyValueStore}.
+ *
+ * @param  The key type
+ * @param  The value type
+ */
+public class KeyValueRawStoreWrapper implements StateStore {
+
+public static final long PUT_RETURN_CODE_IS_LATEST
+= VersionedKeyValueStore.PUT_RETURN_CODE_VALID_TO_UNDEFINED;
+
+private KeyValueStore timestampedStore = null;
+private VersionedKeyValueStore versionedStore = null;
+
+// same as either timestampedStore or versionedStore above. kept merely as 
a convenience
+// to simplify implementation for methods which do not depend on store 
type.
+private StateStore store = null;
+
+@SuppressWarnings("unchecked")
+public KeyValueRawStoreWrapper(final ProcessorContext context, final 
String storeName) {
+try {
+// first try timestamped store
+timestampedStore = ((WrappedStateStore, K, V>) ((WrappedStateStore, K, V>) 
context.getStateStore(storeName)).wrapped()).wrapped();

Review Comment:
   This is pretty much just hacked together right now... (Same in L67 below.)
   
   This nasty unwrapping business resulted in the idea to add a `RawXxxStore` 
interface that `MeteredXxxStore` could implement, and we would just cast to 
`RawXxxStore` here instead of un-wrapping.



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



[GitHub] [kafka] mjsax commented on a diff in pull request #14157: KAFKA-15030: Avoid unnecessary re-serialization in FK-join

2023-08-06 Thread via GitHub


mjsax commented on code in PR #14157:
URL: https://github.com/apache/kafka/pull/14157#discussion_r1285343576


##
streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueRawStoreWrapper.java:
##
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.state.internals;
+
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.query.PositionBound;
+import org.apache.kafka.streams.query.Query;
+import org.apache.kafka.streams.query.QueryConfig;
+import org.apache.kafka.streams.query.QueryResult;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.TimestampedKeyValueStore;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+
+/**
+ * A wrapper class for non-windowed key-value stores used within the DSL. All 
such stores are
+ * instances of either {@link TimestampedKeyValueStore} or {@link 
VersionedKeyValueStore}.
+ *
+ * @param  The key type
+ * @param  The value type
+ */
+public class KeyValueRawStoreWrapper implements StateStore {
+
+public static final long PUT_RETURN_CODE_IS_LATEST
+= VersionedKeyValueStore.PUT_RETURN_CODE_VALID_TO_UNDEFINED;
+
+private KeyValueStore timestampedStore = null;
+private VersionedKeyValueStore versionedStore = null;
+
+// same as either timestampedStore or versionedStore above. kept merely as 
a convenience
+// to simplify implementation for methods which do not depend on store 
type.
+private StateStore store = null;
+
+@SuppressWarnings("unchecked")
+public KeyValueRawStoreWrapper(final ProcessorContext context, final 
String storeName) {
+try {
+// first try timestamped store
+timestampedStore = ((WrappedStateStore, K, V>) ((WrappedStateStore, K, V>) 
context.getStateStore(storeName)).wrapped()).wrapped();

Review Comment:
   This is pretty much just hacked together right now... (Same in L67 below.)



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



[GitHub] [kafka] mjsax commented on a diff in pull request #14157: KAFKA-15030: Avoid unnecessary re-serialization in FK-join

2023-08-06 Thread via GitHub


mjsax commented on code in PR #14157:
URL: https://github.com/apache/kafka/pull/14157#discussion_r1285341728


##
streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ResponseJoinProcessorSupplier.java:
##
@@ -44,72 +45,98 @@
  * @param  Type of foreign values
  * @param  Type of joined result of primary and foreign values
  */
-public class ResponseJoinProcessorSupplier implements 
ProcessorSupplier, K, VR> {
+public class ResponseJoinProcessorSupplier implements 
ProcessorSupplier, K, VR> {
 private static final Logger LOG = 
LoggerFactory.getLogger(ResponseJoinProcessorSupplier.class);
-private final KTableValueGetterSupplier valueGetterSupplier;
-private final Serializer constructionTimeValueSerializer;
+private final KTableValueGetterSupplier 
rawValueGetterSupplier;
+private final Deserializer keyDeserializer;
+private final Deserializer leftValueDeserializer;
+private final Deserializer rightValueDeserializer;
 private final Supplier valueHashSerdePseudoTopicSupplier;
 private final ValueJoiner joiner;
 private final boolean leftJoin;
 
-public ResponseJoinProcessorSupplier(final KTableValueGetterSupplier 
valueGetterSupplier,
- final Serializer valueSerializer,
+public ResponseJoinProcessorSupplier(final 
KTableValueGetterSupplier rawValueGetterSupplier,
+ final Deserializer keyDeserializer,
+ final Deserializer 
leftValueDeserializer,
+ final Deserializer 
rightValueDeserializer,
  final Supplier 
valueHashSerdePseudoTopicSupplier,
  final ValueJoiner joiner,
  final boolean leftJoin) {
-this.valueGetterSupplier = valueGetterSupplier;
-constructionTimeValueSerializer = valueSerializer;
+this.rawValueGetterSupplier = rawValueGetterSupplier;
+this.keyDeserializer = keyDeserializer;
+this.leftValueDeserializer = leftValueDeserializer;
+this.rightValueDeserializer = rightValueDeserializer;
 this.valueHashSerdePseudoTopicSupplier = 
valueHashSerdePseudoTopicSupplier;
 this.joiner = joiner;
 this.leftJoin = leftJoin;
 }
 
 @Override
-public Processor, K, VR> get() {
-return new ContextualProcessor, K, 
VR>() {
+public Processor, K, VR> get() {
+return new ContextualProcessor, K, VR>() {
 private String valueHashSerdePseudoTopic;
-private Serializer runtimeValueSerializer = 
constructionTimeValueSerializer;
-
-private KTableValueGetter valueGetter;
+private Deserializer keyDeserializer = 
ResponseJoinProcessorSupplier.this.keyDeserializer;
+private Deserializer leftValueDeserializer = 
ResponseJoinProcessorSupplier.this.leftValueDeserializer;
+private Deserializer rightValueDeserializer = 
ResponseJoinProcessorSupplier.this.rightValueDeserializer;
+private KTableValueGetter rawValueGetter;
 
 @SuppressWarnings("unchecked")
 @Override
 public void init(final ProcessorContext context) {
 super.init(context);
 valueHashSerdePseudoTopic = 
valueHashSerdePseudoTopicSupplier.get();
-valueGetter = valueGetterSupplier.get();
-valueGetter.init(context);
-if (runtimeValueSerializer == null) {
-runtimeValueSerializer = (Serializer) 
context.valueSerde().serializer();
+rawValueGetter = rawValueGetterSupplier.get();
+rawValueGetter.init(context);
+if (keyDeserializer == null) {
+keyDeserializer = (Deserializer) 
context.keySerde().deserializer();
+}
+if (leftValueDeserializer == null) {
+leftValueDeserializer = (Deserializer) 
context.valueSerde().deserializer();
+}
+if (rightValueDeserializer == null) {
+rightValueDeserializer = (Deserializer) 
context.valueSerde().deserializer();
 }
 }
 
 @Override
-public void process(final Record> record) {
+public void process(final Record> record) {
 if (record.value().getVersion() != 
SubscriptionResponseWrapper.CURRENT_VERSION) {
 //Guard against modifications to 
SubscriptionResponseWrapper. Need to ensure that there is
 //compatibility with previous versions to enable rolling 
upgrades. Must develop a strategy for
 //upgrading from older SubscriptionWrapper versions to 
newer versions.
 throw new 
UnsupportedVersionException("SubscriptionResponseWrapper is of an in

[GitHub] [kafka] mjsax commented on a diff in pull request #14157: KAFKA-15030: Avoid unnecessary re-serialization in FK-join

2023-08-06 Thread via GitHub


mjsax commented on code in PR #14157:
URL: https://github.com/apache/kafka/pull/14157#discussion_r1285341380


##
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java:
##
@@ -266,20 +266,22 @@ private void fetchAndEmit(final Record record,
 
 // Only time ordered (indexed) session store should have 
implemented
 // this function, otherwise a not-supported exception would throw
-final KeyValueIterator, VAgg> windowToEmit = store
-.findSessions(emitRangeLowerBound, emitRangeUpperBound);
-
 int emittedCount = 0;
-while (windowToEmit.hasNext()) {
-emittedCount++;
-final KeyValue, VAgg> kv = windowToEmit.next();
-
-tupleForwarder.maybeForward(
-record.withKey(kv.key)
-.withValue(new Change<>(kv.value, null))
-// set the timestamp as the window end timestamp
-.withTimestamp(kv.key.window().end())
-.withHeaders(record.headers()));
+
+try (final KeyValueIterator, VAgg> windowToEmit =

Review Comment:
   Unrelated side fix to close the iterator.



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



[GitHub] [kafka] mjsax commented on a diff in pull request #14157: KAFKA-15030: Avoid unnecessary re-serialization in FK-join

2023-08-06 Thread via GitHub


mjsax commented on code in PR #14157:
URL: https://github.com/apache/kafka/pull/14157#discussion_r1285341513


##
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableProcessorSupplier.java:
##
@@ -16,12 +16,17 @@
  */
 package org.apache.kafka.streams.kstream.internals;
 
+import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.streams.processor.api.ProcessorSupplier;
 
 public interface KTableProcessorSupplier extends 
ProcessorSupplier, KOut, Change> {
 
 KTableValueGetterSupplier view();
 
+default KTableValueGetterSupplier rawView() {

Review Comment:
   As above: should not have a default impl



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



[GitHub] [kafka] mjsax commented on a diff in pull request #14157: KAFKA-15030: Avoid unnecessary re-serialization in FK-join

2023-08-06 Thread via GitHub


mjsax commented on code in PR #14157:
URL: https://github.com/apache/kafka/pull/14157#discussion_r1285340966


##
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggProcessorSupplier.java:
##
@@ -16,12 +16,17 @@
  */
 package org.apache.kafka.streams.kstream.internals;
 
+import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.streams.processor.api.ProcessorSupplier;
 
 public interface KStreamAggProcessorSupplier extends 
ProcessorSupplier> {
 
 KTableValueGetterSupplier view();
 
+default KTableValueGetterSupplier rawView() {

Review Comment:
   This has only a default to make the code compile -- we should remove a 
default to force all classes to provide a proper implementation -- seems no 
test fails, what means we actually have tests to add for case that should fail 
with NPE now.



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

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

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