Nflrijal commented on code in PR #27353:
URL: https://github.com/apache/flink/pull/27353#discussion_r2669696250


##########
flink-core/src/main/java/org/apache/flink/api/common/serialization/RowFieldExtractorSchema.java:
##########
@@ -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> {

Review Comment:
   I thought of  keeping it in `flink-core` for a few reasons:
   
   1. 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.
   
   2. Consistency: Other serialization schemas like `SimpleStringSchema` and 
`ByteArraySchema` are in `flink-core`, so this fits the existing pattern.
   
   3. Python wrapper already exists: The PyFlink wrapper in 
`flink-python/pyflink/common/serialization.py` already makes it easily 
accessible to Python users, and they can import it naturally via `from 
pyflink.common.serialization import RowFieldExtractorSchema`.
   
   4. Separation of concerns: `flink-core` handles Java serialization logic, 
`flink-python` handles Python-Java bridging.
   
   However, I'm happy to move it to `flink-python` if you think that better 
serves the community! What do you think?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to