Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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 java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;

import static org.apache.flink.util.Preconditions.checkArgument;

/**
* Serialization schema that extracts a specific field from a {@link Row} and serializes it as a
* UTF-8 encoded byte array.
*
* <p>This schema is particularly useful when using Flink with Kafka, where you may want to use a
* specific field as the message key for partition routing.
*
* <p>By default, the serializer uses "UTF-8" for string/byte conversion.
*
* <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)) // Use field 0 as key
* .setValueSerializationSchema(new RowFieldExtractorSchema(1)) // Use field 1 as value
* .build())
* .build();
* }</pre>
*/
@PublicEvolving
public class RowFieldExtractorSchema implements SerializationSchema<Row> {

private static final long serialVersionUID = 1L;

/** The charset to use for string/byte conversion. */
private static final Charset CHARSET = StandardCharsets.UTF_8;

/** 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];
}

return field.toString().getBytes(CHARSET);
}

@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;
}
}
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 testSerializeStringField() {
RowFieldExtractorSchema schema = new RowFieldExtractorSchema(0);
Row row = Row.of("test-value", 123);

byte[] result = schema.serialize(row);

assertThat(new String(result, StandardCharsets.UTF_8)).isEqualTo("test-value");
}

@Test
void testSerializeIntegerField() {
RowFieldExtractorSchema schema = new RowFieldExtractorSchema(1);
Row row = Row.of("key", 42);

byte[] result = schema.serialize(row);

assertThat(new String(result, StandardCharsets.UTF_8)).isEqualTo("42");
}

@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) // Changed from IndexOutOfBoundsException
.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());
}
}
47 changes: 46 additions & 1 deletion flink-python/pyflink/common/serialization.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,8 @@
'SimpleStringSchema',
'ByteArraySchema',
'Encoder',
'BulkWriterFactory'
'BulkWriterFactory',
'RowFieldExtractorSchema',
]


Expand Down Expand Up @@ -126,3 +127,47 @@ def __init__(self, j_bulk_writer_factory, row_type):

def get_row_type(self):
return self._row_type


class RowFieldExtractorSchema(SerializationSchema):
"""
Serialization schema that extracts a specific field from a Row and serializes it as a
UTF-8 encoded byte array.

This schema is particularly useful when using Flink with Kafka, where you may want to use a
specific field as the message key for partition routing.

Example usage with Kafka:
>>> from pyflink.common.serialization import RowFieldExtractorSchema
>>> from pyflink.datastream.connectors.kafka import KafkaSink, \
KafkaRecordSerializationSchema
>>> sink = KafkaSink.builder() \\
... .set_bootstrap_servers("localhost:9092") \\
... .set_record_serializer(
... KafkaRecordSerializationSchema.builder()
... .set_topic("my-topic")
... .set_key_serialization_schema(RowFieldExtractorSchema(0))
# Field 0 as key
... .set_value_serialization_schema(RowFieldExtractorSchema(1))
# Field 1 as value
... .build()
... ) \\
... .build()

:param field_index: The zero-based index of the field to extract from the Row.
"""

def __init__(self, field_index: int):
"""
Creates a new RowFieldExtractorSchema that extracts the field at the specified index.

:param field_index: The zero-based index of the field to extract (must be non-negative).
:raises ValueError: If field_index is negative.
"""
if field_index < 0:
raise ValueError(f"Field index must be non-negative, got: {field_index}")
gateway = get_gateway()
j_row_field_extractor_schema = gateway.jvm.org.apache.flink.api.common.serialization \
.RowFieldExtractorSchema(field_index)
super(RowFieldExtractorSchema, self).__init__(
j_serialization_schema=j_row_field_extractor_schema)
Loading