Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
C0urante merged PR #15379: URL: https://github.com/apache/kafka/pull/15379 -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
C0urante commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1592877739 ## connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/SingleFieldPathTest.java: ## @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +class SingleFieldPathTest { + +@Test void shouldFindField() { +SchemaBuilder barSchema = SchemaBuilder.struct().field("bar", Schema.INT32_SCHEMA); +Schema schema = SchemaBuilder.struct().field("foo", barSchema).build(); + +assertEquals(barSchema.field("bar"), pathV2("foo.bar").fieldFrom(schema)); +assertEquals(schema.field("foo"), pathV2("foo").fieldFrom(schema)); +} + +@Test void shouldReturnNullFieldWhenFieldNotFound() { +SchemaBuilder barSchema = SchemaBuilder.struct().field("bar", Schema.INT32_SCHEMA); +Schema schema = SchemaBuilder.struct().field("foo", barSchema).build(); + +assertNull(pathV2("un.known").fieldFrom(schema)); +assertNull(pathV2("foo.unknown").fieldFrom(schema)); +assertNull(pathV2("unknown").fieldFrom(schema)); +assertNull(pathV2("test").fieldFrom(null)); +} + +@Test void shouldFindValueInMap() { +Map foo = new HashMap<>(); +foo.put("bar", 42); +foo.put("baz", null); +Map map = new HashMap<>(); +map.put("foo", foo); + +assertEquals(42, pathV2("foo.bar").valueFrom(map)); +assertNull(pathV2("foo.baz").valueFrom(map)); +} + +@Test void shouldReturnNullValueWhenFieldNotFoundInMap() { +Map foo = new HashMap<>(); +foo.put("bar", 42); +foo.put("baz", null); +Map map = new HashMap<>(); +map.put("foo", foo); + +assertNull(new SingleFieldPath("un.known", FieldSyntaxVersion.V2).valueFrom(map)); +assertNull(new SingleFieldPath("foo.unknown", FieldSyntaxVersion.V2).valueFrom(map)); +assertNull(new SingleFieldPath("unknown", FieldSyntaxVersion.V2).valueFrom(map)); +assertNull(new SingleFieldPath("foo.baz", FieldSyntaxVersion.V2).valueFrom(map)); +assertNull(new SingleFieldPath("foo.baz.inner", FieldSyntaxVersion.V2).valueFrom(map)); +} + +@Test void shouldFindValueInStruct() { +SchemaBuilder bazSchema = SchemaBuilder.struct() +.field("inner", Schema.STRING_SCHEMA); +SchemaBuilder barSchema = SchemaBuilder.struct() +.field("bar", Schema.INT32_SCHEMA) +.field("baz", bazSchema.optional()); +Schema schema = SchemaBuilder.struct().field("foo", barSchema).build(); +Struct foo = new Struct(barSchema) +.put("bar", 42) +.put("baz", null); +Struct struct = new Struct(schema).put("foo", foo); + +assertEquals(42, pathV2("foo.bar").valueFrom(struct)); +assertNull(pathV2("foo.baz").valueFrom(struct)); +} + +@Test void shouldReturnNullValueWhenFieldNotFoundInStruct() { +SchemaBuilder bazSchema = SchemaBuilder.struct() +.field("inner", Schema.STRING_SCHEMA); +SchemaBuilder barSchema = SchemaBuilder.struct() +.field("bar", Schema.INT32_SCHEMA) +.field("baz", bazSchema.optional()); +Schema schema = SchemaBuilder.struct().field("foo", barSchema).build(); +Struct foo = new Struct(barSchema) +.put("bar", 42) +.put("baz", null); +Struct struct = new Struct(schema).put("foo", foo); + +assertNull(new SingleFieldPath("un.known", FieldSyntaxVersion.V2).valueFrom(struct)); +assertNull(new SingleFieldPath("foo.unknown", FieldSyntaxVersion.V2).valueFrom(struct)); +assertNull(new SingleFieldPath("unknown", FieldSyntaxVersion.V2).valueFrom(struct)); +assertNull(ne
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
jeqo commented on PR #15379: URL: https://github.com/apache/kafka/pull/15379#issuecomment-2096481453 Thanks @C0urante! I've applied most suggestions, PTAL -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
jeqo commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1591285305 ## connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/SingleFieldPathTest.java: ## @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +class SingleFieldPathTest { + +@Test void shouldFindField() { +SchemaBuilder barSchema = SchemaBuilder.struct().field("bar", Schema.INT32_SCHEMA); +Schema schema = SchemaBuilder.struct().field("foo", barSchema).build(); + +assertEquals(barSchema.field("bar"), pathV2("foo.bar").fieldFrom(schema)); +assertEquals(schema.field("foo"), pathV2("foo").fieldFrom(schema)); +} + +@Test void shouldReturnNullFieldWhenFieldNotFound() { +SchemaBuilder barSchema = SchemaBuilder.struct().field("bar", Schema.INT32_SCHEMA); +Schema schema = SchemaBuilder.struct().field("foo", barSchema).build(); + +assertNull(pathV2("un.known").fieldFrom(schema)); +assertNull(pathV2("foo.unknown").fieldFrom(schema)); +assertNull(pathV2("unknown").fieldFrom(schema)); +assertNull(pathV2("test").fieldFrom(null)); +} + +@Test void shouldFindValueInMap() { +Map foo = new HashMap<>(); +foo.put("bar", 42); +foo.put("baz", null); +Map map = new HashMap<>(); +map.put("foo", foo); + +assertEquals(42, pathV2("foo.bar").valueFrom(map)); +assertNull(pathV2("foo.baz").valueFrom(map)); +} + +@Test void shouldReturnNullValueWhenFieldNotFoundInMap() { +Map foo = new HashMap<>(); +foo.put("bar", 42); +foo.put("baz", null); +Map map = new HashMap<>(); +map.put("foo", foo); + +assertNull(new SingleFieldPath("un.known", FieldSyntaxVersion.V2).valueFrom(map)); +assertNull(new SingleFieldPath("foo.unknown", FieldSyntaxVersion.V2).valueFrom(map)); +assertNull(new SingleFieldPath("unknown", FieldSyntaxVersion.V2).valueFrom(map)); +assertNull(new SingleFieldPath("foo.baz", FieldSyntaxVersion.V2).valueFrom(map)); +assertNull(new SingleFieldPath("foo.baz.inner", FieldSyntaxVersion.V2).valueFrom(map)); +} + +@Test void shouldFindValueInStruct() { +SchemaBuilder bazSchema = SchemaBuilder.struct() +.field("inner", Schema.STRING_SCHEMA); +SchemaBuilder barSchema = SchemaBuilder.struct() +.field("bar", Schema.INT32_SCHEMA) +.field("baz", bazSchema.optional()); +Schema schema = SchemaBuilder.struct().field("foo", barSchema).build(); +Struct foo = new Struct(barSchema) +.put("bar", 42) +.put("baz", null); +Struct struct = new Struct(schema).put("foo", foo); + +assertEquals(42, pathV2("foo.bar").valueFrom(struct)); +assertNull(pathV2("foo.baz").valueFrom(struct)); +} + +@Test void shouldReturnNullValueWhenFieldNotFoundInStruct() { +SchemaBuilder bazSchema = SchemaBuilder.struct() +.field("inner", Schema.STRING_SCHEMA); +SchemaBuilder barSchema = SchemaBuilder.struct() +.field("bar", Schema.INT32_SCHEMA) +.field("baz", bazSchema.optional()); +Schema schema = SchemaBuilder.struct().field("foo", barSchema).build(); +Struct foo = new Struct(barSchema) +.put("bar", 42) +.put("baz", null); +Struct struct = new Struct(schema).put("foo", foo); + +assertNull(new SingleFieldPath("un.known", FieldSyntaxVersion.V2).valueFrom(struct)); +assertNull(new SingleFieldPath("foo.unknown", FieldSyntaxVersion.V2).valueFrom(struct)); +assertNull(new SingleFieldPath("unknown", FieldSyntaxVersion.V2).valueFrom(struct)); +assertNull(new Si
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
C0urante commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1583252760 ## connect/transforms/src/main/java/org/apache/kafka/connect/transforms/field/SingleFieldPath.java: ## @@ -0,0 +1,240 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.apache.kafka.connect.transforms.util.Requirements.requireMapOrNull; + +/** + * A SingleFieldPath is composed of one or more field names, known as path steps, + * to access values within a data object (either {@code Struct} or {@code Map}). + * + * The field path semantics are defined by the {@link FieldSyntaxVersion syntax version}. + * + * @see https://cwiki.apache.org/confluence/display/KAFKA/KIP-821%3A+Connect+Transforms+support+for+nested+structures";>KIP-821 + * @see FieldSyntaxVersion + */ +public class SingleFieldPath { +// Invariants: +// - A field path can contain one or more steps +private static final char BACKTICK = '`'; +private static final char DOT = '.'; +private static final char BACKSLASH = '\\'; + +private final FieldSyntaxVersion version; +private final List steps; + +public SingleFieldPath(String pathText, FieldSyntaxVersion version) { +this.version = version; +switch (version) { +case V1: // backward compatibility +this.steps = Collections.singletonList(pathText); +break; +case V2: +this.steps = buildFieldPathV2(pathText); +break; +default: +throw new IllegalArgumentException("Unknown syntax version: " + version); +} +} + +private static List buildFieldPathV2(String path) { +final List steps = new ArrayList<>(); +// path character index to track backticks and dots and break path into steps +int idx = 0; +while (idx < path.length() && idx >= 0) { +if (path.charAt(idx) != BACKTICK) { +final int start = idx; +idx = path.indexOf(String.valueOf(DOT), idx); +if (idx >= 0) { // get path step and move forward +String field = path.substring(start, idx); +steps.add(field); +idx++; +} else { // add all +String field = path.substring(start); +steps.add(field); +} +} else { // has backtick +int backtickAt = idx; +idx++; +StringBuilder field = new StringBuilder(); +int start = idx; +while (true) { +// find closing backtick +idx = path.indexOf(String.valueOf(BACKTICK), idx); +if (idx == -1) { // if not found, then fail +failWhenIncompleteBacktickPair(path, backtickAt); +} + +// backtick escaped if right after backslash +boolean escaped = path.charAt(idx - 1) == BACKSLASH; + +if (idx >= path.length() - 1) { // at the end of path +if (escaped) { // but escaped, then fail +failWhenIncompleteBacktickPair(path, backtickAt); +} +field.append(path, start, idx); +// we've reached the end of the path, and the last character is the backtick +steps.add(field.toString()); +idx++; +break; +} + +if (path.charAt(idx + 1) != DOT) { // not followed by a dot +// this backtick isn't followed by a dot; include it in the field name, but continue +
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
C0urante commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1583247131 ## connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/FieldPathNotationTest.java: ## @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import org.apache.kafka.common.config.ConfigException; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +class FieldPathNotationTest { +final static String[] EMPTY_PATH = new String[] {}; + +@Test +void shouldBuildV1WithDotsAndBacktickPair() { +// Given v1 +// When path contains dots, then single step path +assertArrayEquals( +new String[] {"foo.bar.baz"}, +new SingleFieldPath("foo.bar.baz", FieldSyntaxVersion.V1).path()); +// When path contains backticks, then single step path +assertArrayEquals( +new String[] {"foo`bar`"}, +new SingleFieldPath("foo`bar`", FieldSyntaxVersion.V1).path()); +// When path contains dots and backticks, then single step path +assertArrayEquals( +new String[] {"foo.`bar.baz`"}, +new SingleFieldPath("foo.`bar.baz`", FieldSyntaxVersion.V1).path()); +} + +@Test +void shouldBuildV2WithEmptyPath() { +// Given v2 +// When path is empty +// Then build a path with no steps +assertArrayEquals(EMPTY_PATH, new SingleFieldPath("", FieldSyntaxVersion.V2).path()); +} + +@Test +void shouldBuildV2WithoutDots() { +// Given v2 +// When path without dots +// Then build a single step path +assertArrayEquals(new String[] {"foobarbaz"}, new SingleFieldPath("foobarbaz", FieldSyntaxVersion.V2).path()); +} + +@Test +void shouldBuildV2WhenIncludesDots() { +// Given v2 and fields without dots +// When path includes dots +// Then build a path with steps separated by dots +assertArrayEquals(new String[] {"foo", "bar", "baz"}, new SingleFieldPath("foo.bar.baz", FieldSyntaxVersion.V2).path()); +} + +@Test +void shouldBuildV2WithoutWrappingBackticks() { +// Given v2 and fields without dots +// When backticks are not wrapping a field name +// Then build a single step path including backticks +assertArrayEquals(new String[] {"foo`bar`baz"}, new SingleFieldPath("foo`bar`baz", FieldSyntaxVersion.V2).path()); +} + +@Test +void shouldBuildV2WhenIncludesDotsAndBacktickPair() { +// Given v2 and fields including dots +// When backticks are wrapping a field name (i.e. withing edges or between dots) +// Then build a path with steps separated by dots and not including backticks +assertArrayEquals(new String[] {"foo.bar.baz"}, new SingleFieldPath("`foo.bar.baz`", FieldSyntaxVersion.V2).path()); +assertArrayEquals(new String[] {"foo", "bar.baz"}, new SingleFieldPath("foo.`bar.baz`", FieldSyntaxVersion.V2).path()); +assertArrayEquals(new String[] {"foo.bar", "baz"}, new SingleFieldPath("`foo.bar`.baz", FieldSyntaxVersion.V2).path()); +assertArrayEquals(new String[] {"foo", "bar", "baz"}, new SingleFieldPath("foo.`bar`.baz", FieldSyntaxVersion.V2).path()); +} + +@Test +void shouldBuildV2AndIgnoreBackticksThatAreNotWrapping() { +// Given v2 and fields including dots and backticks +// When backticks are wrapping a field name (i.e. withing edges or between dots) +// Then build a path with steps separated by dots and including non-wrapping backticks +assertArrayEquals(new String[] {"foo", "`bar.baz"}, new SingleFieldPath("foo.``bar.baz`", FieldSyntaxVersion.V2).path()); +assertArrayEquals(new String[] {"foo", "bar.baz`"}, new SingleFieldPath("foo.`bar.baz``", FieldSyntaxVersion.V2).path()); +assertArrayEquals(new String[] {"foo", "ba`r.baz"},
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
jeqo commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1572995864 ## connect/transforms/src/main/java/org/apache/kafka/connect/transforms/field/MultiFieldPaths.java: ## @@ -0,0 +1,581 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Schema.Type; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.transforms.util.SchemaUtil; + +import java.util.AbstractMap; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Multiple field paths to access data objects ({@code Struct} or {@code Map}) efficiently, + * instead of multiple individual {@link SingleFieldPath single-field paths}. + * + * If the SMT requires accessing a single field on the same data object, + * use {@link SingleFieldPath} instead. + * + * @see https://cwiki.apache.org/confluence/display/KAFKA/KIP-821%3A+Connect+Transforms+support+for+nested+structures";>KIP-821 + * @see SingleFieldPath + * @see FieldSyntaxVersion + */ +public class MultiFieldPaths { +final Trie trie = new Trie(); + +MultiFieldPaths(Set paths) { +paths.forEach(trie::insert); +} + +public static MultiFieldPaths of(List fields, FieldSyntaxVersion syntaxVersion) { +return new MultiFieldPaths(fields.stream() +.map(f -> new SingleFieldPath(f, syntaxVersion)) +.collect(Collectors.toSet())); +} + +/** + * Find values at the field paths + * + * @param struct data value + * @return map of field paths and field/values + */ +public Map> fieldAndValuesFrom(Struct struct) { +if (trie.isEmpty()) return Collections.emptyMap(); +return findFieldAndValues(struct, trie.root, new HashMap<>()); +} + +private Map> findFieldAndValues( +Struct originalValue, +TrieNode trieAt, +Map> fieldAndValueMap +) { +for (Map.Entry step : trieAt.steps().entrySet()) { +Field field = originalValue.schema().field(step.getKey()); +if (step.getValue().isLeaf()) { +Map.Entry fieldAndValue = +field != null +? new AbstractMap.SimpleImmutableEntry<>(field, originalValue.get(field)) +: null; +fieldAndValueMap.put(step.getValue().path, fieldAndValue); +} else { +if (field.schema().type() == Type.STRUCT) { +findFieldAndValues( +originalValue.getStruct(field.name()), +step.getValue(), +fieldAndValueMap +); +} +} +} +return fieldAndValueMap; +} + +/** + * Find values at the field paths + * + * @param value data value + * @return map of field paths and field/values + */ +public Map> fieldAndValuesFrom(Map value) { +if (trie.isEmpty()) return Collections.emptyMap(); +return findFieldAndValues(value, trie.root, new HashMap<>()); +} + +@SuppressWarnings("unchecked") +private Map> findFieldAndValues( +Map value, +TrieNode trieAt, +Map> fieldAndValueMap +) { +for (Map.Entry step : trieAt.steps().entrySet()) { +Object fieldValue = value.get(step.getKey()); +if (step.getValue().isLeaf()) { +fieldAndValueMap.put( +step.getValue().path, +new AbstractMap.SimpleImmutableEntry<>(step.getKey(), fieldValue) +); +} else { +if (fieldValue instanceof Map) { +findFieldAndValues( +(Map) fieldValue, +step.getValue(), +fieldAndValueMap +
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
jeqo commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1572509302 ## connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/FieldPathNotationTest.java: ## @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import org.apache.kafka.common.config.ConfigException; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +class FieldPathNotationTest { +final static String[] EMPTY_PATH = new String[] {}; + +@Test +void shouldBuildV1WithDotsAndBacktickPair() { +// Given v1 +// When path contains dots, then single step path +assertArrayEquals( +new String[] {"foo.bar.baz"}, +new SingleFieldPath("foo.bar.baz", FieldSyntaxVersion.V1).path()); +// When path contains backticks, then single step path +assertArrayEquals( +new String[] {"foo`bar`"}, +new SingleFieldPath("foo`bar`", FieldSyntaxVersion.V1).path()); +// When path contains dots and backticks, then single step path +assertArrayEquals( +new String[] {"foo.`bar.baz`"}, +new SingleFieldPath("foo.`bar.baz`", FieldSyntaxVersion.V1).path()); +} + +@Test +void shouldBuildV2WithEmptyPath() { +// Given v2 +// When path is empty +// Then build a path with no steps +assertArrayEquals(EMPTY_PATH, new SingleFieldPath("", FieldSyntaxVersion.V2).path()); +} + +@Test +void shouldBuildV2WithoutDots() { +// Given v2 +// When path without dots +// Then build a single step path +assertArrayEquals(new String[] {"foobarbaz"}, new SingleFieldPath("foobarbaz", FieldSyntaxVersion.V2).path()); +} + +@Test +void shouldBuildV2WhenIncludesDots() { +// Given v2 and fields without dots +// When path includes dots +// Then build a path with steps separated by dots +assertArrayEquals(new String[] {"foo", "bar", "baz"}, new SingleFieldPath("foo.bar.baz", FieldSyntaxVersion.V2).path()); +} + +@Test +void shouldBuildV2WithoutWrappingBackticks() { +// Given v2 and fields without dots +// When backticks are not wrapping a field name +// Then build a single step path including backticks +assertArrayEquals(new String[] {"foo`bar`baz"}, new SingleFieldPath("foo`bar`baz", FieldSyntaxVersion.V2).path()); +} + +@Test +void shouldBuildV2WhenIncludesDotsAndBacktickPair() { +// Given v2 and fields including dots +// When backticks are wrapping a field name (i.e. withing edges or between dots) +// Then build a path with steps separated by dots and not including backticks +assertArrayEquals(new String[] {"foo.bar.baz"}, new SingleFieldPath("`foo.bar.baz`", FieldSyntaxVersion.V2).path()); +assertArrayEquals(new String[] {"foo", "bar.baz"}, new SingleFieldPath("foo.`bar.baz`", FieldSyntaxVersion.V2).path()); +assertArrayEquals(new String[] {"foo.bar", "baz"}, new SingleFieldPath("`foo.bar`.baz", FieldSyntaxVersion.V2).path()); +assertArrayEquals(new String[] {"foo", "bar", "baz"}, new SingleFieldPath("foo.`bar`.baz", FieldSyntaxVersion.V2).path()); +} + +@Test +void shouldBuildV2AndIgnoreBackticksThatAreNotWrapping() { +// Given v2 and fields including dots and backticks +// When backticks are wrapping a field name (i.e. withing edges or between dots) +// Then build a path with steps separated by dots and including non-wrapping backticks +assertArrayEquals(new String[] {"foo", "`bar.baz"}, new SingleFieldPath("foo.``bar.baz`", FieldSyntaxVersion.V2).path()); +assertArrayEquals(new String[] {"foo", "bar.baz`"}, new SingleFieldPath("foo.`bar.baz``", FieldSyntaxVersion.V2).path()); +assertArrayEquals(new String[] {"foo", "ba`r.baz"}, new
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
jeqo commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1572436483 ## connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.java: ## @@ -60,6 +62,30 @@ public void schemaless() { assertEquals(expectedKey, transformedRecord.key()); } +@Test +public void schemalessAndNestedFields() { +Map configs = new HashMap<>(); +configs.put("fields", "a,b.c"); +configs.put(FieldSyntaxVersion.FIELD_SYNTAX_VERSION_CONFIG, FieldSyntaxVersion.V2.name()); +xform.configure(configs); + +final HashMap value = new HashMap<>(); +value.put("a", 1); +final HashMap nested = new HashMap<>(); +nested.put("c", 3); +value.put("b", nested); + +final SinkRecord record = new SinkRecord("", 0, null, null, null, value, 0); +final SinkRecord transformedRecord = xform.apply(record); + +final HashMap expectedKey = new HashMap<>(); +expectedKey.put("a", 1); +expectedKey.put("b.c", 3); Review Comment: Good catch. I don't remember this scenario being discussed in the KIP thread. Was using the original keys as they are meant to be unique, but they could also contain all kind of escape fields making it hard to use. Happy to switch into deriving the key schema from the nesting structure. Will make an update to the KIP to make this explicit as well. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
C0urante commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1571018390 ## connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/SingleFieldPathTest.java: ## @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.junit.jupiter.api.Test; + +class SingleFieldPathTest { + +@Test +void shouldIncludeEmptyFieldNames() { +assertArrayEquals( +new String[] {"", "", ""}, +new SingleFieldPath("..", FieldSyntaxVersion.V2).path() +); +assertArrayEquals( +new String[] {"foo", "", ""}, +new SingleFieldPath("foo..", FieldSyntaxVersion.V2).path() +); +assertArrayEquals( +new String[] {"", "bar", ""}, +new SingleFieldPath(".bar.", FieldSyntaxVersion.V2).path() +); +assertArrayEquals( +new String[] {"", "", "baz"}, +new SingleFieldPath("..baz", FieldSyntaxVersion.V2).path() +); +} Review Comment: This case can probably be moved to `FieldPathNotationTest`, right? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
C0urante commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1571007787 ## connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/FieldPathNotationTest.java: ## @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import org.apache.kafka.common.config.ConfigException; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +class FieldPathNotationTest { +final static String[] EMPTY_PATH = new String[] {}; + +@Test +void shouldBuildV1WithDotsAndBacktickPair() { +// Given v1 +// When path contains dots, then single step path +assertArrayEquals( +new String[] {"foo.bar.baz"}, +new SingleFieldPath("foo.bar.baz", FieldSyntaxVersion.V1).path()); +// When path contains backticks, then single step path +assertArrayEquals( +new String[] {"foo`bar`"}, +new SingleFieldPath("foo`bar`", FieldSyntaxVersion.V1).path()); +// When path contains dots and backticks, then single step path +assertArrayEquals( +new String[] {"foo.`bar.baz`"}, +new SingleFieldPath("foo.`bar.baz`", FieldSyntaxVersion.V1).path()); +} + +@Test +void shouldBuildV2WithEmptyPath() { +// Given v2 +// When path is empty +// Then build a path with no steps +assertArrayEquals(EMPTY_PATH, new SingleFieldPath("", FieldSyntaxVersion.V2).path()); +} + +@Test +void shouldBuildV2WithoutDots() { +// Given v2 +// When path without dots +// Then build a single step path +assertArrayEquals(new String[] {"foobarbaz"}, new SingleFieldPath("foobarbaz", FieldSyntaxVersion.V2).path()); +} + +@Test +void shouldBuildV2WhenIncludesDots() { +// Given v2 and fields without dots +// When path includes dots +// Then build a path with steps separated by dots +assertArrayEquals(new String[] {"foo", "bar", "baz"}, new SingleFieldPath("foo.bar.baz", FieldSyntaxVersion.V2).path()); +} + +@Test +void shouldBuildV2WithoutWrappingBackticks() { +// Given v2 and fields without dots +// When backticks are not wrapping a field name +// Then build a single step path including backticks +assertArrayEquals(new String[] {"foo`bar`baz"}, new SingleFieldPath("foo`bar`baz", FieldSyntaxVersion.V2).path()); +} + +@Test +void shouldBuildV2WhenIncludesDotsAndBacktickPair() { +// Given v2 and fields including dots +// When backticks are wrapping a field name (i.e. withing edges or between dots) +// Then build a path with steps separated by dots and not including backticks +assertArrayEquals(new String[] {"foo.bar.baz"}, new SingleFieldPath("`foo.bar.baz`", FieldSyntaxVersion.V2).path()); +assertArrayEquals(new String[] {"foo", "bar.baz"}, new SingleFieldPath("foo.`bar.baz`", FieldSyntaxVersion.V2).path()); +assertArrayEquals(new String[] {"foo.bar", "baz"}, new SingleFieldPath("`foo.bar`.baz", FieldSyntaxVersion.V2).path()); +assertArrayEquals(new String[] {"foo", "bar", "baz"}, new SingleFieldPath("foo.`bar`.baz", FieldSyntaxVersion.V2).path()); +} + +@Test +void shouldBuildV2AndIgnoreBackticksThatAreNotWrapping() { +// Given v2 and fields including dots and backticks +// When backticks are wrapping a field name (i.e. withing edges or between dots) +// Then build a path with steps separated by dots and including non-wrapping backticks +assertArrayEquals(new String[] {"foo", "`bar.baz"}, new SingleFieldPath("foo.``bar.baz`", FieldSyntaxVersion.V2).path()); +assertArrayEquals(new String[] {"foo", "bar.baz`"}, new SingleFieldPath("foo.`bar.baz``", FieldSyntaxVersion.V2).path()); +assertArrayEquals(new String[] {"foo", "ba`r.baz"},
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
C0urante commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1570998105 ## connect/transforms/src/test/java/org/apache/kafka/connect/transforms/field/FieldPathNotationTest.java: ## @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import org.apache.kafka.common.config.ConfigException; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +class FieldPathNotationTest { Review Comment: We can simplify these tests with a few utility methods: ```java private void assertParseV1(String path) { assertArrayEquals( new String[] {path}, new SingleFieldPath(path, FieldSyntaxVersion.V1).path()); } private void assertParseV2(String inputPath, String... expectedSteps) { assertArrayEquals( expectedSteps, new SingleFieldPath(inputPath, FieldSyntaxVersion.V2).path() ); } private void assertParseV2Error(String inputPath, String expectedMessage) { ConfigException exception = assertThrows( ConfigException.class, () -> new SingleFieldPath(inputPath, FieldSyntaxVersion.V2) ); assertEquals(expectedMessage, exception.getMessage()); } ``` Some simplified test cases would look like this: ```java @Test void shouldBuildV1WithDotsAndBacktickPair() { // Given v1 // When path contains dots, then single step path assertParseV1("foo.bar.baz"); // When path contains backticks, then single step path assertParseV1("foo`bar`"); // When path contains dots and backticks, then single step path assertParseV1("foo.`bar.baz`"); } @Test void shouldBuildV2WhenIncludesDotsAndBacktickPair() { // Given v2 and fields including dots // When backticks are wrapping a field name (i.e. withing edges or between dots) // Then build a path with steps separated by dots and not including backticks assertParseV2("`foo.bar.baz`", "foo.bar.baz"); assertParseV2("foo.`bar.baz`", "foo", "bar.baz"); assertParseV2("`foo.bar`.baz", "foo.bar", "baz"); assertParseV2("foo.`bar`.baz", "foo", "bar", "baz"); } @Test void shouldFailV2WhenIncompleteBackticks() { // Given v2 // When backticks are not closed and not escaped // Then it should fail assertParseV2Error( "`foo.bar.baz", "Incomplete backtick pair in path: [`foo.bar.baz], consider adding a backslash before backtick at position 0 to escape it" ); assertParseV2Error( "foo.`bar.baz", "Incomplete backtick pair in path: [foo.`bar.baz], consider adding a backslash before backtick at position 4 to escape it" ); assertParseV2Error( "foo.bar.`baz", "Incomplete backtick pair in path: [foo.bar.`baz], consider adding a backslash before backtick at position 8 to escape it" ); assertParseV2Error( "foo.bar.`baz\\`", "Incomplete backtick pair in path: [foo.bar.`baz\\`], consider adding a backslash before backtick at position 8 to escape it" ); } ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
C0urante commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1570962332 ## connect/transforms/src/main/java/org/apache/kafka/connect/transforms/field/SingleFieldPath.java: ## @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * A SingleFieldPath is composed of one or more field names, known as path steps, + * to access values within a data object (either {@code Struct} or {@code Map}). + * + * If the SMT requires accessing multiple fields on the same data object, use {@link MultiFieldPaths} instead. + * + * The field path semantics are defined by the {@link FieldSyntaxVersion syntax version}. + * + * @see https://cwiki.apache.org/confluence/display/KAFKA/KIP-821%3A+Connect+Transforms+support+for+nested+structures";>KIP-821 + * @see FieldSyntaxVersion + * @see MultiFieldPaths + */ +public class SingleFieldPath { +// Invariants: +// - A field path can contain one or more steps +private static final char BACKTICK = '`'; +private static final char DOT = '.'; +private static final char BACKSLASH = '\\'; + +private final String originalPath; +private final FieldSyntaxVersion version; +private final List steps; + +public SingleFieldPath(String pathText, FieldSyntaxVersion version) { +this.originalPath = Objects.requireNonNull(pathText, "Field path cannot be null"); +this.version = version; +switch (version) { +case V1: // backward compatibility +this.steps = Collections.singletonList(pathText); +break; +case V2: +this.steps = buildFieldPathV2(pathText); +break; +default: +throw new IllegalArgumentException("Unknown syntax version: " + version); +} +} + +private static List buildFieldPathV2(String path) { +final List steps = new ArrayList<>(); +// path character index to track backticks and dots and break path into steps +int idx = 0; +while (idx < path.length() && idx >= 0) { +if (path.charAt(idx) != BACKTICK) { +final int start = idx; +idx = path.indexOf(String.valueOf(DOT), idx); +if (idx >= 0) { // get path step and move forward +String field = path.substring(start, idx); +steps.add(field); +idx++; +} else { // add all +String field = path.substring(start); +steps.add(field); +} +} else { // has backtick +int backtickAt = idx; +idx++; +StringBuilder field = new StringBuilder(); +int start = idx; +while (true) { +// find closing backtick +idx = path.indexOf(String.valueOf(BACKTICK), idx); +if (idx == -1) { // if not found, then fail +failWhenIncompleteBacktickPair(path, backtickAt); +} + +// backtick escaped if right after backslash +boolean escaped = path.charAt(idx - 1) == BACKSLASH; + +if (idx >= path.length() - 1) { // at the end of path +if (escaped) { // but escaped, then fail +failWhenIncompleteBacktickPair(path, backtickAt); +} +field.append(path, start, idx); +// we've reached the end of the path, and the last character is the backtick +steps.add(field.toString()); +idx++; +break; +} + +if (pat
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
C0urante commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1570960525 ## connect/transforms/src/main/java/org/apache/kafka/connect/transforms/field/SingleFieldPath.java: ## @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * A SingleFieldPath is composed of one or more field names, known as path steps, + * to access values within a data object (either {@code Struct} or {@code Map}). + * + * If the SMT requires accessing multiple fields on the same data object, use {@link MultiFieldPaths} instead. + * + * The field path semantics are defined by the {@link FieldSyntaxVersion syntax version}. + * + * @see https://cwiki.apache.org/confluence/display/KAFKA/KIP-821%3A+Connect+Transforms+support+for+nested+structures";>KIP-821 + * @see FieldSyntaxVersion + * @see MultiFieldPaths + */ +public class SingleFieldPath { +// Invariants: +// - A field path can contain one or more steps +private static final char BACKTICK = '`'; +private static final char DOT = '.'; +private static final char BACKSLASH = '\\'; + +private final String originalPath; +private final FieldSyntaxVersion version; +private final List steps; + +public SingleFieldPath(String pathText, FieldSyntaxVersion version) { +this.originalPath = Objects.requireNonNull(pathText, "Field path cannot be null"); +this.version = version; +switch (version) { +case V1: // backward compatibility +this.steps = Collections.singletonList(pathText); +break; +case V2: +this.steps = buildFieldPathV2(pathText); +break; +default: +throw new IllegalArgumentException("Unknown syntax version: " + version); +} +} + +private static List buildFieldPathV2(String path) { +final List steps = new ArrayList<>(); +// path character index to track backticks and dots and break path into steps +int idx = 0; +while (idx < path.length() && idx >= 0) { +if (path.charAt(idx) != BACKTICK) { +final int start = idx; +idx = path.indexOf(String.valueOf(DOT), idx); +if (idx >= 0) { // get path step and move forward +String field = path.substring(start, idx); +steps.add(field); +idx++; +} else { // add all +String field = path.substring(start); +steps.add(field); +} +} else { // has backtick +int backtickAt = idx; +idx++; +StringBuilder field = new StringBuilder(); +int start = idx; +while (true) { +// find closing backtick +idx = path.indexOf(String.valueOf(BACKTICK), idx); +if (idx == -1) { // if not found, then fail +failWhenIncompleteBacktickPair(path, backtickAt); +} + +// backtick escaped if right after backslash +boolean escaped = path.charAt(idx - 1) == BACKSLASH; + +if (idx >= path.length() - 1) { // at the end of path +if (escaped) { // but escaped, then fail +failWhenIncompleteBacktickPair(path, backtickAt); +} +field.append(path, start, idx); +// we've reached the end of the path, and the last character is the backtick +steps.add(field.toString()); +idx++; +break; +} + +if (pat
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
C0urante commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1570939313 ## connect/transforms/src/main/java/org/apache/kafka/connect/transforms/field/SingleFieldPath.java: ## @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * A SingleFieldPath is composed of one or more field names, known as path steps, + * to access values within a data object (either {@code Struct} or {@code Map}). + * + * If the SMT requires accessing multiple fields on the same data object, use {@link MultiFieldPaths} instead. + * + * The field path semantics are defined by the {@link FieldSyntaxVersion syntax version}. + * + * @see https://cwiki.apache.org/confluence/display/KAFKA/KIP-821%3A+Connect+Transforms+support+for+nested+structures";>KIP-821 + * @see FieldSyntaxVersion + * @see MultiFieldPaths + */ +public class SingleFieldPath { +// Invariants: +// - A field path can contain one or more steps +private static final char BACKTICK = '`'; +private static final char DOT = '.'; +private static final char BACKSLASH = '\\'; + +private final String originalPath; +private final FieldSyntaxVersion version; +private final List steps; + +public SingleFieldPath(String pathText, FieldSyntaxVersion version) { +this.originalPath = Objects.requireNonNull(pathText, "Field path cannot be null"); +this.version = version; +switch (version) { +case V1: // backward compatibility +this.steps = Collections.singletonList(pathText); +break; +case V2: +this.steps = buildFieldPathV2(pathText); +break; +default: +throw new IllegalArgumentException("Unknown syntax version: " + version); +} +} + +private static List buildFieldPathV2(String path) { +final List steps = new ArrayList<>(); +// path character index to track backticks and dots and break path into steps +int idx = 0; +while (idx < path.length() && idx >= 0) { +if (path.charAt(idx) != BACKTICK) { +final int start = idx; +idx = path.indexOf(String.valueOf(DOT), idx); +if (idx >= 0) { // get path step and move forward +String field = path.substring(start, idx); +steps.add(field); +idx++; +} else { // add all +String field = path.substring(start); +steps.add(field); +} +} else { // has backtick +int backtickAt = idx; +idx++; +StringBuilder field = new StringBuilder(); +int start = idx; +while (true) { +// find closing backtick +idx = path.indexOf(String.valueOf(BACKTICK), idx); +if (idx == -1) { // if not found, then fail +failWhenIncompleteBacktickPair(path, backtickAt); +} + +// backtick escaped if right after backslash +boolean escaped = path.charAt(idx - 1) == BACKSLASH; + +if (idx >= path.length() - 1) { // at the end of path +if (escaped) { // but escaped, then fail +failWhenIncompleteBacktickPair(path, backtickAt); +} +field.append(path, start, idx); +// we've reached the end of the path, and the last character is the backtick +steps.add(field.toString()); +idx++; +break; +} + +if (pat
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
C0urante commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1570939313 ## connect/transforms/src/main/java/org/apache/kafka/connect/transforms/field/SingleFieldPath.java: ## @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * A SingleFieldPath is composed of one or more field names, known as path steps, + * to access values within a data object (either {@code Struct} or {@code Map}). + * + * If the SMT requires accessing multiple fields on the same data object, use {@link MultiFieldPaths} instead. + * + * The field path semantics are defined by the {@link FieldSyntaxVersion syntax version}. + * + * @see https://cwiki.apache.org/confluence/display/KAFKA/KIP-821%3A+Connect+Transforms+support+for+nested+structures";>KIP-821 + * @see FieldSyntaxVersion + * @see MultiFieldPaths + */ +public class SingleFieldPath { +// Invariants: +// - A field path can contain one or more steps +private static final char BACKTICK = '`'; +private static final char DOT = '.'; +private static final char BACKSLASH = '\\'; + +private final String originalPath; +private final FieldSyntaxVersion version; +private final List steps; + +public SingleFieldPath(String pathText, FieldSyntaxVersion version) { +this.originalPath = Objects.requireNonNull(pathText, "Field path cannot be null"); +this.version = version; +switch (version) { +case V1: // backward compatibility +this.steps = Collections.singletonList(pathText); +break; +case V2: +this.steps = buildFieldPathV2(pathText); +break; +default: +throw new IllegalArgumentException("Unknown syntax version: " + version); +} +} + +private static List buildFieldPathV2(String path) { +final List steps = new ArrayList<>(); +// path character index to track backticks and dots and break path into steps +int idx = 0; +while (idx < path.length() && idx >= 0) { +if (path.charAt(idx) != BACKTICK) { +final int start = idx; +idx = path.indexOf(String.valueOf(DOT), idx); +if (idx >= 0) { // get path step and move forward +String field = path.substring(start, idx); +steps.add(field); +idx++; +} else { // add all +String field = path.substring(start); +steps.add(field); +} +} else { // has backtick +int backtickAt = idx; +idx++; +StringBuilder field = new StringBuilder(); +int start = idx; +while (true) { +// find closing backtick +idx = path.indexOf(String.valueOf(BACKTICK), idx); +if (idx == -1) { // if not found, then fail +failWhenIncompleteBacktickPair(path, backtickAt); +} + +// backtick escaped if right after backslash +boolean escaped = path.charAt(idx - 1) == BACKSLASH; + +if (idx >= path.length() - 1) { // at the end of path +if (escaped) { // but escaped, then fail +failWhenIncompleteBacktickPair(path, backtickAt); +} +field.append(path, start, idx); +// we've reached the end of the path, and the last character is the backtick +steps.add(field.toString()); +idx++; +break; +} + +if (pat
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
C0urante commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1570939313 ## connect/transforms/src/main/java/org/apache/kafka/connect/transforms/field/SingleFieldPath.java: ## @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * A SingleFieldPath is composed of one or more field names, known as path steps, + * to access values within a data object (either {@code Struct} or {@code Map}). + * + * If the SMT requires accessing multiple fields on the same data object, use {@link MultiFieldPaths} instead. + * + * The field path semantics are defined by the {@link FieldSyntaxVersion syntax version}. + * + * @see https://cwiki.apache.org/confluence/display/KAFKA/KIP-821%3A+Connect+Transforms+support+for+nested+structures";>KIP-821 + * @see FieldSyntaxVersion + * @see MultiFieldPaths + */ +public class SingleFieldPath { +// Invariants: +// - A field path can contain one or more steps +private static final char BACKTICK = '`'; +private static final char DOT = '.'; +private static final char BACKSLASH = '\\'; + +private final String originalPath; +private final FieldSyntaxVersion version; +private final List steps; + +public SingleFieldPath(String pathText, FieldSyntaxVersion version) { +this.originalPath = Objects.requireNonNull(pathText, "Field path cannot be null"); +this.version = version; +switch (version) { +case V1: // backward compatibility +this.steps = Collections.singletonList(pathText); +break; +case V2: +this.steps = buildFieldPathV2(pathText); +break; +default: +throw new IllegalArgumentException("Unknown syntax version: " + version); +} +} + +private static List buildFieldPathV2(String path) { +final List steps = new ArrayList<>(); +// path character index to track backticks and dots and break path into steps +int idx = 0; +while (idx < path.length() && idx >= 0) { +if (path.charAt(idx) != BACKTICK) { +final int start = idx; +idx = path.indexOf(String.valueOf(DOT), idx); +if (idx >= 0) { // get path step and move forward +String field = path.substring(start, idx); +steps.add(field); +idx++; +} else { // add all +String field = path.substring(start); +steps.add(field); +} +} else { // has backtick +int backtickAt = idx; +idx++; +StringBuilder field = new StringBuilder(); +int start = idx; +while (true) { +// find closing backtick +idx = path.indexOf(String.valueOf(BACKTICK), idx); +if (idx == -1) { // if not found, then fail +failWhenIncompleteBacktickPair(path, backtickAt); +} + +// backtick escaped if right after backslash +boolean escaped = path.charAt(idx - 1) == BACKSLASH; + +if (idx >= path.length() - 1) { // at the end of path +if (escaped) { // but escaped, then fail +failWhenIncompleteBacktickPair(path, backtickAt); +} +field.append(path, start, idx); +// we've reached the end of the path, and the last character is the backtick +steps.add(field.toString()); +idx++; +break; +} + +if (pat
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
C0urante commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1570939313 ## connect/transforms/src/main/java/org/apache/kafka/connect/transforms/field/SingleFieldPath.java: ## @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * A SingleFieldPath is composed of one or more field names, known as path steps, + * to access values within a data object (either {@code Struct} or {@code Map}). + * + * If the SMT requires accessing multiple fields on the same data object, use {@link MultiFieldPaths} instead. + * + * The field path semantics are defined by the {@link FieldSyntaxVersion syntax version}. + * + * @see https://cwiki.apache.org/confluence/display/KAFKA/KIP-821%3A+Connect+Transforms+support+for+nested+structures";>KIP-821 + * @see FieldSyntaxVersion + * @see MultiFieldPaths + */ +public class SingleFieldPath { +// Invariants: +// - A field path can contain one or more steps +private static final char BACKTICK = '`'; +private static final char DOT = '.'; +private static final char BACKSLASH = '\\'; + +private final String originalPath; +private final FieldSyntaxVersion version; +private final List steps; + +public SingleFieldPath(String pathText, FieldSyntaxVersion version) { +this.originalPath = Objects.requireNonNull(pathText, "Field path cannot be null"); +this.version = version; +switch (version) { +case V1: // backward compatibility +this.steps = Collections.singletonList(pathText); +break; +case V2: +this.steps = buildFieldPathV2(pathText); +break; +default: +throw new IllegalArgumentException("Unknown syntax version: " + version); +} +} + +private static List buildFieldPathV2(String path) { +final List steps = new ArrayList<>(); +// path character index to track backticks and dots and break path into steps +int idx = 0; +while (idx < path.length() && idx >= 0) { +if (path.charAt(idx) != BACKTICK) { +final int start = idx; +idx = path.indexOf(String.valueOf(DOT), idx); +if (idx >= 0) { // get path step and move forward +String field = path.substring(start, idx); +steps.add(field); +idx++; +} else { // add all +String field = path.substring(start); +steps.add(field); +} +} else { // has backtick +int backtickAt = idx; +idx++; +StringBuilder field = new StringBuilder(); +int start = idx; +while (true) { +// find closing backtick +idx = path.indexOf(String.valueOf(BACKTICK), idx); +if (idx == -1) { // if not found, then fail +failWhenIncompleteBacktickPair(path, backtickAt); +} + +// backtick escaped if right after backslash +boolean escaped = path.charAt(idx - 1) == BACKSLASH; + +if (idx >= path.length() - 1) { // at the end of path +if (escaped) { // but escaped, then fail +failWhenIncompleteBacktickPair(path, backtickAt); +} +field.append(path, start, idx); +// we've reached the end of the path, and the last character is the backtick +steps.add(field.toString()); +idx++; +break; +} + +if (pat
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
C0urante commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1570907754 ## connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.java: ## @@ -60,6 +62,30 @@ public void schemaless() { assertEquals(expectedKey, transformedRecord.key()); } +@Test +public void schemalessAndNestedFields() { +Map configs = new HashMap<>(); +configs.put("fields", "a,b.c"); +configs.put(FieldSyntaxVersion.FIELD_SYNTAX_VERSION_CONFIG, FieldSyntaxVersion.V2.name()); +xform.configure(configs); + +final HashMap value = new HashMap<>(); +value.put("a", 1); +final HashMap nested = new HashMap<>(); +nested.put("c", 3); +value.put("b", nested); + +final SinkRecord record = new SinkRecord("", 0, null, null, null, value, 0); +final SinkRecord transformedRecord = xform.apply(record); + +final HashMap expectedKey = new HashMap<>(); +expectedKey.put("a", 1); +expectedKey.put("b.c", 3); Review Comment: Hmmm... I thought that the nesting structure of the value would be matched in the derived key. Does the KIP go into detail anywhere about this? I'm not a huge fan of reusing the dot notation here but I can't remember if this was discussed on the KIP thread or not. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
jeqo commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1568385972 ## connect/transforms/src/main/java/org/apache/kafka/connect/transforms/field/MultiFieldPaths.java: ## @@ -0,0 +1,581 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Schema.Type; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.transforms.util.SchemaUtil; + +import java.util.AbstractMap; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Multiple field paths to access data objects ({@code Struct} or {@code Map}) efficiently, + * instead of multiple individual {@link SingleFieldPath single-field paths}. + * + * If the SMT requires accessing a single field on the same data object, + * use {@link SingleFieldPath} instead. + * + * @see https://cwiki.apache.org/confluence/display/KAFKA/KIP-821%3A+Connect+Transforms+support+for+nested+structures";>KIP-821 + * @see SingleFieldPath + * @see FieldSyntaxVersion + */ +public class MultiFieldPaths { +final Trie trie = new Trie(); + +MultiFieldPaths(Set paths) { +paths.forEach(trie::insert); +} + +public static MultiFieldPaths of(List fields, FieldSyntaxVersion syntaxVersion) { +return new MultiFieldPaths(fields.stream() +.map(f -> new SingleFieldPath(f, syntaxVersion)) +.collect(Collectors.toSet())); +} + +/** + * Find values at the field paths + * + * @param struct data value + * @return map of field paths and field/values + */ +public Map> fieldAndValuesFrom(Struct struct) { +if (trie.isEmpty()) return Collections.emptyMap(); +return findFieldAndValues(struct, trie.root, new HashMap<>()); +} + +private Map> findFieldAndValues( +Struct originalValue, +TrieNode trieAt, +Map> fieldAndValueMap +) { +for (Map.Entry step : trieAt.steps().entrySet()) { +Field field = originalValue.schema().field(step.getKey()); +if (step.getValue().isLeaf()) { +Map.Entry fieldAndValue = +field != null +? new AbstractMap.SimpleImmutableEntry<>(field, originalValue.get(field)) +: null; +fieldAndValueMap.put(step.getValue().path, fieldAndValue); +} else { +if (field.schema().type() == Type.STRUCT) { +findFieldAndValues( +originalValue.getStruct(field.name()), +step.getValue(), +fieldAndValueMap +); +} +} +} +return fieldAndValueMap; +} + +/** + * Find values at the field paths + * + * @param value data value + * @return map of field paths and field/values + */ +public Map> fieldAndValuesFrom(Map value) { +if (trie.isEmpty()) return Collections.emptyMap(); +return findFieldAndValues(value, trie.root, new HashMap<>()); +} + +@SuppressWarnings("unchecked") +private Map> findFieldAndValues( +Map value, +TrieNode trieAt, +Map> fieldAndValueMap +) { +for (Map.Entry step : trieAt.steps().entrySet()) { +Object fieldValue = value.get(step.getKey()); +if (step.getValue().isLeaf()) { +fieldAndValueMap.put( +step.getValue().path, +new AbstractMap.SimpleImmutableEntry<>(step.getKey(), fieldValue) +); +} else { +if (fieldValue instanceof Map) { +findFieldAndValues( +(Map) fieldValue, +step.getValue(), +fieldAndValueMap +
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
jeqo commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1568385072 ## connect/transforms/src/main/java/org/apache/kafka/connect/transforms/field/MultiFieldPaths.java: ## @@ -0,0 +1,581 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Schema.Type; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.transforms.util.SchemaUtil; + +import java.util.AbstractMap; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Multiple field paths to access data objects ({@code Struct} or {@code Map}) efficiently, + * instead of multiple individual {@link SingleFieldPath single-field paths}. + * + * If the SMT requires accessing a single field on the same data object, + * use {@link SingleFieldPath} instead. + * + * @see https://cwiki.apache.org/confluence/display/KAFKA/KIP-821%3A+Connect+Transforms+support+for+nested+structures";>KIP-821 + * @see SingleFieldPath + * @see FieldSyntaxVersion + */ +public class MultiFieldPaths { +final Trie trie = new Trie(); + +MultiFieldPaths(Set paths) { +paths.forEach(trie::insert); +} + +public static MultiFieldPaths of(List fields, FieldSyntaxVersion syntaxVersion) { +return new MultiFieldPaths(fields.stream() +.map(f -> new SingleFieldPath(f, syntaxVersion)) +.collect(Collectors.toSet())); +} + +/** + * Find values at the field paths + * + * @param struct data value + * @return map of field paths and field/values + */ +public Map> fieldAndValuesFrom(Struct struct) { +if (trie.isEmpty()) return Collections.emptyMap(); +return findFieldAndValues(struct, trie.root, new HashMap<>()); +} + +private Map> findFieldAndValues( +Struct originalValue, +TrieNode trieAt, +Map> fieldAndValueMap +) { +for (Map.Entry step : trieAt.steps().entrySet()) { +Field field = originalValue.schema().field(step.getKey()); +if (step.getValue().isLeaf()) { +Map.Entry fieldAndValue = +field != null +? new AbstractMap.SimpleImmutableEntry<>(field, originalValue.get(field)) +: null; +fieldAndValueMap.put(step.getValue().path, fieldAndValue); +} else { +if (field.schema().type() == Type.STRUCT) { +findFieldAndValues( +originalValue.getStruct(field.name()), +step.getValue(), +fieldAndValueMap +); +} +} +} +return fieldAndValueMap; +} + +/** + * Find values at the field paths + * + * @param value data value + * @return map of field paths and field/values + */ +public Map> fieldAndValuesFrom(Map value) { +if (trie.isEmpty()) return Collections.emptyMap(); +return findFieldAndValues(value, trie.root, new HashMap<>()); +} + +@SuppressWarnings("unchecked") +private Map> findFieldAndValues( +Map value, +TrieNode trieAt, +Map> fieldAndValueMap +) { +for (Map.Entry step : trieAt.steps().entrySet()) { +Object fieldValue = value.get(step.getKey()); +if (step.getValue().isLeaf()) { +fieldAndValueMap.put( +step.getValue().path, +new AbstractMap.SimpleImmutableEntry<>(step.getKey(), fieldValue) +); +} else { +if (fieldValue instanceof Map) { +findFieldAndValues( +(Map) fieldValue, +step.getValue(), +fieldAndValueMap +
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
C0urante commented on code in PR #15379: URL: https://github.com/apache/kafka/pull/15379#discussion_r1556834149 ## connect/transforms/src/main/java/org/apache/kafka/connect/transforms/field/MultiFieldPaths.java: ## @@ -0,0 +1,581 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.field; + +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Schema.Type; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.transforms.util.SchemaUtil; + +import java.util.AbstractMap; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Multiple field paths to access data objects ({@code Struct} or {@code Map}) efficiently, + * instead of multiple individual {@link SingleFieldPath single-field paths}. + * + * If the SMT requires accessing a single field on the same data object, + * use {@link SingleFieldPath} instead. + * + * @see https://cwiki.apache.org/confluence/display/KAFKA/KIP-821%3A+Connect+Transforms+support+for+nested+structures";>KIP-821 + * @see SingleFieldPath + * @see FieldSyntaxVersion + */ +public class MultiFieldPaths { +final Trie trie = new Trie(); + +MultiFieldPaths(Set paths) { +paths.forEach(trie::insert); +} + +public static MultiFieldPaths of(List fields, FieldSyntaxVersion syntaxVersion) { +return new MultiFieldPaths(fields.stream() +.map(f -> new SingleFieldPath(f, syntaxVersion)) +.collect(Collectors.toSet())); +} + +/** + * Find values at the field paths + * + * @param struct data value + * @return map of field paths and field/values + */ +public Map> fieldAndValuesFrom(Struct struct) { +if (trie.isEmpty()) return Collections.emptyMap(); +return findFieldAndValues(struct, trie.root, new HashMap<>()); +} + +private Map> findFieldAndValues( +Struct originalValue, +TrieNode trieAt, +Map> fieldAndValueMap +) { +for (Map.Entry step : trieAt.steps().entrySet()) { +Field field = originalValue.schema().field(step.getKey()); +if (step.getValue().isLeaf()) { +Map.Entry fieldAndValue = +field != null +? new AbstractMap.SimpleImmutableEntry<>(field, originalValue.get(field)) +: null; +fieldAndValueMap.put(step.getValue().path, fieldAndValue); +} else { +if (field.schema().type() == Type.STRUCT) { +findFieldAndValues( +originalValue.getStruct(field.name()), +step.getValue(), +fieldAndValueMap +); +} +} +} +return fieldAndValueMap; +} + +/** + * Find values at the field paths + * + * @param value data value + * @return map of field paths and field/values + */ +public Map> fieldAndValuesFrom(Map value) { +if (trie.isEmpty()) return Collections.emptyMap(); +return findFieldAndValues(value, trie.root, new HashMap<>()); +} + +@SuppressWarnings("unchecked") +private Map> findFieldAndValues( +Map value, +TrieNode trieAt, +Map> fieldAndValueMap +) { +for (Map.Entry step : trieAt.steps().entrySet()) { +Object fieldValue = value.get(step.getKey()); +if (step.getValue().isLeaf()) { +fieldAndValueMap.put( +step.getValue().path, +new AbstractMap.SimpleImmutableEntry<>(step.getKey(), fieldValue) +); +} else { +if (fieldValue instanceof Map) { +findFieldAndValues( +(Map) fieldValue, +step.getValue(), +fieldAndValueMap +
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
C0urante commented on PR #15379: URL: https://github.com/apache/kafka/pull/15379#issuecomment-2044091861 I'm also wondering if it's necessary to have the `SingleFieldPath` class at all. Would it be significantly more expensive to just use the `MultiFieldPaths` class for everything for now? It'd reduce the complexity of the implementation and probably make review easier. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
C0urante commented on PR #15379: URL: https://github.com/apache/kafka/pull/15379#issuecomment-2044089149 @jeqo This is difficult to review without seeing how this code is actually used. Maybe we could move incrementally and introduce a commit that only touches on 1-3 SMTs, and only introduces the internal changes (i.e., methods in the `SingleFieldPath` and `MultiFieldPaths` classes) necessary in order to touch on those SMTs? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org
Re: [PR] KAFKA-14226: feat(connect:transform): Introduce FieldPath abstraction [kafka]
jeqo commented on PR #15379: URL: https://github.com/apache/kafka/pull/15379#issuecomment-1947314563 cc @C0urante @yashmayya @mimaison -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org