-
Notifications
You must be signed in to change notification settings - Fork 486
[lake/iceberg] support nested row types for Iceberg tiering #2278
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
48a3455
3bf4b79
c5b4491
ac215c8
10fb75c
98575d8
ae3a2b5
373035c
d954ee7
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 |
|---|---|---|
|
|
@@ -21,6 +21,7 @@ | |
| import org.apache.fluss.metadata.TablePath; | ||
| import org.apache.fluss.row.GenericRow; | ||
| import org.apache.fluss.row.InternalRow; | ||
| import org.apache.fluss.types.DataField; | ||
| import org.apache.fluss.types.DataType; | ||
| import org.apache.fluss.types.DataTypes; | ||
| import org.apache.fluss.types.RowType; | ||
|
|
@@ -38,6 +39,7 @@ | |
|
|
||
| import javax.annotation.Nullable; | ||
|
|
||
| import java.util.ArrayList; | ||
| import java.util.List; | ||
|
|
||
| import static org.apache.fluss.metadata.ResolvedPartitionSpec.PARTITION_SPEC_SEPARATOR; | ||
|
|
@@ -128,7 +130,16 @@ private static DataType convertIcebergTypeToFlussType(Type icebergType) { | |
| } else if (icebergType instanceof Types.ListType) { | ||
| Types.ListType listType = (Types.ListType) icebergType; | ||
| return DataTypes.ARRAY(convertIcebergTypeToFlussType(listType.elementType())); | ||
| } else if (icebergType.isStructType()) { | ||
| Types.StructType structType = icebergType.asStructType(); | ||
| List<DataField> fields = new ArrayList<>(); | ||
| for (Types.NestedField nestedField : structType.fields()) { | ||
| DataType fieldType = convertIcebergTypeToFlussType(nestedField.type()); | ||
| fields.add(new DataField(nestedField.name(), fieldType)); | ||
| } | ||
| return DataTypes.ROW(fields.toArray(new DataField[0])); | ||
| } | ||
|
Comment on lines
+133
to
141
|
||
|
|
||
| throw new UnsupportedOperationException( | ||
| "Unsupported data type conversion for Iceberg type: " | ||
| + icebergType.getClass().getName()); | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.