-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-38189][core][python] Add RowFieldExtractorSchema for Row field serialization #27353
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
dianfu
merged 3 commits into
apache:master
from
Nflrijal:flink-38189-custom-field-schema
Jan 9, 2026
+584
−2
Merged
Changes from all commits
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
122 changes: 122 additions & 0 deletions
122
...core/src/main/java/org/apache/flink/api/common/serialization/RowFieldExtractorSchema.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,122 @@ | ||
| /* | ||
| * 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.flink.api.common.serialization; | ||
|
|
||
| import org.apache.flink.annotation.PublicEvolving; | ||
| import org.apache.flink.annotation.VisibleForTesting; | ||
| import org.apache.flink.types.Row; | ||
|
|
||
| import javax.annotation.Nullable; | ||
|
|
||
| import static org.apache.flink.util.Preconditions.checkArgument; | ||
|
|
||
| /** | ||
| * Serialization schema that extracts a specific field from a {@link Row} and returns it as a byte | ||
| * array. | ||
| * | ||
| * <p>The field is required to be of type {@code byte[]}. This schema is particularly useful when | ||
| * using Flink with Kafka, where you may want to use one Row field as the message key and another as | ||
| * the value and perform the conversion to bytes explicitly in user code. | ||
| * | ||
| * <p>Example usage with Kafka: | ||
| * | ||
| * <pre>{@code | ||
| * KafkaSink<Row> sink = KafkaSink.<Row>builder() | ||
| * .setBootstrapServers(bootstrapServers) | ||
| * .setRecordSerializer( | ||
| * KafkaRecordSerializationSchema.builder() | ||
| * .setTopic("my-topic") | ||
| * .setKeySerializationSchema(new RowFieldExtractorSchema(0)) // field 0 as key | ||
| * .setValueSerializationSchema(new RowFieldExtractorSchema(1)) // field 1 as value | ||
| * .build()) | ||
| * .build(); | ||
| * }</pre> | ||
| */ | ||
| @PublicEvolving | ||
| public class RowFieldExtractorSchema implements SerializationSchema<Row> { | ||
|
|
||
| private static final long serialVersionUID = 1L; | ||
|
|
||
| /** The index of the field to extract from the Row. */ | ||
| private final int fieldIndex; | ||
|
|
||
| /** | ||
| * Creates a new RowFieldExtractorSchema that extracts the field at the specified index. | ||
| * | ||
| * @param fieldIndex the zero-based index of the field to extract | ||
| * @throws IllegalArgumentException if fieldIndex is negative | ||
| */ | ||
| public RowFieldExtractorSchema(int fieldIndex) { | ||
| checkArgument(fieldIndex >= 0, "Field index must be non-negative, got: %s", fieldIndex); | ||
| this.fieldIndex = fieldIndex; | ||
| } | ||
|
|
||
| /** | ||
| * Gets the field index being extracted. | ||
| * | ||
| * @return the field index | ||
| */ | ||
| @VisibleForTesting | ||
| public int getFieldIndex() { | ||
| return fieldIndex; | ||
| } | ||
|
|
||
| @Override | ||
| public byte[] serialize(@Nullable Row element) { | ||
| if (element == null) { | ||
| return new byte[0]; | ||
| } | ||
|
|
||
| checkArgument( | ||
| fieldIndex < element.getArity(), | ||
| "Cannot access field %s in Row with arity %s", | ||
| fieldIndex, | ||
| element.getArity()); | ||
|
|
||
| Object field = element.getField(fieldIndex); | ||
| if (field == null) { | ||
| return new byte[0]; | ||
| } | ||
|
|
||
| if (!(field instanceof byte[])) { | ||
| throw new IllegalArgumentException( | ||
| String.format( | ||
| "Field at index %s must be of type byte[], but was %s", | ||
| fieldIndex, field.getClass().getName())); | ||
| } | ||
|
|
||
| return (byte[]) field; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean equals(Object o) { | ||
| if (this == o) { | ||
| return true; | ||
| } | ||
| if (o == null || getClass() != o.getClass()) { | ||
| return false; | ||
| } | ||
| RowFieldExtractorSchema that = (RowFieldExtractorSchema) o; | ||
| return fieldIndex == that.fieldIndex; | ||
| } | ||
|
|
||
| @Override | ||
| public int hashCode() { | ||
| return fieldIndex; | ||
| } | ||
| } | ||
119 changes: 119 additions & 0 deletions
119
.../src/test/java/org/apache/flink/api/common/serialization/RowFieldExtractorSchemaTest.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,119 @@ | ||
| /* | ||
| * 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.flink.api.common.serialization; | ||
|
|
||
| import org.apache.flink.types.Row; | ||
| import org.apache.flink.util.InstantiationUtil; | ||
|
|
||
| import org.junit.jupiter.api.Test; | ||
|
|
||
| import java.io.IOException; | ||
| import java.nio.charset.StandardCharsets; | ||
|
|
||
| import static org.assertj.core.api.Assertions.assertThat; | ||
| import static org.assertj.core.api.Assertions.assertThatThrownBy; | ||
|
|
||
| /** Tests for {@link RowFieldExtractorSchema}. */ | ||
| class RowFieldExtractorSchemaTest { | ||
|
|
||
| @Test | ||
| void testSerializeByteArrayField() { | ||
| RowFieldExtractorSchema schema = new RowFieldExtractorSchema(0); | ||
| byte[] value = "test-value".getBytes(StandardCharsets.UTF_8); | ||
| Row row = Row.of(value, 123); | ||
|
|
||
| byte[] result = schema.serialize(row); | ||
|
|
||
| assertThat(result).isEqualTo(value); | ||
| } | ||
|
|
||
| @Test | ||
| void testSerializeNonByteArrayFieldThrowsException() { | ||
| RowFieldExtractorSchema schema = new RowFieldExtractorSchema(1); | ||
| Row row = Row.of("key", 42); // field 1 is Integer, not byte[] | ||
|
|
||
| assertThatThrownBy(() -> schema.serialize(row)) | ||
| .isInstanceOf(IllegalArgumentException.class) | ||
| .hasMessageContaining("must be of type byte[]"); | ||
| } | ||
|
|
||
| @Test | ||
| void testSerializeNullRow() { | ||
| RowFieldExtractorSchema schema = new RowFieldExtractorSchema(0); | ||
|
|
||
| byte[] result = schema.serialize(null); | ||
|
|
||
| assertThat(result).isEmpty(); | ||
| } | ||
|
|
||
| @Test | ||
| void testSerializeNullField() { | ||
| RowFieldExtractorSchema schema = new RowFieldExtractorSchema(0); | ||
| Row row = Row.of(null, "value"); | ||
|
|
||
| byte[] result = schema.serialize(row); | ||
|
|
||
| assertThat(result).isEmpty(); | ||
| } | ||
|
|
||
| @Test | ||
| void testSerializeOutOfBoundsIndex() { | ||
| RowFieldExtractorSchema schema = new RowFieldExtractorSchema(5); | ||
| Row row = Row.of("field0", "field1"); | ||
|
|
||
| assertThatThrownBy(() -> schema.serialize(row)) | ||
| .isInstanceOf(IllegalArgumentException.class) | ||
| .hasMessageContaining("Cannot access field 5 in Row with arity 2"); | ||
| } | ||
|
|
||
| @Test | ||
| void testNegativeFieldIndexThrowsException() { | ||
| assertThatThrownBy(() -> new RowFieldExtractorSchema(-1)) | ||
| .isInstanceOf(IllegalArgumentException.class) | ||
| .hasMessageContaining("Field index must be non-negative"); | ||
| } | ||
|
|
||
| @Test | ||
| void testSerializability() throws IOException, ClassNotFoundException { | ||
| RowFieldExtractorSchema schema = new RowFieldExtractorSchema(3); | ||
|
|
||
| RowFieldExtractorSchema deserialized = | ||
| InstantiationUtil.deserializeObject( | ||
| InstantiationUtil.serializeObject(schema), getClass().getClassLoader()); | ||
|
|
||
| assertThat(deserialized.getFieldIndex()).isEqualTo(3); | ||
| } | ||
|
|
||
| @Test | ||
| void testEquals() { | ||
| RowFieldExtractorSchema schema1 = new RowFieldExtractorSchema(1); | ||
| RowFieldExtractorSchema schema2 = new RowFieldExtractorSchema(1); | ||
| RowFieldExtractorSchema schema3 = new RowFieldExtractorSchema(2); | ||
|
|
||
| assertThat(schema1).isEqualTo(schema2); | ||
| assertThat(schema1).isNotEqualTo(schema3); | ||
| } | ||
|
|
||
| @Test | ||
| void testHashCode() { | ||
| RowFieldExtractorSchema schema1 = new RowFieldExtractorSchema(1); | ||
| RowFieldExtractorSchema schema2 = new RowFieldExtractorSchema(1); | ||
|
|
||
| assertThat(schema1.hashCode()).isEqualTo(schema2.hashCode()); | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What about moving this class to module
flink-python? This class should be more useful for Python users.Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I thought of keeping it in
flink-corefor a few reasons:General-purpose functionality: While Python users will benefit from this, Java/Scala users can also use it for Kafka or other sinks that need field extraction.
Consistency: Other serialization schemas like
SimpleStringSchemaandByteArraySchemaare inflink-core, so this fits the existing pattern.Python wrapper already exists: The PyFlink wrapper in
flink-python/pyflink/common/serialization.pyalready makes it easily accessible to Python users, and they can import it naturally viafrom pyflink.common.serialization import RowFieldExtractorSchema.Separation of concerns:
flink-corehandles Java serialization logic,flink-pythonhandles Python-Java bridging.However, I'm happy to move it to
flink-pythonif you think that better serves the community! What do you think @dianfu