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


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/StoreSerdeInitializer.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.config.ConfigException;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.streams.errors.StreamsException;
+import org.apache.kafka.streams.kstream.internals.WrappingNullableUtils;
+import org.apache.kafka.streams.processor.ProcessorContext;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.SerdeGetter;
+import org.apache.kafka.streams.state.StateSerdes;
+
+
+public class StoreSerdeInitializer {
+    static <K, V> StateSerdes<K, V> prepareStoreSerde(final StateStoreContext 
context, final String storeName,
+                                                      final String 
changelogTopic, final Serde<K> keySerde,

Review Comment:
   nit formatting. We should have a single parameter per line, not multiple 
(both line above) -- also below



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java:
##########
@@ -58,8 +60,21 @@ public void addChild(final ProcessorNode<Void, Void, ?, ?> 
child) {
     public void init(final InternalProcessorContext<Void, Void> context) {
         super.init(context);
         this.context = context;
-        keySerializer = prepareKeySerializer(keySerializer, context, 
this.name());
-        valSerializer = prepareValueSerializer(valSerializer, context, 
this.name());
+        try {
+            keySerializer = prepareKeySerializer(keySerializer, context, 
this.name());
+        } catch (final ConfigException e) {
+            throw new ConfigException(String.format("Failed to initialize key 
serdes for sink node %s", name()));

Review Comment:
   Should we somehow preserve `e.getMessage()` -- it seems useful?



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java:
##########
@@ -58,8 +60,21 @@ public void addChild(final ProcessorNode<Void, Void, ?, ?> 
child) {
     public void init(final InternalProcessorContext<Void, Void> context) {
         super.init(context);
         this.context = context;
-        keySerializer = prepareKeySerializer(keySerializer, context, 
this.name());
-        valSerializer = prepareValueSerializer(valSerializer, context, 
this.name());
+        try {
+            keySerializer = prepareKeySerializer(keySerializer, context, 
this.name());

Review Comment:
   I did dig into `prepareKeySerializer` and `prepareValueSerializer` which 
both use `WrappingNullableUtils#prepareSerializer()` which might call both 
`context.keySerde()` and `context.valueSerde()`, and thus, I believe we could 
currently get an exception when trying to get the key serde, even if default 
key serde is set, but default value serde is not set?
   
   I think this internal helper method needs some updated, too.



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java:
##########
@@ -58,8 +60,21 @@ public void addChild(final ProcessorNode<Void, Void, ?, ?> 
child) {
     public void init(final InternalProcessorContext<Void, Void> context) {
         super.init(context);
         this.context = context;
-        keySerializer = prepareKeySerializer(keySerializer, context, 
this.name());
-        valSerializer = prepareValueSerializer(valSerializer, context, 
this.name());
+        try {
+            keySerializer = prepareKeySerializer(keySerializer, context, 
this.name());
+        } catch (final ConfigException e) {
+            throw new ConfigException(String.format("Failed to initialize key 
serdes for sink node %s", name()));
+        } catch (final StreamsException e) {

Review Comment:
   Why are we catching `StreamsException`?  Seems the only exception that might 
bubble up her is a `ConfigException`?



##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java:
##########
@@ -156,8 +156,7 @@ public void testTopologyLevelConfigException() {
 
         final ConfigException se = assertThrows(ConfigException.class, () -> 
new TopologyTestDriver(topology));
         final String msg = se.getMessage();
-        assertTrue("Error about class cast with serdes", 
msg.contains("StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG"));

Review Comment:
   Relates to my other commend -- preserving the original message, which tells 
the user a relevant config seems useful, and we should keep it?



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