-
Notifications
You must be signed in to change notification settings - Fork 3k
Spark: Support metadata columns in 3.2 #3373
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 |
|---|---|---|
| @@ -0,0 +1,51 @@ | ||
| /* | ||
| * 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.iceberg.spark.source; | ||
|
|
||
| import org.apache.spark.sql.connector.catalog.MetadataColumn; | ||
| import org.apache.spark.sql.types.DataType; | ||
|
|
||
| public class SparkMetadataColumn implements MetadataColumn { | ||
|
|
||
| private final String name; | ||
| private final DataType dataType; | ||
| private final boolean isNullable; | ||
|
|
||
| public SparkMetadataColumn(String name, DataType dataType, boolean isNullable) { | ||
| this.name = name; | ||
| this.dataType = dataType; | ||
| this.isNullable = isNullable; | ||
| } | ||
|
|
||
| @Override | ||
| public String name() { | ||
| return name; | ||
| } | ||
|
|
||
| @Override | ||
| public DataType dataType() { | ||
| return dataType; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean isNullable() { | ||
| return isNullable; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,7 +24,9 @@ | |
| import java.util.Set; | ||
| import org.apache.iceberg.DataFile; | ||
| import org.apache.iceberg.FileScanTask; | ||
| import org.apache.iceberg.MetadataColumns; | ||
| import org.apache.iceberg.PartitionSpec; | ||
| import org.apache.iceberg.Partitioning; | ||
| import org.apache.iceberg.Table; | ||
| import org.apache.iceberg.TableProperties; | ||
| import org.apache.iceberg.TableScan; | ||
|
|
@@ -45,7 +47,9 @@ | |
| import org.apache.iceberg.spark.SparkSchemaUtil; | ||
| import org.apache.iceberg.spark.SparkWriteOptions; | ||
| import org.apache.spark.sql.SparkSession; | ||
| import org.apache.spark.sql.connector.catalog.MetadataColumn; | ||
| import org.apache.spark.sql.connector.catalog.SupportsDelete; | ||
| import org.apache.spark.sql.connector.catalog.SupportsMetadataColumns; | ||
| import org.apache.spark.sql.connector.catalog.SupportsRead; | ||
| import org.apache.spark.sql.connector.catalog.SupportsWrite; | ||
| import org.apache.spark.sql.connector.catalog.TableCapability; | ||
|
|
@@ -56,6 +60,8 @@ | |
| import org.apache.spark.sql.connector.write.LogicalWriteInfo; | ||
| import org.apache.spark.sql.connector.write.WriteBuilder; | ||
| import org.apache.spark.sql.sources.Filter; | ||
| import org.apache.spark.sql.types.DataType; | ||
| import org.apache.spark.sql.types.DataTypes; | ||
| import org.apache.spark.sql.types.StructType; | ||
| import org.apache.spark.sql.util.CaseInsensitiveStringMap; | ||
| import org.slf4j.Logger; | ||
|
|
@@ -69,7 +75,7 @@ | |
| import static org.apache.iceberg.TableProperties.UPDATE_MODE_DEFAULT; | ||
|
|
||
| public class SparkTable implements org.apache.spark.sql.connector.catalog.Table, | ||
| SupportsRead, SupportsWrite, SupportsDelete, SupportsMerge { | ||
| SupportsRead, SupportsWrite, SupportsDelete, SupportsMerge, SupportsMetadataColumns { | ||
|
|
||
| private static final Logger LOG = LoggerFactory.getLogger(SparkTable.class); | ||
|
|
||
|
|
@@ -168,6 +174,17 @@ public Set<TableCapability> capabilities() { | |
| return CAPABILITIES; | ||
| } | ||
|
|
||
| @Override | ||
| public MetadataColumn[] metadataColumns() { | ||
| DataType sparkPartitionType = SparkSchemaUtil.convert(Partitioning.partitionType(table())); | ||
| return new MetadataColumn[] { | ||
| new SparkMetadataColumn(MetadataColumns.SPEC_ID.name(), DataTypes.IntegerType, false), | ||
| new SparkMetadataColumn(MetadataColumns.PARTITION_COLUMN_NAME, sparkPartitionType, true), | ||
|
Contributor
Author
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. Only the partition column is nullable (e.g. unpartitioned tables).
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. I like that we can project the partition. I've been meaning to add a way to project the individual partition fields, but this is probably way easier. |
||
| new SparkMetadataColumn(MetadataColumns.FILE_PATH.name(), DataTypes.StringType, false), | ||
| new SparkMetadataColumn(MetadataColumns.ROW_POSITION.name(), DataTypes.LongType, false) | ||
|
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. I am not too familiar with the metadata columns, but I see 5 currently defined in Iceberg's
Contributor
Author
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. I can match the order in I am not sure how useful
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.
|
||
| }; | ||
| } | ||
|
|
||
| @Override | ||
| public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { | ||
| if (options.containsKey(SparkReadOptions.FILE_SCAN_TASK_SET_ID)) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,6 +19,9 @@ | |
|
|
||
| package org.apache.iceberg.spark.source; | ||
|
|
||
| import org.apache.iceberg.catalog.Namespace; | ||
| import org.apache.iceberg.catalog.TableIdentifier; | ||
| import org.apache.iceberg.spark.Spark3Util; | ||
| import org.apache.iceberg.spark.SparkSessionCatalog; | ||
| import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; | ||
| import org.apache.spark.sql.connector.catalog.Identifier; | ||
|
|
@@ -30,14 +33,14 @@ public class TestSparkCatalog<T extends TableCatalog & SupportsNamespaces> exten | |
|
|
||
| @Override | ||
| public Table loadTable(Identifier ident) throws NoSuchTableException { | ||
| String[] parts = ident.name().split("\\$", 2); | ||
|
Contributor
Author
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. I removed the ugly workaround we had earlier. |
||
| if (parts.length == 2) { | ||
| TestTables.TestTable table = TestTables.load(parts[0]); | ||
| String[] metadataColumns = parts[1].split(","); | ||
| return new SparkTestTable(table, metadataColumns, false); | ||
| } else { | ||
| TestTables.TestTable table = TestTables.load(ident.name()); | ||
| return new SparkTestTable(table, null, false); | ||
| TableIdentifier tableIdentifier = Spark3Util.identifierToTableIdentifier(ident); | ||
| Namespace namespace = tableIdentifier.namespace(); | ||
|
|
||
| TestTables.TestTable table = TestTables.load(tableIdentifier.toString()); | ||
| if (table == null && namespace.equals(Namespace.of("default"))) { | ||
| table = TestTables.load(tableIdentifier.name()); | ||
| } | ||
|
|
||
| return new SparkTable(table, false); | ||
|
Comment on lines
+40
to
+44
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. If I guess since this is for testing it's not as much of a concern, but should we throw
Contributor
Author
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. Yeah, the way we use |
||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.
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.
As far as I know, Spark does not offer a utility for creating metadata columns similarly to
Expressions. That's why I had to implement it in Iceberg. We should probably move it to Spark.