-
Notifications
You must be signed in to change notification settings - Fork 2.5k
feat: add variant type adapter for Flink #18702
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,6 +18,8 @@ | |
|
|
||
| package org.apache.hudi.util; | ||
|
|
||
| import org.apache.hudi.adapter.DataTypeAdapter; | ||
|
|
||
| import lombok.AccessLevel; | ||
| import lombok.NoArgsConstructor; | ||
| import org.apache.avro.generic.GenericFixed; | ||
|
|
@@ -155,6 +157,9 @@ public static AvroToRowDataConverter createConverter(LogicalType type, boolean u | |
| case MULTISET: | ||
| return createMapConverter(type, utcTimezone); | ||
| default: | ||
| if (DataTypeAdapter.isVariantType(type)) { | ||
| return createVariantConverter(); | ||
| } | ||
| throw new UnsupportedOperationException("Unsupported type: " + type); | ||
| } | ||
| } | ||
|
|
@@ -212,6 +217,18 @@ private static AvroToRowDataConverter createMapConverter(LogicalType type, boole | |
| }; | ||
| } | ||
|
|
||
| /** | ||
| * Creates a converter for Flink 2.1+ VARIANT LogicalType. The converter receives an Avro | ||
| * GenericRecord carrying metadata/value binary fields and produces a Flink | ||
| * {@code BinaryVariant}. | ||
| */ | ||
| private static AvroToRowDataConverter createVariantConverter() { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 🤖 nit: - AI-generated; verify before applying. React 👍/👎 to flag quality. |
||
| return avroObject -> { | ||
| IndexedRecord record = (IndexedRecord) avroObject; | ||
| return DataTypeAdapter.createVariant(convertToBytes(record.get(1)), convertToBytes(record.get(0))); | ||
| }; | ||
| } | ||
|
|
||
| private static AvroToRowDataConverter createTimestampConverter(int precision, boolean utcTimezone) { | ||
| final ChronoUnit chronoUnit; | ||
| if (precision <= 3) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,6 +18,7 @@ | |
|
|
||
| package org.apache.hudi.util; | ||
|
|
||
| import org.apache.hudi.adapter.DataTypeAdapter; | ||
| import org.apache.hudi.common.schema.HoodieSchema; | ||
| import org.apache.hudi.common.schema.HoodieSchemaField; | ||
| import org.apache.hudi.common.schema.HoodieSchemaType; | ||
|
|
@@ -239,6 +240,10 @@ public Object convert(HoodieSchema schema, Object object) { | |
| break; | ||
| case RAW: | ||
| default: | ||
| if (DataTypeAdapter.isVariantType(type)) { | ||
| converter = createVariantConverter(); | ||
| break; | ||
| } | ||
| throw new UnsupportedOperationException("Unsupported type: " + type); | ||
| } | ||
|
|
||
|
|
@@ -357,5 +362,28 @@ public Object convert(HoodieSchema schema, Object object) { | |
| } | ||
| }; | ||
| } | ||
|
|
||
| /** | ||
| * Creates a converter for Flink 2.1+ VARIANT LogicalType. The converter receives a Flink | ||
| * {@code Variant} object at runtime and extracts the raw metadata/value byte arrays, | ||
| * then packs them into an Avro GenericRecord with the Variant schema. | ||
| * | ||
| * <p>No shredded-variant check is needed here: {@code HoodieSchemaConverter.convertVariant()} | ||
| * already rejects shredded variants before a Flink type or converter is ever constructed, | ||
| * and Flink 2.1 itself only supports unshredded variants (FLIP-521). | ||
| */ | ||
| private static RowDataToAvroConverter createVariantConverter() { | ||
| return new RowDataToAvroConverter() { | ||
| private static final long serialVersionUID = 1L; | ||
|
|
||
| @Override | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 🤖 nit: the string literals - AI-generated; verify before applying. React 👍/👎 to flag quality. |
||
| public Object convert(HoodieSchema schema, Object object) { | ||
| final GenericRecord record = new GenericData.Record(schema.toAvroSchema()); | ||
| record.put(HoodieSchema.Variant.VARIANT_METADATA_FIELD, ByteBuffer.wrap(DataTypeAdapter.getVariantMetadata(object))); | ||
| record.put(HoodieSchema.Variant.VARIANT_VALUE_FIELD, ByteBuffer.wrap(DataTypeAdapter.getVariantValue(object))); | ||
| return record; | ||
| } | ||
| }; | ||
| } | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,53 @@ | ||
| /* | ||
| * 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.hudi.adapter; | ||
|
|
||
| import org.apache.flink.table.data.RowData; | ||
| import org.apache.flink.table.types.DataType; | ||
| import org.apache.flink.table.types.logical.LogicalType; | ||
| import org.apache.flink.types.variant.Variant; | ||
|
|
||
| /** | ||
| * Adapter utils to provide {@code DataType} utilities. | ||
| */ | ||
| public class DataTypeAdapter { | ||
| public static Variant getVariant(RowData rowData, int pos) { | ||
| throw new UnsupportedOperationException("Variant is not supported yet."); | ||
| } | ||
|
|
||
| public static Object createVariant(byte[] value, byte[] metadata) { | ||
| throw new UnsupportedOperationException("Variant is not supported yet."); | ||
| } | ||
|
|
||
| public static boolean isVariantType(LogicalType logicalType) { | ||
| return false; | ||
| } | ||
|
|
||
| public static DataType createVariantType() { | ||
| throw new UnsupportedOperationException("Variant is not supported yet."); | ||
| } | ||
|
|
||
| public static byte[] getVariantMetadata(Object obj) { | ||
| throw new UnsupportedOperationException("Variant is not supported yet."); | ||
| } | ||
|
|
||
| public static byte[] getVariantValue(Object obj) { | ||
| throw new UnsupportedOperationException("Variant is not supported yet."); | ||
| } | ||
| } |
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.
Since the writer does not attach the variant logical annotation (todo in line 227), a Flink-written variant column will read back as a plain row and HoodieSchemaConverter.convertToSchema() will lose the VARIANT type?
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.
hudi/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieParquetReadSupport.scala
Lines 113 to 120 in 47bf4e4
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.
Yeah based on my understanding, in order to make sure a Flink-written variant field (or any variant field written without the parquet variant annotation — which is all current Hudi writes since the code has a TODO to add it like you pointed out) is re-read as variant in Flink, we would need a subsequent PR like #18539 that add HoodieSchema for detection or we need to actually attach the annotation at write time.
Maybe we can add a TODO comment here, just so that readers know this state of Flink variant integration (in case for some reason we don't fully "fix" this by 1.3)?
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.
yeah, add a TODO to add variant on writer side first.