-
Notifications
You must be signed in to change notification settings - Fork 2.5k
feat(lance): Add VariantType support to Lance base files #18599
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
base: master
Are you sure you want to change the base?
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,7 @@ | |
|
|
||
| package org.apache.hudi.io.storage; | ||
|
|
||
| import org.apache.hudi.SparkAdapterSupport$; | ||
| import org.apache.hudi.common.bloom.BloomFilter; | ||
| import org.apache.hudi.common.config.HoodieStorageConfig; | ||
| import org.apache.hudi.common.engine.TaskContextSupplier; | ||
|
|
@@ -26,6 +27,7 @@ | |
| import org.apache.hudi.common.schema.HoodieSchema; | ||
| import org.apache.hudi.common.schema.HoodieSchemaType; | ||
| import org.apache.hudi.common.util.Option; | ||
| import org.apache.hudi.common.util.collection.Pair; | ||
| import org.apache.hudi.exception.HoodieNotSupportedException; | ||
| import org.apache.hudi.io.lance.HoodieBaseLanceWriter; | ||
| import org.apache.hudi.io.storage.row.HoodieBloomFilterRowWriteSupport; | ||
|
|
@@ -34,16 +36,18 @@ | |
| import org.apache.hudi.storage.StoragePath; | ||
|
|
||
| import org.lance.spark.arrow.LanceArrowWriter; | ||
| import lombok.AllArgsConstructor; | ||
| import lombok.Builder; | ||
| import org.apache.arrow.vector.VectorSchemaRoot; | ||
| import org.apache.arrow.vector.types.pojo.ArrowType; | ||
| import org.apache.arrow.vector.types.pojo.Field; | ||
| import org.apache.arrow.vector.types.pojo.FieldType; | ||
| import org.apache.arrow.vector.types.pojo.Schema; | ||
| import org.apache.spark.sql.catalyst.InternalRow; | ||
| import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; | ||
| import org.apache.spark.sql.catalyst.expressions.SpecializedGetters; | ||
| import org.apache.spark.sql.types.ArrayType; | ||
| import org.apache.spark.sql.types.DataType; | ||
| import org.apache.spark.sql.types.DataTypes; | ||
| import org.apache.spark.sql.types.MapType; | ||
| import org.apache.spark.sql.types.Metadata; | ||
| import org.apache.spark.sql.types.MetadataBuilder; | ||
|
|
@@ -58,6 +62,7 @@ | |
| import java.util.HashMap; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.function.BiConsumer; | ||
| import java.util.function.Function; | ||
|
|
||
| import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD; | ||
|
|
@@ -88,7 +93,11 @@ public class HoodieSparkLanceWriter extends HoodieBaseLanceWriter<InternalRow, U | |
| private static final String LANCE_BLOB_ENCODING_KEY = "lance-encoding:blob"; | ||
| private static final String LANCE_BLOB_ENCODING_VALUE = "true"; | ||
|
|
||
| private static final int[] EMPTY_INT_ARRAY = new int[0]; | ||
|
|
||
| private final StructType sparkSchema; | ||
| private final StructType inputSparkSchema; | ||
| private final int[] variantOrdinals; | ||
| private final Schema arrowSchema; | ||
| private final UTF8String fileName; | ||
| private final UTF8String instantTime; | ||
|
|
@@ -157,7 +166,10 @@ private HoodieSparkLanceWriter(StoragePath file, | |
| long flushByteWatermark) { | ||
| super(file, DEFAULT_BATCH_SIZE, allocatorSize, flushByteWatermark, | ||
| bloomFilterOpt.map(HoodieBloomFilterRowWriteSupport::new)); | ||
| this.sparkSchema = enrichSparkSchemaForLance(sparkSchema); | ||
| this.inputSparkSchema = sparkSchema; | ||
| Pair<StructType, int[]> variantPlan = enrichForLanceVariant(sparkSchema); | ||
|
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. |
||
| this.variantOrdinals = variantPlan.getRight(); | ||
| this.sparkSchema = enrichSparkSchemaForLance(variantPlan.getLeft()); | ||
| Schema baseArrow = LanceArrowUtils.toArrowSchema(this.sparkSchema, DEFAULT_TIMEZONE, true); | ||
| // Force LargeBinary + `lance-encoding:blob=true` on each BLOB's nested `data` Arrow leaf. | ||
| // Can't be expressed Spark-side: toArrowSchema drops nested-field metadata, and tagging | ||
|
|
@@ -293,6 +305,54 @@ private static Field rewriteBlobDataChild(Field blobStructField) { | |
| return new Field(blobStructField.getName(), blobStructField.getFieldType(), rebuilt); | ||
| } | ||
|
|
||
| /** | ||
| * Single-pass walk that returns (a) the enriched schema with top-level | ||
| * {@code VariantType} fields replaced by Hudi's canonical | ||
| * {@code Struct[metadata: binary, value: binary]} (tagged {@code hudi_type=VARIANT} | ||
| * so {@code HoodieSparkSchemaConverters} promotes it back on read), and (b) the | ||
| * variant ordinals in ascending order. {@code LanceArrowUtils} has no VariantType | ||
| * case, so we hand it a plain struct. Top-level only - nested variants are not | ||
| * yet supported. | ||
| */ | ||
| private static Pair<StructType, int[]> enrichForLanceVariant(StructType sparkSchema) { | ||
|
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 sibling method is - AI-generated; verify before applying. React 👍/👎 to flag quality.
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. Could |
||
| StructField[] fields = sparkSchema.fields(); | ||
| StructField[] newFields = null; | ||
| List<Integer> ordinals = null; | ||
|
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. 🤖 - AI-generated; verify before applying. React 👍/👎 to flag quality. |
||
| for (int i = 0; i < fields.length; i++) { | ||
| StructField field = fields[i]; | ||
| if (!SparkAdapterSupport$.MODULE$.sparkAdapter().isVariantType(field.dataType())) { | ||
| if (newFields != null) { | ||
| newFields[i] = field; | ||
| } | ||
| continue; | ||
| } | ||
| if (newFields == null) { | ||
| newFields = new StructField[fields.length]; | ||
| System.arraycopy(fields, 0, newFields, 0, i); | ||
| ordinals = new ArrayList<>(); | ||
| } | ||
| ordinals.add(i); | ||
| StructField metaField = new StructField( | ||
| HoodieSchema.Variant.VARIANT_METADATA_FIELD, DataTypes.BinaryType, false, Metadata.empty()); | ||
| StructField valField = new StructField( | ||
| HoodieSchema.Variant.VARIANT_VALUE_FIELD, DataTypes.BinaryType, false, Metadata.empty()); | ||
| StructType variantStruct = new StructType(new StructField[] {metaField, valField}); | ||
| Metadata enriched = new MetadataBuilder() | ||
| .withMetadata(field.metadata()) | ||
| .putString(HoodieSchema.TYPE_METADATA_FIELD, HoodieSchemaType.VARIANT.name()) | ||
| .build(); | ||
| newFields[i] = new StructField(field.name(), variantStruct, field.nullable(), enriched); | ||
| } | ||
| if (newFields == null) { | ||
|
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: could you replace this manual unboxing loop with - AI-generated; verify before applying. React 👍/👎 to flag quality.
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: could you replace the manual unboxing loop with - AI-generated; verify before applying. React 👍/👎 to flag quality. |
||
| return Pair.of(sparkSchema, EMPTY_INT_ARRAY); | ||
| } | ||
| int[] ordinalArr = new int[ordinals.size()]; | ||
|
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 three-line int[] conversion could be collapsed to - AI-generated; verify before applying. React 👍/👎 to flag quality. |
||
| for (int i = 0; i < ordinalArr.length; i++) { | ||
|
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: could you simplify this to - AI-generated; verify before applying. React 👍/👎 to flag quality. |
||
| ordinalArr[i] = ordinals.get(i); | ||
| } | ||
| return Pair.of(new StructType(newFields), ordinalArr); | ||
| } | ||
|
|
||
| @Override | ||
| public void writeRowWithMetadata(HoodieKey key, InternalRow row) throws IOException { | ||
| UTF8String recordKey = UTF8String.fromString(key.getRecordKey()); | ||
|
|
@@ -323,7 +383,8 @@ public void writeRow(InternalRow row) throws IOException { | |
|
|
||
| @Override | ||
| protected ArrowWriter<InternalRow> createArrowWriter(VectorSchemaRoot root) { | ||
| return SparkArrowWriter.of(LanceArrowWriter.create(root, sparkSchema)); | ||
| return SparkArrowWriter.of( | ||
| LanceArrowWriter.create(root, sparkSchema), inputSparkSchema, variantOrdinals); | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -399,13 +460,59 @@ protected void updateRecordMetadata(InternalRow row, | |
| row.update(FILENAME_METADATA_FIELD.ordinal(), fileName); | ||
| } | ||
|
|
||
| @AllArgsConstructor(staticName = "of") | ||
| private static class SparkArrowWriter implements ArrowWriter<InternalRow> { | ||
| /** | ||
| * Forwards rows to the lance-spark {@link LanceArrowWriter}. When the schema | ||
| * has no {@code VariantType} columns, rows are passed through directly. When it | ||
| * does, a single {@link VariantProjectedRow} instance is reused per row to | ||
| * delegate every accessor to the underlying input row except at variant | ||
| * ordinals, where it returns a pre-allocated {@code (metadata, value)} struct | ||
| * populated by {@link org.apache.spark.sql.hudi.SparkAdapter#createVariantValueWriter}. | ||
|
Comment on lines
+464
to
+469
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. Does this projection introduce overhead? Does Lance library or writer provide its own projection or adaptation for Variant Type? |
||
| */ | ||
| private static final class SparkArrowWriter implements ArrowWriter<InternalRow> { | ||
| private final LanceArrowWriter lanceArrowWriter; | ||
| private final VariantProjectedRow projectedRow; | ||
|
|
||
| private SparkArrowWriter(LanceArrowWriter lanceArrowWriter, VariantProjectedRow projectedRow) { | ||
| this.lanceArrowWriter = lanceArrowWriter; | ||
| this.projectedRow = projectedRow; | ||
| } | ||
|
|
||
| static SparkArrowWriter of(LanceArrowWriter lanceArrowWriter, | ||
| StructType inputSchema, | ||
| int[] variantOrdinals) { | ||
| if (variantOrdinals.length == 0) { | ||
| return new SparkArrowWriter(lanceArrowWriter, null); | ||
| } | ||
|
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 - AI-generated; verify before applying. React 👍/👎 to flag quality.
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 two-step init ( - AI-generated; verify before applying. React 👍/👎 to flag quality. |
||
| int numFields = inputSchema.fields().length; | ||
| GenericInternalRow[] structByOrdinal = new GenericInternalRow[numFields]; | ||
| List<BiConsumer<SpecializedGetters, Integer>> extractorByOrdinal = new ArrayList<>(numFields); | ||
|
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 null-fill loop on the next few lines could be replaced by - AI-generated; verify before applying. React 👍/👎 to flag quality. |
||
| for (int i = 0; i < numFields; i++) { | ||
| extractorByOrdinal.add(null); | ||
| } | ||
| for (int o : variantOrdinals) { | ||
| // Pre-allocated per variant column; slot 0 = metadata, slot 1 = value | ||
| // (parquet-variant spec on-disk order). lance-spark's BinaryWriter copies | ||
| // bytes synchronously via VarBinaryVector.setSafe, so the byte[] returned by | ||
| // VariantVal.getMetadata/getValue does not need to outlive a single write(). | ||
| GenericInternalRow struct = new GenericInternalRow(new Object[2]); | ||
| structByOrdinal[o] = struct; | ||
| extractorByOrdinal.set(o, SparkAdapterSupport$.MODULE$.sparkAdapter().createVariantValueWriter( | ||
| inputSchema.fields()[o].dataType(), | ||
| (byte[] v) -> struct.update(1, v), | ||
| (byte[] m) -> struct.update(0, m))); | ||
| } | ||
| VariantProjectedRow projectedRow = SparkAdapterSupport$.MODULE$.sparkAdapter() | ||
| .createVariantProjectedRow(numFields, structByOrdinal, extractorByOrdinal); | ||
| return new SparkArrowWriter(lanceArrowWriter, projectedRow); | ||
| } | ||
|
|
||
| @Override | ||
| public void write(InternalRow row) { | ||
| lanceArrowWriter.write(row); | ||
| if (projectedRow == null) { | ||
| lanceArrowWriter.write(row); | ||
| return; | ||
| } | ||
| lanceArrowWriter.write(projectedRow.wrap(row)); | ||
| } | ||
|
|
||
| @Override | ||
|
|
||
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.
🤖 nit:
variantPlanreads like aLogicalPlanorQueryPlanin the Spark context — could you rename it to something likeschemaAndOrdinalsorvariantEnrichmentto make it clear this is just a(StructType, int[])pair?- AI-generated; verify before applying. React 👍/👎 to flag quality.