From 4d334aa063d3b06d3654fb7e74ad1d4049e46f95 Mon Sep 17 00:00:00 2001 From: caican00 Date: Wed, 1 May 2024 22:46:56 +0800 Subject: [PATCH 01/18] [#2543] feat(spark-connector): support row-level operations to iceberg Table --- integration-test/build.gradle.kts | 4 + .../integration/test/spark/SparkCommonIT.java | 103 +++++++++++++ .../test/spark/hive/SparkHiveCatalogIT.java | 5 + .../spark/iceberg/SparkIcebergCatalogIT.java | 139 ++++++++++++++++++ .../test/util/spark/SparkTableInfo.java | 42 +++++- .../test/util/spark/SparkUtilIT.java | 8 +- .../spark/connector/ConnectorConstants.java | 1 + .../spark/connector/catalog/BaseCatalog.java | 42 ++++-- .../connector/hive/GravitinoHiveCatalog.java | 13 +- .../spark/connector/hive/SparkHiveTable.java | 52 ++++++- .../iceberg/GravitinoIcebergCatalog.java | 13 +- .../connector/iceberg/SparkIcebergTable.java | 67 ++++++--- .../plugin/GravitinoDriverPlugin.java | 21 ++- .../spark/connector/utils/ConnectorUtil.java | 26 ++++ .../SparkBaseTableHelper.java} | 68 ++------- .../connector/utils/TestConnectorUtil.java | 31 ++++ 16 files changed, 518 insertions(+), 117 deletions(-) create mode 100644 spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/utils/ConnectorUtil.java rename spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/{table/SparkBaseTable.java => utils/SparkBaseTableHelper.java} (67%) create mode 100644 spark-connector/spark-connector/src/test/java/com/datastrato/gravitino/spark/connector/utils/TestConnectorUtil.java diff --git a/integration-test/build.gradle.kts b/integration-test/build.gradle.kts index 384f8417b18..95ce862da68 100644 --- a/integration-test/build.gradle.kts +++ b/integration-test/build.gradle.kts @@ -13,6 +13,8 @@ plugins { val scalaVersion: String = project.properties["scalaVersion"] as? String ?: extra["defaultScalaVersion"].toString() val sparkVersion: String = libs.versions.spark.get() +val sparkMajorVersion: String = sparkVersion.substringBeforeLast(".") +val kyuubiVersion: String = libs.versions.kyuubi.get() val icebergVersion: String = libs.versions.iceberg.get() val scalaCollectionCompatVersion: String = libs.versions.scala.collection.compat.get() @@ -114,6 +116,8 @@ dependencies { exclude("io.dropwizard.metrics") exclude("org.rocksdb") } + testImplementation("org.apache.iceberg:iceberg-spark-runtime-${sparkMajorVersion}_$scalaVersion:$icebergVersion") + testImplementation("org.apache.kyuubi:kyuubi-spark-connector-hive_$scalaVersion:$kyuubiVersion") testImplementation(libs.okhttp3.loginterceptor) testImplementation(libs.postgresql.driver) diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/SparkCommonIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/SparkCommonIT.java index 9dab1b46839..498a245228f 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/SparkCommonIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/SparkCommonIT.java @@ -68,11 +68,39 @@ protected static String getDeleteSql(String tableName, String condition) { return String.format("DELETE FROM %s where %s", tableName, condition); } + private static String getUpdateTableSql(String tableName, String setClause, String whereClause) { + return String.format("UPDATE %s SET %s WHERE %s", tableName, setClause, whereClause); + } + + private static String getRowLevelUpdateTableSql( + String targetTableName, String selectClause, String sourceTableName, String onClause) { + return String.format( + "MERGE INTO %s " + + "USING (%s) %s " + + "ON %s " + + "WHEN MATCHED THEN UPDATE SET * " + + "WHEN NOT MATCHED THEN INSERT *", + targetTableName, selectClause, sourceTableName, onClause); + } + + private static String getRowLevelDeleteTableSql( + String targetTableName, String selectClause, String sourceTableName, String onClause) { + return String.format( + "MERGE INTO %s " + + "USING (%s) %s " + + "ON %s " + + "WHEN MATCHED THEN DELETE " + + "WHEN NOT MATCHED THEN INSERT *", + targetTableName, selectClause, sourceTableName, onClause); + } + // Whether supports [CLUSTERED BY col_name3 SORTED BY col_name INTO num_buckets BUCKETS] protected abstract boolean supportsSparkSQLClusteredBy(); protected abstract boolean supportsPartition(); + protected abstract boolean supportsDelete(); + // Use a custom database not the original default database because SparkCommonIT couldn't // read&write data to tables in default database. The main reason is default database location is // determined by `hive.metastore.warehouse.dir` in hive-site.xml which is local HDFS address @@ -702,6 +730,28 @@ void testTableOptions() { checkTableReadWrite(tableInfo); } + @Test + @EnabledIf("supportsDelete") + void testDeleteOperation() { + String tableName = "test_row_level_delete_table"; + dropTableIfExists(tableName); + createSimpleTable(tableName); + + SparkTableInfo table = getTableInfo(tableName); + checkTableColumns(tableName, getSimpleTableColumn(), table); + sql( + String.format( + "INSERT INTO %s VALUES (1, '1', 1),(2, '2', 2),(3, '3', 3),(4, '4', 4),(5, '5', 5)", + tableName)); + List queryResult1 = getTableData(tableName); + Assertions.assertEquals(5, queryResult1.size()); + Assertions.assertEquals("1,1,1;2,2,2;3,3,3;4,4,4;5,5,5", String.join(";", queryResult1)); + sql(getDeleteSql(tableName, "id <= 4")); + List queryResult2 = getTableData(tableName); + Assertions.assertEquals(1, queryResult2.size()); + Assertions.assertEquals("5,5,5", queryResult2.get(0)); + } + protected void checkTableReadWrite(SparkTableInfo table) { String name = table.getTableIdentifier(); boolean isPartitionTable = table.isPartitionTable(); @@ -760,6 +810,49 @@ protected String getExpectedTableData(SparkTableInfo table) { .collect(Collectors.joining(",")); } + protected void checkTableRowLevelUpdate(String tableName) { + writeToEmptyTableAndCheckData(tableName); + String updatedValues = "id = 6, name = '6', age = 6"; + sql(getUpdateTableSql(tableName, updatedValues, "id = 5")); + List queryResult = getQueryData(getSelectAllSqlWithOrder(tableName)); + Assertions.assertEquals(5, queryResult.size()); + Assertions.assertEquals("1,1,1;2,2,2;3,3,3;4,4,4;6,6,6", String.join(";", queryResult)); + } + + protected void checkTableRowLevelDelete(String tableName) { + writeToEmptyTableAndCheckData(tableName); + sql(getDeleteSql(tableName, "id <= 2")); + List queryResult = getQueryData(getSelectAllSqlWithOrder(tableName)); + Assertions.assertEquals(3, queryResult.size()); + Assertions.assertEquals("3,3,3;4,4,4;5,5,5", String.join(";", queryResult)); + } + + protected void checkTableDeleteByMergeInto(String tableName) { + writeToEmptyTableAndCheckData(tableName); + + String sourceTableName = "source_table"; + String selectClause = + "SELECT 1 AS id, '1' AS name, 1 AS age UNION ALL SELECT 6 AS id, '6' AS name, 6 AS age"; + String onClause = String.format("%s.id = %s.id", tableName, sourceTableName); + sql(getRowLevelDeleteTableSql(tableName, selectClause, sourceTableName, onClause)); + List queryResult = getQueryData(getSelectAllSqlWithOrder(tableName)); + Assertions.assertEquals(5, queryResult.size()); + Assertions.assertEquals("2,2,2;3,3,3;4,4,4;5,5,5;6,6,6", String.join(";", queryResult)); + } + + protected void checkTableUpdateByMergeInto(String tableName) { + writeToEmptyTableAndCheckData(tableName); + + String sourceTableName = "source_table"; + String selectClause = + "SELECT 1 AS id, '2' AS name, 2 AS age UNION ALL SELECT 6 AS id, '6' AS name, 6 AS age"; + String onClause = String.format("%s.id = %s.id", tableName, sourceTableName); + sql(getRowLevelUpdateTableSql(tableName, selectClause, sourceTableName, onClause)); + List queryResult = getQueryData(getSelectAllSqlWithOrder(tableName)); + Assertions.assertEquals(6, queryResult.size()); + Assertions.assertEquals("1,2,2;2,2,2;3,3,3;4,4,4;5,5,5;6,6,6", String.join(";", queryResult)); + } + protected String getCreateSimpleTableString(String tableName) { return getCreateSimpleTableString(tableName, false); } @@ -801,6 +894,16 @@ protected void checkTableColumns( .check(tableInfo); } + private void writeToEmptyTableAndCheckData(String tableName) { + sql( + String.format( + "INSERT INTO %s VALUES (1, '1', 1),(2, '2', 2),(3, '3', 3),(4, '4', 4),(5, '5', 5)", + tableName)); + List queryResult = getTableData(tableName); + Assertions.assertEquals(5, queryResult.size()); + Assertions.assertEquals("1,1,1;2,2,2;3,3,3;4,4,4;5,5,5", String.join(";", queryResult)); + } + // partition expression may contain "'", like a='s'/b=1 private String getPartitionExpression(SparkTableInfo table, String delimiter) { return table.getPartitionedColumns().stream() diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/hive/SparkHiveCatalogIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/hive/SparkHiveCatalogIT.java index 1f34c87c10f..f42e0332dd1 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/hive/SparkHiveCatalogIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/hive/SparkHiveCatalogIT.java @@ -55,6 +55,11 @@ protected boolean supportsPartition() { return true; } + @Override + protected boolean supportsDelete() { + return false; + } + @Test public void testCreateHiveFormatPartitionTable() { String tableName = "hive_partition_table"; diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java index b94d6eb5e17..f7da5564809 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java @@ -9,6 +9,7 @@ import com.datastrato.gravitino.integration.test.util.spark.SparkTableInfo; import com.datastrato.gravitino.integration.test.util.spark.SparkTableInfoChecker; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import java.io.File; import java.util.ArrayList; import java.util.Arrays; @@ -18,10 +19,13 @@ import java.util.Map; import java.util.stream.Collectors; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions; +import org.apache.spark.SparkConf; import org.apache.spark.sql.Column; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; @@ -30,13 +34,21 @@ import org.apache.spark.sql.connector.catalog.FunctionCatalog; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; +import org.apache.spark.sql.internal.StaticSQLConf; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructField; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.platform.commons.util.StringUtils; +import scala.Tuple3; public abstract class SparkIcebergCatalogIT extends SparkCommonIT { + private static final String ICEBERG_FORMAT_VERSION = "format-version"; + private static final String ICEBERG_DELETE_MODE = "write.delete.mode"; + private static final String ICEBERG_UPDATE_MODE = "write.update.mode"; + private static final String ICEBERG_MERGE_MODE = "write.merge.mode"; + @Override protected String getCatalogName() { return "iceberg"; @@ -57,6 +69,11 @@ protected boolean supportsPartition() { return true; } + @Override + protected boolean supportsDelete() { + return true; + } + @Override protected String getTableLocation(SparkTableInfo table) { return String.join(File.separator, table.getTableLocation(), "data"); @@ -216,6 +233,24 @@ void testIcebergMetadataColumns() throws NoSuchTableException { testDeleteMetadataColumn(); } + @Test + void testInjectSparkExtensions() { + SparkSession sparkSession = getSparkSession(); + SparkConf conf = sparkSession.sparkContext().getConf(); + Assertions.assertTrue(conf.contains(StaticSQLConf.SPARK_SESSION_EXTENSIONS().key())); + String extensions = conf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS().key()); + Assertions.assertTrue(StringUtils.isNotBlank(extensions)); + Assertions.assertEquals(IcebergSparkSessionExtensions.class.getName(), extensions); + } + + @Test + void testIcebergTableRowLevelOperations() { + testIcebergDeleteOperation(); + testIcebergUpdateOperation(); + testIcebergMergeIntoDeleteOperation(); + testIcebergMergeIntoUpdateOperation(); + } + private void testMetadataColumns() { String tableName = "test_metadata_columns"; dropTableIfExists(tableName); @@ -386,6 +421,88 @@ private void testDeleteMetadataColumn() { Assertions.assertEquals(0, queryResult1.size()); } + private void testIcebergDeleteOperation() { + getIcebergTablePropertyValues() + .forEach( + tuple -> { + String tableName = + String.format("test_iceberg_%s_%s_delete_operation", tuple._1(), tuple._2()); + dropTableIfExists(tableName); + createIcebergTableWithTabProperties( + tableName, + tuple._1(), + ImmutableMap.of( + ICEBERG_FORMAT_VERSION, + String.valueOf(tuple._2()), + ICEBERG_DELETE_MODE, + tuple._3())); + checkTableColumns(tableName, getSimpleTableColumn(), getTableInfo(tableName)); + checkTableRowLevelDelete(tableName); + }); + } + + private void testIcebergUpdateOperation() { + getIcebergTablePropertyValues() + .forEach( + tuple -> { + String tableName = + String.format("test_iceberg_%s_%s_update_operation", tuple._1(), tuple._2()); + dropTableIfExists(tableName); + createIcebergTableWithTabProperties( + tableName, + tuple._1(), + ImmutableMap.of( + ICEBERG_FORMAT_VERSION, + String.valueOf(tuple._2()), + ICEBERG_UPDATE_MODE, + tuple._3())); + checkTableColumns(tableName, getSimpleTableColumn(), getTableInfo(tableName)); + checkTableRowLevelUpdate(tableName); + }); + } + + private void testIcebergMergeIntoDeleteOperation() { + getIcebergTablePropertyValues() + .forEach( + tuple -> { + String tableName = + String.format( + "test_iceberg_%s_%s_mergeinto_delete_operation", tuple._1(), tuple._2()); + dropTableIfExists(tableName); + createIcebergTableWithTabProperties( + tableName, + tuple._1(), + ImmutableMap.of( + ICEBERG_FORMAT_VERSION, + String.valueOf(tuple._2()), + ICEBERG_MERGE_MODE, + tuple._3())); + checkTableColumns(tableName, getSimpleTableColumn(), getTableInfo(tableName)); + checkTableDeleteByMergeInto(tableName); + }); + } + + private void testIcebergMergeIntoUpdateOperation() { + getIcebergTablePropertyValues() + .forEach( + tuple -> { + String tableName = + String.format( + "test_iceberg_%s_%s_mergeinto_update_operation", tuple._1(), tuple._2()); + dropTableIfExists(tableName); + createIcebergTableWithTabProperties( + tableName, + tuple._1(), + ImmutableMap.of( + ICEBERG_FORMAT_VERSION, + String.valueOf(tuple._2()), + ICEBERG_MERGE_MODE, + tuple._3())); + checkTableColumns(tableName, getSimpleTableColumn(), getTableInfo(tableName)); + checkTableUpdateByMergeInto(tableName); + }); + } + private List getIcebergSimpleTableColumn() { return Arrays.asList( SparkTableInfo.SparkColumnInfo.of("id", DataTypes.IntegerType, "id comment"), @@ -416,4 +533,26 @@ private SparkMetadataColumnInfo[] getIcebergMetadataColumns() { new SparkMetadataColumnInfo("_deleted", DataTypes.BooleanType, false) }; } + + private List> getIcebergTablePropertyValues() { + return Arrays.asList( + new Tuple3<>(false, 1, "copy-on-write"), + new Tuple3<>(false, 2, "merge-on-read"), + new Tuple3<>(true, 1, "copy-on-write"), + new Tuple3<>(true, 2, "merge-on-read")); + } + + private void createIcebergTableWithTabProperties( + String tableName, boolean isPartitioned, ImmutableMap tblProperties) { + String partitionedClause = isPartitioned ? " PARTITIONED BY (name) " : ""; + String tblPropertiesStr = + tblProperties.entrySet().stream() + .map(e -> String.format("'%s'='%s'", e.getKey(), e.getValue())) + .collect(Collectors.joining(",")); + String createSql = + String.format( + "CREATE TABLE %s (id INT COMMENT 'id comment', name STRING COMMENT '', age INT) %s TBLPROPERTIES(%s)", + tableName, partitionedClause, tblPropertiesStr); + sql(createSql); + } } diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkTableInfo.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkTableInfo.java index ee08de46ee9..43d3b85adfb 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkTableInfo.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkTableInfo.java @@ -6,7 +6,9 @@ package com.datastrato.gravitino.integration.test.util.spark; import com.datastrato.gravitino.spark.connector.ConnectorConstants; -import com.datastrato.gravitino.spark.connector.table.SparkBaseTable; +import com.datastrato.gravitino.spark.connector.SparkTransformConverter; +import com.datastrato.gravitino.spark.connector.hive.SparkHiveTable; +import com.datastrato.gravitino.spark.connector.iceberg.SparkIcebergTable; import java.util.ArrayList; import java.util.Arrays; import java.util.HashSet; @@ -18,6 +20,7 @@ import lombok.Data; import org.apache.commons.lang3.StringUtils; import org.apache.spark.sql.connector.catalog.SupportsMetadataColumns; +import org.apache.spark.sql.connector.catalog.Table; import org.apache.spark.sql.connector.catalog.TableCatalog; import org.apache.spark.sql.connector.expressions.ApplyTransform; import org.apache.spark.sql.connector.expressions.BucketTransform; @@ -29,6 +32,7 @@ import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.connector.expressions.YearsTransform; import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.StructType; import org.junit.jupiter.api.Assertions; /** SparkTableInfo is used to check the result in test. */ @@ -89,7 +93,7 @@ void addPartition(Transform partition) { } } - static SparkTableInfo create(SparkBaseTable baseTable) { + static SparkTableInfo create(Table baseTable) { SparkTableInfo sparkTableInfo = new SparkTableInfo(); String identifier = baseTable.name(); String[] items = identifier.split("\\."); @@ -98,7 +102,7 @@ static SparkTableInfo create(SparkBaseTable baseTable) { sparkTableInfo.tableName = items[1]; sparkTableInfo.database = items[0]; sparkTableInfo.columns = - Arrays.stream(baseTable.schema().fields()) + Arrays.stream(getSchema(baseTable).fields()) .map( sparkField -> new SparkColumnInfo( @@ -110,7 +114,7 @@ static SparkTableInfo create(SparkBaseTable baseTable) { sparkTableInfo.comment = baseTable.properties().remove(ConnectorConstants.COMMENT); sparkTableInfo.tableProperties = baseTable.properties(); boolean supportsBucketPartition = - baseTable.getSparkTransformConverter().isSupportsBucketPartition(); + getSparkTransformConverter(baseTable).isSupportsBucketPartition(); Arrays.stream(baseTable.partitioning()) .forEach( transform -> { @@ -149,10 +153,6 @@ static SparkTableInfo create(SparkBaseTable baseTable) { return sparkTableInfo; } - private static boolean isBucketPartition(boolean supportsBucketPartition, Transform transform) { - return supportsBucketPartition && !(transform instanceof SortedBucketTransform); - } - public List getUnPartitionedColumns() { return columns.stream() .filter(column -> !partitionColumnNames.contains(column.name)) @@ -165,6 +165,32 @@ public List getPartitionedColumns() { .collect(Collectors.toList()); } + private static boolean isBucketPartition(boolean supportsBucketPartition, Transform transform) { + return supportsBucketPartition && !(transform instanceof SortedBucketTransform); + } + + private static SparkTransformConverter getSparkTransformConverter(Table baseTable) { + if (baseTable instanceof SparkHiveTable) { + return ((SparkHiveTable) baseTable).getSparkTransformConverter(); + } else if (baseTable instanceof SparkIcebergTable) { + return ((SparkIcebergTable) baseTable).getSparkTransformConverter(); + } else { + throw new IllegalArgumentException( + "Doesn't support Spark table: " + baseTable.getClass().getName()); + } + } + + private static StructType getSchema(Table baseTable) { + if (baseTable instanceof SparkHiveTable) { + return ((SparkHiveTable) baseTable).schema(); + } else if (baseTable instanceof SparkIcebergTable) { + return ((SparkIcebergTable) baseTable).schema(); + } else { + throw new IllegalArgumentException( + "Doesn't support Spark table: " + baseTable.getClass().getName()); + } + } + @Data public static class SparkColumnInfo { private String name; diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkUtilIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkUtilIT.java index 2603fbe8f73..bad6fa0cb62 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkUtilIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkUtilIT.java @@ -20,7 +20,6 @@ package com.datastrato.gravitino.integration.test.util.spark; import com.datastrato.gravitino.integration.test.util.AbstractIT; -import com.datastrato.gravitino.spark.connector.table.SparkBaseTable; import java.sql.Timestamp; import java.text.SimpleDateFormat; import java.util.Arrays; @@ -130,8 +129,7 @@ protected SparkTableInfo getTableInfo(String tableName) { CommandResult result = (CommandResult) ds.logicalPlan(); DescribeRelation relation = (DescribeRelation) result.commandLogicalPlan(); ResolvedTable table = (ResolvedTable) relation.child(); - SparkBaseTable baseTable = (SparkBaseTable) table.table(); - return SparkTableInfo.create(baseTable); + return SparkTableInfo.create(table.table()); } protected void dropTableIfExists(String tableName) { @@ -159,6 +157,10 @@ protected void insertTableAsSelect(String tableName, String newName) { sql(String.format("INSERT INTO TABLE %s SELECT * FROM %s", newName, tableName)); } + protected static String getSelectAllSqlWithOrder(String tableName) { + return String.format("SELECT * FROM %s ORDER BY id", tableName); + } + private static String getSelectAllSql(String tableName) { return String.format("SELECT * FROM %s", tableName); } diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/ConnectorConstants.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/ConnectorConstants.java index 3a49a21470f..9758ff42196 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/ConnectorConstants.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/ConnectorConstants.java @@ -14,6 +14,7 @@ public class ConnectorConstants { public static final String LOCATION = "location"; public static final String DOT = "."; + public static final String COMMA = ","; private ConnectorConstants() {} } diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/catalog/BaseCatalog.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/catalog/BaseCatalog.java index f5994b4ce86..1cfc98de6ef 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/catalog/BaseCatalog.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/catalog/BaseCatalog.java @@ -19,7 +19,6 @@ import com.datastrato.gravitino.spark.connector.SparkTransformConverter; import com.datastrato.gravitino.spark.connector.SparkTransformConverter.DistributionAndSortOrdersInfo; import com.datastrato.gravitino.spark.connector.SparkTypeConverter; -import com.datastrato.gravitino.spark.connector.table.SparkBaseTable; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import java.util.Arrays; @@ -27,6 +26,7 @@ import java.util.Map; import java.util.Optional; import javax.ws.rs.NotSupportedException; +import lombok.SneakyThrows; import org.apache.commons.lang3.StringUtils; import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; @@ -93,15 +93,17 @@ protected abstract TableCatalog createAndInitSparkCatalog( * * @param identifier Spark's table identifier * @param gravitinoTable Gravitino table to do DDL operations + * @param sparkTable specific Spark table to do IO operations * @param sparkCatalog specific Spark catalog to do IO operations * @param propertiesConverter transform properties between Gravitino and Spark * @param sparkTransformConverter sparkTransformConverter convert transforms between Gravitino and * Spark * @return a specific Spark table */ - protected abstract SparkBaseTable createSparkTable( + protected abstract Table createSparkTable( Identifier identifier, com.datastrato.gravitino.rel.Table gravitinoTable, + Table sparkTable, TableCatalog sparkCatalog, PropertiesConverter propertiesConverter, SparkTransformConverter sparkTransformConverter); @@ -162,10 +164,10 @@ public Identifier[] listTables(String[] namespace) throws NoSuchNamespaceExcepti } } + @SneakyThrows @Override public Table createTable( - Identifier ident, Column[] columns, Transform[] transforms, Map properties) - throws TableAlreadyExistsException, NoSuchNamespaceException { + Identifier ident, Column[] columns, Transform[] transforms, Map properties) { NameIdentifier gravitinoIdentifier = NameIdentifier.of(metalakeName, catalogName, getDatabase(ident), ident.name()); com.datastrato.gravitino.rel.Column[] gravitinoColumns = @@ -184,7 +186,7 @@ public Table createTable( sparkTransformConverter.toGravitinoPartitionings(transforms); try { - com.datastrato.gravitino.rel.Table table = + com.datastrato.gravitino.rel.Table gravitinoTable = gravitinoCatalogClient .asTableCatalog() .createTable( @@ -195,12 +197,20 @@ public Table createTable( partitionings, distributionAndSortOrdersInfo.getDistribution(), distributionAndSortOrdersInfo.getSortOrders()); + Table sparkTable = sparkCatalog.loadTable(ident); return createSparkTable( - ident, table, sparkCatalog, propertiesConverter, sparkTransformConverter); + ident, + gravitinoTable, + sparkTable, + sparkCatalog, + propertiesConverter, + sparkTransformConverter); } catch (NoSuchSchemaException e) { throw new NoSuchNamespaceException(ident.namespace()); } catch (com.datastrato.gravitino.exceptions.TableAlreadyExistsException e) { throw new TableAlreadyExistsException(ident); + } catch (NoSuchTableException e) { + throw new NoSuchTableException(ident); } } @@ -208,13 +218,19 @@ public Table createTable( public Table loadTable(Identifier ident) throws NoSuchTableException { try { String database = getDatabase(ident); - com.datastrato.gravitino.rel.Table table = + com.datastrato.gravitino.rel.Table gravitinoTable = gravitinoCatalogClient .asTableCatalog() .loadTable(NameIdentifier.of(metalakeName, catalogName, database, ident.name())); + Table sparkTable = sparkCatalog.loadTable(ident); // Will create a catalog specific table return createSparkTable( - ident, table, sparkCatalog, propertiesConverter, sparkTransformConverter); + ident, + gravitinoTable, + sparkTable, + sparkCatalog, + propertiesConverter, + sparkTransformConverter); } catch (com.datastrato.gravitino.exceptions.NoSuchTableException e) { throw new NoSuchTableException(ident); } @@ -235,14 +251,20 @@ public Table alterTable(Identifier ident, TableChange... changes) throws NoSuchT .map(BaseCatalog::transformTableChange) .toArray(com.datastrato.gravitino.rel.TableChange[]::new); try { - com.datastrato.gravitino.rel.Table table = + com.datastrato.gravitino.rel.Table gravitinoTable = gravitinoCatalogClient .asTableCatalog() .alterTable( NameIdentifier.of(metalakeName, catalogName, getDatabase(ident), ident.name()), gravitinoTableChanges); + Table sparkTable = sparkCatalog.loadTable(ident); return createSparkTable( - ident, table, sparkCatalog, propertiesConverter, sparkTransformConverter); + ident, + gravitinoTable, + sparkTable, + sparkCatalog, + propertiesConverter, + sparkTransformConverter); } catch (com.datastrato.gravitino.exceptions.NoSuchTableException e) { throw new NoSuchTableException(ident); } diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/GravitinoHiveCatalog.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/GravitinoHiveCatalog.java index 6ffca1ff9f4..a1cefdaf3a9 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/GravitinoHiveCatalog.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/GravitinoHiveCatalog.java @@ -10,7 +10,6 @@ import com.datastrato.gravitino.spark.connector.PropertiesConverter; import com.datastrato.gravitino.spark.connector.SparkTransformConverter; import com.datastrato.gravitino.spark.connector.catalog.BaseCatalog; -import com.datastrato.gravitino.spark.connector.table.SparkBaseTable; import com.google.common.base.Preconditions; import java.util.HashMap; import java.util.Map; @@ -42,14 +41,20 @@ protected TableCatalog createAndInitSparkCatalog( } @Override - protected SparkBaseTable createSparkTable( + protected org.apache.spark.sql.connector.catalog.Table createSparkTable( Identifier identifier, Table gravitinoTable, - TableCatalog sparkCatalog, + org.apache.spark.sql.connector.catalog.Table sparkTable, + TableCatalog sparkHiveCatalog, PropertiesConverter propertiesConverter, SparkTransformConverter sparkTransformConverter) { return new SparkHiveTable( - identifier, gravitinoTable, sparkCatalog, propertiesConverter, sparkTransformConverter); + identifier, + gravitinoTable, + sparkTable, + sparkHiveCatalog, + propertiesConverter, + sparkTransformConverter); } @Override diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/SparkHiveTable.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/SparkHiveTable.java index 91f9468178b..ac656e4e639 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/SparkHiveTable.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/SparkHiveTable.java @@ -8,23 +8,61 @@ import com.datastrato.gravitino.rel.Table; import com.datastrato.gravitino.spark.connector.PropertiesConverter; import com.datastrato.gravitino.spark.connector.SparkTransformConverter; -import com.datastrato.gravitino.spark.connector.table.SparkBaseTable; +import com.datastrato.gravitino.spark.connector.utils.SparkBaseTableHelper; +import com.google.common.annotations.VisibleForTesting; +import java.util.Map; +import org.apache.kyuubi.spark.connector.hive.HiveTable; +import org.apache.kyuubi.spark.connector.hive.HiveTableCatalog; +import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.types.StructType; + +/** Keep consistent behavior with the SparkIcebergTable */ +public class SparkHiveTable extends HiveTable { + + private SparkBaseTableHelper sparkBaseTableHelper; -/** May support more capabilities like partition management. */ -public class SparkHiveTable extends SparkBaseTable { public SparkHiveTable( Identifier identifier, Table gravitinoTable, - TableCatalog sparkCatalog, + org.apache.spark.sql.connector.catalog.Table sparkHiveTable, + TableCatalog sparkHiveCatalog, PropertiesConverter propertiesConverter, SparkTransformConverter sparkTransformConverter) { - super(identifier, gravitinoTable, sparkCatalog, propertiesConverter, sparkTransformConverter); + super( + SparkSession.active(), + ((HiveTable) sparkHiveTable).catalogTable(), + (HiveTableCatalog) sparkHiveCatalog); + this.sparkBaseTableHelper = + new SparkBaseTableHelper( + identifier, gravitinoTable, propertiesConverter, sparkTransformConverter); } @Override - protected boolean isCaseSensitive() { - return false; + public String name() { + return sparkBaseTableHelper.name(false); + } + + @Override + @SuppressWarnings("deprecation") + public StructType schema() { + return sparkBaseTableHelper.schema(); + } + + @Override + public Map properties() { + return sparkBaseTableHelper.properties(); + } + + @Override + public Transform[] partitioning() { + return sparkBaseTableHelper.partitioning(); + } + + @VisibleForTesting + public SparkTransformConverter getSparkTransformConverter() { + return sparkBaseTableHelper.getSparkTransformConverter(); } } diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/GravitinoIcebergCatalog.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/GravitinoIcebergCatalog.java index f7a028cad7a..5355dbc3dfd 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/GravitinoIcebergCatalog.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/GravitinoIcebergCatalog.java @@ -9,7 +9,6 @@ import com.datastrato.gravitino.spark.connector.PropertiesConverter; import com.datastrato.gravitino.spark.connector.SparkTransformConverter; import com.datastrato.gravitino.spark.connector.catalog.BaseCatalog; -import com.datastrato.gravitino.spark.connector.table.SparkBaseTable; import com.google.common.base.Preconditions; import java.util.HashMap; import java.util.Locale; @@ -66,14 +65,20 @@ protected TableCatalog createAndInitSparkCatalog( } @Override - protected SparkBaseTable createSparkTable( + protected org.apache.spark.sql.connector.catalog.Table createSparkTable( Identifier identifier, Table gravitinoTable, - TableCatalog sparkCatalog, + org.apache.spark.sql.connector.catalog.Table sparkTable, + TableCatalog sparkIcebergCatalog, PropertiesConverter propertiesConverter, SparkTransformConverter sparkTransformConverter) { return new SparkIcebergTable( - identifier, gravitinoTable, sparkCatalog, propertiesConverter, sparkTransformConverter); + identifier, + gravitinoTable, + sparkTable, + sparkIcebergCatalog, + propertiesConverter, + sparkTransformConverter); } @Override diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/SparkIcebergTable.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/SparkIcebergTable.java index 22dd0bb73a8..5c040e45670 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/SparkIcebergTable.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/SparkIcebergTable.java @@ -8,43 +8,72 @@ import com.datastrato.gravitino.rel.Table; import com.datastrato.gravitino.spark.connector.PropertiesConverter; import com.datastrato.gravitino.spark.connector.SparkTransformConverter; -import com.datastrato.gravitino.spark.connector.table.SparkBaseTable; +import com.datastrato.gravitino.spark.connector.utils.SparkBaseTableHelper; +import com.google.common.annotations.VisibleForTesting; +import java.lang.reflect.Field; +import java.util.Map; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.iceberg.spark.source.SparkTable; import org.apache.spark.sql.connector.catalog.Identifier; -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.TableCatalog; -import org.apache.spark.sql.sources.Filter; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.types.StructType; -public class SparkIcebergTable extends SparkBaseTable - implements SupportsDelete, SupportsMetadataColumns { +/** + * For spark-connector in Iceberg, it explicitly uses SparkTable to identify whether it is an + * Iceberg table, so the SparkIcebergTable must extend SparkTable. + */ +public class SparkIcebergTable extends SparkTable { + + private SparkBaseTableHelper sparkBaseTableHelper; public SparkIcebergTable( Identifier identifier, Table gravitinoTable, + org.apache.spark.sql.connector.catalog.Table sparkIcebergTable, TableCatalog sparkIcebergCatalog, PropertiesConverter propertiesConverter, SparkTransformConverter sparkTransformConverter) { - super( - identifier, - gravitinoTable, - sparkIcebergCatalog, - propertiesConverter, - sparkTransformConverter); + super(((SparkTable) sparkIcebergTable).table(), !isCacheEnabled(sparkIcebergCatalog)); + this.sparkBaseTableHelper = + new SparkBaseTableHelper( + identifier, gravitinoTable, propertiesConverter, sparkTransformConverter); + } + + @Override + public String name() { + return sparkBaseTableHelper.name(true); } @Override - public boolean canDeleteWhere(Filter[] filters) { - return ((SupportsDelete) getSparkTable()).canDeleteWhere(filters); + @SuppressWarnings("deprecation") + public StructType schema() { + return sparkBaseTableHelper.schema(); } @Override - public void deleteWhere(Filter[] filters) { - ((SupportsDelete) getSparkTable()).deleteWhere(filters); + public Map properties() { + return sparkBaseTableHelper.properties(); } @Override - public MetadataColumn[] metadataColumns() { - return ((SupportsMetadataColumns) getSparkTable()).metadataColumns(); + public Transform[] partitioning() { + return sparkBaseTableHelper.partitioning(); + } + + @VisibleForTesting + public SparkTransformConverter getSparkTransformConverter() { + return sparkBaseTableHelper.getSparkTransformConverter(); + } + + private static boolean isCacheEnabled(TableCatalog sparkIcebergCatalog) { + try { + SparkCatalog catalog = ((SparkCatalog) sparkIcebergCatalog); + Field cacheEnabled = catalog.getClass().getDeclaredField("cacheEnabled"); + cacheEnabled.setAccessible(true); + return cacheEnabled.getBoolean(catalog); + } catch (NoSuchFieldException | IllegalAccessException e) { + throw new RuntimeException("Failed to get cacheEnabled field from SparkCatalog", e); + } } } diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java index 3f830de2cdc..201666cc004 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java @@ -5,6 +5,8 @@ package com.datastrato.gravitino.spark.connector.plugin; +import static com.datastrato.gravitino.spark.connector.utils.ConnectorUtil.removeDuplicates; + import com.datastrato.gravitino.Catalog; import com.datastrato.gravitino.spark.connector.GravitinoSparkConfig; import com.datastrato.gravitino.spark.connector.catalog.GravitinoCatalogManager; @@ -15,10 +17,12 @@ import java.util.Locale; import java.util.Map; import org.apache.commons.lang3.StringUtils; +import org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions; import org.apache.spark.SparkConf; import org.apache.spark.SparkContext; import org.apache.spark.api.plugin.DriverPlugin; import org.apache.spark.api.plugin.PluginContext; +import org.apache.spark.sql.internal.StaticSQLConf; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -30,6 +34,8 @@ public class GravitinoDriverPlugin implements DriverPlugin { private static final Logger LOG = LoggerFactory.getLogger(GravitinoDriverPlugin.class); private GravitinoCatalogManager catalogManager; + private static final String[] GRAVITINO_DRIVER_EXTENSIONS = + new String[] {IcebergSparkSessionExtensions.class.getName()}; @Override public Map init(SparkContext sc, PluginContext pluginContext) { @@ -48,7 +54,7 @@ public Map init(SparkContext sc, PluginContext pluginContext) { catalogManager = GravitinoCatalogManager.create(gravitinoUri, metalake); catalogManager.loadRelationalCatalogs(); registerGravitinoCatalogs(conf, catalogManager.getCatalogs()); - registerSqlExtensions(); + registerSqlExtensions(conf); return Collections.emptyMap(); } @@ -103,6 +109,15 @@ private void registerCatalog(SparkConf sparkConf, String catalogName, String pro LOG.info("Register {} catalog to Spark catalog manager.", catalogName); } - // Todo inject Iceberg extensions - private void registerSqlExtensions() {} + private void registerSqlExtensions(SparkConf conf) { + String gravitinoDriverExtensions = String.join(",", GRAVITINO_DRIVER_EXTENSIONS); + if (conf.contains(StaticSQLConf.SPARK_SESSION_EXTENSIONS().key())) { + String sparkSessionExtensions = conf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS().key()); + conf.set( + StaticSQLConf.SPARK_SESSION_EXTENSIONS().key(), + removeDuplicates(GRAVITINO_DRIVER_EXTENSIONS, sparkSessionExtensions)); + } else { + conf.set(StaticSQLConf.SPARK_SESSION_EXTENSIONS().key(), gravitinoDriverExtensions); + } + } } diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/utils/ConnectorUtil.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/utils/ConnectorUtil.java new file mode 100644 index 00000000000..eeaa56c9da2 --- /dev/null +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/utils/ConnectorUtil.java @@ -0,0 +1,26 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ + +package com.datastrato.gravitino.spark.connector.utils; + +import static com.datastrato.gravitino.spark.connector.ConnectorConstants.COMMA; + +import java.util.Arrays; +import java.util.LinkedHashSet; +import java.util.Set; +import org.apache.commons.lang3.StringUtils; + +public class ConnectorUtil { + + public static String removeDuplicates(String[] elements, String otherElements) { + Set uniqueElements = new LinkedHashSet<>(Arrays.asList(elements)); + if (StringUtils.isNotBlank(otherElements)) { + uniqueElements.addAll(Arrays.asList(otherElements.split(COMMA))); + } + return uniqueElements.stream() + .reduce((element1, element2) -> element1 + COMMA + element2) + .orElse(""); + } +} diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/table/SparkBaseTable.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/utils/SparkBaseTableHelper.java similarity index 67% rename from spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/table/SparkBaseTable.java rename to spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/utils/SparkBaseTableHelper.java index d1333135f19..0011968bd35 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/table/SparkBaseTable.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/utils/SparkBaseTableHelper.java @@ -3,7 +3,7 @@ * This software is licensed under the Apache License version 2. */ -package com.datastrato.gravitino.spark.connector.table; +package com.datastrato.gravitino.spark.connector.utils; import com.datastrato.gravitino.rel.expressions.distributions.Distribution; import com.datastrato.gravitino.rel.expressions.sorts.SortOrder; @@ -11,65 +11,47 @@ import com.datastrato.gravitino.spark.connector.PropertiesConverter; import com.datastrato.gravitino.spark.connector.SparkTransformConverter; import com.datastrato.gravitino.spark.connector.SparkTypeConverter; -import com.google.common.annotations.VisibleForTesting; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.Set; import java.util.stream.Collectors; -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.connector.catalog.Identifier; -import org.apache.spark.sql.connector.catalog.SupportsRead; -import org.apache.spark.sql.connector.catalog.SupportsWrite; -import org.apache.spark.sql.connector.catalog.Table; -import org.apache.spark.sql.connector.catalog.TableCapability; -import org.apache.spark.sql.connector.catalog.TableCatalog; import org.apache.spark.sql.connector.expressions.Transform; -import org.apache.spark.sql.connector.read.ScanBuilder; -import org.apache.spark.sql.connector.write.LogicalWriteInfo; -import org.apache.spark.sql.connector.write.WriteBuilder; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.MetadataBuilder; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; /** * Provides schema info from Gravitino, IO from the internal spark table. The specific catalog table * could implement more capabilities like SupportsPartitionManagement for Hive table, SupportsIndex * for JDBC table, SupportsRowLevelOperations for Iceberg table. */ -public abstract class SparkBaseTable implements Table, SupportsRead, SupportsWrite { +public class SparkBaseTableHelper { + private Identifier identifier; private com.datastrato.gravitino.rel.Table gravitinoTable; - private TableCatalog sparkCatalog; - private Table lazySparkTable; private PropertiesConverter propertiesConverter; private SparkTransformConverter sparkTransformConverter; - public SparkBaseTable( + public SparkBaseTableHelper( Identifier identifier, com.datastrato.gravitino.rel.Table gravitinoTable, - TableCatalog sparkCatalog, PropertiesConverter propertiesConverter, SparkTransformConverter sparkTransformConverter) { this.identifier = identifier; this.gravitinoTable = gravitinoTable; - this.sparkCatalog = sparkCatalog; this.propertiesConverter = propertiesConverter; this.sparkTransformConverter = sparkTransformConverter; } - @Override - public String name() { - return getNormalizedIdentifier(identifier, gravitinoTable.name()); + public String name(boolean isCaseSensitive) { + return getNormalizedIdentifier(identifier, gravitinoTable.name(), isCaseSensitive); } - @Override - @SuppressWarnings("deprecation") public StructType schema() { List structs = Arrays.stream(gravitinoTable.columns()) @@ -93,7 +75,6 @@ public StructType schema() { return DataTypes.createStructType(structs); } - @Override public Map properties() { Map properties = new HashMap(); if (gravitinoTable.properties() != null) { @@ -110,22 +91,6 @@ public Map properties() { return properties; } - @Override - public Set capabilities() { - return getSparkTable().capabilities(); - } - - @Override - public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { - return ((SupportsRead) getSparkTable()).newScanBuilder(options); - } - - @Override - public WriteBuilder newWriteBuilder(LogicalWriteInfo info) { - return ((SupportsWrite) getSparkTable()).newWriteBuilder(info); - } - - @Override public Transform[] partitioning() { com.datastrato.gravitino.rel.expressions.transforms.Transform[] partitions = gravitinoTable.partitioning(); @@ -134,35 +99,20 @@ public Transform[] partitioning() { return sparkTransformConverter.toSparkTransform(partitions, distribution, sortOrders); } - protected Table getSparkTable() { - if (lazySparkTable == null) { - try { - this.lazySparkTable = sparkCatalog.loadTable(identifier); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - } - return lazySparkTable; - } - - @VisibleForTesting public SparkTransformConverter getSparkTransformConverter() { return sparkTransformConverter; } - protected boolean isCaseSensitive() { - return true; - } - // The underlying catalogs may not case-sensitive, to keep consistent with the action of SparkSQL, // we should return normalized identifiers. - private String getNormalizedIdentifier(Identifier tableIdentifier, String gravitinoTableName) { + private String getNormalizedIdentifier( + Identifier tableIdentifier, String gravitinoTableName, boolean isCaseSensitive) { if (tableIdentifier.namespace().length == 0) { return gravitinoTableName; } String databaseName = tableIdentifier.namespace()[0]; - if (isCaseSensitive() == false) { + if (!isCaseSensitive) { databaseName = databaseName.toLowerCase(Locale.ROOT); } diff --git a/spark-connector/spark-connector/src/test/java/com/datastrato/gravitino/spark/connector/utils/TestConnectorUtil.java b/spark-connector/spark-connector/src/test/java/com/datastrato/gravitino/spark/connector/utils/TestConnectorUtil.java new file mode 100644 index 00000000000..81d452d28e8 --- /dev/null +++ b/spark-connector/spark-connector/src/test/java/com/datastrato/gravitino/spark/connector/utils/TestConnectorUtil.java @@ -0,0 +1,31 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ + +package com.datastrato.gravitino.spark.connector.utils; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; + +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public class TestConnectorUtil { + + @Test + void testRemoveDuplicates() { + String[] elements = {"a", "b", "c"}; + String otherElements = "a,d,e"; + String result = ConnectorUtil.removeDuplicates(elements, otherElements); + Assertions.assertEquals(result, "a,b,c,d,e"); + + elements = new String[] {"a", "a", "b", "c"}; + otherElements = ""; + result = ConnectorUtil.removeDuplicates(elements, otherElements); + Assertions.assertEquals(result, "a,b,c"); + + elements = new String[] {"a", "a", "b", "c"}; + result = ConnectorUtil.removeDuplicates(elements, null); + Assertions.assertEquals(result, "a,b,c"); + } +} From 90b7be8afb61f61b049eec2ae1a1547d4079d008 Mon Sep 17 00:00:00 2001 From: caican00 Date: Sat, 4 May 2024 14:29:22 +0800 Subject: [PATCH 02/18] [#3264] feat(spark-connector): Support Iceberg time travel in SQL queries --- .../spark/iceberg/SparkIcebergCatalogIT.java | 65 +++++++++++++++++++ 1 file changed, 65 insertions(+) diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java index f7da5564809..dabfb4bc970 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java @@ -11,6 +11,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import java.io.File; +import java.sql.Timestamp; +import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -251,6 +253,62 @@ void testIcebergTableRowLevelOperations() { testIcebergMergeIntoUpdateOperation(); } + @Test + void testIcebergAsOfQuery() { + String tableName = "test_iceberg_as_of_query"; + String fullTableName = + String.format("%s.%s.%s", getCatalogName(), getDefaultDatabase(), tableName); + dropTableIfExists(tableName); + createSimpleTable(tableName); + + checkTableColumns(tableName, getSimpleTableColumn(), getTableInfo(tableName)); + + sql(String.format("INSERT INTO %s VALUES (1, '1', 1)", tableName)); + List snapshots = + getSparkSession() + .sql(String.format("SELECT snapshot_id FROM %s.snapshots", fullTableName)) + .collectAsList(); + Assertions.assertEquals(1, snapshots.size()); + long snapshotId = snapshots.get(0).getLong(0); + List timestamp = + getSparkSession() + .sql(String.format("SELECT committed_at FROM %s.snapshots", fullTableName)) + .collectAsList(); + Assertions.assertEquals(1, timestamp.size()); + Timestamp timestampAt = timestamp.get(0).getTimestamp(0); + waitUntilAfter(timestampAt.getTime()); + Timestamp firstSnapshotTimestamp = + Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); + sql(String.format("INSERT INTO %s VALUES (2, '2', 2)", tableName)); + + List tableData = getQueryData(getSelectAllSqlWithOrder(tableName)); + Assertions.assertEquals(2, tableData.size()); + Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); + + tableData = + getQueryData( + String.format( + "SELECT * FROM %s TIMESTAMP AS OF '%s'", tableName, firstSnapshotTimestamp)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + tableData = + getQueryData( + String.format( + "SELECT * FROM %s FOR SYSTEM_TIME AS OF '%s'", tableName, firstSnapshotTimestamp)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + tableData = + getQueryData(String.format("SELECT * FROM %s VERSION AS OF %d", tableName, snapshotId)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + tableData = + getQueryData( + String.format("SELECT * FROM %s FOR SYSTEM_VERSION AS OF %d", tableName, snapshotId)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + } + private void testMetadataColumns() { String tableName = "test_metadata_columns"; dropTableIfExists(tableName); @@ -555,4 +613,11 @@ private void createIcebergTableWithTabProperties( tableName, partitionedClause, tblPropertiesStr); sql(createSql); } + + private void waitUntilAfter(Long timestampMillis) { + long current = System.currentTimeMillis(); + while (current <= timestampMillis) { + current = System.currentTimeMillis(); + } + } } From 65ef2a4b5493d9537b45d6c01f00350f871ae5e0 Mon Sep 17 00:00:00 2001 From: caican00 Date: Sat, 4 May 2024 14:29:22 +0800 Subject: [PATCH 03/18] update --- .../spark/connector/catalog/BaseCatalog.java | 56 +++++++++++++++++-- .../spark/connector/hive/SparkHiveTable.java | 27 ++++++++- .../connector/iceberg/SparkIcebergTable.java | 55 +++++++++++++++++- .../connector/utils/SparkBaseTableHelper.java | 28 ++++++++++ 4 files changed, 159 insertions(+), 7 deletions(-) diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/catalog/BaseCatalog.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/catalog/BaseCatalog.java index 1cfc98de6ef..55e2779c861 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/catalog/BaseCatalog.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/catalog/BaseCatalog.java @@ -217,11 +217,7 @@ public Table createTable( @Override public Table loadTable(Identifier ident) throws NoSuchTableException { try { - String database = getDatabase(ident); - com.datastrato.gravitino.rel.Table gravitinoTable = - gravitinoCatalogClient - .asTableCatalog() - .loadTable(NameIdentifier.of(metalakeName, catalogName, database, ident.name())); + com.datastrato.gravitino.rel.Table gravitinoTable = loadGravitinoTable(ident); Table sparkTable = sparkCatalog.loadTable(ident); // Will create a catalog specific table return createSparkTable( @@ -236,6 +232,44 @@ public Table loadTable(Identifier ident) throws NoSuchTableException { } } + @Override + public Table loadTable(Identifier ident, String version) throws NoSuchTableException { + try { + com.datastrato.gravitino.rel.Table gravitinoTable = loadGravitinoTable(ident); + // load SparkTable with version + Table sparkTable = sparkCatalog.loadTable(ident, version); + // Will create a catalog specific table + return createSparkTable( + ident, + gravitinoTable, + sparkTable, + sparkCatalog, + propertiesConverter, + sparkTransformConverter); + } catch (com.datastrato.gravitino.exceptions.NoSuchTableException e) { + throw new NoSuchTableException(ident); + } + } + + @Override + public Table loadTable(Identifier ident, long timestamp) throws NoSuchTableException { + try { + com.datastrato.gravitino.rel.Table gravitinoTable = loadGravitinoTable(ident); + // load SparkTable with timestamp + Table sparkTable = sparkCatalog.loadTable(ident, timestamp); + // Will create a catalog specific table + return createSparkTable( + ident, + gravitinoTable, + sparkTable, + sparkCatalog, + propertiesConverter, + sparkTransformConverter); + } catch (com.datastrato.gravitino.exceptions.NoSuchTableException e) { + throw new NoSuchTableException(ident); + } + } + @SuppressWarnings("deprecation") @Override public Table createTable( @@ -520,4 +554,16 @@ private static com.datastrato.gravitino.rel.TableChange.ColumnPosition transform "Unsupported table column position %s", columnPosition.getClass().getName())); } } + + private com.datastrato.gravitino.rel.Table loadGravitinoTable(Identifier ident) + throws NoSuchTableException { + try { + String database = getDatabase(ident); + return gravitinoCatalogClient + .asTableCatalog() + .loadTable(NameIdentifier.of(metalakeName, catalogName, database, ident.name())); + } catch (com.datastrato.gravitino.exceptions.NoSuchTableException e) { + throw new NoSuchTableException(ident); + } + } } diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/SparkHiveTable.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/SparkHiveTable.java index ac656e4e639..48c88f1262d 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/SparkHiveTable.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/SparkHiveTable.java @@ -11,13 +11,19 @@ import com.datastrato.gravitino.spark.connector.utils.SparkBaseTableHelper; import com.google.common.annotations.VisibleForTesting; import java.util.Map; +import java.util.Set; import org.apache.kyuubi.spark.connector.hive.HiveTable; import org.apache.kyuubi.spark.connector.hive.HiveTableCatalog; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCapability; import org.apache.spark.sql.connector.catalog.TableCatalog; import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.connector.read.ScanBuilder; +import org.apache.spark.sql.connector.write.LogicalWriteInfo; +import org.apache.spark.sql.connector.write.WriteBuilder; import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; /** Keep consistent behavior with the SparkIcebergTable */ public class SparkHiveTable extends HiveTable { @@ -37,7 +43,11 @@ public SparkHiveTable( (HiveTableCatalog) sparkHiveCatalog); this.sparkBaseTableHelper = new SparkBaseTableHelper( - identifier, gravitinoTable, propertiesConverter, sparkTransformConverter); + identifier, + gravitinoTable, + sparkHiveTable, + propertiesConverter, + sparkTransformConverter); } @Override @@ -61,6 +71,21 @@ public Transform[] partitioning() { return sparkBaseTableHelper.partitioning(); } + @Override + public Set capabilities() { + return sparkBaseTableHelper.capabilities(); + } + + @Override + public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { + return sparkBaseTableHelper.newScanBuilder(options); + } + + @Override + public WriteBuilder newWriteBuilder(LogicalWriteInfo info) { + return sparkBaseTableHelper.newWriteBuilder(info); + } + @VisibleForTesting public SparkTransformConverter getSparkTransformConverter() { return sparkBaseTableHelper.getSparkTransformConverter(); diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/SparkIcebergTable.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/SparkIcebergTable.java index 5c040e45670..14afb7ef136 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/SparkIcebergTable.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/SparkIcebergTable.java @@ -12,12 +12,25 @@ import com.google.common.annotations.VisibleForTesting; import java.lang.reflect.Field; import java.util.Map; +import java.util.Set; import org.apache.iceberg.spark.SparkCatalog; import org.apache.iceberg.spark.source.SparkTable; import org.apache.spark.sql.connector.catalog.Identifier; +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.SupportsRowLevelOperations; +import org.apache.spark.sql.connector.catalog.TableCapability; import org.apache.spark.sql.connector.catalog.TableCatalog; import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.connector.read.ScanBuilder; +import org.apache.spark.sql.connector.write.LogicalWriteInfo; +import org.apache.spark.sql.connector.write.RowLevelOperationBuilder; +import org.apache.spark.sql.connector.write.RowLevelOperationInfo; +import org.apache.spark.sql.connector.write.WriteBuilder; +import org.apache.spark.sql.sources.Filter; import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; /** * For spark-connector in Iceberg, it explicitly uses SparkTable to identify whether it is an @@ -37,7 +50,11 @@ public SparkIcebergTable( super(((SparkTable) sparkIcebergTable).table(), !isCacheEnabled(sparkIcebergCatalog)); this.sparkBaseTableHelper = new SparkBaseTableHelper( - identifier, gravitinoTable, propertiesConverter, sparkTransformConverter); + identifier, + gravitinoTable, + sparkIcebergTable, + propertiesConverter, + sparkTransformConverter); } @Override @@ -61,6 +78,42 @@ public Transform[] partitioning() { return sparkBaseTableHelper.partitioning(); } + @Override + public Set capabilities() { + return sparkBaseTableHelper.capabilities(); + } + + @Override + public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { + return sparkBaseTableHelper.newScanBuilder(options); + } + + @Override + public WriteBuilder newWriteBuilder(LogicalWriteInfo info) { + return sparkBaseTableHelper.newWriteBuilder(info); + } + + @Override + public boolean canDeleteWhere(Filter[] filters) { + return ((SupportsDelete) sparkBaseTableHelper.getSparkTable()).canDeleteWhere(filters); + } + + @Override + public void deleteWhere(Filter[] filters) { + ((SupportsDelete) sparkBaseTableHelper.getSparkTable()).deleteWhere(filters); + } + + @Override + public MetadataColumn[] metadataColumns() { + return ((SupportsMetadataColumns) sparkBaseTableHelper.getSparkTable()).metadataColumns(); + } + + @Override + public RowLevelOperationBuilder newRowLevelOperationBuilder(RowLevelOperationInfo info) { + return ((SupportsRowLevelOperations) sparkBaseTableHelper.getSparkTable()) + .newRowLevelOperationBuilder(info); + } + @VisibleForTesting public SparkTransformConverter getSparkTransformConverter() { return sparkBaseTableHelper.getSparkTransformConverter(); diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/utils/SparkBaseTableHelper.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/utils/SparkBaseTableHelper.java index 0011968bd35..461ce49384a 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/utils/SparkBaseTableHelper.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/utils/SparkBaseTableHelper.java @@ -16,14 +16,23 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.SupportsRead; +import org.apache.spark.sql.connector.catalog.SupportsWrite; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCapability; import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.connector.read.ScanBuilder; +import org.apache.spark.sql.connector.write.LogicalWriteInfo; +import org.apache.spark.sql.connector.write.WriteBuilder; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.MetadataBuilder; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; /** * Provides schema info from Gravitino, IO from the internal spark table. The specific catalog table @@ -34,16 +43,19 @@ public class SparkBaseTableHelper { private Identifier identifier; private com.datastrato.gravitino.rel.Table gravitinoTable; + private Table sparkTable; private PropertiesConverter propertiesConverter; private SparkTransformConverter sparkTransformConverter; public SparkBaseTableHelper( Identifier identifier, com.datastrato.gravitino.rel.Table gravitinoTable, + Table sparkTable, PropertiesConverter propertiesConverter, SparkTransformConverter sparkTransformConverter) { this.identifier = identifier; this.gravitinoTable = gravitinoTable; + this.sparkTable = sparkTable; this.propertiesConverter = propertiesConverter; this.sparkTransformConverter = sparkTransformConverter; } @@ -99,6 +111,22 @@ public Transform[] partitioning() { return sparkTransformConverter.toSparkTransform(partitions, distribution, sortOrders); } + public Set capabilities() { + return sparkTable.capabilities(); + } + + public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { + return ((SupportsRead) sparkTable).newScanBuilder(options); + } + + public WriteBuilder newWriteBuilder(LogicalWriteInfo info) { + return ((SupportsWrite) sparkTable).newWriteBuilder(info); + } + + public Table getSparkTable() { + return sparkTable; + } + public SparkTransformConverter getSparkTransformConverter() { return sparkTransformConverter; } From 0a8ff3500bd54bf447818e159b8f2bef08687eab Mon Sep 17 00:00:00 2001 From: caican00 Date: Sat, 4 May 2024 23:20:19 +0800 Subject: [PATCH 04/18] [#3187] feat(spark-connector): Support SparkSQL extended syntax in Iceberg --- .../spark/iceberg/SparkIcebergCatalogIT.java | 268 +++++++++++++++++- .../spark-connector/build.gradle.kts | 2 + ...ravitinoIcebergSparkSessionExtensions.java | 22 ++ .../IcebergExtendedDataSourceV2Strategy.java | 235 +++++++++++++++ .../plugin/GravitinoDriverPlugin.java | 6 +- 5 files changed, 531 insertions(+), 2 deletions(-) create mode 100644 spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/GravitinoIcebergSparkSessionExtensions.java create mode 100644 spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/IcebergExtendedDataSourceV2Strategy.java diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java index f7da5564809..0df5e2550ba 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java @@ -8,6 +8,7 @@ import com.datastrato.gravitino.integration.test.util.spark.SparkMetadataColumnInfo; import com.datastrato.gravitino.integration.test.util.spark.SparkTableInfo; import com.datastrato.gravitino.integration.test.util.spark.SparkTableInfoChecker; +import com.datastrato.gravitino.spark.connector.iceberg.extensions.GravitinoIcebergSparkSessionExtensions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import java.io.File; @@ -19,6 +20,7 @@ import java.util.Map; import java.util.stream.Collectors; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions; import org.apache.spark.SparkConf; import org.apache.spark.sql.Column; @@ -48,6 +50,9 @@ public abstract class SparkIcebergCatalogIT extends SparkCommonIT { private static final String ICEBERG_DELETE_MODE = "write.delete.mode"; private static final String ICEBERG_UPDATE_MODE = "write.update.mode"; private static final String ICEBERG_MERGE_MODE = "write.merge.mode"; + private static final String ICEBERG_WRITE_DISTRIBUTION_MODE = "write.distribution-mode"; + private static final String ICEBERG_SORT_ORDER = "sort-order"; + private static final String ICEBERG_IDENTIFIER_FIELDS = "identifier-fields"; @Override protected String getCatalogName() { @@ -240,7 +245,12 @@ void testInjectSparkExtensions() { Assertions.assertTrue(conf.contains(StaticSQLConf.SPARK_SESSION_EXTENSIONS().key())); String extensions = conf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS().key()); Assertions.assertTrue(StringUtils.isNotBlank(extensions)); - Assertions.assertEquals(IcebergSparkSessionExtensions.class.getName(), extensions); + Assertions.assertEquals( + String.join( + ",", + GravitinoIcebergSparkSessionExtensions.class.getName(), + IcebergSparkSessionExtensions.class.getName()), + extensions); } @Test @@ -251,6 +261,15 @@ void testIcebergTableRowLevelOperations() { testIcebergMergeIntoUpdateOperation(); } + @Test + void testIcebergSQLExtensions() { + testIcebergPartitionFieldOperations(); + testIcebergBranchOperations(); + testIcebergTagOperations(); + testIcebergIdentifierOperations(); + testIcebergDistributionAndOrderingOperations(); + } + private void testMetadataColumns() { String tableName = "test_metadata_columns"; dropTableIfExists(tableName); @@ -503,6 +522,253 @@ private void testIcebergMergeIntoUpdateOperation() { }); } + private void testIcebergPartitionFieldOperations() { + List partitionFields = + Arrays.asList("name", "truncate(1, name)", "bucket(16, id)", "days(ts)"); + String partitionExpression = "name=a/name_trunc=a/id_bucket=4/ts_day=2024-01-01"; + String tableName = "test_iceberg_partition_field_operations"; + dropTableIfExists(tableName); + sql(getCreateIcebergSimpleTableString(tableName)); + + // add partition fields + SparkTableInfo tableInfo = getTableInfo(tableName); + SparkTableInfoChecker checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withColumns(getIcebergSimpleTableColumn()); + checker.check(tableInfo); + + partitionFields.forEach( + partitionField -> + sql(String.format("ALTER TABLE %s ADD PARTITION FIELD %s", tableName, partitionField))); + + tableInfo = getTableInfo(tableName); + checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withColumns(getIcebergSimpleTableColumn()) + .withIdentifyPartition(Collections.singletonList("name")) + .withTruncatePartition(1, "name") + .withBucketPartition(16, Collections.singletonList("id")) + .withDayPartition("ts"); + checker.check(tableInfo); + + sql( + String.format( + "INSERT INTO %s VALUES(2,'a',cast('2024-01-01 12:00:00' as timestamp));", tableName)); + List queryResult = getTableData(tableName); + Assertions.assertEquals(1, queryResult.size()); + Assertions.assertEquals("2,a,2024-01-01 12:00:00", queryResult.get(0)); + Path partitionPath = new Path(getTableLocation(tableInfo), partitionExpression); + checkDirExists(partitionPath); + + // replace partition fields + sql(String.format("ALTER TABLE %s REPLACE PARTITION FIELD ts_day WITH months(ts)", tableName)); + tableInfo = getTableInfo(tableName); + checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withColumns(getIcebergSimpleTableColumn()) + .withIdentifyPartition(Collections.singletonList("name")) + .withTruncatePartition(1, "name") + .withBucketPartition(16, Collections.singletonList("id")) + .withMonthPartition("ts"); + checker.check(tableInfo); + + // drop partition fields + sql(String.format("ALTER TABLE %s DROP PARTITION FIELD months(ts)", tableName)); + tableInfo = getTableInfo(tableName); + checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withColumns(getIcebergSimpleTableColumn()) + .withIdentifyPartition(Collections.singletonList("name")) + .withTruncatePartition(1, "name") + .withBucketPartition(16, Collections.singletonList("id")); + checker.check(tableInfo); + } + + private void testIcebergBranchOperations() { + String tableName = "test_iceberg_branch_operations"; + String fullTableName = + String.format("%s.%s.%s", getCatalogName(), getDefaultDatabase(), tableName); + String branch1 = "branch1"; + dropTableIfExists(tableName); + createSimpleTable(tableName); + + // create branch and query data using branch + sql(String.format("INSERT INTO %s VALUES(1, '1', 1);", tableName)); + List tableData = getTableData(tableName); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + List snapshots = + getSparkSession() + .sql(String.format("SELECT snapshot_id FROM %s.snapshots", fullTableName)) + .collectAsList(); + Assertions.assertEquals(1, snapshots.size()); + long snapshotId = snapshots.get(0).getLong(0); + + sql(String.format("ALTER TABLE %s CREATE BRANCH IF NOT EXISTS `%s`", tableName, branch1)); + sql(String.format("INSERT INTO %s VALUES(2, '2', 2);", tableName)); + tableData = getQueryData(getSelectAllSqlWithOrder(tableName)); + Assertions.assertEquals(2, tableData.size()); + Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); + + tableData = + getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, branch1)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + sql(String.format("ALTER TABLE %s CREATE OR REPLACE BRANCH `%s`", tableName, branch1)); + tableData = + getQueryData( + String.format("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, branch1)); + Assertions.assertEquals(2, tableData.size()); + Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); + + // replace branch + sql( + String.format( + "ALTER TABLE %s REPLACE BRANCH `%s` AS OF VERSION %d RETAIN 1 DAYS", + tableName, branch1, snapshotId)); + tableData = + getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, branch1)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + // drop branch + sql(String.format("ALTER TABLE %s DROP BRANCH `%s`", tableName, branch1)); + Assertions.assertThrows( + ValidationException.class, + () -> sql(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, branch1))); + } + + private void testIcebergTagOperations() { + String tableName = "test_iceberg_tag_operations"; + String fullTableName = + String.format("%s.%s.%s", getCatalogName(), getDefaultDatabase(), tableName); + String tag1 = "tag1"; + dropTableIfExists(tableName); + createSimpleTable(tableName); + + // create tag and query data using tag + sql(String.format("INSERT INTO %s VALUES(1, '1', 1);", tableName)); + List tableData = getTableData(tableName); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + List snapshots = + getSparkSession() + .sql(String.format("SELECT snapshot_id FROM %s.snapshots", fullTableName)) + .collectAsList(); + Assertions.assertEquals(1, snapshots.size()); + long snapshotId = snapshots.get(0).getLong(0); + + sql(String.format("ALTER TABLE %s CREATE TAG IF NOT EXISTS `%s`", tableName, tag1)); + sql(String.format("INSERT INTO %s VALUES(2, '2', 2);", tableName)); + tableData = getQueryData(getSelectAllSqlWithOrder(tableName)); + Assertions.assertEquals(2, tableData.size()); + Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); + + tableData = getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, tag1)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + sql(String.format("ALTER TABLE %s CREATE OR REPLACE TAG `%s`", tableName, tag1)); + tableData = + getQueryData( + String.format("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, tag1)); + Assertions.assertEquals(2, tableData.size()); + Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); + + // replace tag + sql( + String.format( + "ALTER TABLE %s REPLACE TAG `%s` AS OF VERSION %d RETAIN 1 DAYS", + tableName, tag1, snapshotId)); + tableData = getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, tag1)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + // drop tag + sql(String.format("ALTER TABLE %s DROP TAG `%s`", tableName, tag1)); + Assertions.assertThrows( + ValidationException.class, + () -> sql(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, tag1))); + } + + private void testIcebergIdentifierOperations() { + String tableName = "test_iceberg_identifier_operations"; + List columnInfos = + Arrays.asList( + SparkTableInfo.SparkColumnInfo.of("id", DataTypes.IntegerType, "id comment", false), + SparkTableInfo.SparkColumnInfo.of("name", DataTypes.StringType, "", false), + SparkTableInfo.SparkColumnInfo.of("ts", DataTypes.TimestampType, null, true)); + dropTableIfExists(tableName); + sql( + String.format( + "CREATE TABLE %s (id INT COMMENT 'id comment' NOT NULL, name STRING COMMENT '' NOT NULL, age INT)", + tableName)); + SparkTableInfo tableInfo = getTableInfo(tableName); + SparkTableInfoChecker checker = + SparkTableInfoChecker.create().withName(tableName).withColumns(columnInfos); + checker.check(tableInfo); + Map tableProperties = tableInfo.getTableProperties(); + Assertions.assertNull(tableProperties.get(ICEBERG_IDENTIFIER_FIELDS)); + + // add identifier + sql(String.format("ALTER TABLE %s SET IDENTIFIER FIELDS id, name", tableName)); + tableInfo = getTableInfo(tableName); + tableProperties = tableInfo.getTableProperties(); + Assertions.assertEquals("[name,id]", tableProperties.get(ICEBERG_IDENTIFIER_FIELDS)); + + // drop identifier + sql(String.format("ALTER TABLE %s DROP IDENTIFIER 'id1'", tableName)); + tableInfo = getTableInfo(tableName); + tableProperties = tableInfo.getTableProperties(); + Assertions.assertNull(tableProperties.get(ICEBERG_IDENTIFIER_FIELDS)); + } + + private void testIcebergDistributionAndOrderingOperations() { + String tableName = "test_iceberg_distribution_and_ordering_operations"; + dropTableIfExists(tableName); + createSimpleTable(tableName); + + SparkTableInfo tableInfo = getTableInfo(tableName); + Map tableProperties = tableInfo.getTableProperties(); + Assertions.assertNull(tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + Assertions.assertNull(tableProperties.get(ICEBERG_SORT_ORDER)); + + // set globally ordering + sql(String.format("ALTER TABLE %s WRITE ORDERED BY id DESC", tableName)); + tableInfo = getTableInfo(tableName); + tableProperties = tableInfo.getTableProperties(); + Assertions.assertEquals("range", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + Assertions.assertEquals("id DESC NULLS LAST", tableProperties.get(ICEBERG_SORT_ORDER)); + + // set locally ordering + sql(String.format("ALTER TABLE %s WRITE LOCALLY ORDERED BY id DESC", tableName)); + tableInfo = getTableInfo(tableName); + tableProperties = tableInfo.getTableProperties(); + Assertions.assertEquals("none", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + Assertions.assertEquals("id DESC NULLS LAST", tableProperties.get(ICEBERG_SORT_ORDER)); + + // set distribution + sql(String.format("ALTER TABLE %s WRITE DISTRIBUTED BY PARTITION", tableName)); + tableInfo = getTableInfo(tableName); + tableProperties = tableInfo.getTableProperties(); + Assertions.assertEquals("hash", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + Assertions.assertNull(tableProperties.get(ICEBERG_SORT_ORDER)); + + // set distribution with locally ordering + sql( + String.format( + "ALTER TABLE %s WRITE DISTRIBUTED BY PARTITION LOCALLY ORDERED BY id desc", tableName)); + tableInfo = getTableInfo(tableName); + tableProperties = tableInfo.getTableProperties(); + Assertions.assertEquals("hash", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + Assertions.assertEquals("id DESC NULLS LAST", tableProperties.get(ICEBERG_SORT_ORDER)); + } + private List getIcebergSimpleTableColumn() { return Arrays.asList( SparkTableInfo.SparkColumnInfo.of("id", DataTypes.IntegerType, "id comment"), diff --git a/spark-connector/spark-connector/build.gradle.kts b/spark-connector/spark-connector/build.gradle.kts index 1cd7a7f2fb0..9461d13fe1a 100644 --- a/spark-connector/spark-connector/build.gradle.kts +++ b/spark-connector/spark-connector/build.gradle.kts @@ -19,6 +19,7 @@ val sparkMajorVersion: String = sparkVersion.substringBeforeLast(".") val icebergVersion: String = libs.versions.iceberg.get() val kyuubiVersion: String = libs.versions.kyuubi.get() val scalaJava8CompatVersion: String = libs.versions.scala.java.compat.get() +val scalaCollectionCompatVersion: String = libs.versions.scala.collection.compat.get() dependencies { implementation(project(":catalogs:bundled-catalog", configuration = "shadow")) @@ -30,6 +31,7 @@ dependencies { compileOnly("org.apache.spark:spark-catalyst_$scalaVersion:$sparkVersion") compileOnly("org.apache.spark:spark-sql_$scalaVersion:$sparkVersion") compileOnly("org.scala-lang.modules:scala-java8-compat_$scalaVersion:$scalaJava8CompatVersion") + compileOnly("org.scala-lang.modules:scala-collection-compat_$scalaVersion:$scalaCollectionCompatVersion") annotationProcessor(libs.lombok) compileOnly(libs.lombok) diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/GravitinoIcebergSparkSessionExtensions.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/GravitinoIcebergSparkSessionExtensions.java new file mode 100644 index 00000000000..3da60c979be --- /dev/null +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/GravitinoIcebergSparkSessionExtensions.java @@ -0,0 +1,22 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.spark.connector.iceberg.extensions; + +import org.apache.spark.sql.SparkSessionExtensions; +import scala.Function1; + +public class GravitinoIcebergSparkSessionExtensions + implements Function1 { + + @Override + public Void apply(SparkSessionExtensions extensions) { + + // planner extensions + extensions.injectPlannerStrategy(IcebergExtendedDataSourceV2Strategy::new); + + // There must be a return value, and Void only supports returning null, not other types. + return null; + } +} diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/IcebergExtendedDataSourceV2Strategy.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/IcebergExtendedDataSourceV2Strategy.java new file mode 100644 index 00000000000..ab452ffad5a --- /dev/null +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/IcebergExtendedDataSourceV2Strategy.java @@ -0,0 +1,235 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.spark.connector.iceberg.extensions; + +import com.datastrato.gravitino.spark.connector.iceberg.GravitinoIcebergCatalog; +import java.util.Collections; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.plans.logical.AddPartitionField; +import org.apache.spark.sql.catalyst.plans.logical.CreateOrReplaceBranch; +import org.apache.spark.sql.catalyst.plans.logical.CreateOrReplaceTag; +import org.apache.spark.sql.catalyst.plans.logical.DropBranch; +import org.apache.spark.sql.catalyst.plans.logical.DropIdentifierFields; +import org.apache.spark.sql.catalyst.plans.logical.DropPartitionField; +import org.apache.spark.sql.catalyst.plans.logical.DropTag; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.catalyst.plans.logical.ReplacePartitionField; +import org.apache.spark.sql.catalyst.plans.logical.SetIdentifierFields; +import org.apache.spark.sql.catalyst.plans.logical.SetWriteDistributionAndOrdering; +import org.apache.spark.sql.connector.catalog.CatalogPlugin; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.execution.SparkPlan; +import org.apache.spark.sql.execution.datasources.v2.AddPartitionFieldExec; +import org.apache.spark.sql.execution.datasources.v2.CreateOrReplaceBranchExec; +import org.apache.spark.sql.execution.datasources.v2.CreateOrReplaceTagExec; +import org.apache.spark.sql.execution.datasources.v2.DropBranchExec; +import org.apache.spark.sql.execution.datasources.v2.DropIdentifierFieldsExec; +import org.apache.spark.sql.execution.datasources.v2.DropPartitionFieldExec; +import org.apache.spark.sql.execution.datasources.v2.DropTagExec; +import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Strategy; +import org.apache.spark.sql.execution.datasources.v2.ReplacePartitionFieldExec; +import org.apache.spark.sql.execution.datasources.v2.SetIdentifierFieldsExec; +import org.apache.spark.sql.execution.datasources.v2.SetWriteDistributionAndOrderingExec; +import scala.Option; +import scala.Some; +import scala.Tuple2; +import scala.collection.JavaConverters; +import scala.collection.Seq; + +public class IcebergExtendedDataSourceV2Strategy extends ExtendedDataSourceV2Strategy { + + private final SparkSession spark; + + public IcebergExtendedDataSourceV2Strategy(SparkSession spark) { + super(spark); + this.spark = spark; + } + + @Override + public Seq apply(LogicalPlan plan) { + if (plan instanceof AddPartitionField) { + AddPartitionField addPartitionField = (AddPartitionField) plan; + return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier(spark, addPartitionField.table()) + .map( + catalogAndIdentifier -> { + TableCatalog catalog = catalogAndIdentifier._1(); + Identifier identifier = catalogAndIdentifier._2(); + AddPartitionFieldExec addPartitionFieldExec = + new AddPartitionFieldExec( + catalog, + identifier, + addPartitionField.transform(), + addPartitionField.name()); + return toSeq(addPartitionFieldExec); + }) + .getOrElse(() -> super.apply(plan)); + } else if (plan instanceof CreateOrReplaceBranch) { + CreateOrReplaceBranch createOrReplaceBranch = (CreateOrReplaceBranch) plan; + return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier( + spark, createOrReplaceBranch.table()) + .map( + catalogAndIdentifier -> { + TableCatalog catalog = catalogAndIdentifier._1(); + Identifier identifier = catalogAndIdentifier._2(); + CreateOrReplaceBranchExec createOrReplaceBranchExec = + new CreateOrReplaceBranchExec( + catalog, + identifier, + createOrReplaceBranch.branch(), + createOrReplaceBranch.branchOptions(), + createOrReplaceBranch.replace(), + createOrReplaceBranch.ifNotExists()); + return toSeq(createOrReplaceBranchExec); + }) + .getOrElse(() -> super.apply(plan)); + } else if (plan instanceof CreateOrReplaceTag) { + CreateOrReplaceTag createOrReplaceTag = (CreateOrReplaceTag) plan; + return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier( + spark, createOrReplaceTag.table()) + .map( + catalogAndIdentifier -> { + TableCatalog catalog = catalogAndIdentifier._1(); + Identifier identifier = catalogAndIdentifier._2(); + CreateOrReplaceTagExec createOrReplaceTagExec = + new CreateOrReplaceTagExec( + catalog, + identifier, + createOrReplaceTag.tag(), + createOrReplaceTag.tagOptions(), + createOrReplaceTag.replace(), + createOrReplaceTag.ifNotExists()); + return toSeq(createOrReplaceTagExec); + }) + .getOrElse(() -> super.apply(plan)); + } else if (plan instanceof DropBranch) { + DropBranch dropBranch = (DropBranch) plan; + return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier(spark, dropBranch.table()) + .map( + catalogAndIdentifier -> { + TableCatalog catalog = catalogAndIdentifier._1(); + Identifier identifier = catalogAndIdentifier._2(); + DropBranchExec dropBranchExec = + new DropBranchExec( + catalog, identifier, dropBranch.branch(), dropBranch.ifExists()); + return toSeq(dropBranchExec); + }) + .getOrElse(() -> super.apply(plan)); + } else if (plan instanceof DropTag) { + DropTag dropTag = (DropTag) plan; + return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier(spark, dropTag.table()) + .map( + catalogAndIdentifier -> { + TableCatalog catalog = catalogAndIdentifier._1(); + Identifier identifier = catalogAndIdentifier._2(); + DropTagExec dropTagExec = + new DropTagExec(catalog, identifier, dropTag.tag(), dropTag.ifExists()); + return toSeq(dropTagExec); + }) + .getOrElse(() -> super.apply(plan)); + } else if (plan instanceof DropPartitionField) { + DropPartitionField dropPartitionField = (DropPartitionField) plan; + return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier( + spark, dropPartitionField.table()) + .map( + catalogAndIdentifier -> { + TableCatalog catalog = catalogAndIdentifier._1(); + Identifier identifier = catalogAndIdentifier._2(); + DropPartitionFieldExec dropPartitionFieldExec = + new DropPartitionFieldExec(catalog, identifier, dropPartitionField.transform()); + return toSeq(dropPartitionFieldExec); + }) + .getOrElse(() -> super.apply(plan)); + } else if (plan instanceof ReplacePartitionField) { + ReplacePartitionField replacePartitionField = (ReplacePartitionField) plan; + return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier( + spark, replacePartitionField.table()) + .map( + catalogAndIdentifier -> { + TableCatalog catalog = catalogAndIdentifier._1(); + Identifier identifier = catalogAndIdentifier._2(); + ReplacePartitionFieldExec replacePartitionFieldExec = + new ReplacePartitionFieldExec( + catalog, + identifier, + replacePartitionField.transformFrom(), + replacePartitionField.transformTo(), + replacePartitionField.name()); + return toSeq(replacePartitionFieldExec); + }) + .getOrElse(() -> super.apply(plan)); + } else if (plan instanceof SetIdentifierFields) { + SetIdentifierFields setIdentifierFields = (SetIdentifierFields) plan; + return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier( + spark, setIdentifierFields.table()) + .map( + catalogAndIdentifier -> { + TableCatalog catalog = catalogAndIdentifier._1(); + Identifier identifier = catalogAndIdentifier._2(); + SetIdentifierFieldsExec setIdentifierFieldsExec = + new SetIdentifierFieldsExec(catalog, identifier, setIdentifierFields.fields()); + return toSeq(setIdentifierFieldsExec); + }) + .getOrElse(() -> super.apply(plan)); + } else if (plan instanceof DropIdentifierFields) { + DropIdentifierFields dropIdentifierFields = (DropIdentifierFields) plan; + return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier( + spark, dropIdentifierFields.table()) + .map( + catalogAndIdentifier -> { + TableCatalog catalog = catalogAndIdentifier._1(); + Identifier identifier = catalogAndIdentifier._2(); + DropIdentifierFieldsExec dropIdentifierFieldsExec = + new DropIdentifierFieldsExec( + catalog, identifier, dropIdentifierFields.fields()); + return toSeq(dropIdentifierFieldsExec); + }) + .getOrElse(() -> super.apply(plan)); + } else if (plan instanceof SetWriteDistributionAndOrdering) { + SetWriteDistributionAndOrdering setWriteDistributionAndOrdering = + (SetWriteDistributionAndOrdering) plan; + return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier( + spark, setWriteDistributionAndOrdering.table()) + .map( + catalogAndIdentifier -> { + TableCatalog catalog = catalogAndIdentifier._1(); + Identifier identifier = catalogAndIdentifier._2(); + SetWriteDistributionAndOrderingExec setWriteDistributionAndOrderingExec = + new SetWriteDistributionAndOrderingExec( + catalog, + identifier, + setWriteDistributionAndOrdering.distributionMode(), + setWriteDistributionAndOrdering.sortOrder()); + return toSeq(setWriteDistributionAndOrderingExec); + }) + .getOrElse(() -> super.apply(plan)); + } else { + return super.apply(plan); + } + } + + private Seq toSeq(SparkPlan plan) { + return JavaConverters.asScalaIteratorConverter(Collections.singletonList(plan).listIterator()) + .asScala() + .toSeq(); + } + + static class IcebergCatalogAndIdentifier { + static Option> buildCatalogAndIdentifier( + SparkSession spark, Seq identifier) { + Spark3Util.CatalogAndIdentifier catalogAndIdentifier = + Spark3Util.catalogAndIdentifier( + spark, scala.collection.JavaConversions.seqAsJavaList(identifier)); + CatalogPlugin catalog = catalogAndIdentifier.catalog(); + if (catalog instanceof GravitinoIcebergCatalog) { + return Some.apply(new Tuple2<>((TableCatalog) catalog, catalogAndIdentifier.identifier())); + } else { + // TODO: support SparkSessionCatalog + return Option.empty(); + } + } + } +} diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java index 201666cc004..57fdb6bfa47 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java @@ -12,6 +12,7 @@ import com.datastrato.gravitino.spark.connector.catalog.GravitinoCatalogManager; import com.datastrato.gravitino.spark.connector.hive.GravitinoHiveCatalog; import com.datastrato.gravitino.spark.connector.iceberg.GravitinoIcebergCatalog; +import com.datastrato.gravitino.spark.connector.iceberg.extensions.GravitinoIcebergSparkSessionExtensions; import com.google.common.base.Preconditions; import java.util.Collections; import java.util.Locale; @@ -35,7 +36,10 @@ public class GravitinoDriverPlugin implements DriverPlugin { private GravitinoCatalogManager catalogManager; private static final String[] GRAVITINO_DRIVER_EXTENSIONS = - new String[] {IcebergSparkSessionExtensions.class.getName()}; + new String[] { + GravitinoIcebergSparkSessionExtensions.class.getName(), + IcebergSparkSessionExtensions.class.getName() + }; @Override public Map init(SparkContext sc, PluginContext pluginContext) { From 90b8d14c8c43b248836932bd38f74b55eb73b1fa Mon Sep 17 00:00:00 2001 From: caican00 Date: Mon, 13 May 2024 20:02:22 +0800 Subject: [PATCH 05/18] update --- .../spark/iceberg/SparkIcebergCatalogIT.java | 163 +++++++++++------- .../spark/connector/catalog/BaseCatalog.java | 104 ++++++++++- .../connector/hive/GravitinoHiveCatalog.java | 12 +- .../spark/connector/hive/SparkHiveTable.java | 10 ++ .../iceberg/GravitinoIcebergCatalog.java | 16 +- .../connector/iceberg/SparkIcebergTable.java | 14 ++ 6 files changed, 227 insertions(+), 92 deletions(-) diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java index 8928924a040..f96b97975fd 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java @@ -8,27 +8,24 @@ import com.datastrato.gravitino.integration.test.util.spark.SparkMetadataColumnInfo; import com.datastrato.gravitino.integration.test.util.spark.SparkTableInfo; import com.datastrato.gravitino.integration.test.util.spark.SparkTableInfoChecker; +import com.datastrato.gravitino.spark.connector.iceberg.SparkIcebergTable; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import java.io.File; -import java.sql.Timestamp; -import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import lombok.Data; import org.apache.hadoop.fs.Path; -import org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions; -import org.apache.spark.SparkConf; import org.apache.spark.sql.Column; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; @@ -36,8 +33,9 @@ import org.apache.spark.sql.connector.catalog.CatalogPlugin; import org.apache.spark.sql.connector.catalog.FunctionCatalog; import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCatalog; import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; -import org.apache.spark.sql.internal.StaticSQLConf; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructField; import org.junit.jupiter.api.Assertions; @@ -245,63 +243,76 @@ void testIcebergTableRowLevelOperations(IcebergTableWriteProperties icebergTable testIcebergMergeIntoUpdateOperation(icebergTableWriteProperties); } - @Test - void testIcebergAsOfQuery() { - String tableName = "test_iceberg_as_of_query"; - String fullTableName = - String.format("%s.%s.%s", getCatalogName(), getDefaultDatabase(), tableName); - dropTableIfExists(tableName); - createSimpleTable(tableName); - - checkTableColumns(tableName, getSimpleTableColumn(), getTableInfo(tableName)); - - sql(String.format("INSERT INTO %s VALUES (1, '1', 1)", tableName)); - List snapshots = - getSparkSession() - .sql(String.format("SELECT snapshot_id FROM %s.snapshots", fullTableName)) - .collectAsList(); - Assertions.assertEquals(1, snapshots.size()); - long snapshotId = snapshots.get(0).getLong(0); - List timestamp = - getSparkSession() - .sql(String.format("SELECT committed_at FROM %s.snapshots", fullTableName)) - .collectAsList(); - Assertions.assertEquals(1, timestamp.size()); - Timestamp timestampAt = timestamp.get(0).getTimestamp(0); - waitUntilAfter(timestampAt.getTime()); - Timestamp firstSnapshotTimestamp = - Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); - sql(String.format("INSERT INTO %s VALUES (2, '2', 2)", tableName)); - - List tableData = getQueryData(getSelectAllSqlWithOrder(tableName)); - Assertions.assertEquals(2, tableData.size()); - Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); - - tableData = - getQueryData( - String.format( - "SELECT * FROM %s TIMESTAMP AS OF '%s'", tableName, firstSnapshotTimestamp)); - Assertions.assertEquals(1, tableData.size()); - Assertions.assertEquals("1,1,1", tableData.get(0)); - tableData = - getQueryData( - String.format( - "SELECT * FROM %s FOR SYSTEM_TIME AS OF '%s'", tableName, firstSnapshotTimestamp)); - Assertions.assertEquals(1, tableData.size()); - Assertions.assertEquals("1,1,1", tableData.get(0)); - - tableData = - getQueryData(String.format("SELECT * FROM %s VERSION AS OF %d", tableName, snapshotId)); - Assertions.assertEquals(1, tableData.size()); - Assertions.assertEquals("1,1,1", tableData.get(0)); - tableData = - getQueryData( - String.format("SELECT * FROM %s FOR SYSTEM_VERSION AS OF %d", tableName, snapshotId)); - Assertions.assertEquals(1, tableData.size()); - Assertions.assertEquals("1,1,1", tableData.get(0)); - } + @Test + void testIcebergAsOfQuery() throws NoSuchTableException { + String tableName = "test_iceberg_as_of_query"; + dropTableIfExists(tableName); + createSimpleTable(tableName); + checkTableColumns(tableName, getSimpleTableColumn(), getTableInfo(tableName)); - private void testMetadataColumns() { + sql(String.format("INSERT INTO %s VALUES (1, '1', 1)", tableName)); + List tableData = getQueryData(getSelectAllSqlWithOrder(tableName, "id")); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + SparkIcebergTable sparkIcebergTable = getSparkIcebergTableInstance(tableName); + long snapshotId = getCurrentSnapshotId(tableName); + sparkIcebergTable.table().manageSnapshots().createBranch("test_branch", snapshotId).commit(); + sparkIcebergTable.table().manageSnapshots().createTag("test_tag", snapshotId).commit(); + long snapshotTimestamp = getCurrentSnapshotTimestamp(tableName); + long timestamp = waitUntilAfter(snapshotTimestamp + 1000); + waitUntilAfter(timestamp + 1000); + long timestampInSeconds = TimeUnit.MILLISECONDS.toSeconds(timestamp); + + // create a second snapshot + sql(String.format("INSERT INTO %s VALUES (2, '2', 2)", tableName)); + tableData = getQueryData(getSelectAllSqlWithOrder(tableName, "id")); + Assertions.assertEquals(2, tableData.size()); + Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); + + tableData = + getQueryData( + String.format("SELECT * FROM %s TIMESTAMP AS OF %s", tableName, timestampInSeconds)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + tableData = + getQueryData( + String.format( + "SELECT * FROM %s FOR SYSTEM_TIME AS OF %s", tableName, timestampInSeconds)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + tableData = + getQueryData(String.format("SELECT * FROM %s VERSION AS OF %d", tableName, snapshotId)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + tableData = + getQueryData( + String.format("SELECT * FROM %s FOR SYSTEM_VERSION AS OF %d", tableName, snapshotId)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + tableData = + getQueryData(String.format("SELECT * FROM %s VERSION AS OF 'test_branch'", tableName)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + tableData = + getQueryData( + String.format("SELECT * FROM %s FOR SYSTEM_VERSION AS OF 'test_branch'", tableName)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + tableData = getQueryData(String.format("SELECT * FROM %s VERSION AS OF 'test_tag'", tableName)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + tableData = + getQueryData( + String.format("SELECT * FROM %s FOR SYSTEM_VERSION AS OF 'test_tag'", tableName)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + } + + private void testMetadataColumns() { String tableName = "test_metadata_columns"; dropTableIfExists(tableName); String createTableSQL = getCreateSimpleTableString(tableName); @@ -621,4 +632,32 @@ static IcebergTableWriteProperties of( return new IcebergTableWriteProperties(isPartitionedTable, formatVersion, writeMode); } } + + private SparkIcebergTable getSparkIcebergTableInstance(String tableName) + throws NoSuchTableException { + CatalogPlugin catalogPlugin = + getSparkSession().sessionState().catalogManager().catalog(getCatalogName()); + Assertions.assertInstanceOf(TableCatalog.class, catalogPlugin); + TableCatalog catalog = (TableCatalog) catalogPlugin; + Table table = catalog.loadTable(Identifier.of(new String[] {getDefaultDatabase()}, tableName)); + return (SparkIcebergTable) table; + } + + private long getCurrentSnapshotTimestamp(String tableName) throws NoSuchTableException { + SparkIcebergTable sparkIcebergTable = getSparkIcebergTableInstance(tableName); + return sparkIcebergTable.table().currentSnapshot().timestampMillis(); + } + + private long getCurrentSnapshotId(String tableName) throws NoSuchTableException { + SparkIcebergTable sparkIcebergTable = getSparkIcebergTableInstance(tableName); + return sparkIcebergTable.table().currentSnapshot().snapshotId(); + } + + private long waitUntilAfter(Long timestampMillis) { + long current = System.currentTimeMillis(); + while (current <= timestampMillis) { + current = System.currentTimeMillis(); + } + return current; + } } diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/catalog/BaseCatalog.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/catalog/BaseCatalog.java index 703b113527f..4284c5cd585 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/catalog/BaseCatalog.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/catalog/BaseCatalog.java @@ -92,6 +92,7 @@ protected abstract TableCatalog createAndInitSparkCatalog( * * @param identifier Spark's table identifier * @param gravitinoTable Gravitino table to do DDL operations + * @param sparkTable Spark internal table to do IO operations * @param sparkCatalog specific Spark catalog to do IO operations * @param propertiesConverter transform properties between Gravitino and Spark * @param sparkTransformConverter sparkTransformConverter convert transforms between Gravitino and @@ -101,6 +102,7 @@ protected abstract TableCatalog createAndInitSparkCatalog( protected abstract Table createSparkTable( Identifier identifier, com.datastrato.gravitino.rel.Table gravitinoTable, + Table sparkTable, TableCatalog sparkCatalog, PropertiesConverter propertiesConverter, SparkTransformConverter sparkTransformConverter); @@ -194,8 +196,14 @@ public Table createTable( partitionings, distributionAndSortOrdersInfo.getDistribution(), distributionAndSortOrdersInfo.getSortOrders()); + org.apache.spark.sql.connector.catalog.Table sparkTable = loadSparkTable(ident); return createSparkTable( - ident, gravitinoTable, sparkCatalog, propertiesConverter, sparkTransformConverter); + ident, + gravitinoTable, + sparkTable, + sparkCatalog, + propertiesConverter, + sparkTransformConverter); } catch (NoSuchSchemaException e) { throw new NoSuchNamespaceException(ident.namespace()); } catch (com.datastrato.gravitino.exceptions.TableAlreadyExistsException e) { @@ -206,14 +214,52 @@ public Table createTable( @Override public Table loadTable(Identifier ident) throws NoSuchTableException { try { - String database = getDatabase(ident); - com.datastrato.gravitino.rel.Table gravitinoTable = - gravitinoCatalogClient - .asTableCatalog() - .loadTable(NameIdentifier.of(metalakeName, catalogName, database, ident.name())); + com.datastrato.gravitino.rel.Table gravitinoTable = loadGravitinoTable(ident); + org.apache.spark.sql.connector.catalog.Table sparkTable = loadSparkTable(ident); // Will create a catalog specific table return createSparkTable( - ident, gravitinoTable, sparkCatalog, propertiesConverter, sparkTransformConverter); + ident, + gravitinoTable, + sparkTable, + sparkCatalog, + propertiesConverter, + sparkTransformConverter); + } catch (com.datastrato.gravitino.exceptions.NoSuchTableException e) { + throw new NoSuchTableException(ident); + } + } + + @Override + public Table loadTable(Identifier ident, String version) throws NoSuchTableException { + try { + com.datastrato.gravitino.rel.Table gravitinoTable = loadGravitinoTable(ident); + org.apache.spark.sql.connector.catalog.Table sparkTable = loadSparkTable(ident, version); + // Will create a catalog specific table + return createSparkTable( + ident, + gravitinoTable, + sparkTable, + sparkCatalog, + propertiesConverter, + sparkTransformConverter); + } catch (com.datastrato.gravitino.exceptions.NoSuchTableException e) { + throw new NoSuchTableException(ident); + } + } + + @Override + public Table loadTable(Identifier ident, long timestamp) throws NoSuchTableException { + try { + com.datastrato.gravitino.rel.Table gravitinoTable = loadGravitinoTable(ident); + org.apache.spark.sql.connector.catalog.Table sparkTable = loadSparkTable(ident, timestamp); + // Will create a catalog specific table + return createSparkTable( + ident, + gravitinoTable, + sparkTable, + sparkCatalog, + propertiesConverter, + sparkTransformConverter); } catch (com.datastrato.gravitino.exceptions.NoSuchTableException e) { throw new NoSuchTableException(ident); } @@ -240,8 +286,14 @@ public Table alterTable(Identifier ident, TableChange... changes) throws NoSuchT .alterTable( NameIdentifier.of(metalakeName, catalogName, getDatabase(ident), ident.name()), gravitinoTableChanges); + org.apache.spark.sql.connector.catalog.Table sparkTable = loadSparkTable(ident); return createSparkTable( - ident, gravitinoTable, sparkCatalog, propertiesConverter, sparkTransformConverter); + ident, + gravitinoTable, + sparkTable, + sparkCatalog, + propertiesConverter, + sparkTransformConverter); } catch (com.datastrato.gravitino.exceptions.NoSuchTableException e) { throw new NoSuchTableException(ident); } @@ -509,4 +561,40 @@ private com.datastrato.gravitino.rel.Table loadGravitinoTable(Identifier ident) throw new NoSuchTableException(ident); } } + + private Table loadSparkTable(Identifier ident) { + try { + return sparkCatalog.loadTable(ident); + } catch (NoSuchTableException e) { + throw new RuntimeException( + String.format( + "Failed to load the real sparkTable: %s", + String.join(".", getDatabase(ident), ident.name())), + e); + } + } + + private Table loadSparkTable(Identifier ident, String version) { + try { + return sparkCatalog.loadTable(ident, version); + } catch (NoSuchTableException e) { + throw new RuntimeException( + String.format( + "Failed to load the real sparkTable: %s", + String.join(".", getDatabase(ident), ident.name())), + e); + } + } + + private Table loadSparkTable(Identifier ident, long timestamp) { + try { + return sparkCatalog.loadTable(ident, timestamp); + } catch (NoSuchTableException e) { + throw new RuntimeException( + String.format( + "Failed to load the real sparkTable: %s", + String.join(".", getDatabase(ident), ident.name())), + e); + } + } } diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/GravitinoHiveCatalog.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/GravitinoHiveCatalog.java index cbfd09a4d15..feff8ad760d 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/GravitinoHiveCatalog.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/GravitinoHiveCatalog.java @@ -12,7 +12,6 @@ import java.util.Map; import org.apache.kyuubi.spark.connector.hive.HiveTable; import org.apache.kyuubi.spark.connector.hive.HiveTableCatalog; -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.TableCatalog; import org.apache.spark.sql.util.CaseInsensitiveStringMap; @@ -34,19 +33,10 @@ protected TableCatalog createAndInitSparkCatalog( protected org.apache.spark.sql.connector.catalog.Table createSparkTable( Identifier identifier, Table gravitinoTable, + org.apache.spark.sql.connector.catalog.Table sparkTable, TableCatalog sparkHiveCatalog, PropertiesConverter propertiesConverter, SparkTransformConverter sparkTransformConverter) { - org.apache.spark.sql.connector.catalog.Table sparkTable; - try { - sparkTable = sparkHiveCatalog.loadTable(identifier); - } catch (NoSuchTableException e) { - throw new RuntimeException( - String.format( - "Failed to load the real sparkTable: %s", - String.join(".", getDatabase(identifier), identifier.name())), - e); - } return new SparkHiveTable( identifier, gravitinoTable, diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/SparkHiveTable.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/SparkHiveTable.java index e27916af283..48dbc5b80bb 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/SparkHiveTable.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/SparkHiveTable.java @@ -15,12 +15,15 @@ import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; /** Keep consistent behavior with the SparkIcebergTable */ public class SparkHiveTable extends HiveTable { private GravitinoTableInfoHelper gravitinoTableInfoHelper; + private org.apache.spark.sql.connector.catalog.Table sparkTable; public SparkHiveTable( Identifier identifier, @@ -33,6 +36,7 @@ public SparkHiveTable( this.gravitinoTableInfoHelper = new GravitinoTableInfoHelper( false, identifier, gravitinoTable, propertiesConverter, sparkTransformConverter); + this.sparkTable = hiveTable; } @Override @@ -55,4 +59,10 @@ public Map properties() { public Transform[] partitioning() { return gravitinoTableInfoHelper.partitioning(); } + + /** to keep consistent behavior with SparkIcebergTable. */ + @Override + public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { + return ((HiveTable) sparkTable).newScanBuilder(options); + } } diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/GravitinoIcebergCatalog.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/GravitinoIcebergCatalog.java index d44dd1edb5e..4d9bc7b1f93 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/GravitinoIcebergCatalog.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/GravitinoIcebergCatalog.java @@ -14,7 +14,6 @@ import org.apache.iceberg.spark.source.SparkTable; import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.connector.catalog.FunctionCatalog; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.TableCatalog; @@ -40,23 +39,18 @@ protected TableCatalog createAndInitSparkCatalog( return icebergCatalog; } + /** + * Migrated `loadTable(identifier)` to the BaseCatalog class and execute `loadTable(identifier)` + * before createSparkTable to load sparkTable with different parameters easily. + */ @Override protected org.apache.spark.sql.connector.catalog.Table createSparkTable( Identifier identifier, Table gravitinoTable, + org.apache.spark.sql.connector.catalog.Table sparkTable, TableCatalog sparkIcebergCatalog, PropertiesConverter propertiesConverter, SparkTransformConverter sparkTransformConverter) { - org.apache.spark.sql.connector.catalog.Table sparkTable; - try { - sparkTable = sparkIcebergCatalog.loadTable(identifier); - } catch (NoSuchTableException e) { - throw new RuntimeException( - String.format( - "Failed to load the real sparkTable: %s", - String.join(".", getDatabase(identifier), identifier.name())), - e); - } return new SparkIcebergTable( identifier, gravitinoTable, diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/SparkIcebergTable.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/SparkIcebergTable.java index 870ff535f88..714a04f2057 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/SparkIcebergTable.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/SparkIcebergTable.java @@ -15,7 +15,9 @@ import org.apache.iceberg.spark.source.SparkTable; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; /** * For spark-connector in Iceberg, it explicitly uses SparkTable to identify whether it is an @@ -24,6 +26,7 @@ public class SparkIcebergTable extends SparkTable { private GravitinoTableInfoHelper gravitinoTableInfoHelper; + private org.apache.spark.sql.connector.catalog.Table sparkTable; public SparkIcebergTable( Identifier identifier, @@ -36,6 +39,7 @@ public SparkIcebergTable( this.gravitinoTableInfoHelper = new GravitinoTableInfoHelper( true, identifier, gravitinoTable, propertiesConverter, sparkTransformConverter); + this.sparkTable = sparkTable; } @Override @@ -59,6 +63,16 @@ public Transform[] partitioning() { return gravitinoTableInfoHelper.partitioning(); } + /** + * Although SparkIcebergTable extended SparkTable, it also needs to initialize its member variable + * , such as snapshotId or branch, before it reused newScanBuilder from the parent class. In + * contrast, overriding newScanBuilder to support time travel is simpler and more concise. + */ + @Override + public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { + return ((SparkTable) sparkTable).newScanBuilder(options); + } + private static boolean isCacheEnabled(SparkCatalog sparkCatalog) { try { Field cacheEnabled = sparkCatalog.getClass().getDeclaredField("cacheEnabled"); From b367b65f56dbaa807a4157933b2990b14a6f25b8 Mon Sep 17 00:00:00 2001 From: caican00 Date: Thu, 16 May 2024 16:00:43 +0800 Subject: [PATCH 06/18] update --- .../spark/iceberg/SparkIcebergCatalogIT.java | 559 ++++++++++-------- .../IcebergExtendedDataSourceV2Strategy.java | 93 +-- .../plugin/GravitinoDriverPlugin.java | 10 +- .../connector/utils/SparkBaseTableHelper.java | 0 4 files changed, 358 insertions(+), 304 deletions(-) delete mode 100644 spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/utils/SparkBaseTableHelper.java diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java index ba2b1c199b7..28f21ad0024 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java @@ -8,18 +8,23 @@ import com.datastrato.gravitino.integration.test.util.spark.SparkMetadataColumnInfo; import com.datastrato.gravitino.integration.test.util.spark.SparkTableInfo; import com.datastrato.gravitino.integration.test.util.spark.SparkTableInfoChecker; +import com.datastrato.gravitino.spark.connector.iceberg.SparkIcebergTable; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import java.io.File; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; import lombok.Data; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.exceptions.ValidationException; import org.apache.spark.sql.Column; import org.apache.spark.sql.Dataset; @@ -32,6 +37,8 @@ import org.apache.spark.sql.connector.catalog.CatalogPlugin; import org.apache.spark.sql.connector.catalog.FunctionCatalog; import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCatalog; import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructField; @@ -48,7 +55,6 @@ public abstract class SparkIcebergCatalogIT extends SparkCommonIT { private static final String ICEBERG_MERGE_MODE = "write.merge.mode"; private static final String ICEBERG_WRITE_DISTRIBUTION_MODE = "write.distribution-mode"; private static final String ICEBERG_SORT_ORDER = "sort-order"; - private static final String ICEBERG_IDENTIFIER_FIELDS = "identifier-fields"; @Override protected String getCatalogName() { @@ -243,14 +249,14 @@ void testIcebergTableRowLevelOperations(IcebergTableWriteProperties icebergTable testIcebergMergeIntoUpdateOperation(icebergTableWriteProperties); } - @Test - void testIcebergSQLExtensions() { - testIcebergPartitionFieldOperations(); - testIcebergBranchOperations(); - testIcebergTagOperations(); - testIcebergIdentifierOperations(); - testIcebergDistributionAndOrderingOperations(); - } + @Test + void testIcebergSQLExtensions() throws NoSuchTableException, IOException { + testIcebergPartitionFieldOperations(); + testIcebergBranchOperations(); + testIcebergTagOperations(); + testIcebergIdentifierOperations(); + testIcebergDistributionAndOrderingOperations(); + } private void testMetadataColumns() { String tableName = "test_metadata_columns"; @@ -500,259 +506,283 @@ private void testIcebergMergeIntoUpdateOperation( checkTableUpdateByMergeInto(tableName); } - private void testIcebergPartitionFieldOperations() { - List partitionFields = - Arrays.asList("name", "truncate(1, name)", "bucket(16, id)", "days(ts)"); - String partitionExpression = "name=a/name_trunc=a/id_bucket=4/ts_day=2024-01-01"; - String tableName = "test_iceberg_partition_field_operations"; - dropTableIfExists(tableName); - sql(getCreateIcebergSimpleTableString(tableName)); - - // add partition fields - SparkTableInfo tableInfo = getTableInfo(tableName); - SparkTableInfoChecker checker = - SparkTableInfoChecker.create() - .withName(tableName) - .withColumns(getIcebergSimpleTableColumn()); - checker.check(tableInfo); - - partitionFields.forEach( - partitionField -> - sql(String.format("ALTER TABLE %s ADD PARTITION FIELD %s", tableName, partitionField))); - - tableInfo = getTableInfo(tableName); - checker = - SparkTableInfoChecker.create() - .withName(tableName) - .withColumns(getIcebergSimpleTableColumn()) - .withIdentifyPartition(Collections.singletonList("name")) - .withTruncatePartition(1, "name") - .withBucketPartition(16, Collections.singletonList("id")) - .withDayPartition("ts"); - checker.check(tableInfo); - - sql( - String.format( - "INSERT INTO %s VALUES(2,'a',cast('2024-01-01 12:00:00' as timestamp));", tableName)); - List queryResult = getTableData(tableName); - Assertions.assertEquals(1, queryResult.size()); - Assertions.assertEquals("2,a,2024-01-01 12:00:00", queryResult.get(0)); - Path partitionPath = new Path(getTableLocation(tableInfo), partitionExpression); - checkDirExists(partitionPath); - - // replace partition fields - sql(String.format("ALTER TABLE %s REPLACE PARTITION FIELD ts_day WITH months(ts)", tableName)); - tableInfo = getTableInfo(tableName); - checker = - SparkTableInfoChecker.create() - .withName(tableName) - .withColumns(getIcebergSimpleTableColumn()) - .withIdentifyPartition(Collections.singletonList("name")) - .withTruncatePartition(1, "name") - .withBucketPartition(16, Collections.singletonList("id")) - .withMonthPartition("ts"); - checker.check(tableInfo); - - // drop partition fields - sql(String.format("ALTER TABLE %s DROP PARTITION FIELD months(ts)", tableName)); - tableInfo = getTableInfo(tableName); - checker = - SparkTableInfoChecker.create() - .withName(tableName) - .withColumns(getIcebergSimpleTableColumn()) - .withIdentifyPartition(Collections.singletonList("name")) - .withTruncatePartition(1, "name") - .withBucketPartition(16, Collections.singletonList("id")); - checker.check(tableInfo); - } + private void testIcebergPartitionFieldOperations() { + List partitionFields = + Arrays.asList("name", "truncate(1, name)", "bucket(16, id)", "days(ts)"); + String partitionExpression = "name=a/name_trunc_1=a/id_bucket_16=4/ts_day=2024-01-01"; + String tableName = "test_iceberg_partition_field_operations"; + dropTableIfExists(tableName); + sql(getCreateIcebergSimpleTableString(tableName)); - private void testIcebergBranchOperations() { - String tableName = "test_iceberg_branch_operations"; - String fullTableName = - String.format("%s.%s.%s", getCatalogName(), getDefaultDatabase(), tableName); - String branch1 = "branch1"; - dropTableIfExists(tableName); - createSimpleTable(tableName); - - // create branch and query data using branch - sql(String.format("INSERT INTO %s VALUES(1, '1', 1);", tableName)); - List tableData = getTableData(tableName); - Assertions.assertEquals(1, tableData.size()); - Assertions.assertEquals("1,1,1", tableData.get(0)); - List snapshots = - getSparkSession() - .sql(String.format("SELECT snapshot_id FROM %s.snapshots", fullTableName)) - .collectAsList(); - Assertions.assertEquals(1, snapshots.size()); - long snapshotId = snapshots.get(0).getLong(0); - - sql(String.format("ALTER TABLE %s CREATE BRANCH IF NOT EXISTS `%s`", tableName, branch1)); - sql(String.format("INSERT INTO %s VALUES(2, '2', 2);", tableName)); - tableData = getQueryData(getSelectAllSqlWithOrder(tableName)); - Assertions.assertEquals(2, tableData.size()); - Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); - - tableData = - getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, branch1)); - Assertions.assertEquals(1, tableData.size()); - Assertions.assertEquals("1,1,1", tableData.get(0)); - - sql(String.format("ALTER TABLE %s CREATE OR REPLACE BRANCH `%s`", tableName, branch1)); - tableData = - getQueryData( - String.format("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, branch1)); - Assertions.assertEquals(2, tableData.size()); - Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); - - // replace branch - sql( - String.format( - "ALTER TABLE %s REPLACE BRANCH `%s` AS OF VERSION %d RETAIN 1 DAYS", - tableName, branch1, snapshotId)); - tableData = - getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, branch1)); - Assertions.assertEquals(1, tableData.size()); - Assertions.assertEquals("1,1,1", tableData.get(0)); - - // drop branch - sql(String.format("ALTER TABLE %s DROP BRANCH `%s`", tableName, branch1)); - Assertions.assertThrows( - ValidationException.class, - () -> sql(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, branch1))); - } + // add partition fields + SparkTableInfo tableInfo = getTableInfo(tableName); + SparkTableInfoChecker checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withColumns(getIcebergSimpleTableColumn()); + checker.check(tableInfo); - private void testIcebergTagOperations() { - String tableName = "test_iceberg_tag_operations"; - String fullTableName = - String.format("%s.%s.%s", getCatalogName(), getDefaultDatabase(), tableName); - String tag1 = "tag1"; - dropTableIfExists(tableName); - createSimpleTable(tableName); - - // create tag and query data using tag - sql(String.format("INSERT INTO %s VALUES(1, '1', 1);", tableName)); - List tableData = getTableData(tableName); - Assertions.assertEquals(1, tableData.size()); - Assertions.assertEquals("1,1,1", tableData.get(0)); - List snapshots = - getSparkSession() - .sql(String.format("SELECT snapshot_id FROM %s.snapshots", fullTableName)) - .collectAsList(); - Assertions.assertEquals(1, snapshots.size()); - long snapshotId = snapshots.get(0).getLong(0); - - sql(String.format("ALTER TABLE %s CREATE TAG IF NOT EXISTS `%s`", tableName, tag1)); - sql(String.format("INSERT INTO %s VALUES(2, '2', 2);", tableName)); - tableData = getQueryData(getSelectAllSqlWithOrder(tableName)); - Assertions.assertEquals(2, tableData.size()); - Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); - - tableData = getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, tag1)); - Assertions.assertEquals(1, tableData.size()); - Assertions.assertEquals("1,1,1", tableData.get(0)); - - sql(String.format("ALTER TABLE %s CREATE OR REPLACE TAG `%s`", tableName, tag1)); - tableData = - getQueryData( - String.format("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, tag1)); - Assertions.assertEquals(2, tableData.size()); - Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); - - // replace tag - sql( - String.format( - "ALTER TABLE %s REPLACE TAG `%s` AS OF VERSION %d RETAIN 1 DAYS", - tableName, tag1, snapshotId)); - tableData = getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, tag1)); - Assertions.assertEquals(1, tableData.size()); - Assertions.assertEquals("1,1,1", tableData.get(0)); - - // drop tag - sql(String.format("ALTER TABLE %s DROP TAG `%s`", tableName, tag1)); - Assertions.assertThrows( - ValidationException.class, - () -> sql(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, tag1))); - } + partitionFields.forEach( + partitionField -> + sql(String.format("ALTER TABLE %s ADD PARTITION FIELD %s", tableName, partitionField))); - private void testIcebergIdentifierOperations() { - String tableName = "test_iceberg_identifier_operations"; - List columnInfos = - Arrays.asList( - SparkTableInfo.SparkColumnInfo.of("id", DataTypes.IntegerType, "id comment", false), - SparkTableInfo.SparkColumnInfo.of("name", DataTypes.StringType, "", false), - SparkTableInfo.SparkColumnInfo.of("ts", DataTypes.TimestampType, null, true)); - dropTableIfExists(tableName); - sql( - String.format( - "CREATE TABLE %s (id INT COMMENT 'id comment' NOT NULL, name STRING COMMENT '' NOT NULL, age INT)", - tableName)); - SparkTableInfo tableInfo = getTableInfo(tableName); - SparkTableInfoChecker checker = - SparkTableInfoChecker.create().withName(tableName).withColumns(columnInfos); - checker.check(tableInfo); - Map tableProperties = tableInfo.getTableProperties(); - Assertions.assertNull(tableProperties.get(ICEBERG_IDENTIFIER_FIELDS)); - - // add identifier - sql(String.format("ALTER TABLE %s SET IDENTIFIER FIELDS id, name", tableName)); - tableInfo = getTableInfo(tableName); - tableProperties = tableInfo.getTableProperties(); - Assertions.assertEquals("[name,id]", tableProperties.get(ICEBERG_IDENTIFIER_FIELDS)); - - // drop identifier - sql(String.format("ALTER TABLE %s DROP IDENTIFIER 'id1'", tableName)); - tableInfo = getTableInfo(tableName); - tableProperties = tableInfo.getTableProperties(); - Assertions.assertNull(tableProperties.get(ICEBERG_IDENTIFIER_FIELDS)); - } + tableInfo = getTableInfo(tableName); + checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withColumns(getIcebergSimpleTableColumn()) + .withIdentifyPartition(Collections.singletonList("name")) + .withTruncatePartition(1, "name") + .withBucketPartition(16, Collections.singletonList("id")) + .withDayPartition("ts"); + checker.check(tableInfo); - private void testIcebergDistributionAndOrderingOperations() { - String tableName = "test_iceberg_distribution_and_ordering_operations"; - dropTableIfExists(tableName); - createSimpleTable(tableName); - - SparkTableInfo tableInfo = getTableInfo(tableName); - Map tableProperties = tableInfo.getTableProperties(); - Assertions.assertNull(tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); - Assertions.assertNull(tableProperties.get(ICEBERG_SORT_ORDER)); - - // set globally ordering - sql(String.format("ALTER TABLE %s WRITE ORDERED BY id DESC", tableName)); - tableInfo = getTableInfo(tableName); - tableProperties = tableInfo.getTableProperties(); - Assertions.assertEquals("range", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); - Assertions.assertEquals("id DESC NULLS LAST", tableProperties.get(ICEBERG_SORT_ORDER)); - - // set locally ordering - sql(String.format("ALTER TABLE %s WRITE LOCALLY ORDERED BY id DESC", tableName)); - tableInfo = getTableInfo(tableName); - tableProperties = tableInfo.getTableProperties(); - Assertions.assertEquals("none", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); - Assertions.assertEquals("id DESC NULLS LAST", tableProperties.get(ICEBERG_SORT_ORDER)); - - // set distribution - sql(String.format("ALTER TABLE %s WRITE DISTRIBUTED BY PARTITION", tableName)); - tableInfo = getTableInfo(tableName); - tableProperties = tableInfo.getTableProperties(); - Assertions.assertEquals("hash", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); - Assertions.assertNull(tableProperties.get(ICEBERG_SORT_ORDER)); - - // set distribution with locally ordering - sql( - String.format( - "ALTER TABLE %s WRITE DISTRIBUTED BY PARTITION LOCALLY ORDERED BY id desc", tableName)); - tableInfo = getTableInfo(tableName); - tableProperties = tableInfo.getTableProperties(); - Assertions.assertEquals("hash", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); - Assertions.assertEquals("id DESC NULLS LAST", tableProperties.get(ICEBERG_SORT_ORDER)); - } + sql( + String.format( + "INSERT INTO %s VALUES(2,'a',cast('2024-01-01 12:00:00' as timestamp));", tableName)); + List queryResult = getTableData(tableName); + Assertions.assertEquals(1, queryResult.size()); + Assertions.assertEquals("2,a,2024-01-01 12:00:00", queryResult.get(0)); + Path partitionPath = new Path(getTableLocation(tableInfo), partitionExpression); + checkDirExists(partitionPath); + + // replace partition fields + sql(String.format("ALTER TABLE %s REPLACE PARTITION FIELD ts_day WITH months(ts)", tableName)); + tableInfo = getTableInfo(tableName); + checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withColumns(getIcebergSimpleTableColumn()) + .withIdentifyPartition(Collections.singletonList("name")) + .withTruncatePartition(1, "name") + .withBucketPartition(16, Collections.singletonList("id")) + .withMonthPartition("ts"); + checker.check(tableInfo); - private List getIcebergSimpleTableColumn() { - return Arrays.asList( - SparkTableInfo.SparkColumnInfo.of("id", DataTypes.IntegerType, "id comment"), - SparkTableInfo.SparkColumnInfo.of("name", DataTypes.StringType, ""), - SparkTableInfo.SparkColumnInfo.of("ts", DataTypes.TimestampType, null)); - } + // drop partition fields + sql(String.format("ALTER TABLE %s DROP PARTITION FIELD months(ts)", tableName)); + tableInfo = getTableInfo(tableName); + checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withColumns(getIcebergSimpleTableColumn()) + .withIdentifyPartition(Collections.singletonList("name")) + .withTruncatePartition(1, "name") + .withBucketPartition(16, Collections.singletonList("id")); + checker.check(tableInfo); + } + + private void testIcebergBranchOperations() throws NoSuchTableException { + String tableName = "test_iceberg_branch_operations"; + String branch1 = "branch1"; + dropTableIfExists(tableName); + createSimpleTable(tableName); + + // create branch and query data using branch + sql(String.format("INSERT INTO %s VALUES(1, '1', 1);", tableName)); + List tableData = getTableData(tableName); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + long snapshotId = getCurrentSnapshotId(tableName); + sql(String.format("ALTER TABLE %s CREATE BRANCH IF NOT EXISTS `%s`", tableName, branch1)); + + sql(String.format("INSERT INTO %s VALUES(2, '2', 2);", tableName)); + tableData = getQueryData(getSelectAllSqlWithOrder(tableName, "id")); + Assertions.assertEquals(2, tableData.size()); + Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); + + tableData = + getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, branch1)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + sql(String.format("ALTER TABLE %s CREATE OR REPLACE BRANCH `%s`", tableName, branch1)); + tableData = + getQueryData( + String.format("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, branch1)); + Assertions.assertEquals(2, tableData.size()); + Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); + + // replace branch + sql( + String.format( + "ALTER TABLE %s REPLACE BRANCH `%s` AS OF VERSION %d RETAIN 1 DAYS", + tableName, branch1, snapshotId)); + tableData = + getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, branch1)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + // drop branch + sql(String.format("ALTER TABLE %s DROP BRANCH `%s`", tableName, branch1)); + Assertions.assertThrows( + ValidationException.class, + () -> sql(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, branch1))); + } + + private void testIcebergTagOperations() throws NoSuchTableException { + String tableName = "test_iceberg_tag_operations"; + String tag1 = "tag1"; + dropTableIfExists(tableName); + createSimpleTable(tableName); + + // create tag and query data using tag + sql(String.format("INSERT INTO %s VALUES(1, '1', 1);", tableName)); + List tableData = getTableData(tableName); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + long snapshotId = getCurrentSnapshotId(tableName); + sql(String.format("ALTER TABLE %s CREATE TAG IF NOT EXISTS `%s`", tableName, tag1)); + + sql(String.format("INSERT INTO %s VALUES(2, '2', 2);", tableName)); + tableData = getQueryData(getSelectAllSqlWithOrder(tableName, "id")); + Assertions.assertEquals(2, tableData.size()); + Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); + + tableData = getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, tag1)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + sql(String.format("ALTER TABLE %s CREATE OR REPLACE TAG `%s`", tableName, tag1)); + tableData = + getQueryData( + String.format("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, tag1)); + Assertions.assertEquals(2, tableData.size()); + Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); + + // replace tag + sql( + String.format( + "ALTER TABLE %s REPLACE TAG `%s` AS OF VERSION %d RETAIN 1 DAYS", + tableName, tag1, snapshotId)); + tableData = getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, tag1)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + // drop tag + sql(String.format("ALTER TABLE %s DROP TAG `%s`", tableName, tag1)); + Assertions.assertThrows( + ValidationException.class, + () -> sql(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, tag1))); + } + + private void testIcebergIdentifierOperations() throws NoSuchTableException { + String tableName = "test_iceberg_identifier_operations"; + // The Identifier fields must be non-null, so a new schema with non-null fields is created here. + List columnInfos = + Arrays.asList( + SparkTableInfo.SparkColumnInfo.of("id", DataTypes.IntegerType, "id comment", false), + SparkTableInfo.SparkColumnInfo.of("name", DataTypes.StringType, "", false), + SparkTableInfo.SparkColumnInfo.of("age", DataTypes.IntegerType, null, true)); + dropTableIfExists(tableName); + + sql( + String.format( + "CREATE TABLE %s (id INT COMMENT 'id comment' NOT NULL, name STRING COMMENT '' NOT NULL, age INT)", + tableName)); + SparkTableInfo tableInfo = getTableInfo(tableName); + SparkTableInfoChecker checker = + SparkTableInfoChecker.create().withName(tableName).withColumns(columnInfos); + checker.check(tableInfo); + + SparkIcebergTable sparkIcebergTable = getSparkIcebergTableInstance(tableName); + org.apache.iceberg.Table icebergTable = sparkIcebergTable.table(); + Set identifierFieldNames = icebergTable.schema().identifierFieldNames(); + Assertions.assertEquals(0, identifierFieldNames.size()); + + // add identifier fields + sql(String.format("ALTER TABLE %s SET IDENTIFIER FIELDS id, name", tableName)); + icebergTable.refresh(); + identifierFieldNames = icebergTable.schema().identifierFieldNames(); + Assertions.assertTrue(identifierFieldNames.contains("id")); + Assertions.assertTrue(identifierFieldNames.contains("name")); + + // drop identifier fields + sql(String.format("ALTER TABLE %s DROP IDENTIFIER FIELDS name", tableName)); + icebergTable.refresh(); + identifierFieldNames = icebergTable.schema().identifierFieldNames(); + Assertions.assertTrue(identifierFieldNames.contains("id")); + Assertions.assertFalse(identifierFieldNames.contains("name")); + } + + private void testIcebergDistributionAndOrderingOperations() throws NoSuchTableException { + String tableName = "test_iceberg_distribution_and_ordering_operations"; + dropTableIfExists(tableName); + createSimpleTable(tableName); + + SparkTableInfo tableInfo = getTableInfo(tableName); + Map tableProperties = tableInfo.getTableProperties(); + Assertions.assertEquals("none", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + Assertions.assertNull(tableProperties.get(ICEBERG_SORT_ORDER)); + + SparkIcebergTable sparkIcebergTable = getSparkIcebergTableInstance(tableName); + org.apache.iceberg.Table icebergTable = sparkIcebergTable.table(); + + // set globally ordering + sql(String.format("ALTER TABLE %s WRITE ORDERED BY id DESC", tableName)); + icebergTable.refresh(); + tableInfo = getTableInfo(tableName); + tableProperties = tableInfo.getTableProperties(); + Assertions.assertEquals("range", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + SortOrder sortOrder = + SortOrder.builderFor(icebergTable.schema()) + .withOrderId(1) + .desc("id", NullOrder.NULLS_LAST) + .build(); + Assertions.assertEquals(sortOrder, icebergTable.sortOrder()); + + // set locally ordering + sql(String.format("ALTER TABLE %s WRITE LOCALLY ORDERED BY id DESC", tableName)); + icebergTable.refresh(); + tableInfo = getTableInfo(tableName); + tableProperties = tableInfo.getTableProperties(); + Assertions.assertEquals("none", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + sortOrder = + SortOrder.builderFor(icebergTable.schema()) + .withOrderId(1) + .desc("id", NullOrder.NULLS_LAST) + .build(); + Assertions.assertEquals(sortOrder, icebergTable.sortOrder()); + + // set distribution + sql( + String.format( + "ALTER TABLE %s WRITE ORDERED BY id DESC DISTRIBUTED BY PARTITION", tableName)); + icebergTable.refresh(); + tableInfo = getTableInfo(tableName); + tableProperties = tableInfo.getTableProperties(); + Assertions.assertEquals("hash", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + sortOrder = + SortOrder.builderFor(icebergTable.schema()) + .withOrderId(1) + .desc("id", NullOrder.NULLS_LAST) + .build(); + Assertions.assertEquals(sortOrder, icebergTable.sortOrder()); + + // set distribution with locally ordering + sql( + String.format( + "ALTER TABLE %s WRITE DISTRIBUTED BY PARTITION LOCALLY ORDERED BY id desc", tableName)); + icebergTable.refresh(); + tableInfo = getTableInfo(tableName); + tableProperties = tableInfo.getTableProperties(); + Assertions.assertEquals("hash", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + sortOrder = + SortOrder.builderFor(icebergTable.schema()) + .withOrderId(1) + .desc("id", NullOrder.NULLS_LAST) + .build(); + Assertions.assertEquals(sortOrder, icebergTable.sortOrder()); + } + + private List getIcebergSimpleTableColumn() { + return Arrays.asList( + SparkTableInfo.SparkColumnInfo.of("id", DataTypes.IntegerType, "id comment"), + SparkTableInfo.SparkColumnInfo.of("name", DataTypes.StringType, ""), + SparkTableInfo.SparkColumnInfo.of("ts", DataTypes.TimestampType, null)); + } /** * Here we build a new `createIcebergSql` String for creating a table with a field of timestamp @@ -800,6 +830,21 @@ private void createIcebergTableWithTableProperties( sql(createSql); } + private SparkIcebergTable getSparkIcebergTableInstance(String tableName) + throws NoSuchTableException { + CatalogPlugin catalogPlugin = + getSparkSession().sessionState().catalogManager().catalog(getCatalogName()); + Assertions.assertInstanceOf(TableCatalog.class, catalogPlugin); + TableCatalog catalog = (TableCatalog) catalogPlugin; + Table table = catalog.loadTable(Identifier.of(new String[] {getDefaultDatabase()}, tableName)); + return (SparkIcebergTable) table; + } + + private long getCurrentSnapshotId(String tableName) throws NoSuchTableException { + SparkIcebergTable sparkIcebergTable = getSparkIcebergTableInstance(tableName); + return sparkIcebergTable.table().currentSnapshot().snapshotId(); + } + @Data private static class IcebergTableWriteProperties { diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/IcebergExtendedDataSourceV2Strategy.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/IcebergExtendedDataSourceV2Strategy.java index ab452ffad5a..52262d4df22 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/IcebergExtendedDataSourceV2Strategy.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/IcebergExtendedDataSourceV2Strategy.java @@ -36,7 +36,6 @@ import org.apache.spark.sql.execution.datasources.v2.SetWriteDistributionAndOrderingExec; import scala.Option; import scala.Some; -import scala.Tuple2; import scala.collection.JavaConverters; import scala.collection.Seq; @@ -56,12 +55,10 @@ public Seq apply(LogicalPlan plan) { return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier(spark, addPartitionField.table()) .map( catalogAndIdentifier -> { - TableCatalog catalog = catalogAndIdentifier._1(); - Identifier identifier = catalogAndIdentifier._2(); AddPartitionFieldExec addPartitionFieldExec = new AddPartitionFieldExec( - catalog, - identifier, + catalogAndIdentifier.catalog, + catalogAndIdentifier.identifier, addPartitionField.transform(), addPartitionField.name()); return toSeq(addPartitionFieldExec); @@ -73,12 +70,10 @@ public Seq apply(LogicalPlan plan) { spark, createOrReplaceBranch.table()) .map( catalogAndIdentifier -> { - TableCatalog catalog = catalogAndIdentifier._1(); - Identifier identifier = catalogAndIdentifier._2(); CreateOrReplaceBranchExec createOrReplaceBranchExec = new CreateOrReplaceBranchExec( - catalog, - identifier, + catalogAndIdentifier.catalog, + catalogAndIdentifier.identifier, createOrReplaceBranch.branch(), createOrReplaceBranch.branchOptions(), createOrReplaceBranch.replace(), @@ -92,12 +87,10 @@ public Seq apply(LogicalPlan plan) { spark, createOrReplaceTag.table()) .map( catalogAndIdentifier -> { - TableCatalog catalog = catalogAndIdentifier._1(); - Identifier identifier = catalogAndIdentifier._2(); CreateOrReplaceTagExec createOrReplaceTagExec = new CreateOrReplaceTagExec( - catalog, - identifier, + catalogAndIdentifier.catalog, + catalogAndIdentifier.identifier, createOrReplaceTag.tag(), createOrReplaceTag.tagOptions(), createOrReplaceTag.replace(), @@ -110,11 +103,12 @@ public Seq apply(LogicalPlan plan) { return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier(spark, dropBranch.table()) .map( catalogAndIdentifier -> { - TableCatalog catalog = catalogAndIdentifier._1(); - Identifier identifier = catalogAndIdentifier._2(); DropBranchExec dropBranchExec = new DropBranchExec( - catalog, identifier, dropBranch.branch(), dropBranch.ifExists()); + catalogAndIdentifier.catalog, + catalogAndIdentifier.identifier, + dropBranch.branch(), + dropBranch.ifExists()); return toSeq(dropBranchExec); }) .getOrElse(() -> super.apply(plan)); @@ -123,10 +117,12 @@ public Seq apply(LogicalPlan plan) { return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier(spark, dropTag.table()) .map( catalogAndIdentifier -> { - TableCatalog catalog = catalogAndIdentifier._1(); - Identifier identifier = catalogAndIdentifier._2(); DropTagExec dropTagExec = - new DropTagExec(catalog, identifier, dropTag.tag(), dropTag.ifExists()); + new DropTagExec( + catalogAndIdentifier.catalog, + catalogAndIdentifier.identifier, + dropTag.tag(), + dropTag.ifExists()); return toSeq(dropTagExec); }) .getOrElse(() -> super.apply(plan)); @@ -136,10 +132,11 @@ public Seq apply(LogicalPlan plan) { spark, dropPartitionField.table()) .map( catalogAndIdentifier -> { - TableCatalog catalog = catalogAndIdentifier._1(); - Identifier identifier = catalogAndIdentifier._2(); DropPartitionFieldExec dropPartitionFieldExec = - new DropPartitionFieldExec(catalog, identifier, dropPartitionField.transform()); + new DropPartitionFieldExec( + catalogAndIdentifier.catalog, + catalogAndIdentifier.identifier, + dropPartitionField.transform()); return toSeq(dropPartitionFieldExec); }) .getOrElse(() -> super.apply(plan)); @@ -149,12 +146,10 @@ public Seq apply(LogicalPlan plan) { spark, replacePartitionField.table()) .map( catalogAndIdentifier -> { - TableCatalog catalog = catalogAndIdentifier._1(); - Identifier identifier = catalogAndIdentifier._2(); ReplacePartitionFieldExec replacePartitionFieldExec = new ReplacePartitionFieldExec( - catalog, - identifier, + catalogAndIdentifier.catalog, + catalogAndIdentifier.identifier, replacePartitionField.transformFrom(), replacePartitionField.transformTo(), replacePartitionField.name()); @@ -167,10 +162,11 @@ public Seq apply(LogicalPlan plan) { spark, setIdentifierFields.table()) .map( catalogAndIdentifier -> { - TableCatalog catalog = catalogAndIdentifier._1(); - Identifier identifier = catalogAndIdentifier._2(); SetIdentifierFieldsExec setIdentifierFieldsExec = - new SetIdentifierFieldsExec(catalog, identifier, setIdentifierFields.fields()); + new SetIdentifierFieldsExec( + catalogAndIdentifier.catalog, + catalogAndIdentifier.identifier, + setIdentifierFields.fields()); return toSeq(setIdentifierFieldsExec); }) .getOrElse(() -> super.apply(plan)); @@ -180,11 +176,11 @@ public Seq apply(LogicalPlan plan) { spark, dropIdentifierFields.table()) .map( catalogAndIdentifier -> { - TableCatalog catalog = catalogAndIdentifier._1(); - Identifier identifier = catalogAndIdentifier._2(); DropIdentifierFieldsExec dropIdentifierFieldsExec = new DropIdentifierFieldsExec( - catalog, identifier, dropIdentifierFields.fields()); + catalogAndIdentifier.catalog, + catalogAndIdentifier.identifier, + dropIdentifierFields.fields()); return toSeq(dropIdentifierFieldsExec); }) .getOrElse(() -> super.apply(plan)); @@ -195,12 +191,10 @@ public Seq apply(LogicalPlan plan) { spark, setWriteDistributionAndOrdering.table()) .map( catalogAndIdentifier -> { - TableCatalog catalog = catalogAndIdentifier._1(); - Identifier identifier = catalogAndIdentifier._2(); SetWriteDistributionAndOrderingExec setWriteDistributionAndOrderingExec = new SetWriteDistributionAndOrderingExec( - catalog, - identifier, + catalogAndIdentifier.catalog, + catalogAndIdentifier.identifier, setWriteDistributionAndOrdering.distributionMode(), setWriteDistributionAndOrdering.sortOrder()); return toSeq(setWriteDistributionAndOrderingExec); @@ -218,17 +212,32 @@ private Seq toSeq(SparkPlan plan) { } static class IcebergCatalogAndIdentifier { - static Option> buildCatalogAndIdentifier( - SparkSession spark, Seq identifier) { + + private final TableCatalog catalog; + private final Identifier identifier; + + private IcebergCatalogAndIdentifier(TableCatalog catalog, Identifier identifier) { + this.catalog = catalog; + this.identifier = identifier; + } + + private static IcebergCatalogAndIdentifier of(TableCatalog catalog, Identifier identifier) { + return new IcebergCatalogAndIdentifier(catalog, identifier); + } + + static Option buildCatalogAndIdentifier( + SparkSession spark, Seq identifiers) { Spark3Util.CatalogAndIdentifier catalogAndIdentifier = - Spark3Util.catalogAndIdentifier( - spark, scala.collection.JavaConversions.seqAsJavaList(identifier)); + Spark3Util.catalogAndIdentifier(spark, JavaConverters.seqAsJavaList(identifiers)); CatalogPlugin catalog = catalogAndIdentifier.catalog(); if (catalog instanceof GravitinoIcebergCatalog) { - return Some.apply(new Tuple2<>((TableCatalog) catalog, catalogAndIdentifier.identifier())); + return new Some<>( + IcebergCatalogAndIdentifier.of( + (TableCatalog) catalog, catalogAndIdentifier.identifier())); } else { // TODO: support SparkSessionCatalog - return Option.empty(); + throw new UnsupportedOperationException( + "Unsupported catalog type: " + catalog.getClass().getName()); } } } diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java index 80271d9948c..43a86ca5ea2 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java @@ -36,11 +36,11 @@ public class GravitinoDriverPlugin implements DriverPlugin { private static final Logger LOG = LoggerFactory.getLogger(GravitinoDriverPlugin.class); private GravitinoCatalogManager catalogManager; - private static final String[] GRAVITINO_DRIVER_EXTENSIONS = - new String[] { - GravitinoIcebergSparkSessionExtensions.class.getName(), - IcebergSparkSessionExtensions.class.getName() - }; + private static final String[] GRAVITINO_DRIVER_EXTENSIONS = + new String[] { + GravitinoIcebergSparkSessionExtensions.class.getName(), + IcebergSparkSessionExtensions.class.getName() + }; @Override public Map init(SparkContext sc, PluginContext pluginContext) { diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/utils/SparkBaseTableHelper.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/utils/SparkBaseTableHelper.java deleted file mode 100644 index e69de29bb2d..00000000000 From a8a4d6ba72e38c93cb6556033d7a75c3168fdc01 Mon Sep 17 00:00:00 2001 From: caican00 Date: Thu, 16 May 2024 16:06:47 +0800 Subject: [PATCH 07/18] update --- .../spark/iceberg/SparkIcebergCatalogIT.java | 68 +++++++++---------- 1 file changed, 33 insertions(+), 35 deletions(-) diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java index c6ec8f25cf7..da15197f938 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java @@ -12,15 +12,14 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import java.io.File; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.TimeUnit; import java.util.Set; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import lombok.Data; import org.apache.hadoop.fs.Path; @@ -319,16 +318,16 @@ void testIcebergAsOfQuery() throws NoSuchTableException { Assertions.assertEquals("1,1,1", tableData.get(0)); } - @Test - void testIcebergSQLExtensions() throws NoSuchTableException, IOException { - testIcebergPartitionFieldOperations(); - testIcebergBranchOperations(); - testIcebergTagOperations(); - testIcebergIdentifierOperations(); - testIcebergDistributionAndOrderingOperations(); - } + @Test + void testIcebergSQLExtensions() throws NoSuchTableException { + testIcebergPartitionFieldOperations(); + testIcebergBranchOperations(); + testIcebergTagOperations(); + testIcebergIdentifierOperations(); + testIcebergDistributionAndOrderingOperations(); + } - private void testMetadataColumns() { + private void testMetadataColumns() { String tableName = "test_metadata_columns"; dropTableIfExists(tableName); String createTableSQL = getCreateSimpleTableString(tableName); @@ -900,34 +899,33 @@ private void createIcebergTableWithTableProperties( sql(createSql); } + private SparkIcebergTable getSparkIcebergTableInstance(String tableName) + throws NoSuchTableException { + CatalogPlugin catalogPlugin = + getSparkSession().sessionState().catalogManager().catalog(getCatalogName()); + Assertions.assertInstanceOf(TableCatalog.class, catalogPlugin); + TableCatalog catalog = (TableCatalog) catalogPlugin; + Table table = catalog.loadTable(Identifier.of(new String[] {getDefaultDatabase()}, tableName)); + return (SparkIcebergTable) table; + } - private SparkIcebergTable getSparkIcebergTableInstance(String tableName) - throws NoSuchTableException { - CatalogPlugin catalogPlugin = - getSparkSession().sessionState().catalogManager().catalog(getCatalogName()); - Assertions.assertInstanceOf(TableCatalog.class, catalogPlugin); - TableCatalog catalog = (TableCatalog) catalogPlugin; - Table table = catalog.loadTable(Identifier.of(new String[] {getDefaultDatabase()}, tableName)); - return (SparkIcebergTable) table; - } - - private long getCurrentSnapshotTimestamp(String tableName) throws NoSuchTableException { - SparkIcebergTable sparkIcebergTable = getSparkIcebergTableInstance(tableName); - return sparkIcebergTable.table().currentSnapshot().timestampMillis(); - } + private long getCurrentSnapshotTimestamp(String tableName) throws NoSuchTableException { + SparkIcebergTable sparkIcebergTable = getSparkIcebergTableInstance(tableName); + return sparkIcebergTable.table().currentSnapshot().timestampMillis(); + } - private long getCurrentSnapshotId(String tableName) throws NoSuchTableException { - SparkIcebergTable sparkIcebergTable = getSparkIcebergTableInstance(tableName); - return sparkIcebergTable.table().currentSnapshot().snapshotId(); - } + private long getCurrentSnapshotId(String tableName) throws NoSuchTableException { + SparkIcebergTable sparkIcebergTable = getSparkIcebergTableInstance(tableName); + return sparkIcebergTable.table().currentSnapshot().snapshotId(); + } - private long waitUntilAfter(Long timestampMillis) { - long current = System.currentTimeMillis(); - while (current <= timestampMillis) { - current = System.currentTimeMillis(); - } - return current; + private long waitUntilAfter(Long timestampMillis) { + long current = System.currentTimeMillis(); + while (current <= timestampMillis) { + current = System.currentTimeMillis(); } + return current; + } @Data private static class IcebergTableWriteProperties { From ecc463ebfd11ed9ce3bfea10c006b887e6189a6c Mon Sep 17 00:00:00 2001 From: caican00 Date: Fri, 17 May 2024 11:36:06 +0800 Subject: [PATCH 08/18] update --- .../IcebergExtendedDataSourceV2Strategy.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/IcebergExtendedDataSourceV2Strategy.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/IcebergExtendedDataSourceV2Strategy.java index 52262d4df22..32dc29421f3 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/IcebergExtendedDataSourceV2Strategy.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/IcebergExtendedDataSourceV2Strategy.java @@ -63,7 +63,7 @@ public Seq apply(LogicalPlan plan) { addPartitionField.name()); return toSeq(addPartitionFieldExec); }) - .getOrElse(() -> super.apply(plan)); + .get(); } else if (plan instanceof CreateOrReplaceBranch) { CreateOrReplaceBranch createOrReplaceBranch = (CreateOrReplaceBranch) plan; return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier( @@ -80,7 +80,7 @@ public Seq apply(LogicalPlan plan) { createOrReplaceBranch.ifNotExists()); return toSeq(createOrReplaceBranchExec); }) - .getOrElse(() -> super.apply(plan)); + .get(); } else if (plan instanceof CreateOrReplaceTag) { CreateOrReplaceTag createOrReplaceTag = (CreateOrReplaceTag) plan; return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier( @@ -97,7 +97,7 @@ public Seq apply(LogicalPlan plan) { createOrReplaceTag.ifNotExists()); return toSeq(createOrReplaceTagExec); }) - .getOrElse(() -> super.apply(plan)); + .get(); } else if (plan instanceof DropBranch) { DropBranch dropBranch = (DropBranch) plan; return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier(spark, dropBranch.table()) @@ -111,7 +111,7 @@ public Seq apply(LogicalPlan plan) { dropBranch.ifExists()); return toSeq(dropBranchExec); }) - .getOrElse(() -> super.apply(plan)); + .get(); } else if (plan instanceof DropTag) { DropTag dropTag = (DropTag) plan; return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier(spark, dropTag.table()) @@ -125,7 +125,7 @@ public Seq apply(LogicalPlan plan) { dropTag.ifExists()); return toSeq(dropTagExec); }) - .getOrElse(() -> super.apply(plan)); + .get(); } else if (plan instanceof DropPartitionField) { DropPartitionField dropPartitionField = (DropPartitionField) plan; return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier( @@ -139,7 +139,7 @@ public Seq apply(LogicalPlan plan) { dropPartitionField.transform()); return toSeq(dropPartitionFieldExec); }) - .getOrElse(() -> super.apply(plan)); + .get(); } else if (plan instanceof ReplacePartitionField) { ReplacePartitionField replacePartitionField = (ReplacePartitionField) plan; return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier( @@ -155,7 +155,7 @@ public Seq apply(LogicalPlan plan) { replacePartitionField.name()); return toSeq(replacePartitionFieldExec); }) - .getOrElse(() -> super.apply(plan)); + .get(); } else if (plan instanceof SetIdentifierFields) { SetIdentifierFields setIdentifierFields = (SetIdentifierFields) plan; return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier( @@ -169,7 +169,7 @@ public Seq apply(LogicalPlan plan) { setIdentifierFields.fields()); return toSeq(setIdentifierFieldsExec); }) - .getOrElse(() -> super.apply(plan)); + .get(); } else if (plan instanceof DropIdentifierFields) { DropIdentifierFields dropIdentifierFields = (DropIdentifierFields) plan; return IcebergCatalogAndIdentifier.buildCatalogAndIdentifier( @@ -183,7 +183,7 @@ public Seq apply(LogicalPlan plan) { dropIdentifierFields.fields()); return toSeq(dropIdentifierFieldsExec); }) - .getOrElse(() -> super.apply(plan)); + .get(); } else if (plan instanceof SetWriteDistributionAndOrdering) { SetWriteDistributionAndOrdering setWriteDistributionAndOrdering = (SetWriteDistributionAndOrdering) plan; @@ -199,7 +199,7 @@ public Seq apply(LogicalPlan plan) { setWriteDistributionAndOrdering.sortOrder()); return toSeq(setWriteDistributionAndOrderingExec); }) - .getOrElse(() -> super.apply(plan)); + .get(); } else { return super.apply(plan); } From 30fba3cb4b44543c3d3f3d001210be62768d0196 Mon Sep 17 00:00:00 2001 From: caican00 Date: Fri, 17 May 2024 15:52:56 +0800 Subject: [PATCH 09/18] update --- .../test/spark/iceberg/SparkIcebergCatalogIT.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java index da15197f938..c740310aa52 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java @@ -18,6 +18,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -579,8 +580,7 @@ private void testIcebergPartitionFieldOperations() { List partitionFields = Arrays.asList("name", "truncate(1, name)", "bucket(16, id)", "days(ts)"); String partitionExpression = "name=a/name_trunc_1=a/id_bucket_16=4/ts_day=2024-01-01"; - String tableName = "test_iceberg_partition_field_operations"; - dropTableIfExists(tableName); + String tableName = "test_iceberg_partition_field_operations_" + new Random().nextInt(100); sql(getCreateIcebergSimpleTableString(tableName)); // add partition fields @@ -738,7 +738,7 @@ private void testIcebergTagOperations() throws NoSuchTableException { } private void testIcebergIdentifierOperations() throws NoSuchTableException { - String tableName = "test_iceberg_identifier_operations"; + String tableName = "test_iceberg_identifier_operations_" + new Random().nextInt(100); // The Identifier fields must be non-null, so a new schema with non-null fields is created here. List columnInfos = Arrays.asList( @@ -777,7 +777,8 @@ private void testIcebergIdentifierOperations() throws NoSuchTableException { } private void testIcebergDistributionAndOrderingOperations() throws NoSuchTableException { - String tableName = "test_iceberg_distribution_and_ordering_operations"; + String tableName = + "test_iceberg_distribution_and_ordering_operations_" + new Random().nextInt(100); dropTableIfExists(tableName); createSimpleTable(tableName); From b97c325a26ec1d4611e20f1edb2690acb1ff42a9 Mon Sep 17 00:00:00 2001 From: caican00 Date: Fri, 17 May 2024 18:10:03 +0800 Subject: [PATCH 10/18] update --- .../test/spark/iceberg/SparkIcebergCatalogIT.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java index c740310aa52..19e9dcfa49c 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java @@ -580,7 +580,8 @@ private void testIcebergPartitionFieldOperations() { List partitionFields = Arrays.asList("name", "truncate(1, name)", "bucket(16, id)", "days(ts)"); String partitionExpression = "name=a/name_trunc_1=a/id_bucket_16=4/ts_day=2024-01-01"; - String tableName = "test_iceberg_partition_field_operations_" + new Random().nextInt(100); + String tableName = "test_iceberg_partition_field_operations_" + new Random().nextInt(10); + dropTableIfExists(tableName); sql(getCreateIcebergSimpleTableString(tableName)); // add partition fields @@ -738,7 +739,7 @@ private void testIcebergTagOperations() throws NoSuchTableException { } private void testIcebergIdentifierOperations() throws NoSuchTableException { - String tableName = "test_iceberg_identifier_operations_" + new Random().nextInt(100); + String tableName = "test_iceberg_identifier_operations_" + new Random().nextInt(10); // The Identifier fields must be non-null, so a new schema with non-null fields is created here. List columnInfos = Arrays.asList( From 560d6add11a2a67f1e5ae33b0316fb57cacae64e Mon Sep 17 00:00:00 2001 From: caican00 Date: Tue, 21 May 2024 19:28:14 +0800 Subject: [PATCH 11/18] update --- .../spark/iceberg/SparkIcebergCatalogIT.java | 553 +++++++++--------- 1 file changed, 276 insertions(+), 277 deletions(-) diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java index 5be501c61bb..967aabcdef2 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java @@ -18,7 +18,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.TimeUnit; import java.util.Random; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -329,14 +328,14 @@ void testIcebergTimeTravelQuery() throws NoSuchTableException { Assertions.assertEquals("1,1,1", tableData.get(0)); } - @Test - void testIcebergSQLExtensions() throws NoSuchTableException { - testIcebergPartitionFieldOperations(); - testIcebergBranchOperations(); - testIcebergTagOperations(); - testIcebergIdentifierOperations(); - testIcebergDistributionAndOrderingOperations(); - } + @Test + void testIcebergSQLExtensions() throws NoSuchTableException { + testIcebergPartitionFieldOperations(); + testIcebergBranchOperations(); + testIcebergTagOperations(); + testIcebergIdentifierOperations(); + testIcebergDistributionAndOrderingOperations(); + } private void testMetadataColumns() { String tableName = "test_metadata_columns"; @@ -735,279 +734,279 @@ private void testIcebergCallRewritePositionDeleteFiles() { Assertions.assertEquals(1, callResult.get(0).getInt(1)); } - private void testIcebergPartitionFieldOperations() { - List partitionFields = - Arrays.asList("name", "truncate(1, name)", "bucket(16, id)", "days(ts)"); - String partitionExpression = "name=a/name_trunc_1=a/id_bucket_16=4/ts_day=2024-01-01"; - String tableName = "test_iceberg_partition_field_operations_" + new Random().nextInt(10); - dropTableIfExists(tableName); - sql(getCreateIcebergSimpleTableString(tableName)); - - // add partition fields - SparkTableInfo tableInfo = getTableInfo(tableName); - SparkTableInfoChecker checker = - SparkTableInfoChecker.create() - .withName(tableName) - .withColumns(getIcebergSimpleTableColumn()); - checker.check(tableInfo); - - partitionFields.forEach( - partitionField -> - sql(String.format("ALTER TABLE %s ADD PARTITION FIELD %s", tableName, partitionField))); - - tableInfo = getTableInfo(tableName); - checker = - SparkTableInfoChecker.create() - .withName(tableName) - .withColumns(getIcebergSimpleTableColumn()) - .withIdentifyPartition(Collections.singletonList("name")) - .withTruncatePartition(1, "name") - .withBucketPartition(16, Collections.singletonList("id")) - .withDayPartition("ts"); - checker.check(tableInfo); - - sql( - String.format( - "INSERT INTO %s VALUES(2,'a',cast('2024-01-01 12:00:00' as timestamp));", tableName)); - List queryResult = getTableData(tableName); - Assertions.assertEquals(1, queryResult.size()); - Assertions.assertEquals("2,a,2024-01-01 12:00:00", queryResult.get(0)); - Path partitionPath = new Path(getTableLocation(tableInfo), partitionExpression); - checkDirExists(partitionPath); - - // replace partition fields - sql(String.format("ALTER TABLE %s REPLACE PARTITION FIELD ts_day WITH months(ts)", tableName)); - tableInfo = getTableInfo(tableName); - checker = - SparkTableInfoChecker.create() - .withName(tableName) - .withColumns(getIcebergSimpleTableColumn()) - .withIdentifyPartition(Collections.singletonList("name")) - .withTruncatePartition(1, "name") - .withBucketPartition(16, Collections.singletonList("id")) - .withMonthPartition("ts"); - checker.check(tableInfo); - - // drop partition fields - sql(String.format("ALTER TABLE %s DROP PARTITION FIELD months(ts)", tableName)); - tableInfo = getTableInfo(tableName); - checker = - SparkTableInfoChecker.create() - .withName(tableName) - .withColumns(getIcebergSimpleTableColumn()) - .withIdentifyPartition(Collections.singletonList("name")) - .withTruncatePartition(1, "name") - .withBucketPartition(16, Collections.singletonList("id")); - checker.check(tableInfo); - } + private void testIcebergPartitionFieldOperations() { + List partitionFields = + Arrays.asList("name", "truncate(1, name)", "bucket(16, id)", "days(ts)"); + String partitionExpression = "name=a/name_trunc_1=a/id_bucket_16=4/ts_day=2024-01-01"; + String tableName = "test_iceberg_partition_field_operations_" + new Random().nextInt(10); + dropTableIfExists(tableName); + sql(getCreateIcebergSimpleTableString(tableName)); - private void testIcebergBranchOperations() throws NoSuchTableException { - String tableName = "test_iceberg_branch_operations"; - String branch1 = "branch1"; - dropTableIfExists(tableName); - createSimpleTable(tableName); - - // create branch and query data using branch - sql(String.format("INSERT INTO %s VALUES(1, '1', 1);", tableName)); - List tableData = getTableData(tableName); - Assertions.assertEquals(1, tableData.size()); - Assertions.assertEquals("1,1,1", tableData.get(0)); - - long snapshotId = getCurrentSnapshotId(tableName); - sql(String.format("ALTER TABLE %s CREATE BRANCH IF NOT EXISTS `%s`", tableName, branch1)); - - sql(String.format("INSERT INTO %s VALUES(2, '2', 2);", tableName)); - tableData = getQueryData(getSelectAllSqlWithOrder(tableName, "id")); - Assertions.assertEquals(2, tableData.size()); - Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); - - tableData = - getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, branch1)); - Assertions.assertEquals(1, tableData.size()); - Assertions.assertEquals("1,1,1", tableData.get(0)); - - sql(String.format("ALTER TABLE %s CREATE OR REPLACE BRANCH `%s`", tableName, branch1)); - tableData = - getQueryData( - String.format("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, branch1)); - Assertions.assertEquals(2, tableData.size()); - Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); - - // replace branch - sql( - String.format( - "ALTER TABLE %s REPLACE BRANCH `%s` AS OF VERSION %d RETAIN 1 DAYS", - tableName, branch1, snapshotId)); - tableData = - getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, branch1)); - Assertions.assertEquals(1, tableData.size()); - Assertions.assertEquals("1,1,1", tableData.get(0)); - - // drop branch - sql(String.format("ALTER TABLE %s DROP BRANCH `%s`", tableName, branch1)); - Assertions.assertThrows( - ValidationException.class, - () -> sql(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, branch1))); - } + // add partition fields + SparkTableInfo tableInfo = getTableInfo(tableName); + SparkTableInfoChecker checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withColumns(getIcebergSimpleTableColumn()); + checker.check(tableInfo); - private void testIcebergTagOperations() throws NoSuchTableException { - String tableName = "test_iceberg_tag_operations"; - String tag1 = "tag1"; - dropTableIfExists(tableName); - createSimpleTable(tableName); - - // create tag and query data using tag - sql(String.format("INSERT INTO %s VALUES(1, '1', 1);", tableName)); - List tableData = getTableData(tableName); - Assertions.assertEquals(1, tableData.size()); - Assertions.assertEquals("1,1,1", tableData.get(0)); - - long snapshotId = getCurrentSnapshotId(tableName); - sql(String.format("ALTER TABLE %s CREATE TAG IF NOT EXISTS `%s`", tableName, tag1)); - - sql(String.format("INSERT INTO %s VALUES(2, '2', 2);", tableName)); - tableData = getQueryData(getSelectAllSqlWithOrder(tableName, "id")); - Assertions.assertEquals(2, tableData.size()); - Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); - - tableData = getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, tag1)); - Assertions.assertEquals(1, tableData.size()); - Assertions.assertEquals("1,1,1", tableData.get(0)); - - sql(String.format("ALTER TABLE %s CREATE OR REPLACE TAG `%s`", tableName, tag1)); - tableData = - getQueryData( - String.format("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, tag1)); - Assertions.assertEquals(2, tableData.size()); - Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); - - // replace tag - sql( - String.format( - "ALTER TABLE %s REPLACE TAG `%s` AS OF VERSION %d RETAIN 1 DAYS", - tableName, tag1, snapshotId)); - tableData = getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, tag1)); - Assertions.assertEquals(1, tableData.size()); - Assertions.assertEquals("1,1,1", tableData.get(0)); - - // drop tag - sql(String.format("ALTER TABLE %s DROP TAG `%s`", tableName, tag1)); - Assertions.assertThrows( - ValidationException.class, - () -> sql(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, tag1))); - } + partitionFields.forEach( + partitionField -> + sql(String.format("ALTER TABLE %s ADD PARTITION FIELD %s", tableName, partitionField))); - private void testIcebergIdentifierOperations() throws NoSuchTableException { - String tableName = "test_iceberg_identifier_operations_" + new Random().nextInt(10); - // The Identifier fields must be non-null, so a new schema with non-null fields is created here. - List columnInfos = - Arrays.asList( - SparkTableInfo.SparkColumnInfo.of("id", DataTypes.IntegerType, "id comment", false), - SparkTableInfo.SparkColumnInfo.of("name", DataTypes.StringType, "", false), - SparkTableInfo.SparkColumnInfo.of("age", DataTypes.IntegerType, null, true)); - dropTableIfExists(tableName); - - sql( - String.format( - "CREATE TABLE %s (id INT COMMENT 'id comment' NOT NULL, name STRING COMMENT '' NOT NULL, age INT)", - tableName)); - SparkTableInfo tableInfo = getTableInfo(tableName); - SparkTableInfoChecker checker = - SparkTableInfoChecker.create().withName(tableName).withColumns(columnInfos); - checker.check(tableInfo); - - SparkIcebergTable sparkIcebergTable = getSparkIcebergTableInstance(tableName); - org.apache.iceberg.Table icebergTable = sparkIcebergTable.table(); - Set identifierFieldNames = icebergTable.schema().identifierFieldNames(); - Assertions.assertEquals(0, identifierFieldNames.size()); - - // add identifier fields - sql(String.format("ALTER TABLE %s SET IDENTIFIER FIELDS id, name", tableName)); - icebergTable.refresh(); - identifierFieldNames = icebergTable.schema().identifierFieldNames(); - Assertions.assertTrue(identifierFieldNames.contains("id")); - Assertions.assertTrue(identifierFieldNames.contains("name")); - - // drop identifier fields - sql(String.format("ALTER TABLE %s DROP IDENTIFIER FIELDS name", tableName)); - icebergTable.refresh(); - identifierFieldNames = icebergTable.schema().identifierFieldNames(); - Assertions.assertTrue(identifierFieldNames.contains("id")); - Assertions.assertFalse(identifierFieldNames.contains("name")); - } + tableInfo = getTableInfo(tableName); + checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withColumns(getIcebergSimpleTableColumn()) + .withIdentifyPartition(Collections.singletonList("name")) + .withTruncatePartition(1, "name") + .withBucketPartition(16, Collections.singletonList("id")) + .withDayPartition("ts"); + checker.check(tableInfo); - private void testIcebergDistributionAndOrderingOperations() throws NoSuchTableException { - String tableName = - "test_iceberg_distribution_and_ordering_operations_" + new Random().nextInt(100); - dropTableIfExists(tableName); - createSimpleTable(tableName); - - SparkTableInfo tableInfo = getTableInfo(tableName); - Map tableProperties = tableInfo.getTableProperties(); - Assertions.assertEquals("none", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); - Assertions.assertNull(tableProperties.get(ICEBERG_SORT_ORDER)); - - SparkIcebergTable sparkIcebergTable = getSparkIcebergTableInstance(tableName); - org.apache.iceberg.Table icebergTable = sparkIcebergTable.table(); - - // set globally ordering - sql(String.format("ALTER TABLE %s WRITE ORDERED BY id DESC", tableName)); - icebergTable.refresh(); - tableInfo = getTableInfo(tableName); - tableProperties = tableInfo.getTableProperties(); - Assertions.assertEquals("range", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); - SortOrder sortOrder = - SortOrder.builderFor(icebergTable.schema()) - .withOrderId(1) - .desc("id", NullOrder.NULLS_LAST) - .build(); - Assertions.assertEquals(sortOrder, icebergTable.sortOrder()); - - // set locally ordering - sql(String.format("ALTER TABLE %s WRITE LOCALLY ORDERED BY id DESC", tableName)); - icebergTable.refresh(); - tableInfo = getTableInfo(tableName); - tableProperties = tableInfo.getTableProperties(); - Assertions.assertEquals("none", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); - sortOrder = - SortOrder.builderFor(icebergTable.schema()) - .withOrderId(1) - .desc("id", NullOrder.NULLS_LAST) - .build(); - Assertions.assertEquals(sortOrder, icebergTable.sortOrder()); - - // set distribution - sql( - String.format( - "ALTER TABLE %s WRITE ORDERED BY id DESC DISTRIBUTED BY PARTITION", tableName)); - icebergTable.refresh(); - tableInfo = getTableInfo(tableName); - tableProperties = tableInfo.getTableProperties(); - Assertions.assertEquals("hash", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); - sortOrder = - SortOrder.builderFor(icebergTable.schema()) - .withOrderId(1) - .desc("id", NullOrder.NULLS_LAST) - .build(); - Assertions.assertEquals(sortOrder, icebergTable.sortOrder()); - - // set distribution with locally ordering - sql( - String.format( - "ALTER TABLE %s WRITE DISTRIBUTED BY PARTITION LOCALLY ORDERED BY id desc", tableName)); - icebergTable.refresh(); - tableInfo = getTableInfo(tableName); - tableProperties = tableInfo.getTableProperties(); - Assertions.assertEquals("hash", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); - sortOrder = - SortOrder.builderFor(icebergTable.schema()) - .withOrderId(1) - .desc("id", NullOrder.NULLS_LAST) - .build(); - Assertions.assertEquals(sortOrder, icebergTable.sortOrder()); - } + sql( + String.format( + "INSERT INTO %s VALUES(2,'a',cast('2024-01-01 12:00:00' as timestamp));", tableName)); + List queryResult = getTableData(tableName); + Assertions.assertEquals(1, queryResult.size()); + Assertions.assertEquals("2,a,2024-01-01 12:00:00", queryResult.get(0)); + Path partitionPath = new Path(getTableLocation(tableInfo), partitionExpression); + checkDirExists(partitionPath); + + // replace partition fields + sql(String.format("ALTER TABLE %s REPLACE PARTITION FIELD ts_day WITH months(ts)", tableName)); + tableInfo = getTableInfo(tableName); + checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withColumns(getIcebergSimpleTableColumn()) + .withIdentifyPartition(Collections.singletonList("name")) + .withTruncatePartition(1, "name") + .withBucketPartition(16, Collections.singletonList("id")) + .withMonthPartition("ts"); + checker.check(tableInfo); + + // drop partition fields + sql(String.format("ALTER TABLE %s DROP PARTITION FIELD months(ts)", tableName)); + tableInfo = getTableInfo(tableName); + checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withColumns(getIcebergSimpleTableColumn()) + .withIdentifyPartition(Collections.singletonList("name")) + .withTruncatePartition(1, "name") + .withBucketPartition(16, Collections.singletonList("id")); + checker.check(tableInfo); + } + + private void testIcebergBranchOperations() throws NoSuchTableException { + String tableName = "test_iceberg_branch_operations"; + String branch1 = "branch1"; + dropTableIfExists(tableName); + createSimpleTable(tableName); + + // create branch and query data using branch + sql(String.format("INSERT INTO %s VALUES(1, '1', 1);", tableName)); + List tableData = getTableData(tableName); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + long snapshotId = getCurrentSnapshotId(tableName); + sql(String.format("ALTER TABLE %s CREATE BRANCH IF NOT EXISTS `%s`", tableName, branch1)); + + sql(String.format("INSERT INTO %s VALUES(2, '2', 2);", tableName)); + tableData = getQueryData(getSelectAllSqlWithOrder(tableName, "id")); + Assertions.assertEquals(2, tableData.size()); + Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); + + tableData = + getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, branch1)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + sql(String.format("ALTER TABLE %s CREATE OR REPLACE BRANCH `%s`", tableName, branch1)); + tableData = + getQueryData( + String.format("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, branch1)); + Assertions.assertEquals(2, tableData.size()); + Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); + + // replace branch + sql( + String.format( + "ALTER TABLE %s REPLACE BRANCH `%s` AS OF VERSION %d RETAIN 1 DAYS", + tableName, branch1, snapshotId)); + tableData = + getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, branch1)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + // drop branch + sql(String.format("ALTER TABLE %s DROP BRANCH `%s`", tableName, branch1)); + Assertions.assertThrows( + ValidationException.class, + () -> sql(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, branch1))); + } + + private void testIcebergTagOperations() throws NoSuchTableException { + String tableName = "test_iceberg_tag_operations"; + String tag1 = "tag1"; + dropTableIfExists(tableName); + createSimpleTable(tableName); + + // create tag and query data using tag + sql(String.format("INSERT INTO %s VALUES(1, '1', 1);", tableName)); + List tableData = getTableData(tableName); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + long snapshotId = getCurrentSnapshotId(tableName); + sql(String.format("ALTER TABLE %s CREATE TAG IF NOT EXISTS `%s`", tableName, tag1)); + + sql(String.format("INSERT INTO %s VALUES(2, '2', 2);", tableName)); + tableData = getQueryData(getSelectAllSqlWithOrder(tableName, "id")); + Assertions.assertEquals(2, tableData.size()); + Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); + + tableData = getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, tag1)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + sql(String.format("ALTER TABLE %s CREATE OR REPLACE TAG `%s`", tableName, tag1)); + tableData = + getQueryData( + String.format("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, tag1)); + Assertions.assertEquals(2, tableData.size()); + Assertions.assertEquals("1,1,1;2,2,2", String.join(";", tableData)); + + // replace tag + sql( + String.format( + "ALTER TABLE %s REPLACE TAG `%s` AS OF VERSION %d RETAIN 1 DAYS", + tableName, tag1, snapshotId)); + tableData = getQueryData(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, tag1)); + Assertions.assertEquals(1, tableData.size()); + Assertions.assertEquals("1,1,1", tableData.get(0)); + + // drop tag + sql(String.format("ALTER TABLE %s DROP TAG `%s`", tableName, tag1)); + Assertions.assertThrows( + ValidationException.class, + () -> sql(String.format("SELECT * FROM %s VERSION AS OF '%s'", tableName, tag1))); + } + + private void testIcebergIdentifierOperations() throws NoSuchTableException { + String tableName = "test_iceberg_identifier_operations_" + new Random().nextInt(10); + // The Identifier fields must be non-null, so a new schema with non-null fields is created here. + List columnInfos = + Arrays.asList( + SparkTableInfo.SparkColumnInfo.of("id", DataTypes.IntegerType, "id comment", false), + SparkTableInfo.SparkColumnInfo.of("name", DataTypes.StringType, "", false), + SparkTableInfo.SparkColumnInfo.of("age", DataTypes.IntegerType, null, true)); + dropTableIfExists(tableName); + + sql( + String.format( + "CREATE TABLE %s (id INT COMMENT 'id comment' NOT NULL, name STRING COMMENT '' NOT NULL, age INT)", + tableName)); + SparkTableInfo tableInfo = getTableInfo(tableName); + SparkTableInfoChecker checker = + SparkTableInfoChecker.create().withName(tableName).withColumns(columnInfos); + checker.check(tableInfo); + + SparkIcebergTable sparkIcebergTable = getSparkIcebergTableInstance(tableName); + org.apache.iceberg.Table icebergTable = sparkIcebergTable.table(); + Set identifierFieldNames = icebergTable.schema().identifierFieldNames(); + Assertions.assertEquals(0, identifierFieldNames.size()); + + // add identifier fields + sql(String.format("ALTER TABLE %s SET IDENTIFIER FIELDS id, name", tableName)); + icebergTable.refresh(); + identifierFieldNames = icebergTable.schema().identifierFieldNames(); + Assertions.assertTrue(identifierFieldNames.contains("id")); + Assertions.assertTrue(identifierFieldNames.contains("name")); + + // drop identifier fields + sql(String.format("ALTER TABLE %s DROP IDENTIFIER FIELDS name", tableName)); + icebergTable.refresh(); + identifierFieldNames = icebergTable.schema().identifierFieldNames(); + Assertions.assertTrue(identifierFieldNames.contains("id")); + Assertions.assertFalse(identifierFieldNames.contains("name")); + } + + private void testIcebergDistributionAndOrderingOperations() throws NoSuchTableException { + String tableName = + "test_iceberg_distribution_and_ordering_operations_" + new Random().nextInt(100); + dropTableIfExists(tableName); + createSimpleTable(tableName); + + SparkTableInfo tableInfo = getTableInfo(tableName); + Map tableProperties = tableInfo.getTableProperties(); + Assertions.assertEquals("none", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + Assertions.assertNull(tableProperties.get(ICEBERG_SORT_ORDER)); + + SparkIcebergTable sparkIcebergTable = getSparkIcebergTableInstance(tableName); + org.apache.iceberg.Table icebergTable = sparkIcebergTable.table(); + + // set globally ordering + sql(String.format("ALTER TABLE %s WRITE ORDERED BY id DESC", tableName)); + icebergTable.refresh(); + tableInfo = getTableInfo(tableName); + tableProperties = tableInfo.getTableProperties(); + Assertions.assertEquals("range", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + SortOrder sortOrder = + SortOrder.builderFor(icebergTable.schema()) + .withOrderId(1) + .desc("id", NullOrder.NULLS_LAST) + .build(); + Assertions.assertEquals(sortOrder, icebergTable.sortOrder()); + + // set locally ordering + sql(String.format("ALTER TABLE %s WRITE LOCALLY ORDERED BY id DESC", tableName)); + icebergTable.refresh(); + tableInfo = getTableInfo(tableName); + tableProperties = tableInfo.getTableProperties(); + Assertions.assertEquals("none", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + sortOrder = + SortOrder.builderFor(icebergTable.schema()) + .withOrderId(1) + .desc("id", NullOrder.NULLS_LAST) + .build(); + Assertions.assertEquals(sortOrder, icebergTable.sortOrder()); + + // set distribution + sql( + String.format( + "ALTER TABLE %s WRITE ORDERED BY id DESC DISTRIBUTED BY PARTITION", tableName)); + icebergTable.refresh(); + tableInfo = getTableInfo(tableName); + tableProperties = tableInfo.getTableProperties(); + Assertions.assertEquals("hash", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + sortOrder = + SortOrder.builderFor(icebergTable.schema()) + .withOrderId(1) + .desc("id", NullOrder.NULLS_LAST) + .build(); + Assertions.assertEquals(sortOrder, icebergTable.sortOrder()); + + // set distribution with locally ordering + sql( + String.format( + "ALTER TABLE %s WRITE DISTRIBUTED BY PARTITION LOCALLY ORDERED BY id desc", tableName)); + icebergTable.refresh(); + tableInfo = getTableInfo(tableName); + tableProperties = tableInfo.getTableProperties(); + Assertions.assertEquals("hash", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + sortOrder = + SortOrder.builderFor(icebergTable.schema()) + .withOrderId(1) + .desc("id", NullOrder.NULLS_LAST) + .build(); + Assertions.assertEquals(sortOrder, icebergTable.sortOrder()); + } - private List getIcebergSimpleTableColumn() { + private List getIcebergSimpleTableColumn() { return Arrays.asList( SparkTableInfo.SparkColumnInfo.of("id", DataTypes.IntegerType, "id comment"), SparkTableInfo.SparkColumnInfo.of("name", DataTypes.StringType, ""), From b87d9f00cfaa27db8c947ac25f0839a167becb08 Mon Sep 17 00:00:00 2001 From: caican00 Date: Tue, 21 May 2024 19:31:45 +0800 Subject: [PATCH 12/18] update --- .../gravitino/spark/connector/hive/SparkHiveTable.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/SparkHiveTable.java b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/SparkHiveTable.java index 48dbc5b80bb..e27916af283 100644 --- a/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/SparkHiveTable.java +++ b/spark-connector/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/SparkHiveTable.java @@ -15,15 +15,12 @@ import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.expressions.Transform; -import org.apache.spark.sql.connector.read.ScanBuilder; import org.apache.spark.sql.types.StructType; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; /** Keep consistent behavior with the SparkIcebergTable */ public class SparkHiveTable extends HiveTable { private GravitinoTableInfoHelper gravitinoTableInfoHelper; - private org.apache.spark.sql.connector.catalog.Table sparkTable; public SparkHiveTable( Identifier identifier, @@ -36,7 +33,6 @@ public SparkHiveTable( this.gravitinoTableInfoHelper = new GravitinoTableInfoHelper( false, identifier, gravitinoTable, propertiesConverter, sparkTransformConverter); - this.sparkTable = hiveTable; } @Override @@ -59,10 +55,4 @@ public Map properties() { public Transform[] partitioning() { return gravitinoTableInfoHelper.partitioning(); } - - /** to keep consistent behavior with SparkIcebergTable. */ - @Override - public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { - return ((HiveTable) sparkTable).newScanBuilder(options); - } } From ecceee97d3f7254014928a22135a4ef03dc30c92 Mon Sep 17 00:00:00 2001 From: caican00 Date: Wed, 22 May 2024 10:07:25 +0800 Subject: [PATCH 13/18] update --- .../spark/iceberg/SparkIcebergCatalogIT.java | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java index 967aabcdef2..0ff8e370624 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java @@ -18,8 +18,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.Set; +import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -738,7 +738,7 @@ private void testIcebergPartitionFieldOperations() { List partitionFields = Arrays.asList("name", "truncate(1, name)", "bucket(16, id)", "days(ts)"); String partitionExpression = "name=a/name_trunc_1=a/id_bucket_16=4/ts_day=2024-01-01"; - String tableName = "test_iceberg_partition_field_operations_" + new Random().nextInt(10); + String tableName = String.format("test_partition_field_%s", genRandomName()); dropTableIfExists(tableName); sql(getCreateIcebergSimpleTableString(tableName)); @@ -801,7 +801,7 @@ private void testIcebergPartitionFieldOperations() { } private void testIcebergBranchOperations() throws NoSuchTableException { - String tableName = "test_iceberg_branch_operations"; + String tableName = "test_branchs"; String branch1 = "branch1"; dropTableIfExists(tableName); createSimpleTable(tableName); @@ -850,7 +850,7 @@ private void testIcebergBranchOperations() throws NoSuchTableException { } private void testIcebergTagOperations() throws NoSuchTableException { - String tableName = "test_iceberg_tag_operations"; + String tableName = "test_tags"; String tag1 = "tag1"; dropTableIfExists(tableName); createSimpleTable(tableName); @@ -897,7 +897,7 @@ private void testIcebergTagOperations() throws NoSuchTableException { } private void testIcebergIdentifierOperations() throws NoSuchTableException { - String tableName = "test_iceberg_identifier_operations_" + new Random().nextInt(10); + String tableName = String.format("test_identifier_%s", genRandomName()); // The Identifier fields must be non-null, so a new schema with non-null fields is created here. List columnInfos = Arrays.asList( @@ -936,8 +936,7 @@ private void testIcebergIdentifierOperations() throws NoSuchTableException { } private void testIcebergDistributionAndOrderingOperations() throws NoSuchTableException { - String tableName = - "test_iceberg_distribution_and_ordering_operations_" + new Random().nextInt(100); + String tableName = String.format("test_distribution_order_%s", genRandomName()); dropTableIfExists(tableName); createSimpleTable(tableName); @@ -1087,6 +1086,10 @@ private long waitUntilAfter(Long timestampMillis) { return current; } + private String genRandomName() { + return UUID.randomUUID().toString().replace("-", ""); + } + @Data private static class IcebergTableWriteProperties { From 7d859baaac9d1202b80ba566cbfcc08eff58a464 Mon Sep 17 00:00:00 2001 From: caican00 Date: Wed, 5 Jun 2024 22:43:04 +0800 Subject: [PATCH 14/18] update --- .../extensions/IcebergExtendedDataSourceV2Strategy.java | 2 ++ .../spark/connector/plugin/GravitinoDriverPlugin.java | 7 +++---- .../integration/test/iceberg/SparkIcebergCatalogIT.java | 2 +- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/spark-connector/spark-common/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/IcebergExtendedDataSourceV2Strategy.java b/spark-connector/spark-common/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/IcebergExtendedDataSourceV2Strategy.java index 32dc29421f3..bb33ac3b8d6 100644 --- a/spark-connector/spark-common/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/IcebergExtendedDataSourceV2Strategy.java +++ b/spark-connector/spark-common/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/extensions/IcebergExtendedDataSourceV2Strategy.java @@ -76,6 +76,7 @@ public Seq apply(LogicalPlan plan) { catalogAndIdentifier.identifier, createOrReplaceBranch.branch(), createOrReplaceBranch.branchOptions(), + createOrReplaceBranch.create(), createOrReplaceBranch.replace(), createOrReplaceBranch.ifNotExists()); return toSeq(createOrReplaceBranchExec); @@ -93,6 +94,7 @@ public Seq apply(LogicalPlan plan) { catalogAndIdentifier.identifier, createOrReplaceTag.tag(), createOrReplaceTag.tagOptions(), + createOrReplaceTag.create(), createOrReplaceTag.replace(), createOrReplaceTag.ifNotExists()); return toSeq(createOrReplaceTagExec); diff --git a/spark-connector/spark-common/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java b/spark-connector/spark-common/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java index ebd008d4f82..27a127e7faa 100644 --- a/spark-connector/spark-common/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java +++ b/spark-connector/spark-common/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java @@ -11,11 +11,9 @@ import com.datastrato.gravitino.Catalog; import com.datastrato.gravitino.spark.connector.GravitinoSparkConfig; import com.datastrato.gravitino.spark.connector.catalog.GravitinoCatalogManager; +import com.datastrato.gravitino.spark.connector.iceberg.extensions.GravitinoIcebergSparkSessionExtensions; import com.datastrato.gravitino.spark.connector.version.CatalogNameAdaptor; import com.google.common.annotations.VisibleForTesting; -import com.datastrato.gravitino.spark.connector.hive.GravitinoHiveCatalog; -import com.datastrato.gravitino.spark.connector.iceberg.GravitinoIcebergCatalog; -import com.datastrato.gravitino.spark.connector.iceberg.extensions.GravitinoIcebergSparkSessionExtensions; import com.google.common.base.Preconditions; import java.util.ArrayList; import java.util.Arrays; @@ -41,7 +39,8 @@ public class GravitinoDriverPlugin implements DriverPlugin { private static final Logger LOG = LoggerFactory.getLogger(GravitinoDriverPlugin.class); private GravitinoCatalogManager catalogManager; - private List toRegisteredDriverExtensions = Arrays.asList(GravitinoIcebergSparkSessionExtensions.class.getName()); + private List toRegisteredDriverExtensions = + Arrays.asList(GravitinoIcebergSparkSessionExtensions.class.getName()); private List gravitinoDriverExtensions = new ArrayList<>(); private boolean enableIcebergSupport = false; diff --git a/spark-connector/spark-common/src/test/java/com/datastrato/gravitino/spark/connector/integration/test/iceberg/SparkIcebergCatalogIT.java b/spark-connector/spark-common/src/test/java/com/datastrato/gravitino/spark/connector/integration/test/iceberg/SparkIcebergCatalogIT.java index 085713eb655..9fad3e5d2d0 100644 --- a/spark-connector/spark-common/src/test/java/com/datastrato/gravitino/spark/connector/integration/test/iceberg/SparkIcebergCatalogIT.java +++ b/spark-connector/spark-common/src/test/java/com/datastrato/gravitino/spark/connector/integration/test/iceberg/SparkIcebergCatalogIT.java @@ -26,8 +26,8 @@ import lombok.Data; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.fs.Path; -import org.apache.iceberg.ReplaceSortOrder; import org.apache.iceberg.NullOrder; +import org.apache.iceberg.ReplaceSortOrder; import org.apache.iceberg.SortOrder; import org.apache.iceberg.exceptions.ValidationException; import org.apache.spark.sql.Column; From 801b90728c0920dc3f78aa6031d60a5e2f0fb047 Mon Sep 17 00:00:00 2001 From: caican00 Date: Thu, 6 Jun 2024 00:01:04 +0800 Subject: [PATCH 15/18] update --- .../spark/connector/plugin/GravitinoDriverPlugin.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/spark-connector/spark-common/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java b/spark-connector/spark-common/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java index 27a127e7faa..7d9b367521c 100644 --- a/spark-connector/spark-common/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java +++ b/spark-connector/spark-common/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java @@ -39,9 +39,9 @@ public class GravitinoDriverPlugin implements DriverPlugin { private static final Logger LOG = LoggerFactory.getLogger(GravitinoDriverPlugin.class); private GravitinoCatalogManager catalogManager; - private List toRegisteredDriverExtensions = + private final List toRegisteredDriverExtensions = Arrays.asList(GravitinoIcebergSparkSessionExtensions.class.getName()); - private List gravitinoDriverExtensions = new ArrayList<>(); + private final List gravitinoDriverExtensions = new ArrayList<>(); private boolean enableIcebergSupport = false; @VisibleForTesting @@ -62,6 +62,8 @@ public Map init(SparkContext sc, PluginContext pluginContext) { String.format( "%s:%s, should not be empty", GravitinoSparkConfig.GRAVITINO_METALAKE, metalake)); + gravitinoDriverExtensions.addAll(toRegisteredDriverExtensions); + this.enableIcebergSupport = conf.getBoolean(GravitinoSparkConfig.GRAVITINO_ENABLE_ICEBERG_SUPPORT, false); if (enableIcebergSupport) { From 066ca1531b4b4240714b22f711257964ae5b918f Mon Sep 17 00:00:00 2001 From: caican00 Date: Thu, 6 Jun 2024 00:41:47 +0800 Subject: [PATCH 16/18] update --- .../integration/test/iceberg/SparkIcebergCatalogIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark-connector/spark-common/src/test/java/com/datastrato/gravitino/spark/connector/integration/test/iceberg/SparkIcebergCatalogIT.java b/spark-connector/spark-common/src/test/java/com/datastrato/gravitino/spark/connector/integration/test/iceberg/SparkIcebergCatalogIT.java index 9fad3e5d2d0..e16b9846f94 100644 --- a/spark-connector/spark-common/src/test/java/com/datastrato/gravitino/spark/connector/integration/test/iceberg/SparkIcebergCatalogIT.java +++ b/spark-connector/spark-common/src/test/java/com/datastrato/gravitino/spark/connector/integration/test/iceberg/SparkIcebergCatalogIT.java @@ -966,7 +966,7 @@ private void testIcebergIdentifierOperations() throws NoSuchTableException { sql( String.format( - "CREATE TABLE %s (id INT COMMENT 'id comment' NOT NULL, name STRING COMMENT '' NOT NULL, age INT)", + "CREATE TABLE %s (id INT NOT NULL COMMENT 'id comment', name STRING NOT NULL COMMENT '', age INT)", tableName)); SparkTableInfo tableInfo = getTableInfo(tableName); SparkTableInfoChecker checker = From 71fb5591c3335636b18e0eadd733baf7b910dddb Mon Sep 17 00:00:00 2001 From: caican00 Date: Fri, 7 Jun 2024 17:12:02 +0800 Subject: [PATCH 17/18] update --- .../test/iceberg/SparkIcebergCatalogIT.java | 68 ++++++++++++++++++- 1 file changed, 66 insertions(+), 2 deletions(-) diff --git a/spark-connector/spark-common/src/test/java/com/datastrato/gravitino/spark/connector/integration/test/iceberg/SparkIcebergCatalogIT.java b/spark-connector/spark-common/src/test/java/com/datastrato/gravitino/spark/connector/integration/test/iceberg/SparkIcebergCatalogIT.java index e16b9846f94..7f88abbd214 100644 --- a/spark-connector/spark-common/src/test/java/com/datastrato/gravitino/spark/connector/integration/test/iceberg/SparkIcebergCatalogIT.java +++ b/spark-connector/spark-common/src/test/java/com/datastrato/gravitino/spark/connector/integration/test/iceberg/SparkIcebergCatalogIT.java @@ -844,6 +844,13 @@ private void testIcebergPartitionFieldOperations() { .withBucketPartition(16, Collections.singletonList("id")) .withMonthPartition("ts"); checker.check(tableInfo); + sql( + String.format( + "INSERT INTO %s VALUES(2,'a',cast('2024-01-01 12:00:00' as timestamp));", tableName)); + queryResult = getTableData(tableName); + Assertions.assertEquals(2, queryResult.size()); + Assertions.assertEquals( + "2,a,2024-01-01 12:00:00;2,a,2024-01-01 12:00:00", String.join(";", queryResult)); // drop partition fields sql(String.format("ALTER TABLE %s DROP PARTITION FIELD months(ts)", tableName)); @@ -856,6 +863,14 @@ private void testIcebergPartitionFieldOperations() { .withTruncatePartition(1, "name") .withBucketPartition(16, Collections.singletonList("id")); checker.check(tableInfo); + sql( + String.format( + "INSERT INTO %s VALUES(2,'a',cast('2024-01-01 12:00:00' as timestamp));", tableName)); + queryResult = getTableData(tableName); + Assertions.assertEquals(3, queryResult.size()); + Assertions.assertEquals( + "2,a,2024-01-01 12:00:00;2,a,2024-01-01 12:00:00;2,a,2024-01-01 12:00:00", + String.join(";", queryResult)); } private void testIcebergBranchOperations() throws NoSuchTableException { @@ -864,7 +879,7 @@ private void testIcebergBranchOperations() throws NoSuchTableException { dropTableIfExists(tableName); createSimpleTable(tableName); - // create branch and query data using branch + // create a branch and query data using branch sql(String.format("INSERT INTO %s VALUES(1, '1', 1);", tableName)); List tableData = getTableData(tableName); Assertions.assertEquals(1, tableData.size()); @@ -969,6 +984,9 @@ private void testIcebergIdentifierOperations() throws NoSuchTableException { "CREATE TABLE %s (id INT NOT NULL COMMENT 'id comment', name STRING NOT NULL COMMENT '', age INT)", tableName)); SparkTableInfo tableInfo = getTableInfo(tableName); + Map tableProperties = tableInfo.getTableProperties(); + Assertions.assertFalse( + tableProperties.containsKey(IcebergPropertiesConstants.ICEBERG_IDENTIFIER_FIELDS)); SparkTableInfoChecker checker = SparkTableInfoChecker.create().withName(tableName).withColumns(columnInfos); checker.check(tableInfo); @@ -984,6 +1002,15 @@ private void testIcebergIdentifierOperations() throws NoSuchTableException { identifierFieldNames = icebergTable.schema().identifierFieldNames(); Assertions.assertTrue(identifierFieldNames.contains("id")); Assertions.assertTrue(identifierFieldNames.contains("name")); + tableInfo = getTableInfo(tableName); + tableProperties = tableInfo.getTableProperties(); + Assertions.assertEquals( + "[name,id]", tableProperties.get(IcebergPropertiesConstants.ICEBERG_IDENTIFIER_FIELDS)); + + sql(String.format("INSERT INTO %s VALUES(1, '1', 1);", tableName)); + List queryResult = getTableData(tableName); + Assertions.assertEquals(1, queryResult.size()); + Assertions.assertEquals("1,1,1", queryResult.get(0)); // drop identifier fields sql(String.format("ALTER TABLE %s DROP IDENTIFIER FIELDS name", tableName)); @@ -991,6 +1018,15 @@ private void testIcebergIdentifierOperations() throws NoSuchTableException { identifierFieldNames = icebergTable.schema().identifierFieldNames(); Assertions.assertTrue(identifierFieldNames.contains("id")); Assertions.assertFalse(identifierFieldNames.contains("name")); + tableInfo = getTableInfo(tableName); + tableProperties = tableInfo.getTableProperties(); + Assertions.assertEquals( + "[id]", tableProperties.get(IcebergPropertiesConstants.ICEBERG_IDENTIFIER_FIELDS)); + + sql(String.format("INSERT INTO %s VALUES(1, '1', 1);", tableName)); + queryResult = getTableData(tableName); + Assertions.assertEquals(2, queryResult.size()); + Assertions.assertEquals("1,1,1;1,1,1", String.join(";", queryResult)); } private void testIcebergDistributionAndOrderingOperations() throws NoSuchTableException { @@ -1012,6 +1048,8 @@ private void testIcebergDistributionAndOrderingOperations() throws NoSuchTableEx tableInfo = getTableInfo(tableName); tableProperties = tableInfo.getTableProperties(); Assertions.assertEquals("range", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + Assertions.assertEquals( + "id DESC NULLS LAST", tableProperties.get(IcebergPropertiesConstants.ICEBERG_SORT_ORDER)); SortOrder sortOrder = SortOrder.builderFor(icebergTable.schema()) .withOrderId(1) @@ -1019,12 +1057,19 @@ private void testIcebergDistributionAndOrderingOperations() throws NoSuchTableEx .build(); Assertions.assertEquals(sortOrder, icebergTable.sortOrder()); + sql(String.format("INSERT INTO %s VALUES(1, '1', 1);", tableName)); + List queryResult = getTableData(tableName); + Assertions.assertEquals(1, queryResult.size()); + Assertions.assertEquals("1,1,1", queryResult.get(0)); + // set locally ordering sql(String.format("ALTER TABLE %s WRITE LOCALLY ORDERED BY id DESC", tableName)); icebergTable.refresh(); tableInfo = getTableInfo(tableName); tableProperties = tableInfo.getTableProperties(); Assertions.assertEquals("none", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + Assertions.assertEquals( + "id DESC NULLS LAST", tableProperties.get(IcebergPropertiesConstants.ICEBERG_SORT_ORDER)); sortOrder = SortOrder.builderFor(icebergTable.schema()) .withOrderId(1) @@ -1032,6 +1077,11 @@ private void testIcebergDistributionAndOrderingOperations() throws NoSuchTableEx .build(); Assertions.assertEquals(sortOrder, icebergTable.sortOrder()); + sql(String.format("INSERT INTO %s VALUES(1, '1', 1);", tableName)); + queryResult = getTableData(tableName); + Assertions.assertEquals(2, queryResult.size()); + Assertions.assertEquals("1,1,1;1,1,1", String.join(";", queryResult)); + // set distribution sql( String.format( @@ -1040,6 +1090,8 @@ private void testIcebergDistributionAndOrderingOperations() throws NoSuchTableEx tableInfo = getTableInfo(tableName); tableProperties = tableInfo.getTableProperties(); Assertions.assertEquals("hash", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + Assertions.assertEquals( + "id DESC NULLS LAST", tableProperties.get(IcebergPropertiesConstants.ICEBERG_SORT_ORDER)); sortOrder = SortOrder.builderFor(icebergTable.schema()) .withOrderId(1) @@ -1047,20 +1099,32 @@ private void testIcebergDistributionAndOrderingOperations() throws NoSuchTableEx .build(); Assertions.assertEquals(sortOrder, icebergTable.sortOrder()); + sql(String.format("INSERT INTO %s VALUES(1, '1', 1);", tableName)); + queryResult = getTableData(tableName); + Assertions.assertEquals(3, queryResult.size()); + Assertions.assertEquals("1,1,1;1,1,1;1,1,1", String.join(";", queryResult)); + // set distribution with locally ordering sql( String.format( - "ALTER TABLE %s WRITE DISTRIBUTED BY PARTITION LOCALLY ORDERED BY id desc", tableName)); + "ALTER TABLE %s WRITE DISTRIBUTED BY PARTITION LOCALLY ORDERED BY id DESC", tableName)); icebergTable.refresh(); tableInfo = getTableInfo(tableName); tableProperties = tableInfo.getTableProperties(); Assertions.assertEquals("hash", tableProperties.get(ICEBERG_WRITE_DISTRIBUTION_MODE)); + Assertions.assertEquals( + "id DESC NULLS LAST", tableProperties.get(IcebergPropertiesConstants.ICEBERG_SORT_ORDER)); sortOrder = SortOrder.builderFor(icebergTable.schema()) .withOrderId(1) .desc("id", NullOrder.NULLS_LAST) .build(); Assertions.assertEquals(sortOrder, icebergTable.sortOrder()); + + sql(String.format("INSERT INTO %s VALUES(1, '1', 1);", tableName)); + queryResult = getTableData(tableName); + Assertions.assertEquals(4, queryResult.size()); + Assertions.assertEquals("1,1,1;1,1,1;1,1,1;1,1,1", String.join(";", queryResult)); } private List getIcebergSimpleTableColumn() { From 52f46eaa52517e035e4d0164e2ed37953d0ddf7a Mon Sep 17 00:00:00 2001 From: caican00 Date: Thu, 13 Jun 2024 20:42:30 +0800 Subject: [PATCH 18/18] update --- .../connector/plugin/GravitinoDriverPlugin.java | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/spark-connector/spark-common/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java b/spark-connector/spark-common/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java index 7d9b367521c..86ded097c27 100644 --- a/spark-connector/spark-common/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java +++ b/spark-connector/spark-common/src/main/java/com/datastrato/gravitino/spark/connector/plugin/GravitinoDriverPlugin.java @@ -38,16 +38,17 @@ public class GravitinoDriverPlugin implements DriverPlugin { private static final Logger LOG = LoggerFactory.getLogger(GravitinoDriverPlugin.class); - private GravitinoCatalogManager catalogManager; - private final List toRegisteredDriverExtensions = - Arrays.asList(GravitinoIcebergSparkSessionExtensions.class.getName()); - private final List gravitinoDriverExtensions = new ArrayList<>(); - private boolean enableIcebergSupport = false; - @VisibleForTesting static final String ICEBERG_SPARK_EXTENSIONS = "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions"; + private GravitinoCatalogManager catalogManager; + private final List gravitinoIcebergExtensions = + Arrays.asList( + GravitinoIcebergSparkSessionExtensions.class.getName(), ICEBERG_SPARK_EXTENSIONS); + private final List gravitinoDriverExtensions = new ArrayList<>(); + private boolean enableIcebergSupport = false; + @Override public Map init(SparkContext sc, PluginContext pluginContext) { SparkConf conf = sc.conf(); @@ -62,12 +63,10 @@ public Map init(SparkContext sc, PluginContext pluginContext) { String.format( "%s:%s, should not be empty", GravitinoSparkConfig.GRAVITINO_METALAKE, metalake)); - gravitinoDriverExtensions.addAll(toRegisteredDriverExtensions); - this.enableIcebergSupport = conf.getBoolean(GravitinoSparkConfig.GRAVITINO_ENABLE_ICEBERG_SUPPORT, false); if (enableIcebergSupport) { - gravitinoDriverExtensions.add(ICEBERG_SPARK_EXTENSIONS); + gravitinoDriverExtensions.addAll(gravitinoIcebergExtensions); } this.catalogManager = GravitinoCatalogManager.create(gravitinoUri, metalake);