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


##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java:
##########
@@ -45,34 +49,87 @@ 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:
+            case StreamsConfig.UPGRADE_FROM_34:
+                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}
+        // {UINT32 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 = UINT32_SIZE + newDataLength + 
oldDataLength + NEW_OLD_FLAG_SIZE;
+                buf = ByteBuffer.allocate(capacity);
+                ByteUtils.writeUnsignedInt(buf, newDataLength);

Review Comment:
   I agree that we gain little from using `writeUnsignedInt`. I mostly made 
this change to accommodate some 
[feedback](https://lists.apache.org/thread/2gov29vshy2x2mv3mhtcmhwq4w9627rc) 
from the KIP-904 discussion thread. 
   
   I tried using the `writeVarint`/`readVarint` methods as per your suggestion 
but my unit test fails. 
   
   If we still want something simpler, I would honestly just revert this 
[commit](https://github.com/apache/kafka/pull/10747/commits/46c48f26dfe85ec7970b5db1b410b7076d6d2bf8)
 and go back to using the `byteBuffer.putInt`/`byteBuffer.getInt` methods. I'm 
just not sure if i would need to go update the KIP document and inform folks 
before making this change; I don't think this is necessary for such a small 
detail but let me know your thoughts. 
   
   



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