You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "fqaiser94 (via GitHub)" <gi...@apache.org> on 2023/02/06 13:27:54 UTC

[GitHub] [kafka] fqaiser94 commented on a diff in pull request #10747: KAFKA-12446: Call subtractor before adder if key is the same

fqaiser94 commented on code in PR #10747:
URL: https://github.com/apache/kafka/pull/10747#discussion_r1097375552


##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java:
##########
@@ -45,34 +47,85 @@ public void setIfUnset(final SerdeGetter getter) {
         }
     }
 
+    @SuppressWarnings("checkstyle:cyclomaticComplexity")
+    private boolean isUpgrade(final Map<String, ?> configs) {
+        final Object upgradeFrom = configs.get(StreamsConfig.UPGRADE_FROM_CONFIG);
+        if (upgradeFrom == null) {
+            return false;
+        }
+
+        switch ((String) upgradeFrom) {
+            case StreamsConfig.UPGRADE_FROM_0100:
+            case StreamsConfig.UPGRADE_FROM_0101:
+            case StreamsConfig.UPGRADE_FROM_0102:
+            case StreamsConfig.UPGRADE_FROM_0110:
+            case StreamsConfig.UPGRADE_FROM_10:
+            case StreamsConfig.UPGRADE_FROM_11:
+            case StreamsConfig.UPGRADE_FROM_20:
+            case StreamsConfig.UPGRADE_FROM_21:
+            case StreamsConfig.UPGRADE_FROM_22:
+            case StreamsConfig.UPGRADE_FROM_23:
+            case StreamsConfig.UPGRADE_FROM_24:
+            case StreamsConfig.UPGRADE_FROM_25:
+            case StreamsConfig.UPGRADE_FROM_26:
+            case StreamsConfig.UPGRADE_FROM_27:
+            case StreamsConfig.UPGRADE_FROM_28:
+            case StreamsConfig.UPGRADE_FROM_30:
+            case StreamsConfig.UPGRADE_FROM_31:
+            case StreamsConfig.UPGRADE_FROM_32:
+            case StreamsConfig.UPGRADE_FROM_33:
+                return true;
+            default:
+                return false;
+        }
+    }
+
+    @Override
+    public void configure(final Map<String, ?> configs, final boolean isKey) {
+        this.isUpgrade = isUpgrade(configs);
+    }
+
     /**
      * @throws StreamsException if both old and new values of data are null, or if
-     * both values are not null
+     * both values are not null and is upgrading from a version less than 3.4
      */
     @Override
     public byte[] serialize(final String topic, final Headers headers, final Change<T> data) {
-        final byte[] serializedKey;
-
-        // only one of the old / new values would be not null
-        if (data.newValue != null) {
-            if (data.oldValue != null) {
+        final boolean oldValueIsNull = data.oldValue == null;
+        final boolean newValueIsNull = data.newValue == null;
+
+        final byte[] newData = inner.serialize(topic, headers, data.newValue);
+        final byte[] oldData = inner.serialize(topic, headers, data.oldValue);
+
+        final int newDataLength = newValueIsNull ? 0 : newData.length;
+        final int oldDataLength = oldValueIsNull ? 0 : oldData.length;
+
+        // The serialization format is:
+        // {BYTE_ARRAY oldValue}{BYTE newOldFlag=0}
+        // {BYTE_ARRAY newValue}{BYTE newOldFlag=1}
+        // {INT newDataLength}{BYTE_ARRAY newValue}{BYTE_ARRAY oldValue}{BYTE newOldFlag=2}
+        final ByteBuffer buf;
+        if (!newValueIsNull && !oldValueIsNull) {
+            if (isUpgrade) {
                 throw new StreamsException("Both old and new values are not null (" + data.oldValue
-                    + " : " + data.newValue + ") in ChangeSerializer, which is not allowed.");
+                        + " : " + data.newValue + ") in ChangeSerializer, which is not allowed unless upgrading.");
+            } else {
+                final int capacity = Integer.BYTES + newDataLength + oldDataLength + NEW_OLD_FLAG_SIZE;
+                buf = ByteBuffer.allocate(capacity);
+                buf.putInt(newDataLength).put(newData).put(oldData).put((byte) 2);
             }
-
-            serializedKey = inner.serialize(topic, headers, data.newValue);
+        } else if (!newValueIsNull) {
+            final int capacity = newDataLength + NEW_OLD_FLAG_SIZE;
+            buf = ByteBuffer.allocate(capacity);
+            buf.put(newData).put((byte) 1);
+        } else if (!oldValueIsNull) {
+            final int capacity = oldDataLength + NEW_OLD_FLAG_SIZE;
+            buf = ByteBuffer.allocate(capacity);
+            buf.put(oldData).put((byte) 0);
         } else {
-            if (data.oldValue == null) {
-                throw new StreamsException("Both old and new values are null in ChangeSerializer, which is not allowed.");
-            }
-
-            serializedKey = inner.serialize(topic, headers, data.oldValue);
+            throw new StreamsException("Both old and new values are null in ChangeSerializer, which is not allowed.");
         }
 
-        final ByteBuffer buf = ByteBuffer.allocate(serializedKey.length + NEWFLAG_SIZE);
-        buf.put(serializedKey);
-        buf.put((byte) (data.newValue != null ? 1 : 0));
-
         return buf.array();
     }
 

Review Comment:
   Thanks for your feedback!
   Just FYI, I've created the KIP [here](https://cwiki.apache.org/confluence/x/P5VbDg). 
   And started a discussion thread [here](https://lists.apache.org/thread/38rznfn748yx6gmz2b0ldohl45hq3b42). 



-- 
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-unsubscribe@kafka.apache.org

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