From 442dab6c2e9804a0490636d34d78271d4eb5b985 Mon Sep 17 00:00:00 2001
From: Steven Zhang <35498506+stevenpyzhang@users.noreply.github.com>
Date: Wed, 17 Jun 2020 13:59:20 -0700
Subject: [PATCH 1/3] feat: implements ARRAY_JOIN as requested in (#5028)
 (#5474) (#5638)

Co-authored-by: Hans-Peter Grahsl <hpgrahsl@users.noreply.github.com>
---
 .../ksqldb-reference/scalar-functions.md      |  11 ++
 .../ksql/function/udf/array/ArrayJoin.java    |  84 +++++++++++
 .../function/udf/array/ArrayJoinTest.java     | 131 ++++++++++++++++++
 .../6.0.0_1591955240274/plan.json             | 126 +++++++++++++++++
 .../6.0.0_1591955240274/spec.json             |  73 ++++++++++
 .../6.0.0_1591955240274/topology              |  13 ++
 .../6.0.0_1591953360045/plan.json             | 126 +++++++++++++++++
 .../6.0.0_1591953360045/spec.json             |  73 ++++++++++
 .../6.0.0_1591953360045/topology              |  13 ++
 .../6.0.0_1591955240399/plan.json             | 126 +++++++++++++++++
 .../6.0.0_1591955240399/spec.json             |  73 ++++++++++
 .../6.0.0_1591955240399/topology              |  13 ++
 .../query-validation-tests/arrayjoin.json     | 109 +++++++++++++++
 13 files changed, 971 insertions(+)
 create mode 100644 ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/array/ArrayJoin.java
 create mode 100644 ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/array/ArrayJoinTest.java
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/plan.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/spec.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/topology
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/plan.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/spec.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/topology
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/plan.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/spec.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/topology
 create mode 100644 ksqldb-functional-tests/src/test/resources/query-validation-tests/arrayjoin.json

diff --git a/docs/developer-guide/ksqldb-reference/scalar-functions.md b/docs/developer-guide/ksqldb-reference/scalar-functions.md
index fc219284cb62..fe96cd5c6e29 100644
--- a/docs/developer-guide/ksqldb-reference/scalar-functions.md
+++ b/docs/developer-guide/ksqldb-reference/scalar-functions.md
@@ -400,6 +400,17 @@ SLICE(col1, from, to)
 Slices a list based on the supplied indices. The indices start at 1 and
 include both endpoints.
 
+### `ARRAY_JOIN`
+
+```sql
+ARRAY_JOIN(col1, delimiter)
+```
+
+Creates a flat string representation of all the elements contained in the given array.
+The elements in the resulting string are separated by the chosen `delimiter`, 
+which is an optional parameter that falls back to a comma `,`. The current implementation only
+allows for array elements of primitive ksqlDB types.
+
 ## Strings
 
 ### `CONCAT`
diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/array/ArrayJoin.java b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/array/ArrayJoin.java
new file mode 100644
index 000000000000..ebabda5e58fb
--- /dev/null
+++ b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/array/ArrayJoin.java
@@ -0,0 +1,84 @@
+/*
+ * Copyright 2020 Confluent Inc.
+ *
+ * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OF ANY KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+
+package io.confluent.ksql.function.udf.array;
+
+import com.google.common.collect.ImmutableSet;
+import io.confluent.ksql.function.KsqlFunctionException;
+import io.confluent.ksql.function.udf.Udf;
+import io.confluent.ksql.function.udf.UdfDescription;
+import io.confluent.ksql.function.udf.UdfParameter;
+import io.confluent.ksql.util.KsqlConstants;
+import java.math.BigDecimal;
+import java.util.List;
+import java.util.Set;
+import java.util.StringJoiner;
+
+@SuppressWarnings("MethodMayBeStatic") // UDF methods can not be static.
+@UdfDescription(
+    name = "ARRAY_JOIN",
+    description = "joins the array elements into a flat string representation",
+    author = KsqlConstants.CONFLUENT_AUTHOR
+)
+public class ArrayJoin {
+
+  private static final String DEFAULT_DELIMITER = ",";
+  private static final Set<Class> KSQL_PRIMITIVES = ImmutableSet.of(
+      Boolean.class,Integer.class,Long.class,Double.class,BigDecimal.class,String.class
+  );
+
+  @Udf
+  public <T> String join(
+      @UdfParameter(description = "the array to join using the default delimiter '"
+          + DEFAULT_DELIMITER + "'") final List<T> array
+  ) {
+    return join(array, DEFAULT_DELIMITER);
+  }
+
+  @Udf
+  public <T> String join(
+      @UdfParameter(description = "the array to join using the specified delimiter")
+      final List<T> array,
+      @UdfParameter(description = "the string to be used as element delimiter")
+      final String delimiter
+  ) {
+
+    if (array == null) {
+      return null;
+    }
+
+    final StringJoiner sj = new StringJoiner(delimiter == null ? "" : delimiter);
+    array.forEach(e -> processElement(e, sj));
+    return sj.toString();
+
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <T> void processElement(final T element, final StringJoiner joiner) {
+
+    if (element == null || KSQL_PRIMITIVES.contains(element.getClass())) {
+      handlePrimitiveType(element, joiner);
+    } else {
+      throw new KsqlFunctionException("error: hit element of type "
+          + element.getClass().getTypeName() + " which is currently not supported");
+    }
+
+  }
+
+  private static void handlePrimitiveType(final Object element, final StringJoiner joiner) {
+    joiner.add(element != null ? element.toString() : null);
+  }
+
+}
diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/array/ArrayJoinTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/array/ArrayJoinTest.java
new file mode 100644
index 000000000000..d15e841b55cb
--- /dev/null
+++ b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/array/ArrayJoinTest.java
@@ -0,0 +1,131 @@
+/*
+ * Copyright 2020 Confluent Inc.
+ *
+ * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OF ANY KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+
+package io.confluent.ksql.function.udf.array;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.nullValue;
+import static org.junit.Assert.assertThrows;
+
+import io.confluent.ksql.function.KsqlFunctionException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import org.junit.Test;
+
+public class ArrayJoinTest {
+
+  private static final String CUSTOM_DELIMITER = "|";
+
+  private final ArrayJoin arrayJoinUDF = new ArrayJoin();
+
+  @Test
+  public void shouldReturnNullForNullInput() {
+    assertThat(arrayJoinUDF.join(null), nullValue());
+    assertThat(arrayJoinUDF.join(null,CUSTOM_DELIMITER), nullValue());
+  }
+
+  @Test
+  public void shouldReturnEmptyStringForEmptyArrays() {
+    assertThat(arrayJoinUDF.join(Collections.emptyList()).isEmpty(),is(true));
+    assertThat(arrayJoinUDF.join(Collections.emptyList(),CUSTOM_DELIMITER).isEmpty(),is(true));
+  }
+
+  @Test
+  public void shouldReturnCorrectStringForFlatArraysWithPrimitiveTypes() {
+
+    assertThat(arrayJoinUDF.join(Arrays.asList(true, null, false),""),
+        is("truenullfalse")
+    );
+    assertThat(arrayJoinUDF.join(Arrays.asList(true, null, false)),
+        is("true,null,false")
+    );
+    assertThat(arrayJoinUDF.join(Arrays.asList(true,null,false),CUSTOM_DELIMITER),
+        is("true"+CUSTOM_DELIMITER+"null"+CUSTOM_DELIMITER+"false")
+    );
+
+    assertThat(arrayJoinUDF.join(Arrays.asList(1,23,-42,0),null), is("123-420"));
+    assertThat(arrayJoinUDF.join(Arrays.asList(1,23,-42,0)), is("1,23,-42,0"));
+    assertThat(arrayJoinUDF.join(Arrays.asList(1,23,-42,0),CUSTOM_DELIMITER),
+        is("1"+CUSTOM_DELIMITER+"23"+CUSTOM_DELIMITER+"-42"+CUSTOM_DELIMITER+"0")
+    );
+
+    assertThat(arrayJoinUDF.join(Arrays.asList(-4294967297L, 8589934592L),""),
+        is("-42949672978589934592")
+    );
+    assertThat(arrayJoinUDF.join(Arrays.asList(-4294967297L, 8589934592L)),
+        is("-4294967297,8589934592")
+    );
+    assertThat(arrayJoinUDF.join(Arrays.asList(-4294967297L, 8589934592L), CUSTOM_DELIMITER),
+        is("-4294967297"+CUSTOM_DELIMITER+"8589934592")
+    );
+
+    assertThat(arrayJoinUDF.join(Arrays.asList(1.23,-23.42,0.0),null),
+        is("1.23-23.420.0")
+    );
+    assertThat(arrayJoinUDF.join(Arrays.asList(1.23,-23.42,0.0)),
+        is("1.23,-23.42,0.0")
+    );
+    assertThat(arrayJoinUDF.join(Arrays.asList(1.23,-23.42,0.0),CUSTOM_DELIMITER),
+        is("1.23"+CUSTOM_DELIMITER+"-23.42"+CUSTOM_DELIMITER+"0.0")
+    );
+
+    assertThat(arrayJoinUDF.join(
+        Arrays.asList(new BigDecimal("123.45"), new BigDecimal("987.65")),null
+        ),
+        is("123.45987.65")
+    );
+    assertThat(arrayJoinUDF.join(Arrays.asList(new BigDecimal("123.45"), new BigDecimal("987.65"))),
+        is("123.45,987.65")
+    );
+    assertThat(arrayJoinUDF.join(
+        Arrays.asList(new BigDecimal("123.45"), new BigDecimal("987.65")),CUSTOM_DELIMITER),
+        is("123.45"+CUSTOM_DELIMITER+"987.65")
+    );
+
+    assertThat(arrayJoinUDF.join(Arrays.asList("Hello","From","","Ksqldb","Udf"),""),
+        is("HelloFromKsqldbUdf")
+    );
+    assertThat(arrayJoinUDF.join(Arrays.asList("Hello","From","","Ksqldb","Udf")),
+        is("Hello,From,,Ksqldb,Udf")
+    );
+    assertThat(
+        arrayJoinUDF.join(Arrays.asList("hello","from","","ksqldb","udf",null),CUSTOM_DELIMITER),
+        is("hello"+CUSTOM_DELIMITER+"from"+CUSTOM_DELIMITER+CUSTOM_DELIMITER
+            +"ksqldb"+CUSTOM_DELIMITER+"udf"+CUSTOM_DELIMITER+"null")
+    );
+
+  }
+
+  @Test
+  public void shouldThrowExceptionForExamplesOfUnsupportedElementTypes() {
+    assertThrows(KsqlFunctionException.class,
+        () -> arrayJoinUDF.join(Arrays.asList('a','b')));
+    assertThrows(KsqlFunctionException.class,
+        () -> arrayJoinUDF.join(Arrays.asList(BigInteger.ONE,BigInteger.ZERO)));
+    assertThrows(KsqlFunctionException.class,
+        () -> arrayJoinUDF.join(Arrays.asList(-23.0f,42.42f,0.0f)));
+    assertThrows(KsqlFunctionException.class,
+        () -> arrayJoinUDF.join(Arrays.asList(
+            new HashSet<>(Arrays.asList("foo", "blah")),
+            new HashSet<>(Arrays.asList("ksqlDB", "UDF"))
+        ))
+    );
+  }
+
+}
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/plan.json
new file mode 100644
index 000000000000..b5535fca092f
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/plan.json
@@ -0,0 +1,126 @@
+{
+  "plan" : [ {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY<BOOLEAN>, INTARRAY ARRAY<INTEGER>, BIGINTARRAY ARRAY<BIGINT>, DOUBLEARRAY ARRAY<DOUBLE>, DECIMALARRAY ARRAY<DECIMAL(5, 2)>, STRINGARRAY ARRAY<STRING>) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "TEST",
+      "schema" : "`ID` STRING KEY, `BOOLEANARRAY` ARRAY<BOOLEAN>, `INTARRAY` ARRAY<INTEGER>, `BIGINTARRAY` ARRAY<BIGINT>, `DOUBLEARRAY` ARRAY<DOUBLE>, `DECIMALARRAY` ARRAY<DECIMAL(5, 2)>, `STRINGARRAY` ARRAY<STRING>",
+      "topicName" : "test_topic",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "JSON"
+        }
+      }
+    }
+  }, {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM OUTPUT AS SELECT\n  TEST.ID ID,\n  ARRAY_JOIN(TEST.BOOLEANARRAY, '|') KSQL_COL_0,\n  ARRAY_JOIN(TEST.INTARRAY, '?') KSQL_COL_1,\n  ARRAY_JOIN(TEST.BIGINTARRAY, ';') KSQL_COL_2,\n  ARRAY_JOIN(TEST.DOUBLEARRAY, ' ') KSQL_COL_3,\n  ARRAY_JOIN(TEST.DECIMALARRAY, '#') KSQL_COL_4,\n  ARRAY_JOIN(TEST.STRINGARRAY, '_') KSQL_COL_5\nFROM TEST TEST\nEMIT CHANGES",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "OUTPUT",
+      "schema" : "`ID` STRING KEY, `KSQL_COL_0` STRING, `KSQL_COL_1` STRING, `KSQL_COL_2` STRING, `KSQL_COL_3` STRING, `KSQL_COL_4` STRING, `KSQL_COL_5` STRING",
+      "topicName" : "OUTPUT",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "JSON"
+        }
+      }
+    },
+    "queryPlan" : {
+      "sources" : [ "TEST" ],
+      "sink" : "OUTPUT",
+      "physicalPlan" : {
+        "@type" : "streamSinkV1",
+        "properties" : {
+          "queryContext" : "OUTPUT"
+        },
+        "source" : {
+          "@type" : "streamSelectV1",
+          "properties" : {
+            "queryContext" : "Project"
+          },
+          "source" : {
+            "@type" : "streamSourceV1",
+            "properties" : {
+              "queryContext" : "KsqlTopic/Source"
+            },
+            "topicName" : "test_topic",
+            "formats" : {
+              "keyFormat" : {
+                "format" : "KAFKA"
+              },
+              "valueFormat" : {
+                "format" : "JSON"
+              }
+            },
+            "sourceSchema" : "`ID` STRING KEY, `BOOLEANARRAY` ARRAY<BOOLEAN>, `INTARRAY` ARRAY<INTEGER>, `BIGINTARRAY` ARRAY<BIGINT>, `DOUBLEARRAY` ARRAY<DOUBLE>, `DECIMALARRAY` ARRAY<DECIMAL(5, 2)>, `STRINGARRAY` ARRAY<STRING>"
+          },
+          "keyColumnNames" : [ "ID" ],
+          "selectExpressions" : [ "ARRAY_JOIN(BOOLEANARRAY, '|') AS KSQL_COL_0", "ARRAY_JOIN(INTARRAY, '?') AS KSQL_COL_1", "ARRAY_JOIN(BIGINTARRAY, ';') AS KSQL_COL_2", "ARRAY_JOIN(DOUBLEARRAY, ' ') AS KSQL_COL_3", "ARRAY_JOIN(DECIMALARRAY, '#') AS KSQL_COL_4", "ARRAY_JOIN(STRINGARRAY, '_') AS KSQL_COL_5" ]
+        },
+        "formats" : {
+          "keyFormat" : {
+            "format" : "KAFKA"
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          }
+        },
+        "topicName" : "OUTPUT"
+      },
+      "queryId" : "CSAS_OUTPUT_0"
+    }
+  } ],
+  "configs" : {
+    "ksql.extension.dir" : "ext",
+    "ksql.streams.cache.max.bytes.buffering" : "0",
+    "ksql.security.extension.class" : null,
+    "ksql.transient.prefix" : "transient_",
+    "ksql.persistence.wrap.single.values" : "true",
+    "ksql.authorization.cache.expiry.time.secs" : "30",
+    "ksql.schema.registry.url" : "",
+    "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler",
+    "ksql.output.topic.name.prefix" : "",
+    "ksql.streams.auto.offset.reset" : "earliest",
+    "ksql.query.pull.enable.standby.reads" : "false",
+    "ksql.connect.url" : "http://localhost:8083",
+    "ksql.service.id" : "some.ksql.service.id",
+    "ksql.internal.topic.min.insync.replicas" : "1",
+    "ksql.streams.shutdown.timeout.ms" : "300000",
+    "ksql.internal.topic.replicas" : "1",
+    "ksql.insert.into.values.enabled" : "true",
+    "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807",
+    "ksql.query.pull.max.qps" : "2147483647",
+    "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler",
+    "ksql.access.validator.enable" : "auto",
+    "ksql.streams.bootstrap.servers" : "localhost:0",
+    "ksql.streams.commit.interval.ms" : "2000",
+    "ksql.metric.reporters" : "",
+    "ksql.query.pull.metrics.enabled" : "false",
+    "ksql.streams.auto.commit.interval.ms" : "0",
+    "ksql.metrics.extension" : null,
+    "ksql.streams.topology.optimization" : "all",
+    "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.streams.num.stream.threads" : "4",
+    "ksql.timestamp.throw.on.invalid" : "false",
+    "ksql.authorization.cache.max.entries" : "10000",
+    "ksql.metrics.tags.custom" : "",
+    "ksql.pull.queries.enable" : "true",
+    "ksql.udfs.enabled" : "true",
+    "ksql.udf.enable.security.manager" : "true",
+    "ksql.connect.worker.config" : "",
+    "ksql.sink.window.change.log.additional.retention" : "1000000",
+    "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.udf.collect.metrics" : "false",
+    "ksql.persistent.prefix" : "query_",
+    "ksql.query.persistent.active.limit" : "2147483647",
+    "ksql.error.classifier.regex" : ""
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/spec.json
new file mode 100644
index 000000000000..12a2c5dfee3b
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/spec.json
@@ -0,0 +1,73 @@
+{
+  "version" : "6.0.0",
+  "timestamp" : 1591955240274,
+  "path" : "query-validation-tests/arrayjoin.json",
+  "schemas" : {
+    "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT<BOOLEANARRAY ARRAY<BOOLEAN>, INTARRAY ARRAY<INT>, BIGINTARRAY ARRAY<BIGINT>, DOUBLEARRAY ARRAY<DOUBLE>, DECIMALARRAY ARRAY<DECIMAL(5, 2)>, STRINGARRAY ARRAY<VARCHAR>> NOT NULL",
+    "CSAS_OUTPUT_0.OUTPUT" : "STRUCT<KSQL_COL_0 VARCHAR, KSQL_COL_1 VARCHAR, KSQL_COL_2 VARCHAR, KSQL_COL_3 VARCHAR, KSQL_COL_4 VARCHAR, KSQL_COL_5 VARCHAR> NOT NULL"
+  },
+  "testCase" : {
+    "name" : "join flat arrays with primitive ksqldb types and custom delimiters",
+    "inputs" : [ {
+      "topic" : "test_topic",
+      "key" : "1",
+      "value" : {
+        "BOOLEANARRAY" : [ true, null, false ],
+        "INTARRAY" : [ 1, 23, -42, 0 ],
+        "BIGINTARRAY" : [ -4294967297, 8589934592 ],
+        "DOUBLEARRAY" : [ 1.23, -23.42, 0.0 ],
+        "DECIMALARRAY" : [ 123.45, 987.65 ],
+        "STRINGARRAY" : [ "Hello", "From", "", "Ksqldb", "Udf" ]
+      }
+    } ],
+    "outputs" : [ {
+      "topic" : "OUTPUT",
+      "key" : "1",
+      "value" : {
+        "KSQL_COL_0" : "true|null|false",
+        "KSQL_COL_1" : "1?23?-42?0",
+        "KSQL_COL_2" : "-4294967297;8589934592",
+        "KSQL_COL_3" : "1.23 -23.42 0.0",
+        "KSQL_COL_4" : "123.45#987.65",
+        "KSQL_COL_5" : "Hello_From__Ksqldb_Udf"
+      }
+    } ],
+    "topics" : [ {
+      "name" : "OUTPUT",
+      "replicas" : 1,
+      "numPartitions" : 4
+    }, {
+      "name" : "test_topic",
+      "replicas" : 1,
+      "numPartitions" : 4
+    } ],
+    "statements" : [ "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY<BOOLEAN>, INTARRAY ARRAY<INT>, BIGINTARRAY ARRAY<BIGINT>, DOUBLEARRAY ARRAY<DOUBLE>, DECIMALARRAY ARRAY<DECIMAL(5,2)>, STRINGARRAY ARRAY<STRING>) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT ID, ARRAY_JOIN(BOOLEANARRAY,'|'), ARRAY_JOIN(INTARRAY,'?'), ARRAY_JOIN(BIGINTARRAY,';'), ARRAY_JOIN(DOUBLEARRAY,' '), ARRAY_JOIN(DECIMALARRAY,'#'), ARRAY_JOIN(STRINGARRAY,'_') FROM TEST;" ],
+    "post" : {
+      "topics" : {
+        "topics" : [ {
+          "name" : "OUTPUT",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          },
+          "partitions" : 4
+        }, {
+          "name" : "test_topic",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          },
+          "partitions" : 4
+        } ]
+      }
+    }
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/topology
new file mode 100644
index 000000000000..441a8f282644
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_custom_delimiters/6.0.0_1591955240274/topology
@@ -0,0 +1,13 @@
+Topologies:
+   Sub-topology: 0
+    Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic])
+      --> KSTREAM-TRANSFORMVALUES-0000000001
+    Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: [])
+      --> Project
+      <-- KSTREAM-SOURCE-0000000000
+    Processor: Project (stores: [])
+      --> KSTREAM-SINK-0000000003
+      <-- KSTREAM-TRANSFORMVALUES-0000000001
+    Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT)
+      <-- Project
+
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/plan.json
new file mode 100644
index 000000000000..75e075eb8ae4
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/plan.json
@@ -0,0 +1,126 @@
+{
+  "plan" : [ {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY<BOOLEAN>, INTARRAY ARRAY<INTEGER>, BIGINTARRAY ARRAY<BIGINT>, DOUBLEARRAY ARRAY<DOUBLE>, DECIMALARRAY ARRAY<DECIMAL(5, 2)>, STRINGARRAY ARRAY<STRING>) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "TEST",
+      "schema" : "`ID` STRING KEY, `BOOLEANARRAY` ARRAY<BOOLEAN>, `INTARRAY` ARRAY<INTEGER>, `BIGINTARRAY` ARRAY<BIGINT>, `DOUBLEARRAY` ARRAY<DOUBLE>, `DECIMALARRAY` ARRAY<DECIMAL(5, 2)>, `STRINGARRAY` ARRAY<STRING>",
+      "topicName" : "test_topic",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "JSON"
+        }
+      }
+    }
+  }, {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM OUTPUT AS SELECT\n  TEST.ID ID,\n  ARRAY_JOIN(TEST.BOOLEANARRAY) KSQL_COL_0,\n  ARRAY_JOIN(TEST.INTARRAY) KSQL_COL_1,\n  ARRAY_JOIN(TEST.BIGINTARRAY) KSQL_COL_2,\n  ARRAY_JOIN(TEST.DOUBLEARRAY) KSQL_COL_3,\n  ARRAY_JOIN(TEST.DECIMALARRAY) KSQL_COL_4,\n  ARRAY_JOIN(TEST.STRINGARRAY) KSQL_COL_5\nFROM TEST TEST\nEMIT CHANGES",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "OUTPUT",
+      "schema" : "`ID` STRING KEY, `KSQL_COL_0` STRING, `KSQL_COL_1` STRING, `KSQL_COL_2` STRING, `KSQL_COL_3` STRING, `KSQL_COL_4` STRING, `KSQL_COL_5` STRING",
+      "topicName" : "OUTPUT",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "JSON"
+        }
+      }
+    },
+    "queryPlan" : {
+      "sources" : [ "TEST" ],
+      "sink" : "OUTPUT",
+      "physicalPlan" : {
+        "@type" : "streamSinkV1",
+        "properties" : {
+          "queryContext" : "OUTPUT"
+        },
+        "source" : {
+          "@type" : "streamSelectV1",
+          "properties" : {
+            "queryContext" : "Project"
+          },
+          "source" : {
+            "@type" : "streamSourceV1",
+            "properties" : {
+              "queryContext" : "KsqlTopic/Source"
+            },
+            "topicName" : "test_topic",
+            "formats" : {
+              "keyFormat" : {
+                "format" : "KAFKA"
+              },
+              "valueFormat" : {
+                "format" : "JSON"
+              }
+            },
+            "sourceSchema" : "`ID` STRING KEY, `BOOLEANARRAY` ARRAY<BOOLEAN>, `INTARRAY` ARRAY<INTEGER>, `BIGINTARRAY` ARRAY<BIGINT>, `DOUBLEARRAY` ARRAY<DOUBLE>, `DECIMALARRAY` ARRAY<DECIMAL(5, 2)>, `STRINGARRAY` ARRAY<STRING>"
+          },
+          "keyColumnNames" : [ "ID" ],
+          "selectExpressions" : [ "ARRAY_JOIN(BOOLEANARRAY) AS KSQL_COL_0", "ARRAY_JOIN(INTARRAY) AS KSQL_COL_1", "ARRAY_JOIN(BIGINTARRAY) AS KSQL_COL_2", "ARRAY_JOIN(DOUBLEARRAY) AS KSQL_COL_3", "ARRAY_JOIN(DECIMALARRAY) AS KSQL_COL_4", "ARRAY_JOIN(STRINGARRAY) AS KSQL_COL_5" ]
+        },
+        "formats" : {
+          "keyFormat" : {
+            "format" : "KAFKA"
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          }
+        },
+        "topicName" : "OUTPUT"
+      },
+      "queryId" : "CSAS_OUTPUT_0"
+    }
+  } ],
+  "configs" : {
+    "ksql.extension.dir" : "ext",
+    "ksql.streams.cache.max.bytes.buffering" : "0",
+    "ksql.security.extension.class" : null,
+    "ksql.transient.prefix" : "transient_",
+    "ksql.persistence.wrap.single.values" : "true",
+    "ksql.authorization.cache.expiry.time.secs" : "30",
+    "ksql.schema.registry.url" : "",
+    "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler",
+    "ksql.output.topic.name.prefix" : "",
+    "ksql.streams.auto.offset.reset" : "earliest",
+    "ksql.query.pull.enable.standby.reads" : "false",
+    "ksql.connect.url" : "http://localhost:8083",
+    "ksql.service.id" : "some.ksql.service.id",
+    "ksql.internal.topic.min.insync.replicas" : "1",
+    "ksql.streams.shutdown.timeout.ms" : "300000",
+    "ksql.internal.topic.replicas" : "1",
+    "ksql.insert.into.values.enabled" : "true",
+    "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807",
+    "ksql.query.pull.max.qps" : "2147483647",
+    "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler",
+    "ksql.access.validator.enable" : "auto",
+    "ksql.streams.bootstrap.servers" : "localhost:0",
+    "ksql.streams.commit.interval.ms" : "2000",
+    "ksql.metric.reporters" : "",
+    "ksql.query.pull.metrics.enabled" : "false",
+    "ksql.streams.auto.commit.interval.ms" : "0",
+    "ksql.metrics.extension" : null,
+    "ksql.streams.topology.optimization" : "all",
+    "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.streams.num.stream.threads" : "4",
+    "ksql.timestamp.throw.on.invalid" : "false",
+    "ksql.authorization.cache.max.entries" : "10000",
+    "ksql.metrics.tags.custom" : "",
+    "ksql.pull.queries.enable" : "true",
+    "ksql.udfs.enabled" : "true",
+    "ksql.udf.enable.security.manager" : "true",
+    "ksql.connect.worker.config" : "",
+    "ksql.sink.window.change.log.additional.retention" : "1000000",
+    "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.udf.collect.metrics" : "false",
+    "ksql.persistent.prefix" : "query_",
+    "ksql.query.persistent.active.limit" : "2147483647",
+    "ksql.error.classifier.regex" : ""
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/spec.json
new file mode 100644
index 000000000000..7fc811f9adb6
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/spec.json
@@ -0,0 +1,73 @@
+{
+  "version" : "6.0.0",
+  "timestamp" : 1591953360045,
+  "path" : "query-validation-tests/arrayjoin.json",
+  "schemas" : {
+    "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT<BOOLEANARRAY ARRAY<BOOLEAN>, INTARRAY ARRAY<INT>, BIGINTARRAY ARRAY<BIGINT>, DOUBLEARRAY ARRAY<DOUBLE>, DECIMALARRAY ARRAY<DECIMAL(5, 2)>, STRINGARRAY ARRAY<VARCHAR>> NOT NULL",
+    "CSAS_OUTPUT_0.OUTPUT" : "STRUCT<KSQL_COL_0 VARCHAR, KSQL_COL_1 VARCHAR, KSQL_COL_2 VARCHAR, KSQL_COL_3 VARCHAR, KSQL_COL_4 VARCHAR, KSQL_COL_5 VARCHAR> NOT NULL"
+  },
+  "testCase" : {
+    "name" : "join flat arrays with primitive ksqldb types and default delimiter",
+    "inputs" : [ {
+      "topic" : "test_topic",
+      "key" : "1",
+      "value" : {
+        "BOOLEANARRAY" : [ true, null, false ],
+        "INTARRAY" : [ 1, 23, -42, 0 ],
+        "BIGINTARRAY" : [ -4294967297, 8589934592 ],
+        "DOUBLEARRAY" : [ 1.23, -23.42, 0.0 ],
+        "DECIMALARRAY" : [ 123.45, 987.65 ],
+        "STRINGARRAY" : [ "Hello", "From", "", "Ksqldb", "Udf" ]
+      }
+    } ],
+    "outputs" : [ {
+      "topic" : "OUTPUT",
+      "key" : "1",
+      "value" : {
+        "KSQL_COL_0" : "true,null,false",
+        "KSQL_COL_1" : "1,23,-42,0",
+        "KSQL_COL_2" : "-4294967297,8589934592",
+        "KSQL_COL_3" : "1.23,-23.42,0.0",
+        "KSQL_COL_4" : "123.45,987.65",
+        "KSQL_COL_5" : "Hello,From,,Ksqldb,Udf"
+      }
+    } ],
+    "topics" : [ {
+      "name" : "OUTPUT",
+      "replicas" : 1,
+      "numPartitions" : 4
+    }, {
+      "name" : "test_topic",
+      "replicas" : 1,
+      "numPartitions" : 4
+    } ],
+    "statements" : [ "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY<BOOLEAN>, INTARRAY ARRAY<INT>, BIGINTARRAY ARRAY<BIGINT>, DOUBLEARRAY ARRAY<DOUBLE>, DECIMALARRAY ARRAY<DECIMAL(5,2)>, STRINGARRAY ARRAY<STRING>) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT ID, ARRAY_JOIN(BOOLEANARRAY), ARRAY_JOIN(INTARRAY), ARRAY_JOIN(BIGINTARRAY), ARRAY_JOIN(DOUBLEARRAY), ARRAY_JOIN(DECIMALARRAY), ARRAY_JOIN(STRINGARRAY) FROM TEST;" ],
+    "post" : {
+      "topics" : {
+        "topics" : [ {
+          "name" : "OUTPUT",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          },
+          "partitions" : 4
+        }, {
+          "name" : "test_topic",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          },
+          "partitions" : 4
+        } ]
+      }
+    }
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/topology
new file mode 100644
index 000000000000..441a8f282644
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_default_delimiter/6.0.0_1591953360045/topology
@@ -0,0 +1,13 @@
+Topologies:
+   Sub-topology: 0
+    Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic])
+      --> KSTREAM-TRANSFORMVALUES-0000000001
+    Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: [])
+      --> Project
+      <-- KSTREAM-SOURCE-0000000000
+    Processor: Project (stores: [])
+      --> KSTREAM-SINK-0000000003
+      <-- KSTREAM-TRANSFORMVALUES-0000000001
+    Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT)
+      <-- Project
+
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/plan.json
new file mode 100644
index 000000000000..64eb4ec14fbb
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/plan.json
@@ -0,0 +1,126 @@
+{
+  "plan" : [ {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY<BOOLEAN>, INTARRAY ARRAY<INTEGER>, BIGINTARRAY ARRAY<BIGINT>, DOUBLEARRAY ARRAY<DOUBLE>, DECIMALARRAY ARRAY<DECIMAL(5, 2)>, STRINGARRAY ARRAY<STRING>) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "TEST",
+      "schema" : "`ID` STRING KEY, `BOOLEANARRAY` ARRAY<BOOLEAN>, `INTARRAY` ARRAY<INTEGER>, `BIGINTARRAY` ARRAY<BIGINT>, `DOUBLEARRAY` ARRAY<DOUBLE>, `DECIMALARRAY` ARRAY<DECIMAL(5, 2)>, `STRINGARRAY` ARRAY<STRING>",
+      "topicName" : "test_topic",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "JSON"
+        }
+      }
+    }
+  }, {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM OUTPUT AS SELECT\n  TEST.ID ID,\n  ARRAY_JOIN(TEST.BOOLEANARRAY, '') KSQL_COL_0,\n  ARRAY_JOIN(TEST.INTARRAY, null) KSQL_COL_1,\n  ARRAY_JOIN(TEST.BIGINTARRAY, '') KSQL_COL_2,\n  ARRAY_JOIN(TEST.DOUBLEARRAY, null) KSQL_COL_3,\n  ARRAY_JOIN(TEST.DECIMALARRAY, '') KSQL_COL_4,\n  ARRAY_JOIN(TEST.STRINGARRAY, null) KSQL_COL_5\nFROM TEST TEST\nEMIT CHANGES",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "OUTPUT",
+      "schema" : "`ID` STRING KEY, `KSQL_COL_0` STRING, `KSQL_COL_1` STRING, `KSQL_COL_2` STRING, `KSQL_COL_3` STRING, `KSQL_COL_4` STRING, `KSQL_COL_5` STRING",
+      "topicName" : "OUTPUT",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "JSON"
+        }
+      }
+    },
+    "queryPlan" : {
+      "sources" : [ "TEST" ],
+      "sink" : "OUTPUT",
+      "physicalPlan" : {
+        "@type" : "streamSinkV1",
+        "properties" : {
+          "queryContext" : "OUTPUT"
+        },
+        "source" : {
+          "@type" : "streamSelectV1",
+          "properties" : {
+            "queryContext" : "Project"
+          },
+          "source" : {
+            "@type" : "streamSourceV1",
+            "properties" : {
+              "queryContext" : "KsqlTopic/Source"
+            },
+            "topicName" : "test_topic",
+            "formats" : {
+              "keyFormat" : {
+                "format" : "KAFKA"
+              },
+              "valueFormat" : {
+                "format" : "JSON"
+              }
+            },
+            "sourceSchema" : "`ID` STRING KEY, `BOOLEANARRAY` ARRAY<BOOLEAN>, `INTARRAY` ARRAY<INTEGER>, `BIGINTARRAY` ARRAY<BIGINT>, `DOUBLEARRAY` ARRAY<DOUBLE>, `DECIMALARRAY` ARRAY<DECIMAL(5, 2)>, `STRINGARRAY` ARRAY<STRING>"
+          },
+          "keyColumnNames" : [ "ID" ],
+          "selectExpressions" : [ "ARRAY_JOIN(BOOLEANARRAY, '') AS KSQL_COL_0", "ARRAY_JOIN(INTARRAY, null) AS KSQL_COL_1", "ARRAY_JOIN(BIGINTARRAY, '') AS KSQL_COL_2", "ARRAY_JOIN(DOUBLEARRAY, null) AS KSQL_COL_3", "ARRAY_JOIN(DECIMALARRAY, '') AS KSQL_COL_4", "ARRAY_JOIN(STRINGARRAY, null) AS KSQL_COL_5" ]
+        },
+        "formats" : {
+          "keyFormat" : {
+            "format" : "KAFKA"
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          }
+        },
+        "topicName" : "OUTPUT"
+      },
+      "queryId" : "CSAS_OUTPUT_0"
+    }
+  } ],
+  "configs" : {
+    "ksql.extension.dir" : "ext",
+    "ksql.streams.cache.max.bytes.buffering" : "0",
+    "ksql.security.extension.class" : null,
+    "ksql.transient.prefix" : "transient_",
+    "ksql.persistence.wrap.single.values" : "true",
+    "ksql.authorization.cache.expiry.time.secs" : "30",
+    "ksql.schema.registry.url" : "",
+    "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler",
+    "ksql.output.topic.name.prefix" : "",
+    "ksql.streams.auto.offset.reset" : "earliest",
+    "ksql.query.pull.enable.standby.reads" : "false",
+    "ksql.connect.url" : "http://localhost:8083",
+    "ksql.service.id" : "some.ksql.service.id",
+    "ksql.internal.topic.min.insync.replicas" : "1",
+    "ksql.streams.shutdown.timeout.ms" : "300000",
+    "ksql.internal.topic.replicas" : "1",
+    "ksql.insert.into.values.enabled" : "true",
+    "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807",
+    "ksql.query.pull.max.qps" : "2147483647",
+    "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler",
+    "ksql.access.validator.enable" : "auto",
+    "ksql.streams.bootstrap.servers" : "localhost:0",
+    "ksql.streams.commit.interval.ms" : "2000",
+    "ksql.metric.reporters" : "",
+    "ksql.query.pull.metrics.enabled" : "false",
+    "ksql.streams.auto.commit.interval.ms" : "0",
+    "ksql.metrics.extension" : null,
+    "ksql.streams.topology.optimization" : "all",
+    "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.streams.num.stream.threads" : "4",
+    "ksql.timestamp.throw.on.invalid" : "false",
+    "ksql.authorization.cache.max.entries" : "10000",
+    "ksql.metrics.tags.custom" : "",
+    "ksql.pull.queries.enable" : "true",
+    "ksql.udfs.enabled" : "true",
+    "ksql.udf.enable.security.manager" : "true",
+    "ksql.connect.worker.config" : "",
+    "ksql.sink.window.change.log.additional.retention" : "1000000",
+    "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.udf.collect.metrics" : "false",
+    "ksql.persistent.prefix" : "query_",
+    "ksql.query.persistent.active.limit" : "2147483647",
+    "ksql.error.classifier.regex" : ""
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/spec.json
new file mode 100644
index 000000000000..7d30338528ad
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/spec.json
@@ -0,0 +1,73 @@
+{
+  "version" : "6.0.0",
+  "timestamp" : 1591955240399,
+  "path" : "query-validation-tests/arrayjoin.json",
+  "schemas" : {
+    "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT<BOOLEANARRAY ARRAY<BOOLEAN>, INTARRAY ARRAY<INT>, BIGINTARRAY ARRAY<BIGINT>, DOUBLEARRAY ARRAY<DOUBLE>, DECIMALARRAY ARRAY<DECIMAL(5, 2)>, STRINGARRAY ARRAY<VARCHAR>> NOT NULL",
+    "CSAS_OUTPUT_0.OUTPUT" : "STRUCT<KSQL_COL_0 VARCHAR, KSQL_COL_1 VARCHAR, KSQL_COL_2 VARCHAR, KSQL_COL_3 VARCHAR, KSQL_COL_4 VARCHAR, KSQL_COL_5 VARCHAR> NOT NULL"
+  },
+  "testCase" : {
+    "name" : "join flat arrays with primitive ksqldb types and empty or null delimiter",
+    "inputs" : [ {
+      "topic" : "test_topic",
+      "key" : "1",
+      "value" : {
+        "BOOLEANARRAY" : [ true, null, false ],
+        "INTARRAY" : [ 1, 23, -42, 0 ],
+        "BIGINTARRAY" : [ -4294967297, 8589934592 ],
+        "DOUBLEARRAY" : [ 1.23, -23.42, 0.0 ],
+        "DECIMALARRAY" : [ 123.45, 987.65 ],
+        "STRINGARRAY" : [ "Hello", "From", "", "Ksqldb", "Udf" ]
+      }
+    } ],
+    "outputs" : [ {
+      "topic" : "OUTPUT",
+      "key" : "1",
+      "value" : {
+        "KSQL_COL_0" : "truenullfalse",
+        "KSQL_COL_1" : "123-420",
+        "KSQL_COL_2" : "-42949672978589934592",
+        "KSQL_COL_3" : "1.23-23.420.0",
+        "KSQL_COL_4" : "123.45987.65",
+        "KSQL_COL_5" : "HelloFromKsqldbUdf"
+      }
+    } ],
+    "topics" : [ {
+      "name" : "OUTPUT",
+      "replicas" : 1,
+      "numPartitions" : 4
+    }, {
+      "name" : "test_topic",
+      "replicas" : 1,
+      "numPartitions" : 4
+    } ],
+    "statements" : [ "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY<BOOLEAN>, INTARRAY ARRAY<INT>, BIGINTARRAY ARRAY<BIGINT>, DOUBLEARRAY ARRAY<DOUBLE>, DECIMALARRAY ARRAY<DECIMAL(5,2)>, STRINGARRAY ARRAY<STRING>) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT ID, ARRAY_JOIN(BOOLEANARRAY,''), ARRAY_JOIN(INTARRAY,null), ARRAY_JOIN(BIGINTARRAY,''), ARRAY_JOIN(DOUBLEARRAY,null), ARRAY_JOIN(DECIMALARRAY,''), ARRAY_JOIN(STRINGARRAY,null) FROM TEST;" ],
+    "post" : {
+      "topics" : {
+        "topics" : [ {
+          "name" : "OUTPUT",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          },
+          "partitions" : 4
+        }, {
+          "name" : "test_topic",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          },
+          "partitions" : 4
+        } ]
+      }
+    }
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/topology
new file mode 100644
index 000000000000..441a8f282644
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/arrayjoin_-_join_flat_arrays_with_primitive_ksqldb_types_and_empty_or_null_delimiter/6.0.0_1591955240399/topology
@@ -0,0 +1,13 @@
+Topologies:
+   Sub-topology: 0
+    Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic])
+      --> KSTREAM-TRANSFORMVALUES-0000000001
+    Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: [])
+      --> Project
+      <-- KSTREAM-SOURCE-0000000000
+    Processor: Project (stores: [])
+      --> KSTREAM-SINK-0000000003
+      <-- KSTREAM-TRANSFORMVALUES-0000000001
+    Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT)
+      <-- Project
+
diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/arrayjoin.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/arrayjoin.json
new file mode 100644
index 000000000000..ad0a7a5801a6
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/arrayjoin.json
@@ -0,0 +1,109 @@
+{
+  "comments": [
+    "Tests covering the use of the ARRAY_JOIN function."
+  ],
+  "tests": [
+    {
+      "name": "join flat arrays with primitive ksqldb types and default delimiter",
+      "statements": [
+        "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY<BOOLEAN>, INTARRAY ARRAY<INT>, BIGINTARRAY ARRAY<BIGINT>, DOUBLEARRAY ARRAY<DOUBLE>, DECIMALARRAY ARRAY<DECIMAL(5,2)>, STRINGARRAY ARRAY<STRING>) WITH (kafka_topic='test_topic', value_format='JSON');",
+        "CREATE STREAM OUTPUT AS SELECT ID, ARRAY_JOIN(BOOLEANARRAY), ARRAY_JOIN(INTARRAY), ARRAY_JOIN(BIGINTARRAY), ARRAY_JOIN(DOUBLEARRAY), ARRAY_JOIN(DECIMALARRAY), ARRAY_JOIN(STRINGARRAY) FROM TEST;"
+      ],
+      "inputs": [
+        {
+          "topic": "test_topic", "key": "1",
+          "value": {
+            "BOOLEANARRAY": [true, null, false],
+            "INTARRAY": [1, 23, -42, 0],
+            "BIGINTARRAY": [-4294967297, 8589934592],
+            "DOUBLEARRAY": [1.23, -23.42, 0.0],
+            "DECIMALARRAY": [123.45, 987.65],
+            "STRINGARRAY": ["Hello", "From", "", "Ksqldb", "Udf"]
+          }
+        }
+      ],
+      "outputs": [
+        {
+          "topic": "OUTPUT",
+          "key": "1",
+          "value": {
+            "KSQL_COL_0": "true,null,false",
+            "KSQL_COL_1": "1,23,-42,0",
+            "KSQL_COL_2": "-4294967297,8589934592",
+            "KSQL_COL_3": "1.23,-23.42,0.0",
+            "KSQL_COL_4": "123.45,987.65",
+            "KSQL_COL_5": "Hello,From,,Ksqldb,Udf"
+          }
+        }
+      ]
+    },
+    {
+      "name": "join flat arrays with primitive ksqldb types and custom delimiters",
+      "statements": [
+        "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY<BOOLEAN>, INTARRAY ARRAY<INT>, BIGINTARRAY ARRAY<BIGINT>, DOUBLEARRAY ARRAY<DOUBLE>, DECIMALARRAY ARRAY<DECIMAL(5,2)>, STRINGARRAY ARRAY<STRING>) WITH (kafka_topic='test_topic', value_format='JSON');",
+        "CREATE STREAM OUTPUT AS SELECT ID, ARRAY_JOIN(BOOLEANARRAY,'|'), ARRAY_JOIN(INTARRAY,'?'), ARRAY_JOIN(BIGINTARRAY,';'), ARRAY_JOIN(DOUBLEARRAY,' '), ARRAY_JOIN(DECIMALARRAY,'#'), ARRAY_JOIN(STRINGARRAY,'_') FROM TEST;"
+      ],
+      "inputs": [
+        {
+          "topic": "test_topic", "key": "1",
+          "value": {
+            "BOOLEANARRAY": [true, null, false],
+            "INTARRAY": [1, 23, -42, 0],
+            "BIGINTARRAY": [-4294967297, 8589934592],
+            "DOUBLEARRAY": [1.23, -23.42, 0.0],
+            "DECIMALARRAY": [123.45, 987.65],
+            "STRINGARRAY": ["Hello", "From", "", "Ksqldb", "Udf"]
+          }
+        }
+      ],
+      "outputs": [
+        {
+          "topic": "OUTPUT",
+          "key": "1",
+          "value": {
+            "KSQL_COL_0": "true|null|false",
+            "KSQL_COL_1": "1?23?-42?0",
+            "KSQL_COL_2": "-4294967297;8589934592",
+            "KSQL_COL_3": "1.23 -23.42 0.0",
+            "KSQL_COL_4": "123.45#987.65",
+            "KSQL_COL_5": "Hello_From__Ksqldb_Udf"
+          }
+        }
+      ]
+    },
+    {
+      "name": "join flat arrays with primitive ksqldb types and empty or null delimiter",
+      "statements": [
+        "CREATE STREAM TEST (ID STRING KEY, BOOLEANARRAY ARRAY<BOOLEAN>, INTARRAY ARRAY<INT>, BIGINTARRAY ARRAY<BIGINT>, DOUBLEARRAY ARRAY<DOUBLE>, DECIMALARRAY ARRAY<DECIMAL(5,2)>, STRINGARRAY ARRAY<STRING>) WITH (kafka_topic='test_topic', value_format='JSON');",
+        "CREATE STREAM OUTPUT AS SELECT ID, ARRAY_JOIN(BOOLEANARRAY,''), ARRAY_JOIN(INTARRAY,null), ARRAY_JOIN(BIGINTARRAY,''), ARRAY_JOIN(DOUBLEARRAY,null), ARRAY_JOIN(DECIMALARRAY,''), ARRAY_JOIN(STRINGARRAY,null) FROM TEST;"
+      ],
+      "inputs": [
+        {
+          "topic": "test_topic", "key": "1",
+          "value": {
+            "BOOLEANARRAY": [true, null, false],
+            "INTARRAY": [1, 23, -42, 0],
+            "BIGINTARRAY": [-4294967297, 8589934592],
+            "DOUBLEARRAY": [1.23, -23.42, 0.0],
+            "DECIMALARRAY": [123.45, 987.65],
+            "STRINGARRAY": ["Hello", "From", "", "Ksqldb", "Udf"]
+          }
+        }
+      ],
+      "outputs": [
+        {
+          "topic": "OUTPUT",
+          "key": "1",
+          "value": {
+            "KSQL_COL_0": "truenullfalse",
+            "KSQL_COL_1": "123-420",
+            "KSQL_COL_2": "-42949672978589934592",
+            "KSQL_COL_3": "1.23-23.420.0",
+            "KSQL_COL_4": "123.45987.65",
+            "KSQL_COL_5": "HelloFromKsqldbUdf"
+          }
+        }
+      ]
+    }
+  ]
+}
\ No newline at end of file

From 9ea1fdfa7761cd73dc6e56999989157deddada37 Mon Sep 17 00:00:00 2001
From: Nick Dearden <blueedgenick@users.noreply.github.com>
Date: Thu, 11 Jun 2020 13:36:08 -0700
Subject: [PATCH 2/3] feat: new split_to_map udf (#5563)

New UDF split_to_map(input, entryDelimiter, kvDelimiter) to build a map from a string.

Useful for taking messages from upstream systems and converting them into a more structured and usable format.
---
 .../ksqldb-reference/scalar-functions.md      |  17 ++
 .../ksql/function/udf/string/SplitToMap.java  |  61 ++++++
 .../function/udf/string/SplitToMapTest.java   | 132 ++++++++++++
 .../6.0.0_1591481024082/plan.json             | 126 +++++++++++
 .../6.0.0_1591481024082/spec.json             | 201 ++++++++++++++++++
 .../6.0.0_1591481024082/topology              |  13 ++
 .../query-validation-tests/split-to-map.json  |  38 ++++
 7 files changed, 588 insertions(+)
 create mode 100644 ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/string/SplitToMap.java
 create mode 100644 ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/string/SplitToMapTest.java
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/split-to-map_-_split_to_map/6.0.0_1591481024082/plan.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/split-to-map_-_split_to_map/6.0.0_1591481024082/spec.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/split-to-map_-_split_to_map/6.0.0_1591481024082/topology
 create mode 100644 ksqldb-functional-tests/src/test/resources/query-validation-tests/split-to-map.json

diff --git a/docs/developer-guide/ksqldb-reference/scalar-functions.md b/docs/developer-guide/ksqldb-reference/scalar-functions.md
index fe96cd5c6e29..cba94d790782 100644
--- a/docs/developer-guide/ksqldb-reference/scalar-functions.md
+++ b/docs/developer-guide/ksqldb-reference/scalar-functions.md
@@ -732,6 +732,23 @@ If the delimiter is found at the beginning or end
 of the string, or there are contiguous delimiters,
 then an empty space is added to the array.
 
+### `SPLIT_TO_MAP`
+
+```sql
+SPLIT_TO_MAP(input, entryDelimiter, kvDelimiter)
+```
+
+Splits a string into key-value pairs and creates a map from them. The 
+`entryDelimiter` splits the string into key-value pairs which are then split by `kvDelimiter`. If the same key is present multiple times in the input, the latest value for that key is returned. 
+
+Returns NULL if the input text is NULL.
+Returns NULL if either of the delimiters is NULL or an empty string.
+
+Example:
+```sql
+SPLIT_TO_MAP('apple':='green'/'cherry':='red', '/', ':=')  => { 'apple':'green', 'cherry':'red'}
+```
+
 ### `SUBSTRING`
 
 ```sql
diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/string/SplitToMap.java b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/string/SplitToMap.java
new file mode 100644
index 000000000000..ce3d14432230
--- /dev/null
+++ b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/string/SplitToMap.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2020 Confluent Inc.
+ *
+ * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package io.confluent.ksql.function.udf.string;
+
+import com.google.common.base.Splitter;
+import io.confluent.ksql.function.udf.Udf;
+import io.confluent.ksql.function.udf.UdfDescription;
+import io.confluent.ksql.function.udf.UdfParameter;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+
+@UdfDescription(
+    name = "split_to_map",
+    description = "Splits a string into key-value pairs and creates a map from them. The "
+        + "'entryDelimiter' splits the string into key-value pairs which are then split by "
+        + "'kvDelimiter'. If the same key is present multiple times in the input, the latest "
+        + "value for that key is returned. Returns NULL f the input text or either of the "
+        + "delimiters is NULL.")
+public class SplitToMap {
+  @Udf
+  public Map<String, String> splitToMap(
+      @UdfParameter(
+          description = "Separator string and values to join") final String input,
+      @UdfParameter(
+          description = "Separator string and values to join") final String entryDelimiter,
+      @UdfParameter(
+          description = "Separator string and values to join") final String kvDelimiter) {
+
+    if (input == null || entryDelimiter == null || kvDelimiter == null) {
+      return null;
+    }
+
+    if (entryDelimiter.isEmpty() || kvDelimiter.isEmpty() || entryDelimiter.equals(kvDelimiter)) {
+      return null;
+    }
+
+    final Iterable<String> entries = Splitter.on(entryDelimiter).omitEmptyStrings().split(input);
+    final Map<String, String> output = StreamSupport.stream(entries.spliterator(), false)
+        .filter(e -> e.contains(kvDelimiter))
+        .map(kv -> Splitter.on(kvDelimiter).split(kv).iterator())
+        .collect(Collectors.toMap(
+            kvIter -> kvIter.next(), 
+            kvIter -> kvIter.next(),
+            (v1, v2) -> v2));
+
+    return output;
+  }
+}
diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/string/SplitToMapTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/string/SplitToMapTest.java
new file mode 100644
index 000000000000..a37c576e2787
--- /dev/null
+++ b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/string/SplitToMapTest.java
@@ -0,0 +1,132 @@
+/*
+ * Copyright 2018 Confluent Inc.
+ *
+ * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OF ANY KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+
+package io.confluent.ksql.function.udf.string;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.hasEntry;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.nullValue;
+
+import java.util.Collections;
+import java.util.Map;
+import org.junit.Test;
+
+public class SplitToMapTest {
+  private static final SplitToMap udf = new SplitToMap();
+
+  @Test
+  public void shouldSplitStringByGivenDelimiterChars() {
+    Map<String, String> result = udf.splitToMap("foo=apple;bar=cherry", ";", "=");
+    assertThat(result, hasEntry("foo", "apple"));
+    assertThat(result, hasEntry("bar", "cherry"));
+    assertThat(result.size(), equalTo(2));
+  }
+
+  @Test
+  public void shouldSplitStringGivenMultiCharDelimiters() {
+    Map<String, String> result = udf.splitToMap("foo:=apple||bar:=cherry", "||", ":=");
+    assertThat(result, hasEntry("foo", "apple"));
+    assertThat(result, hasEntry("bar", "cherry"));
+    assertThat(result.size(), equalTo(2));
+  }
+
+  @Test
+  public void shouldSplitStringWithOnlyOneEntry() {
+    Map<String, String> result = udf.splitToMap("foo=apple", ";", "=");
+    assertThat(result, hasEntry("foo", "apple"));
+    assertThat(result.size(), equalTo(1));
+  }
+
+  @Test
+  public void shouldRetainWhitespacebetweenDelimiters() {
+    Map<String, String> result = udf.splitToMap("foo :=\tapple || bar:=cherry", "||", ":=");
+    assertThat(result, hasEntry("foo ", "\tapple "));
+    assertThat(result, hasEntry(" bar", "cherry"));
+    assertThat(result.size(), equalTo(2));
+  }
+
+  @Test
+  public void shouldDropEmptyEntriesFromSplit() {
+    Map<String, String> result = udf.splitToMap("/foo:=apple//bar:=cherry/", "/", ":=");
+    assertThat(result, hasEntry("foo", "apple"));
+    assertThat(result, hasEntry("bar", "cherry"));
+    assertThat(result.size(), equalTo(2));
+  }
+
+  @Test
+  public void shouldDropEntriesWithoutKeyAndValue() {
+    Map<String, String> result = udf.splitToMap("foo:=apple/cherry", "/", ":=");
+    assertThat(result, hasEntry("foo", "apple"));
+    assertThat(result.size(), equalTo(1));
+  }
+
+  @Test
+  public void shouldReturnEmptyForInputWithoutDelimiters() {
+    Map<String, String> result = udf.splitToMap("cherry", "/", ":=");
+    assertThat(result, is(Collections.EMPTY_MAP));
+  }
+
+  @Test
+  public void shouldReturnEmptyForEmptyInput() {
+    Map<String, String> result = udf.splitToMap("", "/", ":=");
+    assertThat(result, is(Collections.EMPTY_MAP));
+  }
+
+  @Test
+  public void shouldKeepLatestValueForDuplicateKey() {
+    Map<String, String> result = udf.splitToMap("/foo:=apple/foo:=cherry/", "/", ":=");
+    assertThat(result, hasEntry("foo", "cherry"));
+    assertThat(result.size(), equalTo(1));
+  }
+
+  @Test
+  public void shouldReturnNullOnNullInputString() {
+    Map<String, String> result = udf.splitToMap(null, "/", ":=");
+    assertThat(result, is(nullValue()));
+  }
+
+  @Test
+  public void shouldReturnNullOnSameDelimiterChars() {
+    Map<String, String> result = udf.splitToMap("foo:=apple/bar:=cherry", "/", "/");
+    assertThat(result, is(nullValue()));
+  }
+
+  @Test
+  public void shouldReturnNullOnEmptyEntryDelimiter() {
+    Map<String, String> result = udf.splitToMap("foo:=apple/bar:=cherry", "", ":=");
+    assertThat(result, is(nullValue()));
+  }
+
+  @Test
+  public void shouldReturnNullOnNullEntryDelimiter() {
+    Map<String, String> result = udf.splitToMap("foo:=apple/bar:=cherry", null, ":=");
+    assertThat(result, is(nullValue()));
+  }
+
+  @Test
+  public void shouldReturnNullOnEmptyValueDelimiter() {
+    Map<String, String> result = udf.splitToMap("foo:=apple/bar:=cherry", "/", "");
+    assertThat(result, is(nullValue()));
+  }
+
+  @Test
+  public void shouldReturnNullOnNullValueDelimiter() {
+    Map<String, String> result = udf.splitToMap("foo:=apple/bar:=cherry", "/", null);
+    assertThat(result, is(nullValue()));
+  }
+
+}
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/split-to-map_-_split_to_map/6.0.0_1591481024082/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/split-to-map_-_split_to_map/6.0.0_1591481024082/plan.json
new file mode 100644
index 000000000000..085072de14d3
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/split-to-map_-_split_to_map/6.0.0_1591481024082/plan.json
@@ -0,0 +1,126 @@
+{
+  "plan" : [ {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM INPUT (ID STRING KEY, INPUT STRING, ENTRYDELIM STRING, KVDELIM STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "INPUT",
+      "schema" : "`ID` STRING KEY, `INPUT` STRING, `ENTRYDELIM` STRING, `KVDELIM` STRING",
+      "topicName" : "test_topic",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "JSON"
+        }
+      }
+    }
+  }, {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM OUTPUT AS SELECT\n  INPUT.ID ID,\n  SPLIT_TO_MAP(INPUT.INPUT, INPUT.ENTRYDELIM, INPUT.KVDELIM) RESULT\nFROM INPUT INPUT\nEMIT CHANGES",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "OUTPUT",
+      "schema" : "`ID` STRING KEY, `RESULT` MAP<STRING, STRING>",
+      "topicName" : "OUTPUT",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "JSON"
+        }
+      }
+    },
+    "queryPlan" : {
+      "sources" : [ "INPUT" ],
+      "sink" : "OUTPUT",
+      "physicalPlan" : {
+        "@type" : "streamSinkV1",
+        "properties" : {
+          "queryContext" : "OUTPUT"
+        },
+        "source" : {
+          "@type" : "streamSelectV1",
+          "properties" : {
+            "queryContext" : "Project"
+          },
+          "source" : {
+            "@type" : "streamSourceV1",
+            "properties" : {
+              "queryContext" : "KsqlTopic/Source"
+            },
+            "topicName" : "test_topic",
+            "formats" : {
+              "keyFormat" : {
+                "format" : "KAFKA"
+              },
+              "valueFormat" : {
+                "format" : "JSON"
+              }
+            },
+            "sourceSchema" : "`ID` STRING KEY, `INPUT` STRING, `ENTRYDELIM` STRING, `KVDELIM` STRING"
+          },
+          "keyColumnNames" : [ "ID" ],
+          "selectExpressions" : [ "SPLIT_TO_MAP(INPUT, ENTRYDELIM, KVDELIM) AS RESULT" ]
+        },
+        "formats" : {
+          "keyFormat" : {
+            "format" : "KAFKA"
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          }
+        },
+        "topicName" : "OUTPUT"
+      },
+      "queryId" : "CSAS_OUTPUT_0"
+    }
+  } ],
+  "configs" : {
+    "ksql.extension.dir" : "ext",
+    "ksql.streams.cache.max.bytes.buffering" : "0",
+    "ksql.security.extension.class" : null,
+    "ksql.transient.prefix" : "transient_",
+    "ksql.persistence.wrap.single.values" : "true",
+    "ksql.authorization.cache.expiry.time.secs" : "30",
+    "ksql.schema.registry.url" : "",
+    "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler",
+    "ksql.output.topic.name.prefix" : "",
+    "ksql.streams.auto.offset.reset" : "earliest",
+    "ksql.query.pull.enable.standby.reads" : "false",
+    "ksql.connect.url" : "http://localhost:8083",
+    "ksql.service.id" : "some.ksql.service.id",
+    "ksql.internal.topic.min.insync.replicas" : "1",
+    "ksql.streams.shutdown.timeout.ms" : "300000",
+    "ksql.internal.topic.replicas" : "1",
+    "ksql.insert.into.values.enabled" : "true",
+    "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807",
+    "ksql.query.pull.max.qps" : "2147483647",
+    "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler",
+    "ksql.access.validator.enable" : "auto",
+    "ksql.streams.bootstrap.servers" : "localhost:0",
+    "ksql.streams.commit.interval.ms" : "2000",
+    "ksql.metric.reporters" : "",
+    "ksql.query.pull.metrics.enabled" : "false",
+    "ksql.streams.auto.commit.interval.ms" : "0",
+    "ksql.metrics.extension" : null,
+    "ksql.streams.topology.optimization" : "all",
+    "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.streams.num.stream.threads" : "4",
+    "ksql.timestamp.throw.on.invalid" : "false",
+    "ksql.authorization.cache.max.entries" : "10000",
+    "ksql.metrics.tags.custom" : "",
+    "ksql.pull.queries.enable" : "true",
+    "ksql.udfs.enabled" : "true",
+    "ksql.udf.enable.security.manager" : "true",
+    "ksql.connect.worker.config" : "",
+    "ksql.sink.window.change.log.additional.retention" : "1000000",
+    "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.udf.collect.metrics" : "false",
+    "ksql.persistent.prefix" : "query_",
+    "ksql.query.persistent.active.limit" : "2147483647",
+    "ksql.error.classifier.regex" : ""
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/split-to-map_-_split_to_map/6.0.0_1591481024082/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/split-to-map_-_split_to_map/6.0.0_1591481024082/spec.json
new file mode 100644
index 000000000000..74553a2abef9
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/split-to-map_-_split_to_map/6.0.0_1591481024082/spec.json
@@ -0,0 +1,201 @@
+{
+  "version" : "6.0.0",
+  "timestamp" : 1591481024082,
+  "path" : "query-validation-tests\\split-to-map.json",
+  "schemas" : {
+    "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT<INPUT VARCHAR, ENTRYDELIM VARCHAR, KVDELIM VARCHAR> NOT NULL",
+    "CSAS_OUTPUT_0.OUTPUT" : "STRUCT<RESULT MAP<VARCHAR, VARCHAR>> NOT NULL"
+  },
+  "testCase" : {
+    "name" : "split_to_map",
+    "inputs" : [ {
+      "topic" : "test_topic",
+      "key" : "r1",
+      "value" : {
+        "input" : "apple:green/banana:yellow/cherry:red",
+        "entryDelim" : "/",
+        "kvDelim" : ":"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r2",
+      "value" : {
+        "input" : "apple:green/banana:yellow/cherry:red",
+        "entryDelim" : "&&",
+        "kvDelim" : ":"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r3",
+      "value" : {
+        "input" : "apple:green//banana:yellow//cherry:red/",
+        "entryDelim" : "/",
+        "kvDelim" : ":"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r4",
+      "value" : {
+        "input" : "apple:green/banana:yellow/cherry:red",
+        "entryDelim" : "/",
+        "kvDelim" : "="
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r5",
+      "value" : {
+        "input" : "apple:green/banana:yellow/cherry:red",
+        "entryDelim" : "",
+        "kvDelim" : ":"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r6",
+      "value" : {
+        "input" : "apple:green/banana:yellow/cherry:red",
+        "entryDelim" : "/",
+        "kvDelim" : ""
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r7",
+      "value" : {
+        "input" : "apple:green/banana:yellow/cherry:red",
+        "entryDelim" : null,
+        "kvDelim" : ":"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r8",
+      "value" : {
+        "input" : "apple:green/banana:yellow/cherry:red",
+        "entryDelim" : "/",
+        "kvDelim" : null
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r9",
+      "value" : {
+        "input" : "",
+        "entryDelim" : "/",
+        "kvDelim" : ":"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r10",
+      "value" : {
+        "input" : null,
+        "entryDelim" : "/",
+        "kvDelim" : ":"
+      }
+    } ],
+    "outputs" : [ {
+      "topic" : "OUTPUT",
+      "key" : "r1",
+      "value" : {
+        "RESULT" : {
+          "apple" : "green",
+          "banana" : "yellow",
+          "cherry" : "red"
+        }
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r2",
+      "value" : {
+        "RESULT" : {
+          "apple" : "green/banana"
+        }
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r3",
+      "value" : {
+        "RESULT" : {
+          "apple" : "green",
+          "banana" : "yellow",
+          "cherry" : "red"
+        }
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r4",
+      "value" : {
+        "RESULT" : { }
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r5",
+      "value" : {
+        "RESULT" : null
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r6",
+      "value" : {
+        "RESULT" : null
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r7",
+      "value" : {
+        "RESULT" : null
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r8",
+      "value" : {
+        "RESULT" : null
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r9",
+      "value" : {
+        "RESULT" : { }
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r10",
+      "value" : {
+        "RESULT" : null
+      }
+    } ],
+    "topics" : [ {
+      "name" : "OUTPUT",
+      "replicas" : 1,
+      "numPartitions" : 4
+    }, {
+      "name" : "test_topic",
+      "replicas" : 1,
+      "numPartitions" : 4
+    } ],
+    "statements" : [ "CREATE STREAM INPUT (id STRING KEY, input STRING, entryDelim STRING, kvDelim STRING) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT id, split_to_map(input, entryDelim, kvDelim) as result FROM INPUT;" ],
+    "post" : {
+      "topics" : {
+        "topics" : [ {
+          "name" : "OUTPUT",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          },
+          "partitions" : 4
+        }, {
+          "name" : "test_topic",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          },
+          "partitions" : 4
+        } ]
+      }
+    }
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/split-to-map_-_split_to_map/6.0.0_1591481024082/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/split-to-map_-_split_to_map/6.0.0_1591481024082/topology
new file mode 100644
index 000000000000..441a8f282644
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/split-to-map_-_split_to_map/6.0.0_1591481024082/topology
@@ -0,0 +1,13 @@
+Topologies:
+   Sub-topology: 0
+    Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic])
+      --> KSTREAM-TRANSFORMVALUES-0000000001
+    Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: [])
+      --> Project
+      <-- KSTREAM-SOURCE-0000000000
+    Processor: Project (stores: [])
+      --> KSTREAM-SINK-0000000003
+      <-- KSTREAM-TRANSFORMVALUES-0000000001
+    Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT)
+      <-- Project
+
diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/split-to-map.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/split-to-map.json
new file mode 100644
index 000000000000..27056330ecd6
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/split-to-map.json
@@ -0,0 +1,38 @@
+{
+  "comments": [
+    "Tests covering use of SPLIT_TO_MAP UDF."
+  ],
+  "tests": [
+    {
+      "name": "split_to_map",
+      "statements": [
+        "CREATE STREAM INPUT (id STRING KEY, input STRING, entryDelim STRING, kvDelim STRING) WITH (kafka_topic='test_topic', value_format='JSON');",
+        "CREATE STREAM OUTPUT AS SELECT id, split_to_map(input, entryDelim, kvDelim) as result FROM INPUT;"
+      ],
+      "inputs": [
+        {"topic": "test_topic", "key": "r1", "value": {"input": "apple:green/banana:yellow/cherry:red", "entryDelim": "/", "kvDelim": ":"}},
+        {"topic": "test_topic", "key": "r2", "value": {"input": "apple:green/banana:yellow/cherry:red", "entryDelim": "&&", "kvDelim": ":"}},
+        {"topic": "test_topic", "key": "r3", "value": {"input": "apple:green//banana:yellow//cherry:red/", "entryDelim": "/", "kvDelim": ":"}},
+        {"topic": "test_topic", "key": "r4", "value": {"input": "apple:green/banana:yellow/cherry:red", "entryDelim": "/", "kvDelim": "="}},
+        {"topic": "test_topic", "key": "r5", "value": {"input": "apple:green/banana:yellow/cherry:red", "entryDelim": "", "kvDelim": ":"}},
+        {"topic": "test_topic", "key": "r6", "value": {"input": "apple:green/banana:yellow/cherry:red", "entryDelim": "/", "kvDelim": ""}},
+        {"topic": "test_topic", "key": "r7", "value": {"input": "apple:green/banana:yellow/cherry:red", "entryDelim": null, "kvDelim": ":"}},
+        {"topic": "test_topic", "key": "r8", "value": {"input": "apple:green/banana:yellow/cherry:red", "entryDelim": "/", "kvDelim": null}},
+        {"topic": "test_topic", "key": "r9", "value": {"input": "", "entryDelim": "/", "kvDelim": ":"}},
+        {"topic": "test_topic", "key": "r10", "value": {"input": null, "entryDelim": "/", "kvDelim": ":"}}
+      ],
+      "outputs": [
+        {"topic": "OUTPUT", "key": "r1", "value": {"RESULT": {"apple": "green", "banana": "yellow", "cherry": "red"}}},
+        {"topic": "OUTPUT", "key": "r2", "value": {"RESULT": {"apple": "green/banana"}}},
+        {"topic": "OUTPUT", "key": "r3", "value": {"RESULT": {"apple": "green", "banana": "yellow", "cherry": "red"}}},
+        {"topic": "OUTPUT", "key": "r4", "value": {"RESULT": {}}},
+        {"topic": "OUTPUT", "key": "r5", "value": {"RESULT": null}},
+        {"topic": "OUTPUT", "key": "r6", "value": {"RESULT": null}},
+        {"topic": "OUTPUT", "key": "r7", "value": {"RESULT": null}},
+        {"topic": "OUTPUT", "key": "r8", "value": {"RESULT": null}},
+        {"topic": "OUTPUT", "key": "r9", "value": {"RESULT": {}}},
+        {"topic": "OUTPUT", "key": "r10", "value": {"RESULT": null}}
+      ]
+    }
+  ]
+}
\ No newline at end of file

From 95f9678d5b7a435051110082a83ee49c2cb329e0 Mon Sep 17 00:00:00 2001
From: Nick Dearden <blueedgenick@users.noreply.github.com>
Date: Thu, 11 Jun 2020 13:34:46 -0700
Subject: [PATCH 3/3] feat: add CHR UDF (#5559)

A new UDF, CHR, to turn a number representing a unicode codepoint into a single-character string. Very useful for dealing with non-printable characters (tab, CR, LF, ...) in strings or those characters not easily represented in your local codepage.
---
 .../ksqldb-reference/scalar-functions.md      |  20 +++
 .../ksql/function/udf/string/Chr.java         |  48 ++++++
 .../ksql/function/udf/string/ChrTest.java     | 126 ++++++++++++++
 .../6.0.0_1591421496995/plan.json             | 126 ++++++++++++++
 .../6.0.0_1591421496995/spec.json             | 123 ++++++++++++++
 .../6.0.0_1591421496995/topology              |  13 ++
 .../6.0.0_1591421497086/plan.json             | 126 ++++++++++++++
 .../6.0.0_1591421497086/spec.json             | 111 ++++++++++++
 .../6.0.0_1591421497086/topology              |  13 ++
 .../6.0.0_1591421497142/plan.json             | 126 ++++++++++++++
 .../6.0.0_1591421497142/spec.json             | 159 ++++++++++++++++++
 .../6.0.0_1591421497142/topology              |  13 ++
 .../6.0.0_1591421497199/plan.json             | 126 ++++++++++++++
 .../6.0.0_1591421497199/spec.json             | 147 ++++++++++++++++
 .../6.0.0_1591421497199/topology              |  13 ++
 .../6.0.0_1591421497279/plan.json             | 126 ++++++++++++++
 .../6.0.0_1591421497279/spec.json             |  90 ++++++++++
 .../6.0.0_1591421497279/topology              |  13 ++
 .../6.0.0_1591421497385/plan.json             | 126 ++++++++++++++
 .../6.0.0_1591421497385/spec.json             |  66 ++++++++
 .../6.0.0_1591421497385/topology              |  13 ++
 .../resources/query-validation-tests/chr.json |  72 ++++++++
 22 files changed, 1796 insertions(+)
 create mode 100644 ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/string/Chr.java
 create mode 100644 ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/string/ChrTest.java
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/plan.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/spec.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/topology
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/plan.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/spec.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/topology
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/plan.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/spec.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/topology
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/plan.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/spec.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/topology
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/plan.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/spec.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/topology
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/plan.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/spec.json
 create mode 100644 ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/topology
 create mode 100644 ksqldb-functional-tests/src/test/resources/query-validation-tests/chr.json

diff --git a/docs/developer-guide/ksqldb-reference/scalar-functions.md b/docs/developer-guide/ksqldb-reference/scalar-functions.md
index cba94d790782..2b80052cb16e 100644
--- a/docs/developer-guide/ksqldb-reference/scalar-functions.md
+++ b/docs/developer-guide/ksqldb-reference/scalar-functions.md
@@ -413,6 +413,26 @@ allows for array elements of primitive ksqlDB types.
 
 ## Strings
 
+### `CHR`
+
+```sql
+CHR(decimal_code | utf_string)
+```
+
+Returns a single-character string representing the Unicode code-point described by the input. The input parameter can be either a decimal character code or a string representation of a UTF code.
+
+Returns NULL if the input is NULL or does not represent a valid code-point.
+
+Commonly used to insert control characters such as `Tab` (9), `Line Feed` (10), or `Carriage Return` (13) into strings.
+
+Examples:
+```sql
+CHR(75)        => 'K'
+CHR('\u004b')  => 'K'
+CHR(22909)     => '好'
+CHR('\u597d')  => '好'
+```
+
 ### `CONCAT`
 
 ```sql
diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/string/Chr.java b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/string/Chr.java
new file mode 100644
index 000000000000..73f68a8c7309
--- /dev/null
+++ b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/string/Chr.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2020 Confluent Inc.
+ *
+ * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package io.confluent.ksql.function.udf.string;
+
+import io.confluent.ksql.function.udf.Udf;
+import io.confluent.ksql.function.udf.UdfDescription;
+import io.confluent.ksql.function.udf.UdfParameter;
+import org.apache.commons.lang3.StringEscapeUtils;
+
+@UdfDescription(
+    name = "Chr",
+    description = "Returns a single-character string corresponding to the input character code.")
+public class Chr {
+
+  @Udf
+  public String chr(@UdfParameter(
+      description = "Decimal codepoint") final Integer decimalCode) {
+    if (decimalCode == null) {
+      return null;
+    }
+    if (!Character.isValidCodePoint(decimalCode)) {
+      return null;
+    }
+    final char[] resultChars = Character.toChars(decimalCode.intValue());
+    return String.valueOf(resultChars);
+  }
+
+  @Udf
+  public String chr(@UdfParameter(
+      description = "UTF16 code for the desired character e.g. '\\u004b'") final String utf16Code) {
+    if (utf16Code == null || utf16Code.length() < 6 || !utf16Code.startsWith("\\u")) {
+      return null;
+    }
+    return StringEscapeUtils.unescapeJava(utf16Code);
+  }
+}
diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/string/ChrTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/string/ChrTest.java
new file mode 100644
index 000000000000..defe9ca0648f
--- /dev/null
+++ b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/string/ChrTest.java
@@ -0,0 +1,126 @@
+/*
+ * Copyright 2020 Confluent Inc.
+ *
+ * Licensed under the Confluent Community License (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.confluent.io/confluent-community-license
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and limitations under the
+ * License.
+ */
+
+package io.confluent.ksql.function.udf.string;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import org.junit.Test;
+
+public class ChrTest {
+  private final Chr udf = new Chr();
+
+  @Test
+  public void shouldConvertFromDecimal() {
+    final String result = udf.chr(75);
+    assertThat(result, is("K"));
+  }
+
+  @Test
+  public void shouldConvertFromUTF16String() {
+    final String result = udf.chr("\\u004b");
+    assertThat(result, is("K"));
+  }
+
+  @Test
+  public void shouldConvertFromUTF16StringWithSlash() {
+    final String result = udf.chr("\\u004b");
+    assertThat(result, is("K"));
+  }
+
+  @Test
+  public void shouldConvertZhFromDecimal() {
+    final String result = udf.chr(22909);
+    assertThat(result, is("好"));
+  }
+
+  @Test
+  public void shouldConvertZhFromUTF16() {
+    final String result = udf.chr("\\u597d");
+    assertThat(result, is("好"));
+  }
+
+  @Test
+  public void shouldConvertControlChar() {
+    final String result = udf.chr(9);
+    assertThat(result, is("\t"));
+  }
+
+  @Test
+  public void shouldReturnNullForNullIntegerInput() {
+    final String result = udf.chr((Integer) null);
+    assertThat(result, is(nullValue()));
+  }
+
+  @Test
+  public void shouldReturnNullForNullStringInput() {
+    final String result = udf.chr((String) null);
+    assertThat(result, is(nullValue()));
+  }
+
+  @Test
+  public void shouldReturnNullForEmptyStringInput() {
+    final String result = udf.chr("");
+    assertThat(result, is(nullValue()));
+  }
+
+  @Test
+  public void shouldReturnNullForNegativeDecimalCode() {
+    final String result = udf.chr(-1);
+    assertThat(result, is(nullValue()));
+  }
+
+  @Test
+  public void shouldReturnSingleCharForMaxBMPDecimal() {
+    final String result = udf.chr(65535);
+    assertThat(result.codePointAt(0), is(65535));
+    assertThat(result.toCharArray().length, is(1));
+  }
+
+  @Test
+  public void shouldReturnTwoCharsForNonBMPDecimal() {
+    final String result = udf.chr(65536);
+    assertThat(result.codePointAt(0), is(65536));
+    assertThat(result.toCharArray().length, is(2));
+  }
+
+  @Test
+  public void shouldReturnTwoCharsForMaxUnicodeDecimal() {
+    final String result = udf.chr(1_114_111);
+    assertThat(result.codePointAt(0), is(1_114_111));
+    assertThat(result.toCharArray().length, is(2));
+  }
+
+  @Test
+  public void shouldReturnNullForOutOfRangeDecimal() {
+    final String result = udf.chr(1_114_112);
+    assertThat(result, is(nullValue()));
+  }
+
+  @Test
+  public void shouldReturnNullForTooShortUTF16String() {
+    final String result = udf.chr("\\u065");
+    assertThat(result, is(nullValue()));
+  }
+
+  @Test
+  public void shouldReturnTwoCharsForNonBMPString() {
+    final String result = udf.chr("\\ud800\\udc01");
+    assertThat(result.codePointAt(0), is(65537));
+    assertThat(result.toCharArray().length, is(2));
+  }
+
+}
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/plan.json
new file mode 100644
index 000000000000..33111d1c51ed
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/plan.json
@@ -0,0 +1,126 @@
+{
+  "plan" : [ {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM INPUT (ID STRING KEY, UTFCODE INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='AVRO');",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "INPUT",
+      "schema" : "`ID` STRING KEY, `UTFCODE` INTEGER",
+      "topicName" : "test_topic",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "AVRO"
+        }
+      }
+    }
+  }, {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM OUTPUT AS SELECT\n  INPUT.ID ID,\n  CHR(INPUT.UTFCODE) RESULT\nFROM INPUT INPUT\nEMIT CHANGES",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "OUTPUT",
+      "schema" : "`ID` STRING KEY, `RESULT` STRING",
+      "topicName" : "OUTPUT",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "AVRO"
+        }
+      }
+    },
+    "queryPlan" : {
+      "sources" : [ "INPUT" ],
+      "sink" : "OUTPUT",
+      "physicalPlan" : {
+        "@type" : "streamSinkV1",
+        "properties" : {
+          "queryContext" : "OUTPUT"
+        },
+        "source" : {
+          "@type" : "streamSelectV1",
+          "properties" : {
+            "queryContext" : "Project"
+          },
+          "source" : {
+            "@type" : "streamSourceV1",
+            "properties" : {
+              "queryContext" : "KsqlTopic/Source"
+            },
+            "topicName" : "test_topic",
+            "formats" : {
+              "keyFormat" : {
+                "format" : "KAFKA"
+              },
+              "valueFormat" : {
+                "format" : "AVRO"
+              }
+            },
+            "sourceSchema" : "`ID` STRING KEY, `UTFCODE` INTEGER"
+          },
+          "keyColumnNames" : [ "ID" ],
+          "selectExpressions" : [ "CHR(UTFCODE) AS RESULT" ]
+        },
+        "formats" : {
+          "keyFormat" : {
+            "format" : "KAFKA"
+          },
+          "valueFormat" : {
+            "format" : "AVRO"
+          }
+        },
+        "topicName" : "OUTPUT"
+      },
+      "queryId" : "CSAS_OUTPUT_0"
+    }
+  } ],
+  "configs" : {
+    "ksql.extension.dir" : "ext",
+    "ksql.streams.cache.max.bytes.buffering" : "0",
+    "ksql.security.extension.class" : null,
+    "ksql.transient.prefix" : "transient_",
+    "ksql.persistence.wrap.single.values" : "true",
+    "ksql.authorization.cache.expiry.time.secs" : "30",
+    "ksql.schema.registry.url" : "",
+    "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler",
+    "ksql.output.topic.name.prefix" : "",
+    "ksql.streams.auto.offset.reset" : "earliest",
+    "ksql.query.pull.enable.standby.reads" : "false",
+    "ksql.connect.url" : "http://localhost:8083",
+    "ksql.service.id" : "some.ksql.service.id",
+    "ksql.internal.topic.min.insync.replicas" : "1",
+    "ksql.streams.shutdown.timeout.ms" : "300000",
+    "ksql.internal.topic.replicas" : "1",
+    "ksql.insert.into.values.enabled" : "true",
+    "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807",
+    "ksql.query.pull.max.qps" : "2147483647",
+    "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler",
+    "ksql.access.validator.enable" : "auto",
+    "ksql.streams.bootstrap.servers" : "localhost:0",
+    "ksql.streams.commit.interval.ms" : "2000",
+    "ksql.metric.reporters" : "",
+    "ksql.query.pull.metrics.enabled" : "false",
+    "ksql.streams.auto.commit.interval.ms" : "0",
+    "ksql.metrics.extension" : null,
+    "ksql.streams.topology.optimization" : "all",
+    "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.streams.num.stream.threads" : "4",
+    "ksql.timestamp.throw.on.invalid" : "false",
+    "ksql.authorization.cache.max.entries" : "10000",
+    "ksql.metrics.tags.custom" : "",
+    "ksql.pull.queries.enable" : "true",
+    "ksql.udfs.enabled" : "true",
+    "ksql.udf.enable.security.manager" : "true",
+    "ksql.connect.worker.config" : "",
+    "ksql.sink.window.change.log.additional.retention" : "1000000",
+    "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.udf.collect.metrics" : "false",
+    "ksql.persistent.prefix" : "query_",
+    "ksql.query.persistent.active.limit" : "2147483647",
+    "ksql.error.classifier.regex" : ""
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/spec.json
new file mode 100644
index 000000000000..e55dbca72163
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/spec.json
@@ -0,0 +1,123 @@
+{
+  "version" : "6.0.0",
+  "timestamp" : 1591421496995,
+  "path" : "query-validation-tests\\chr.json",
+  "schemas" : {
+    "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT<UTFCODE INT> NOT NULL",
+    "CSAS_OUTPUT_0.OUTPUT" : "STRUCT<RESULT VARCHAR> NOT NULL"
+  },
+  "testCase" : {
+    "name" : "codepoint from decimal code - AVRO",
+    "inputs" : [ {
+      "topic" : "test_topic",
+      "key" : "r1",
+      "value" : {
+        "utfcode" : 75
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r2",
+      "value" : {
+        "utfcode" : 22909
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r3",
+      "value" : {
+        "utfcode" : 99000
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r4",
+      "value" : {
+        "utfcode" : -1
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r5",
+      "value" : {
+        "utfcode" : null
+      }
+    } ],
+    "outputs" : [ {
+      "topic" : "OUTPUT",
+      "key" : "r1",
+      "value" : {
+        "RESULT" : "K"
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r2",
+      "value" : {
+        "RESULT" : "好"
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r3",
+      "value" : {
+        "RESULT" : "𘊸"
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r4",
+      "value" : {
+        "RESULT" : null
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r5",
+      "value" : {
+        "RESULT" : null
+      }
+    } ],
+    "topics" : [ {
+      "name" : "OUTPUT",
+      "replicas" : 1,
+      "numPartitions" : 4
+    }, {
+      "name" : "test_topic",
+      "schema" : {
+        "type" : "record",
+        "name" : "KsqlDataSourceSchema",
+        "namespace" : "io.confluent.ksql.avro_schemas",
+        "fields" : [ {
+          "name" : "UTFCODE",
+          "type" : [ "null", "int" ],
+          "default" : null
+        } ],
+        "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema"
+      },
+      "format" : "AVRO",
+      "replicas" : 1,
+      "numPartitions" : 4
+    } ],
+    "statements" : [ "CREATE STREAM INPUT (id STRING KEY, utfcode INTEGER) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT id, chr(utfcode) AS result FROM INPUT;" ],
+    "post" : {
+      "topics" : {
+        "topics" : [ {
+          "name" : "OUTPUT",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "AVRO"
+          },
+          "partitions" : 4
+        }, {
+          "name" : "test_topic",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "AVRO"
+          },
+          "partitions" : 4
+        } ]
+      }
+    }
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/topology
new file mode 100644
index 000000000000..441a8f282644
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/topology
@@ -0,0 +1,13 @@
+Topologies:
+   Sub-topology: 0
+    Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic])
+      --> KSTREAM-TRANSFORMVALUES-0000000001
+    Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: [])
+      --> Project
+      <-- KSTREAM-SOURCE-0000000000
+    Processor: Project (stores: [])
+      --> KSTREAM-SINK-0000000003
+      <-- KSTREAM-TRANSFORMVALUES-0000000001
+    Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT)
+      <-- Project
+
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/plan.json
new file mode 100644
index 000000000000..a34c2181cb9f
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/plan.json
@@ -0,0 +1,126 @@
+{
+  "plan" : [ {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM INPUT (ID STRING KEY, UTFCODE INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "INPUT",
+      "schema" : "`ID` STRING KEY, `UTFCODE` INTEGER",
+      "topicName" : "test_topic",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "JSON"
+        }
+      }
+    }
+  }, {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM OUTPUT AS SELECT\n  INPUT.ID ID,\n  CHR(INPUT.UTFCODE) RESULT\nFROM INPUT INPUT\nEMIT CHANGES",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "OUTPUT",
+      "schema" : "`ID` STRING KEY, `RESULT` STRING",
+      "topicName" : "OUTPUT",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "JSON"
+        }
+      }
+    },
+    "queryPlan" : {
+      "sources" : [ "INPUT" ],
+      "sink" : "OUTPUT",
+      "physicalPlan" : {
+        "@type" : "streamSinkV1",
+        "properties" : {
+          "queryContext" : "OUTPUT"
+        },
+        "source" : {
+          "@type" : "streamSelectV1",
+          "properties" : {
+            "queryContext" : "Project"
+          },
+          "source" : {
+            "@type" : "streamSourceV1",
+            "properties" : {
+              "queryContext" : "KsqlTopic/Source"
+            },
+            "topicName" : "test_topic",
+            "formats" : {
+              "keyFormat" : {
+                "format" : "KAFKA"
+              },
+              "valueFormat" : {
+                "format" : "JSON"
+              }
+            },
+            "sourceSchema" : "`ID` STRING KEY, `UTFCODE` INTEGER"
+          },
+          "keyColumnNames" : [ "ID" ],
+          "selectExpressions" : [ "CHR(UTFCODE) AS RESULT" ]
+        },
+        "formats" : {
+          "keyFormat" : {
+            "format" : "KAFKA"
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          }
+        },
+        "topicName" : "OUTPUT"
+      },
+      "queryId" : "CSAS_OUTPUT_0"
+    }
+  } ],
+  "configs" : {
+    "ksql.extension.dir" : "ext",
+    "ksql.streams.cache.max.bytes.buffering" : "0",
+    "ksql.security.extension.class" : null,
+    "ksql.transient.prefix" : "transient_",
+    "ksql.persistence.wrap.single.values" : "true",
+    "ksql.authorization.cache.expiry.time.secs" : "30",
+    "ksql.schema.registry.url" : "",
+    "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler",
+    "ksql.output.topic.name.prefix" : "",
+    "ksql.streams.auto.offset.reset" : "earliest",
+    "ksql.query.pull.enable.standby.reads" : "false",
+    "ksql.connect.url" : "http://localhost:8083",
+    "ksql.service.id" : "some.ksql.service.id",
+    "ksql.internal.topic.min.insync.replicas" : "1",
+    "ksql.streams.shutdown.timeout.ms" : "300000",
+    "ksql.internal.topic.replicas" : "1",
+    "ksql.insert.into.values.enabled" : "true",
+    "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807",
+    "ksql.query.pull.max.qps" : "2147483647",
+    "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler",
+    "ksql.access.validator.enable" : "auto",
+    "ksql.streams.bootstrap.servers" : "localhost:0",
+    "ksql.streams.commit.interval.ms" : "2000",
+    "ksql.metric.reporters" : "",
+    "ksql.query.pull.metrics.enabled" : "false",
+    "ksql.streams.auto.commit.interval.ms" : "0",
+    "ksql.metrics.extension" : null,
+    "ksql.streams.topology.optimization" : "all",
+    "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.streams.num.stream.threads" : "4",
+    "ksql.timestamp.throw.on.invalid" : "false",
+    "ksql.authorization.cache.max.entries" : "10000",
+    "ksql.metrics.tags.custom" : "",
+    "ksql.pull.queries.enable" : "true",
+    "ksql.udfs.enabled" : "true",
+    "ksql.udf.enable.security.manager" : "true",
+    "ksql.connect.worker.config" : "",
+    "ksql.sink.window.change.log.additional.retention" : "1000000",
+    "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.udf.collect.metrics" : "false",
+    "ksql.persistent.prefix" : "query_",
+    "ksql.query.persistent.active.limit" : "2147483647",
+    "ksql.error.classifier.regex" : ""
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/spec.json
new file mode 100644
index 000000000000..7cc21dbd3af8
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/spec.json
@@ -0,0 +1,111 @@
+{
+  "version" : "6.0.0",
+  "timestamp" : 1591421497086,
+  "path" : "query-validation-tests\\chr.json",
+  "schemas" : {
+    "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT<UTFCODE INT> NOT NULL",
+    "CSAS_OUTPUT_0.OUTPUT" : "STRUCT<RESULT VARCHAR> NOT NULL"
+  },
+  "testCase" : {
+    "name" : "codepoint from decimal code - JSON",
+    "inputs" : [ {
+      "topic" : "test_topic",
+      "key" : "r1",
+      "value" : {
+        "utfcode" : 75
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r2",
+      "value" : {
+        "utfcode" : 22909
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r3",
+      "value" : {
+        "utfcode" : 99000
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r4",
+      "value" : {
+        "utfcode" : -1
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r5",
+      "value" : {
+        "utfcode" : null
+      }
+    } ],
+    "outputs" : [ {
+      "topic" : "OUTPUT",
+      "key" : "r1",
+      "value" : {
+        "RESULT" : "K"
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r2",
+      "value" : {
+        "RESULT" : "好"
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r3",
+      "value" : {
+        "RESULT" : "𘊸"
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r4",
+      "value" : {
+        "RESULT" : null
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r5",
+      "value" : {
+        "RESULT" : null
+      }
+    } ],
+    "topics" : [ {
+      "name" : "OUTPUT",
+      "replicas" : 1,
+      "numPartitions" : 4
+    }, {
+      "name" : "test_topic",
+      "replicas" : 1,
+      "numPartitions" : 4
+    } ],
+    "statements" : [ "CREATE STREAM INPUT (id STRING KEY, utfcode INTEGER) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT id, chr(utfcode) AS result FROM INPUT;" ],
+    "post" : {
+      "topics" : {
+        "topics" : [ {
+          "name" : "OUTPUT",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          },
+          "partitions" : 4
+        }, {
+          "name" : "test_topic",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          },
+          "partitions" : 4
+        } ]
+      }
+    }
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/topology
new file mode 100644
index 000000000000..441a8f282644
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/topology
@@ -0,0 +1,13 @@
+Topologies:
+   Sub-topology: 0
+    Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic])
+      --> KSTREAM-TRANSFORMVALUES-0000000001
+    Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: [])
+      --> Project
+      <-- KSTREAM-SOURCE-0000000000
+    Processor: Project (stores: [])
+      --> KSTREAM-SINK-0000000003
+      <-- KSTREAM-TRANSFORMVALUES-0000000001
+    Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT)
+      <-- Project
+
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/plan.json
new file mode 100644
index 000000000000..ac2fb854c475
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/plan.json
@@ -0,0 +1,126 @@
+{
+  "plan" : [ {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM INPUT (ID STRING KEY, UTFCODE STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='AVRO');",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "INPUT",
+      "schema" : "`ID` STRING KEY, `UTFCODE` STRING",
+      "topicName" : "test_topic",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "AVRO"
+        }
+      }
+    }
+  }, {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM OUTPUT AS SELECT\n  INPUT.ID ID,\n  CHR(INPUT.UTFCODE) RESULT\nFROM INPUT INPUT\nEMIT CHANGES",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "OUTPUT",
+      "schema" : "`ID` STRING KEY, `RESULT` STRING",
+      "topicName" : "OUTPUT",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "AVRO"
+        }
+      }
+    },
+    "queryPlan" : {
+      "sources" : [ "INPUT" ],
+      "sink" : "OUTPUT",
+      "physicalPlan" : {
+        "@type" : "streamSinkV1",
+        "properties" : {
+          "queryContext" : "OUTPUT"
+        },
+        "source" : {
+          "@type" : "streamSelectV1",
+          "properties" : {
+            "queryContext" : "Project"
+          },
+          "source" : {
+            "@type" : "streamSourceV1",
+            "properties" : {
+              "queryContext" : "KsqlTopic/Source"
+            },
+            "topicName" : "test_topic",
+            "formats" : {
+              "keyFormat" : {
+                "format" : "KAFKA"
+              },
+              "valueFormat" : {
+                "format" : "AVRO"
+              }
+            },
+            "sourceSchema" : "`ID` STRING KEY, `UTFCODE` STRING"
+          },
+          "keyColumnNames" : [ "ID" ],
+          "selectExpressions" : [ "CHR(UTFCODE) AS RESULT" ]
+        },
+        "formats" : {
+          "keyFormat" : {
+            "format" : "KAFKA"
+          },
+          "valueFormat" : {
+            "format" : "AVRO"
+          }
+        },
+        "topicName" : "OUTPUT"
+      },
+      "queryId" : "CSAS_OUTPUT_0"
+    }
+  } ],
+  "configs" : {
+    "ksql.extension.dir" : "ext",
+    "ksql.streams.cache.max.bytes.buffering" : "0",
+    "ksql.security.extension.class" : null,
+    "ksql.transient.prefix" : "transient_",
+    "ksql.persistence.wrap.single.values" : "true",
+    "ksql.authorization.cache.expiry.time.secs" : "30",
+    "ksql.schema.registry.url" : "",
+    "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler",
+    "ksql.output.topic.name.prefix" : "",
+    "ksql.streams.auto.offset.reset" : "earliest",
+    "ksql.query.pull.enable.standby.reads" : "false",
+    "ksql.connect.url" : "http://localhost:8083",
+    "ksql.service.id" : "some.ksql.service.id",
+    "ksql.internal.topic.min.insync.replicas" : "1",
+    "ksql.streams.shutdown.timeout.ms" : "300000",
+    "ksql.internal.topic.replicas" : "1",
+    "ksql.insert.into.values.enabled" : "true",
+    "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807",
+    "ksql.query.pull.max.qps" : "2147483647",
+    "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler",
+    "ksql.access.validator.enable" : "auto",
+    "ksql.streams.bootstrap.servers" : "localhost:0",
+    "ksql.streams.commit.interval.ms" : "2000",
+    "ksql.metric.reporters" : "",
+    "ksql.query.pull.metrics.enabled" : "false",
+    "ksql.streams.auto.commit.interval.ms" : "0",
+    "ksql.metrics.extension" : null,
+    "ksql.streams.topology.optimization" : "all",
+    "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.streams.num.stream.threads" : "4",
+    "ksql.timestamp.throw.on.invalid" : "false",
+    "ksql.authorization.cache.max.entries" : "10000",
+    "ksql.metrics.tags.custom" : "",
+    "ksql.pull.queries.enable" : "true",
+    "ksql.udfs.enabled" : "true",
+    "ksql.udf.enable.security.manager" : "true",
+    "ksql.connect.worker.config" : "",
+    "ksql.sink.window.change.log.additional.retention" : "1000000",
+    "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.udf.collect.metrics" : "false",
+    "ksql.persistent.prefix" : "query_",
+    "ksql.query.persistent.active.limit" : "2147483647",
+    "ksql.error.classifier.regex" : ""
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/spec.json
new file mode 100644
index 000000000000..27ed7a1ea55a
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/spec.json
@@ -0,0 +1,159 @@
+{
+  "version" : "6.0.0",
+  "timestamp" : 1591421497142,
+  "path" : "query-validation-tests\\chr.json",
+  "schemas" : {
+    "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT<UTFCODE VARCHAR> NOT NULL",
+    "CSAS_OUTPUT_0.OUTPUT" : "STRUCT<RESULT VARCHAR> NOT NULL"
+  },
+  "testCase" : {
+    "name" : "codepoint from text code - AVRO",
+    "inputs" : [ {
+      "topic" : "test_topic",
+      "key" : "r1",
+      "value" : {
+        "utfcode" : "\\u004b"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r2",
+      "value" : {
+        "utfcode" : "\\u597d"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r3",
+      "value" : {
+        "utfcode" : "\\ud820\\udeb8"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r4",
+      "value" : {
+        "utfcode" : "75"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r5",
+      "value" : {
+        "utfcode" : "004b"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r6",
+      "value" : {
+        "utfcode" : "bogus"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r7",
+      "value" : {
+        "utfcode" : ""
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r8",
+      "value" : {
+        "utfcode" : null
+      }
+    } ],
+    "outputs" : [ {
+      "topic" : "OUTPUT",
+      "key" : "r1",
+      "value" : {
+        "RESULT" : "K"
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r2",
+      "value" : {
+        "RESULT" : "好"
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r3",
+      "value" : {
+        "RESULT" : "𘊸"
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r4",
+      "value" : {
+        "RESULT" : null
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r5",
+      "value" : {
+        "RESULT" : null
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r6",
+      "value" : {
+        "RESULT" : null
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r7",
+      "value" : {
+        "RESULT" : null
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r8",
+      "value" : {
+        "RESULT" : null
+      }
+    } ],
+    "topics" : [ {
+      "name" : "OUTPUT",
+      "replicas" : 1,
+      "numPartitions" : 4
+    }, {
+      "name" : "test_topic",
+      "schema" : {
+        "type" : "record",
+        "name" : "KsqlDataSourceSchema",
+        "namespace" : "io.confluent.ksql.avro_schemas",
+        "fields" : [ {
+          "name" : "UTFCODE",
+          "type" : [ "null", "string" ],
+          "default" : null
+        } ],
+        "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema"
+      },
+      "format" : "AVRO",
+      "replicas" : 1,
+      "numPartitions" : 4
+    } ],
+    "statements" : [ "CREATE STREAM INPUT (id STRING KEY, utfcode STRING) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT id, chr(utfcode) AS result FROM INPUT;" ],
+    "post" : {
+      "topics" : {
+        "topics" : [ {
+          "name" : "OUTPUT",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "AVRO"
+          },
+          "partitions" : 4
+        }, {
+          "name" : "test_topic",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "AVRO"
+          },
+          "partitions" : 4
+        } ]
+      }
+    }
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/topology
new file mode 100644
index 000000000000..441a8f282644
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/topology
@@ -0,0 +1,13 @@
+Topologies:
+   Sub-topology: 0
+    Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic])
+      --> KSTREAM-TRANSFORMVALUES-0000000001
+    Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: [])
+      --> Project
+      <-- KSTREAM-SOURCE-0000000000
+    Processor: Project (stores: [])
+      --> KSTREAM-SINK-0000000003
+      <-- KSTREAM-TRANSFORMVALUES-0000000001
+    Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT)
+      <-- Project
+
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/plan.json
new file mode 100644
index 000000000000..0218c9ec6ef8
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/plan.json
@@ -0,0 +1,126 @@
+{
+  "plan" : [ {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM INPUT (ID STRING KEY, UTFCODE STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "INPUT",
+      "schema" : "`ID` STRING KEY, `UTFCODE` STRING",
+      "topicName" : "test_topic",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "JSON"
+        }
+      }
+    }
+  }, {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM OUTPUT AS SELECT\n  INPUT.ID ID,\n  CHR(INPUT.UTFCODE) RESULT\nFROM INPUT INPUT\nEMIT CHANGES",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "OUTPUT",
+      "schema" : "`ID` STRING KEY, `RESULT` STRING",
+      "topicName" : "OUTPUT",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "JSON"
+        }
+      }
+    },
+    "queryPlan" : {
+      "sources" : [ "INPUT" ],
+      "sink" : "OUTPUT",
+      "physicalPlan" : {
+        "@type" : "streamSinkV1",
+        "properties" : {
+          "queryContext" : "OUTPUT"
+        },
+        "source" : {
+          "@type" : "streamSelectV1",
+          "properties" : {
+            "queryContext" : "Project"
+          },
+          "source" : {
+            "@type" : "streamSourceV1",
+            "properties" : {
+              "queryContext" : "KsqlTopic/Source"
+            },
+            "topicName" : "test_topic",
+            "formats" : {
+              "keyFormat" : {
+                "format" : "KAFKA"
+              },
+              "valueFormat" : {
+                "format" : "JSON"
+              }
+            },
+            "sourceSchema" : "`ID` STRING KEY, `UTFCODE` STRING"
+          },
+          "keyColumnNames" : [ "ID" ],
+          "selectExpressions" : [ "CHR(UTFCODE) AS RESULT" ]
+        },
+        "formats" : {
+          "keyFormat" : {
+            "format" : "KAFKA"
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          }
+        },
+        "topicName" : "OUTPUT"
+      },
+      "queryId" : "CSAS_OUTPUT_0"
+    }
+  } ],
+  "configs" : {
+    "ksql.extension.dir" : "ext",
+    "ksql.streams.cache.max.bytes.buffering" : "0",
+    "ksql.security.extension.class" : null,
+    "ksql.transient.prefix" : "transient_",
+    "ksql.persistence.wrap.single.values" : "true",
+    "ksql.authorization.cache.expiry.time.secs" : "30",
+    "ksql.schema.registry.url" : "",
+    "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler",
+    "ksql.output.topic.name.prefix" : "",
+    "ksql.streams.auto.offset.reset" : "earliest",
+    "ksql.query.pull.enable.standby.reads" : "false",
+    "ksql.connect.url" : "http://localhost:8083",
+    "ksql.service.id" : "some.ksql.service.id",
+    "ksql.internal.topic.min.insync.replicas" : "1",
+    "ksql.streams.shutdown.timeout.ms" : "300000",
+    "ksql.internal.topic.replicas" : "1",
+    "ksql.insert.into.values.enabled" : "true",
+    "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807",
+    "ksql.query.pull.max.qps" : "2147483647",
+    "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler",
+    "ksql.access.validator.enable" : "auto",
+    "ksql.streams.bootstrap.servers" : "localhost:0",
+    "ksql.streams.commit.interval.ms" : "2000",
+    "ksql.metric.reporters" : "",
+    "ksql.query.pull.metrics.enabled" : "false",
+    "ksql.streams.auto.commit.interval.ms" : "0",
+    "ksql.metrics.extension" : null,
+    "ksql.streams.topology.optimization" : "all",
+    "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.streams.num.stream.threads" : "4",
+    "ksql.timestamp.throw.on.invalid" : "false",
+    "ksql.authorization.cache.max.entries" : "10000",
+    "ksql.metrics.tags.custom" : "",
+    "ksql.pull.queries.enable" : "true",
+    "ksql.udfs.enabled" : "true",
+    "ksql.udf.enable.security.manager" : "true",
+    "ksql.connect.worker.config" : "",
+    "ksql.sink.window.change.log.additional.retention" : "1000000",
+    "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.udf.collect.metrics" : "false",
+    "ksql.persistent.prefix" : "query_",
+    "ksql.query.persistent.active.limit" : "2147483647",
+    "ksql.error.classifier.regex" : ""
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/spec.json
new file mode 100644
index 000000000000..64d573b7c152
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/spec.json
@@ -0,0 +1,147 @@
+{
+  "version" : "6.0.0",
+  "timestamp" : 1591421497199,
+  "path" : "query-validation-tests\\chr.json",
+  "schemas" : {
+    "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT<UTFCODE VARCHAR> NOT NULL",
+    "CSAS_OUTPUT_0.OUTPUT" : "STRUCT<RESULT VARCHAR> NOT NULL"
+  },
+  "testCase" : {
+    "name" : "codepoint from text code - JSON",
+    "inputs" : [ {
+      "topic" : "test_topic",
+      "key" : "r1",
+      "value" : {
+        "utfcode" : "\\u004b"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r2",
+      "value" : {
+        "utfcode" : "\\u597d"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r3",
+      "value" : {
+        "utfcode" : "\\ud820\\udeb8"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r4",
+      "value" : {
+        "utfcode" : "75"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r5",
+      "value" : {
+        "utfcode" : "004b"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r6",
+      "value" : {
+        "utfcode" : "bogus"
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r7",
+      "value" : {
+        "utfcode" : ""
+      }
+    }, {
+      "topic" : "test_topic",
+      "key" : "r8",
+      "value" : {
+        "utfcode" : null
+      }
+    } ],
+    "outputs" : [ {
+      "topic" : "OUTPUT",
+      "key" : "r1",
+      "value" : {
+        "RESULT" : "K"
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r2",
+      "value" : {
+        "RESULT" : "好"
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r3",
+      "value" : {
+        "RESULT" : "𘊸"
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r4",
+      "value" : {
+        "RESULT" : null
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r5",
+      "value" : {
+        "RESULT" : null
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r6",
+      "value" : {
+        "RESULT" : null
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r7",
+      "value" : {
+        "RESULT" : null
+      }
+    }, {
+      "topic" : "OUTPUT",
+      "key" : "r8",
+      "value" : {
+        "RESULT" : null
+      }
+    } ],
+    "topics" : [ {
+      "name" : "OUTPUT",
+      "replicas" : 1,
+      "numPartitions" : 4
+    }, {
+      "name" : "test_topic",
+      "replicas" : 1,
+      "numPartitions" : 4
+    } ],
+    "statements" : [ "CREATE STREAM INPUT (id STRING KEY, utfcode STRING) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT id, chr(utfcode) AS result FROM INPUT;" ],
+    "post" : {
+      "topics" : {
+        "topics" : [ {
+          "name" : "OUTPUT",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          },
+          "partitions" : 4
+        }, {
+          "name" : "test_topic",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          },
+          "partitions" : 4
+        } ]
+      }
+    }
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/topology
new file mode 100644
index 000000000000..441a8f282644
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/topology
@@ -0,0 +1,13 @@
+Topologies:
+   Sub-topology: 0
+    Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic])
+      --> KSTREAM-TRANSFORMVALUES-0000000001
+    Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: [])
+      --> Project
+      <-- KSTREAM-SOURCE-0000000000
+    Processor: Project (stores: [])
+      --> KSTREAM-SINK-0000000003
+      <-- KSTREAM-TRANSFORMVALUES-0000000001
+    Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT)
+      <-- Project
+
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/plan.json
new file mode 100644
index 000000000000..7ed500c68343
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/plan.json
@@ -0,0 +1,126 @@
+{
+  "plan" : [ {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM INPUT (ID STRING KEY, UTFCODE1 STRING, UTFCODE2 STRING, UTFCODE3 INTEGER, UTFCODE4 INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='AVRO');",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "INPUT",
+      "schema" : "`ID` STRING KEY, `UTFCODE1` STRING, `UTFCODE2` STRING, `UTFCODE3` INTEGER, `UTFCODE4` INTEGER",
+      "topicName" : "test_topic",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "AVRO"
+        }
+      }
+    }
+  }, {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM OUTPUT AS SELECT\n  INPUT.ID ID,\n  CONCAT(CHR(INPUT.UTFCODE1), CHR(INPUT.UTFCODE2), CHR(INPUT.UTFCODE3), CHR(INPUT.UTFCODE4)) RESULT\nFROM INPUT INPUT\nEMIT CHANGES",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "OUTPUT",
+      "schema" : "`ID` STRING KEY, `RESULT` STRING",
+      "topicName" : "OUTPUT",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "AVRO"
+        }
+      }
+    },
+    "queryPlan" : {
+      "sources" : [ "INPUT" ],
+      "sink" : "OUTPUT",
+      "physicalPlan" : {
+        "@type" : "streamSinkV1",
+        "properties" : {
+          "queryContext" : "OUTPUT"
+        },
+        "source" : {
+          "@type" : "streamSelectV1",
+          "properties" : {
+            "queryContext" : "Project"
+          },
+          "source" : {
+            "@type" : "streamSourceV1",
+            "properties" : {
+              "queryContext" : "KsqlTopic/Source"
+            },
+            "topicName" : "test_topic",
+            "formats" : {
+              "keyFormat" : {
+                "format" : "KAFKA"
+              },
+              "valueFormat" : {
+                "format" : "AVRO"
+              }
+            },
+            "sourceSchema" : "`ID` STRING KEY, `UTFCODE1` STRING, `UTFCODE2` STRING, `UTFCODE3` INTEGER, `UTFCODE4` INTEGER"
+          },
+          "keyColumnNames" : [ "ID" ],
+          "selectExpressions" : [ "CONCAT(CHR(UTFCODE1), CHR(UTFCODE2), CHR(UTFCODE3), CHR(UTFCODE4)) AS RESULT" ]
+        },
+        "formats" : {
+          "keyFormat" : {
+            "format" : "KAFKA"
+          },
+          "valueFormat" : {
+            "format" : "AVRO"
+          }
+        },
+        "topicName" : "OUTPUT"
+      },
+      "queryId" : "CSAS_OUTPUT_0"
+    }
+  } ],
+  "configs" : {
+    "ksql.extension.dir" : "ext",
+    "ksql.streams.cache.max.bytes.buffering" : "0",
+    "ksql.security.extension.class" : null,
+    "ksql.transient.prefix" : "transient_",
+    "ksql.persistence.wrap.single.values" : "true",
+    "ksql.authorization.cache.expiry.time.secs" : "30",
+    "ksql.schema.registry.url" : "",
+    "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler",
+    "ksql.output.topic.name.prefix" : "",
+    "ksql.streams.auto.offset.reset" : "earliest",
+    "ksql.query.pull.enable.standby.reads" : "false",
+    "ksql.connect.url" : "http://localhost:8083",
+    "ksql.service.id" : "some.ksql.service.id",
+    "ksql.internal.topic.min.insync.replicas" : "1",
+    "ksql.streams.shutdown.timeout.ms" : "300000",
+    "ksql.internal.topic.replicas" : "1",
+    "ksql.insert.into.values.enabled" : "true",
+    "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807",
+    "ksql.query.pull.max.qps" : "2147483647",
+    "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler",
+    "ksql.access.validator.enable" : "auto",
+    "ksql.streams.bootstrap.servers" : "localhost:0",
+    "ksql.streams.commit.interval.ms" : "2000",
+    "ksql.metric.reporters" : "",
+    "ksql.query.pull.metrics.enabled" : "false",
+    "ksql.streams.auto.commit.interval.ms" : "0",
+    "ksql.metrics.extension" : null,
+    "ksql.streams.topology.optimization" : "all",
+    "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.streams.num.stream.threads" : "4",
+    "ksql.timestamp.throw.on.invalid" : "false",
+    "ksql.authorization.cache.max.entries" : "10000",
+    "ksql.metrics.tags.custom" : "",
+    "ksql.pull.queries.enable" : "true",
+    "ksql.udfs.enabled" : "true",
+    "ksql.udf.enable.security.manager" : "true",
+    "ksql.connect.worker.config" : "",
+    "ksql.sink.window.change.log.additional.retention" : "1000000",
+    "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.udf.collect.metrics" : "false",
+    "ksql.persistent.prefix" : "query_",
+    "ksql.query.persistent.active.limit" : "2147483647",
+    "ksql.error.classifier.regex" : ""
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/spec.json
new file mode 100644
index 000000000000..34a34720d582
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/spec.json
@@ -0,0 +1,90 @@
+{
+  "version" : "6.0.0",
+  "timestamp" : 1591421497279,
+  "path" : "query-validation-tests\\chr.json",
+  "schemas" : {
+    "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT<UTFCODE1 VARCHAR, UTFCODE2 VARCHAR, UTFCODE3 INT, UTFCODE4 INT> NOT NULL",
+    "CSAS_OUTPUT_0.OUTPUT" : "STRUCT<RESULT VARCHAR> NOT NULL"
+  },
+  "testCase" : {
+    "name" : "multiple invocations - AVRO",
+    "inputs" : [ {
+      "topic" : "test_topic",
+      "key" : "r1",
+      "value" : {
+        "utfcode1" : "\\u004b",
+        "utfcode2" : "\\u0053",
+        "utfcode3" : 81,
+        "utfcode4" : 76
+      }
+    } ],
+    "outputs" : [ {
+      "topic" : "OUTPUT",
+      "key" : "r1",
+      "value" : {
+        "RESULT" : "KSQL"
+      }
+    } ],
+    "topics" : [ {
+      "name" : "OUTPUT",
+      "replicas" : 1,
+      "numPartitions" : 4
+    }, {
+      "name" : "test_topic",
+      "schema" : {
+        "type" : "record",
+        "name" : "KsqlDataSourceSchema",
+        "namespace" : "io.confluent.ksql.avro_schemas",
+        "fields" : [ {
+          "name" : "UTFCODE1",
+          "type" : [ "null", "string" ],
+          "default" : null
+        }, {
+          "name" : "UTFCODE2",
+          "type" : [ "null", "string" ],
+          "default" : null
+        }, {
+          "name" : "UTFCODE3",
+          "type" : [ "null", "int" ],
+          "default" : null
+        }, {
+          "name" : "UTFCODE4",
+          "type" : [ "null", "int" ],
+          "default" : null
+        } ],
+        "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema"
+      },
+      "format" : "AVRO",
+      "replicas" : 1,
+      "numPartitions" : 4
+    } ],
+    "statements" : [ "CREATE STREAM INPUT (id STRING KEY, utfcode1 STRING, utfcode2 STRING, utfcode3 INTEGER, utfcode4 INTEGER) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT id, concat(chr(utfcode1), chr(utfcode2), chr(utfcode3), chr(utfcode4)) AS result FROM INPUT;" ],
+    "post" : {
+      "topics" : {
+        "topics" : [ {
+          "name" : "OUTPUT",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "AVRO"
+          },
+          "partitions" : 4
+        }, {
+          "name" : "test_topic",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "AVRO"
+          },
+          "partitions" : 4
+        } ]
+      }
+    }
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/topology
new file mode 100644
index 000000000000..441a8f282644
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/topology
@@ -0,0 +1,13 @@
+Topologies:
+   Sub-topology: 0
+    Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic])
+      --> KSTREAM-TRANSFORMVALUES-0000000001
+    Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: [])
+      --> Project
+      <-- KSTREAM-SOURCE-0000000000
+    Processor: Project (stores: [])
+      --> KSTREAM-SINK-0000000003
+      <-- KSTREAM-TRANSFORMVALUES-0000000001
+    Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT)
+      <-- Project
+
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/plan.json
new file mode 100644
index 000000000000..2c78f35e3004
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/plan.json
@@ -0,0 +1,126 @@
+{
+  "plan" : [ {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM INPUT (ID STRING KEY, UTFCODE1 STRING, UTFCODE2 STRING, UTFCODE3 INTEGER, UTFCODE4 INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "INPUT",
+      "schema" : "`ID` STRING KEY, `UTFCODE1` STRING, `UTFCODE2` STRING, `UTFCODE3` INTEGER, `UTFCODE4` INTEGER",
+      "topicName" : "test_topic",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "JSON"
+        }
+      }
+    }
+  }, {
+    "@type" : "ksqlPlanV1",
+    "statementText" : "CREATE STREAM OUTPUT AS SELECT\n  INPUT.ID ID,\n  CONCAT(CHR(INPUT.UTFCODE1), CHR(INPUT.UTFCODE2), CHR(INPUT.UTFCODE3), CHR(INPUT.UTFCODE4)) RESULT\nFROM INPUT INPUT\nEMIT CHANGES",
+    "ddlCommand" : {
+      "@type" : "createStreamV1",
+      "sourceName" : "OUTPUT",
+      "schema" : "`ID` STRING KEY, `RESULT` STRING",
+      "topicName" : "OUTPUT",
+      "formats" : {
+        "keyFormat" : {
+          "format" : "KAFKA"
+        },
+        "valueFormat" : {
+          "format" : "JSON"
+        }
+      }
+    },
+    "queryPlan" : {
+      "sources" : [ "INPUT" ],
+      "sink" : "OUTPUT",
+      "physicalPlan" : {
+        "@type" : "streamSinkV1",
+        "properties" : {
+          "queryContext" : "OUTPUT"
+        },
+        "source" : {
+          "@type" : "streamSelectV1",
+          "properties" : {
+            "queryContext" : "Project"
+          },
+          "source" : {
+            "@type" : "streamSourceV1",
+            "properties" : {
+              "queryContext" : "KsqlTopic/Source"
+            },
+            "topicName" : "test_topic",
+            "formats" : {
+              "keyFormat" : {
+                "format" : "KAFKA"
+              },
+              "valueFormat" : {
+                "format" : "JSON"
+              }
+            },
+            "sourceSchema" : "`ID` STRING KEY, `UTFCODE1` STRING, `UTFCODE2` STRING, `UTFCODE3` INTEGER, `UTFCODE4` INTEGER"
+          },
+          "keyColumnNames" : [ "ID" ],
+          "selectExpressions" : [ "CONCAT(CHR(UTFCODE1), CHR(UTFCODE2), CHR(UTFCODE3), CHR(UTFCODE4)) AS RESULT" ]
+        },
+        "formats" : {
+          "keyFormat" : {
+            "format" : "KAFKA"
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          }
+        },
+        "topicName" : "OUTPUT"
+      },
+      "queryId" : "CSAS_OUTPUT_0"
+    }
+  } ],
+  "configs" : {
+    "ksql.extension.dir" : "ext",
+    "ksql.streams.cache.max.bytes.buffering" : "0",
+    "ksql.security.extension.class" : null,
+    "ksql.transient.prefix" : "transient_",
+    "ksql.persistence.wrap.single.values" : "true",
+    "ksql.authorization.cache.expiry.time.secs" : "30",
+    "ksql.schema.registry.url" : "",
+    "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler",
+    "ksql.output.topic.name.prefix" : "",
+    "ksql.streams.auto.offset.reset" : "earliest",
+    "ksql.query.pull.enable.standby.reads" : "false",
+    "ksql.connect.url" : "http://localhost:8083",
+    "ksql.service.id" : "some.ksql.service.id",
+    "ksql.internal.topic.min.insync.replicas" : "1",
+    "ksql.streams.shutdown.timeout.ms" : "300000",
+    "ksql.internal.topic.replicas" : "1",
+    "ksql.insert.into.values.enabled" : "true",
+    "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807",
+    "ksql.query.pull.max.qps" : "2147483647",
+    "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler",
+    "ksql.access.validator.enable" : "auto",
+    "ksql.streams.bootstrap.servers" : "localhost:0",
+    "ksql.streams.commit.interval.ms" : "2000",
+    "ksql.metric.reporters" : "",
+    "ksql.query.pull.metrics.enabled" : "false",
+    "ksql.streams.auto.commit.interval.ms" : "0",
+    "ksql.metrics.extension" : null,
+    "ksql.streams.topology.optimization" : "all",
+    "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.streams.num.stream.threads" : "4",
+    "ksql.timestamp.throw.on.invalid" : "false",
+    "ksql.authorization.cache.max.entries" : "10000",
+    "ksql.metrics.tags.custom" : "",
+    "ksql.pull.queries.enable" : "true",
+    "ksql.udfs.enabled" : "true",
+    "ksql.udf.enable.security.manager" : "true",
+    "ksql.connect.worker.config" : "",
+    "ksql.sink.window.change.log.additional.retention" : "1000000",
+    "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
+    "ksql.udf.collect.metrics" : "false",
+    "ksql.persistent.prefix" : "query_",
+    "ksql.query.persistent.active.limit" : "2147483647",
+    "ksql.error.classifier.regex" : ""
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/spec.json
new file mode 100644
index 000000000000..78c3f2b754db
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/spec.json
@@ -0,0 +1,66 @@
+{
+  "version" : "6.0.0",
+  "timestamp" : 1591421497385,
+  "path" : "query-validation-tests\\chr.json",
+  "schemas" : {
+    "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT<UTFCODE1 VARCHAR, UTFCODE2 VARCHAR, UTFCODE3 INT, UTFCODE4 INT> NOT NULL",
+    "CSAS_OUTPUT_0.OUTPUT" : "STRUCT<RESULT VARCHAR> NOT NULL"
+  },
+  "testCase" : {
+    "name" : "multiple invocations - JSON",
+    "inputs" : [ {
+      "topic" : "test_topic",
+      "key" : "r1",
+      "value" : {
+        "utfcode1" : "\\u004b",
+        "utfcode2" : "\\u0053",
+        "utfcode3" : 81,
+        "utfcode4" : 76
+      }
+    } ],
+    "outputs" : [ {
+      "topic" : "OUTPUT",
+      "key" : "r1",
+      "value" : {
+        "RESULT" : "KSQL"
+      }
+    } ],
+    "topics" : [ {
+      "name" : "OUTPUT",
+      "replicas" : 1,
+      "numPartitions" : 4
+    }, {
+      "name" : "test_topic",
+      "replicas" : 1,
+      "numPartitions" : 4
+    } ],
+    "statements" : [ "CREATE STREAM INPUT (id STRING KEY, utfcode1 STRING, utfcode2 STRING, utfcode3 INTEGER, utfcode4 INTEGER) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT id, concat(chr(utfcode1), chr(utfcode2), chr(utfcode3), chr(utfcode4)) AS result FROM INPUT;" ],
+    "post" : {
+      "topics" : {
+        "topics" : [ {
+          "name" : "OUTPUT",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          },
+          "partitions" : 4
+        }, {
+          "name" : "test_topic",
+          "keyFormat" : {
+            "formatInfo" : {
+              "format" : "KAFKA"
+            }
+          },
+          "valueFormat" : {
+            "format" : "JSON"
+          },
+          "partitions" : 4
+        } ]
+      }
+    }
+  }
+}
\ No newline at end of file
diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/topology
new file mode 100644
index 000000000000..441a8f282644
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/topology
@@ -0,0 +1,13 @@
+Topologies:
+   Sub-topology: 0
+    Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic])
+      --> KSTREAM-TRANSFORMVALUES-0000000001
+    Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: [])
+      --> Project
+      <-- KSTREAM-SOURCE-0000000000
+    Processor: Project (stores: [])
+      --> KSTREAM-SINK-0000000003
+      <-- KSTREAM-TRANSFORMVALUES-0000000001
+    Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT)
+      <-- Project
+
diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/chr.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/chr.json
new file mode 100644
index 000000000000..357cb174b2c3
--- /dev/null
+++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/chr.json
@@ -0,0 +1,72 @@
+{
+  "comments": [
+    "Tests covering the use of the CHR function."
+  ],
+  "tests": [
+    {
+      "name": "codepoint from decimal code",
+      "format": ["AVRO", "JSON"],
+      "statements": [
+        "CREATE STREAM INPUT (id STRING KEY, utfcode INTEGER) WITH (kafka_topic='test_topic', value_format='{FORMAT}');",
+        "CREATE STREAM OUTPUT AS SELECT id, chr(utfcode) AS result FROM INPUT;"
+      ],
+      "inputs": [
+        {"topic": "test_topic", "key": "r1", "value": {"utfcode": 75}},
+        {"topic": "test_topic", "key": "r2", "value": {"utfcode": 22909}},
+        {"topic": "test_topic", "key": "r3", "value": {"utfcode": 99000}},
+        {"topic": "test_topic", "key": "r4", "value": {"utfcode": -1}},
+        {"topic": "test_topic", "key": "r5", "value": {"utfcode": null}}
+      ],
+      "outputs": [
+        {"topic": "OUTPUT", "key": "r1", "value": {"RESULT": "K"}},
+        {"topic": "OUTPUT", "key": "r2", "value": {"RESULT": "好"}},
+        {"topic": "OUTPUT", "key": "r3", "value": {"RESULT": "𘊸"}},
+        {"topic": "OUTPUT", "key": "r4", "value": {"RESULT": null}},
+        {"topic": "OUTPUT", "key": "r5", "value": {"RESULT": null}}
+      ]
+    },
+    {
+      "name": "codepoint from text code",
+      "format": ["AVRO", "JSON"],
+      "statements": [
+        "CREATE STREAM INPUT (id STRING KEY, utfcode STRING) WITH (kafka_topic='test_topic', value_format='{FORMAT}');",
+        "CREATE STREAM OUTPUT AS SELECT id, chr(utfcode) AS result FROM INPUT;"
+      ],
+      "inputs": [
+        {"topic": "test_topic", "key": "r1", "value": {"utfcode": "\\u004b"}},
+        {"topic": "test_topic", "key": "r2", "value": {"utfcode": "\\u597d"}},
+        {"topic": "test_topic", "key": "r3", "value": {"utfcode": "\\ud820\\udeb8"}},
+        {"topic": "test_topic", "key": "r4", "value": {"utfcode": "75"}},
+        {"topic": "test_topic", "key": "r5", "value": {"utfcode": "004b"}},
+        {"topic": "test_topic", "key": "r6", "value": {"utfcode": "bogus"}},
+        {"topic": "test_topic", "key": "r7", "value": {"utfcode": ""}},
+        {"topic": "test_topic", "key": "r8", "value": {"utfcode": null}}
+      ],
+      "outputs": [
+        {"topic": "OUTPUT", "key": "r1", "value": {"RESULT": "K"}},
+        {"topic": "OUTPUT", "key": "r2", "value": {"RESULT": "好"}},
+        {"topic": "OUTPUT", "key": "r3", "value": {"RESULT": "𘊸"}},
+        {"topic": "OUTPUT", "key": "r4", "value": {"RESULT": null}},
+        {"topic": "OUTPUT", "key": "r5", "value": {"RESULT": null}},
+        {"topic": "OUTPUT", "key": "r6", "value": {"RESULT": null}},
+        {"topic": "OUTPUT", "key": "r7", "value": {"RESULT": null}},
+        {"topic": "OUTPUT", "key": "r8", "value": {"RESULT": null}}
+      ]
+    },
+    {
+      "name": "multiple invocations",
+      "format": ["AVRO", "JSON"],
+      "statements": [
+        "CREATE STREAM INPUT (id STRING KEY, utfcode1 STRING, utfcode2 STRING, utfcode3 INTEGER, utfcode4 INTEGER) WITH (kafka_topic='test_topic', value_format='{FORMAT}');",
+        "CREATE STREAM OUTPUT AS SELECT id, concat(chr(utfcode1), chr(utfcode2), chr(utfcode3), chr(utfcode4)) AS result FROM INPUT;"
+      ],
+      "inputs": [
+        {"topic": "test_topic", "key": "r1", "value": {"utfcode1": "\\u004b", "utfcode2": "\\u0053", "utfcode3": 81, "utfcode4": 76}}
+      ],
+      "outputs": [
+        {"topic": "OUTPUT", "key": "r1", "value": {"RESULT": "KSQL"}}
+      ]
+    }
+    
+  ]
+}
\ No newline at end of file