From 866c2fcc53209aa66dae7ef9e3ef68138099bca6 Mon Sep 17 00:00:00 2001 From: SteNicholas Date: Wed, 10 Apr 2024 14:51:16 +0800 Subject: [PATCH] [#2738] feat(catalog-lakehouse-paimon): support PaimonCatalog implementation to manage Paimon table operations --- build.gradle.kts | 1 + catalogs/bundled-catalog/build.gradle.kts | 2 + .../catalog-lakehouse-paimon/build.gradle.kts | 111 +++ .../lakehouse/paimon/PaimonCatalog.java | 45 ++ .../paimon/PaimonCatalogBackend.java | 11 + .../paimon/PaimonCatalogOperations.java | 475 ++++++++++++ .../PaimonCatalogPropertiesMetadata.java | 67 ++ .../lakehouse/paimon/PaimonColumn.java | 93 +++ .../lakehouse/paimon/PaimonConfig.java | 72 ++ .../lakehouse/paimon/PaimonSchema.java | 86 +++ .../PaimonSchemaPropertiesMetadata.java | 36 + .../catalog/lakehouse/paimon/PaimonTable.java | 93 +++ .../paimon/PaimonTablePropertiesMetadata.java | 39 + .../lakehouse/paimon/ops/PaimonTableOps.java | 91 +++ .../lakehouse/paimon/utils/CatalogUtils.java | 32 + .../lakehouse/paimon/utils/TableOpsUtils.java | 171 +++++ .../lakehouse/paimon/utils/TypeUtils.java | 242 ++++++ .../com.datastrato.gravitino.CatalogProvider | 5 + .../src/main/resources/lakehouse-paimon.conf | 8 + .../lakehouse/paimon/TestPaimonConfig.java | 43 ++ .../paimon/ops/TestPaimonTableOps.java | 392 ++++++++++ .../paimon/utils/TestCatalogUtils.java | 124 +++ .../paimon/utils/TestTableOpsUtils.java | 278 +++++++ .../lakehouse/paimon/utils/TestTypeUtils.java | 179 +++++ .../resources/hive-schema-3.1.0.derby.sql | 726 ++++++++++++++++++ .../src/test/resources/log4j2.properties | 33 + gradle/libs.versions.toml | 6 + settings.gradle.kts | 1 + 28 files changed, 3462 insertions(+) create mode 100644 catalogs/catalog-lakehouse-paimon/build.gradle.kts create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonCatalog.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonCatalogBackend.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonCatalogOperations.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonCatalogPropertiesMetadata.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonColumn.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonConfig.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonSchema.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonSchemaPropertiesMetadata.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonTable.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonTablePropertiesMetadata.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/ops/PaimonTableOps.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/CatalogUtils.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TableOpsUtils.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TypeUtils.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/resources/META-INF/services/com.datastrato.gravitino.CatalogProvider create mode 100644 catalogs/catalog-lakehouse-paimon/src/main/resources/lakehouse-paimon.conf create mode 100644 catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/TestPaimonConfig.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/ops/TestPaimonTableOps.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TestCatalogUtils.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TestTableOpsUtils.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TestTypeUtils.java create mode 100644 catalogs/catalog-lakehouse-paimon/src/test/resources/hive-schema-3.1.0.derby.sql create mode 100644 catalogs/catalog-lakehouse-paimon/src/test/resources/log4j2.properties diff --git a/build.gradle.kts b/build.gradle.kts index 98b1d48f011..044dd6f8279 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -701,6 +701,7 @@ tasks { dependsOn( ":catalogs:catalog-hive:copyLibAndConfig", ":catalogs:catalog-lakehouse-iceberg:copyLibAndConfig", + ":catalogs:catalog-lakehouse-paimon:copyLibAndConfig", // TODO. Enable packaging the catalog-jdbc-doris module when it is ready for shipping // ":catalogs:catalog-jdbc-doris:copyLibAndConfig", ":catalogs:catalog-jdbc-mysql:copyLibAndConfig", diff --git a/catalogs/bundled-catalog/build.gradle.kts b/catalogs/bundled-catalog/build.gradle.kts index 23ef280c6c2..1fbb3cec302 100644 --- a/catalogs/bundled-catalog/build.gradle.kts +++ b/catalogs/bundled-catalog/build.gradle.kts @@ -16,6 +16,7 @@ dependencies { implementation(project(":catalogs:catalog-jdbc-mysql")) implementation(project(":catalogs:catalog-jdbc-postgresql")) implementation(project(":catalogs:catalog-lakehouse-iceberg")) + implementation(project(":catalogs:catalog-lakehouse-paimon")) implementation(project(":core")) implementation(libs.slf4j.api) } @@ -80,6 +81,7 @@ tasks.jar { tasks.compileJava { dependsOn(":catalogs:catalog-jdbc-postgresql:runtimeJars") dependsOn(":catalogs:catalog-lakehouse-iceberg:runtimeJars") + dependsOn(":catalogs:catalog-lakehouse-paimon:runtimeJars") dependsOn(":catalogs:catalog-jdbc-mysql:runtimeJars") dependsOn(":catalogs:catalog-hive:runtimeJars") dependsOn(":catalogs:catalog-hadoop:runtimeJars") diff --git a/catalogs/catalog-lakehouse-paimon/build.gradle.kts b/catalogs/catalog-lakehouse-paimon/build.gradle.kts new file mode 100644 index 00000000000..baca06cf4d0 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/build.gradle.kts @@ -0,0 +1,111 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +description = "catalog-lakehouse-paimon" + +plugins { + `maven-publish` + id("java") + id("idea") +} + +dependencies { + implementation(project(":api")) + implementation(project(":common")) + implementation(project(":core")) + implementation(libs.bundles.paimon) + implementation(libs.guava) + implementation(libs.hadoop2.common) { + exclude("com.github.spotbugs") + } + implementation(libs.hadoop2.hdfs) + implementation(libs.hive2.exec) { + artifact { + classifier = "core" + } + exclude("com.google.code.findbugs", "jsr305") + exclude("com.google.protobuf") + exclude("org.apache.avro") + exclude("org.apache.calcite") + exclude("org.apache.calcite.avatica") + exclude("org.apache.curator") + exclude("org.apache.hadoop", "hadoop-yarn-server-resourcemanager") + exclude("org.apache.logging.log4j") + exclude("org.apache.zookeeper") + exclude("org.eclipse.jetty.aggregate", "jetty-all") + exclude("org.eclipse.jetty.orbit", "javax.servlet") + exclude("org.openjdk.jol") + exclude("org.pentaho") + exclude("org.slf4j") + } + implementation(libs.hive2.metastore) { + exclude("co.cask.tephra") + exclude("com.github.spotbugs") + exclude("com.google.code.findbugs", "jsr305") + exclude("com.tdunning", "json") + exclude("javax.transaction", "transaction-api") + exclude("org.apache.avro", "avro") + exclude("org.apache.hbase") + exclude("org.apache.hadoop", "hadoop-yarn-api") + exclude("org.apache.hadoop", "hadoop-yarn-server-applicationhistoryservice") + exclude("org.apache.hadoop", "hadoop-yarn-server-common") + exclude("org.apache.hadoop", "hadoop-yarn-server-resourcemanager") + exclude("org.apache.hadoop", "hadoop-yarn-server-web-proxy") + exclude("org.apache.logging.log4j") + exclude("org.apache.parquet", "parquet-hadoop-bundle") + exclude("org.apache.zookeeper") + exclude("org.eclipse.jetty.aggregate", "jetty-all") + exclude("org.eclipse.jetty.orbit", "javax.servlet") + exclude("org.pentaho") // missing dependency + exclude("org.slf4j", "slf4j-log4j12") + exclude("com.zaxxer", "HikariCP") + exclude("com.sun.jersey", "jersey-server") + } + + annotationProcessor(libs.lombok) + compileOnly(libs.lombok) + + testImplementation(libs.junit.jupiter.api) + testImplementation(libs.junit.jupiter.params) + testImplementation(libs.mockito.core) + testImplementation(libs.mockito.inline) + + testRuntimeOnly(libs.junit.jupiter.engine) +} + +tasks { + val runtimeJars by registering(Copy::class) { + from(configurations.runtimeClasspath) + into("build/libs") + } + + val copyCatalogLibs by registering(Copy::class) { + dependsOn("jar", "runtimeJars") + from("build/libs") + into("$rootDir/distribution/package/catalogs/lakehouse-paimon/libs") + } + + val copyCatalogConfig by registering(Copy::class) { + from("src/main/resources") + into("$rootDir/distribution/package/catalogs/lakehouse-paimon/conf") + + include("lakehouse-paimon.conf") + + rename { original -> + if (original.endsWith(".template")) { + original.replace(".template", "") + } else { + original + } + } + + exclude { details -> + details.file.isDirectory() + } + } + + register("copyLibAndConfig", Copy::class) { + dependsOn(copyCatalogLibs, copyCatalogConfig) + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonCatalog.java b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonCatalog.java new file mode 100644 index 00000000000..8b5b5bff60c --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonCatalog.java @@ -0,0 +1,45 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog.lakehouse.paimon; + +import com.datastrato.gravitino.Catalog; +import com.datastrato.gravitino.connector.BaseCatalog; +import com.datastrato.gravitino.connector.CatalogOperations; +import com.datastrato.gravitino.rel.SupportsSchemas; +import com.datastrato.gravitino.rel.TableCatalog; +import java.util.Map; + +/** Implementation of {@link Catalog} that represents a Paimon catalog in Gravitino. */ +public class PaimonCatalog extends BaseCatalog { + + /** @return The short name of the catalog. */ + @Override + public String shortName() { + return "lakehouse-paimon"; + } + + /** + * Creates a new instance of {@link PaimonCatalogOperations} with the provided configuration. + * + * @param config The configuration map for the Paimon catalog operations. + * @return A new instance of {@link PaimonCatalogOperations}. + */ + @Override + protected CatalogOperations newOps(Map config) { + return new PaimonCatalogOperations(); + } + + /** @return The Paimon catalog operations as {@link PaimonCatalogOperations}. */ + @Override + public SupportsSchemas asSchemas() { + return (PaimonCatalogOperations) ops(); + } + + /** @return The Paimon catalog operations as {@link PaimonCatalogOperations}. */ + @Override + public TableCatalog asTableCatalog() { + return (PaimonCatalogOperations) ops(); + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonCatalogBackend.java b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonCatalogBackend.java new file mode 100644 index 00000000000..4e033a61d46 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonCatalogBackend.java @@ -0,0 +1,11 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog.lakehouse.paimon; + +/** The type of Paimon catalog backend. */ +public enum PaimonCatalogBackend { + FILESYSTEM, + HIVE +} diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonCatalogOperations.java b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonCatalogOperations.java new file mode 100644 index 00000000000..e73b6470738 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonCatalogOperations.java @@ -0,0 +1,475 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog.lakehouse.paimon; + +import static com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonConfig.loadPaimonConfig; +import static com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonSchema.fromPaimonSchema; +import static com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonTable.fromPaimonTable; +import static com.datastrato.gravitino.catalog.lakehouse.paimon.utils.TableOpsUtils.checkColumn; +import static com.datastrato.gravitino.connector.BaseCatalog.CATALOG_BYPASS_PREFIX; + +import com.datastrato.gravitino.NameIdentifier; +import com.datastrato.gravitino.Namespace; +import com.datastrato.gravitino.catalog.lakehouse.paimon.ops.PaimonTableOps; +import com.datastrato.gravitino.connector.CatalogInfo; +import com.datastrato.gravitino.connector.CatalogOperations; +import com.datastrato.gravitino.connector.PropertiesMetadata; +import com.datastrato.gravitino.exceptions.NoSuchCatalogException; +import com.datastrato.gravitino.exceptions.NoSuchColumnException; +import com.datastrato.gravitino.exceptions.NoSuchSchemaException; +import com.datastrato.gravitino.exceptions.NoSuchTableException; +import com.datastrato.gravitino.exceptions.NonEmptySchemaException; +import com.datastrato.gravitino.exceptions.SchemaAlreadyExistsException; +import com.datastrato.gravitino.exceptions.TableAlreadyExistsException; +import com.datastrato.gravitino.meta.AuditInfo; +import com.datastrato.gravitino.rel.Column; +import com.datastrato.gravitino.rel.SchemaChange; +import com.datastrato.gravitino.rel.SupportsSchemas; +import com.datastrato.gravitino.rel.TableCatalog; +import com.datastrato.gravitino.rel.TableChange; +import com.datastrato.gravitino.rel.TableChange.RenameTable; +import com.datastrato.gravitino.rel.expressions.distributions.Distribution; +import com.datastrato.gravitino.rel.expressions.sorts.SortOrder; +import com.datastrato.gravitino.rel.expressions.transforms.Transform; +import com.datastrato.gravitino.rel.indexes.Index; +import com.datastrato.gravitino.utils.MapUtils; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import java.io.IOException; +import java.time.Instant; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import org.apache.commons.lang3.ArrayUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.table.Table; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Implementation of {@link CatalogOperations} that represents operations for interacting with the + * Paimon catalog in Gravitino. + */ +public class PaimonCatalogOperations implements CatalogOperations, SupportsSchemas, TableCatalog { + + public static final Logger LOG = LoggerFactory.getLogger(PaimonCatalogOperations.class); + + @VisibleForTesting public PaimonTableOps paimonTableOps; + private PaimonCatalogPropertiesMetadata paimonCatalogPropertiesMetadata; + private PaimonTablePropertiesMetadata paimonTablePropertiesMetadata; + private PaimonSchemaPropertiesMetadata paimonSchemaPropertiesMetadata; + + private static final String NO_SUCH_COLUMN_EXCEPTION = + "Paimon column of table %s does not exist."; + private static final String NO_SUCH_SCHEMA_EXCEPTION = + "Paimon schema (database) %s does not exist."; + private static final String NO_SUCH_TABLE_EXCEPTION = "Paimon table %s does not exist."; + private static final String NON_EMPTY_SCHEMA_EXCEPTION = + "Paimon schema (database) %s is not empty. One or more tables exist."; + private static final String SCHEMA_ALREADY_EXISTS_EXCEPTION = + "Paimon schema (database) %s already exists."; + private static final String TABLE_ALREADY_EXISTS_EXCEPTION = "Paimon table %s already exists."; + + /** + * Initializes the Paimon catalog operations with the provided configuration. + * + * @param conf The configuration map for the Paimon catalog operations. + * @param info The catalog info associated with this operations instance. + * @throws RuntimeException if initialization fails. + */ + @Override + public void initialize(Map conf, CatalogInfo info) throws RuntimeException { + this.paimonTableOps = + new PaimonTableOps(loadPaimonConfig(MapUtils.getPrefixMap(conf, CATALOG_BYPASS_PREFIX))); + this.paimonCatalogPropertiesMetadata = new PaimonCatalogPropertiesMetadata(); + this.paimonTablePropertiesMetadata = new PaimonTablePropertiesMetadata(); + this.paimonSchemaPropertiesMetadata = new PaimonSchemaPropertiesMetadata(); + } + + /** + * Lists the schemas under the specified namespace. + * + * @param namespace The namespace to list the schemas for. + * @return An array of {@link NameIdentifier} representing the schemas in the namespace. + * @throws NoSuchCatalogException If the provided namespace is invalid or does not exist. + */ + @Override + public NameIdentifier[] listSchemas(Namespace namespace) throws NoSuchCatalogException { + return paimonTableOps.listDatabases().stream() + .map(NameIdentifier::of) + .toArray(NameIdentifier[]::new); + } + + /** + * Creates a new schema with the provided identifier, comment, and metadata. + * + * @param identifier The identifier of the schema to create. + * @param comment The comment for the new schema. + * @param properties The properties for the new schema. + * @return The newly created {@link PaimonSchema} instance. + * @throws NoSuchCatalogException If the provided namespace is invalid or does not exist. + * @throws SchemaAlreadyExistsException If a schema with the same name already exists. + */ + @Override + public PaimonSchema createSchema( + NameIdentifier identifier, String comment, Map properties) + throws NoSuchCatalogException, SchemaAlreadyExistsException { + String currentUser = currentUser(); + PaimonSchema createdSchema = + PaimonSchema.builder() + .withName(identifier.name()) + .withComment(comment) + .withProperties(properties) + .withAuditInfo( + AuditInfo.builder().withCreator(currentUser).withCreateTime(Instant.now()).build()) + .build(); + try { + paimonTableOps.createDatabase(createdSchema.toPaimonSchema()); + } catch (Catalog.DatabaseAlreadyExistException e) { + throw new SchemaAlreadyExistsException(e, SCHEMA_ALREADY_EXISTS_EXCEPTION, identifier); + } catch (Exception e) { + throw new RuntimeException(e); + } + LOG.info( + "Created Paimon schema (database): {}.\nCurrent user: {} \nComment: {}.\nMetadata: {}.", + identifier, + currentUser, + comment, + properties); + return createdSchema; + } + + /** + * Loads the schema with the provided identifier. + * + * @param identifier The identifier of the schema to load. + * @return The loaded {@link PaimonSchema} representing the schema. + * @throws NoSuchSchemaException If the schema with the provided identifier does not exist. + */ + @Override + public PaimonSchema loadSchema(NameIdentifier identifier) throws NoSuchSchemaException { + Map properties; + try { + properties = paimonTableOps.loadDatabase(identifier.name()); + } catch (Catalog.DatabaseNotExistException e) { + throw new NoSuchSchemaException(e, NO_SUCH_SCHEMA_EXCEPTION, identifier); + } + LOG.info("Loaded Paimon schema (database) {}.", identifier); + return fromPaimonSchema(identifier.name(), properties); + } + + /** + * Alters the schema with the provided identifier according to the specified {@link SchemaChange} + * changes. + * + * @param identifier The identifier of the schema to alter. + * @param changes The changes to apply to the schema. + * @return The altered {@link PaimonSchema} instance. + * @throws NoSuchSchemaException If the schema with the provided identifier does not exist. + */ + @Override + public PaimonSchema alterSchema(NameIdentifier identifier, SchemaChange... changes) + throws NoSuchSchemaException { + throw new UnsupportedOperationException(); + } + + /** + * Drops the schema with the provided identifier. + * + * @param identifier The identifier of the schema to drop. + * @param cascade If set to true, drops all the tables in the schema as well. + * @return true if the schema is dropped successfully, false otherwise. + * @throws NonEmptySchemaException If the schema is not empty and 'cascade' is set to false. + */ + @Override + public boolean dropSchema(NameIdentifier identifier, boolean cascade) + throws NonEmptySchemaException { + try { + paimonTableOps.dropDatabase(identifier.name(), cascade); + } catch (Catalog.DatabaseNotExistException e) { + LOG.warn("Paimon schema (database) {} does not exist.", identifier); + return false; + } catch (Catalog.DatabaseNotEmptyException e) { + throw new NonEmptySchemaException(e, NON_EMPTY_SCHEMA_EXCEPTION, identifier); + } catch (Exception e) { + throw new RuntimeException(e); + } + LOG.info("Dropped Paimon schema (database) {}.", identifier); + return true; + } + + /** + * Lists all the tables under the specified namespace. + * + * @param namespace The namespace to list tables for. + * @return An array of {@link NameIdentifier} representing the tables in the namespace. + * @throws NoSuchSchemaException If the schema with the provided namespace does not exist. + */ + @Override + public NameIdentifier[] listTables(Namespace namespace) throws NoSuchSchemaException { + NameIdentifier schemaIdentifier = NameIdentifier.of(namespace.levels()); + if (!schemaExists(schemaIdentifier)) { + throw new NoSuchSchemaException(NO_SUCH_SCHEMA_EXCEPTION, schemaIdentifier); + } + List tables; + try { + tables = paimonTableOps.listTables(schemaIdentifier.name()); + } catch (Catalog.DatabaseNotExistException e) { + throw new NoSuchSchemaException(NO_SUCH_SCHEMA_EXCEPTION, schemaIdentifier); + } + return tables.stream() + .map( + tableIdentifier -> + NameIdentifier.of(ArrayUtils.add(namespace.levels(), tableIdentifier))) + .toArray(NameIdentifier[]::new); + } + + /** + * Loads the table with the provided identifier. + * + * @param identifier The identifier of the table to load. + * @return The loaded {@link PaimonTable} instance representing the table. + * @throws NoSuchTableException If the table with the provided identifier does not exist. + */ + @Override + public PaimonTable loadTable(NameIdentifier identifier) throws NoSuchTableException { + Table table; + try { + table = paimonTableOps.loadTable(identifier.toString()); + } catch (Catalog.TableNotExistException e) { + throw new NoSuchTableException(e, NO_SUCH_TABLE_EXCEPTION, identifier); + } + LOG.info("Loaded Paimon table {}.", identifier); + return fromPaimonTable(table); + } + + /** + * Creates a new table with the provided identifier, comment, and metadata. + * + * @param identifier The identifier of the table to create. + * @param columns The array of columns for the new table. + * @param comment The comment for the new table. + * @param properties The properties for the new table. + * @param partitioning The partitioning for the new table. + * @param indexes The indexes for the new table. + * @return The newly created {@link PaimonTable} instance. + * @throws NoSuchSchemaException If the schema with the provided namespace does not exist. + * @throws TableAlreadyExistsException If the table with the same identifier already exists. + */ + @Override + public PaimonTable createTable( + NameIdentifier identifier, + Column[] columns, + String comment, + Map properties, + Transform[] partitioning, + Distribution distribution, + SortOrder[] sortOrders, + Index[] indexes) + throws NoSuchSchemaException, TableAlreadyExistsException { + NameIdentifier schemaIdentifier = NameIdentifier.of(identifier.namespace().levels()); + if (!schemaExists(schemaIdentifier)) { + throw new NoSuchSchemaException(NO_SUCH_SCHEMA_EXCEPTION, schemaIdentifier); + } + String currentUser = currentUser(); + PaimonTable createdTable = + PaimonTable.builder() + .withName(identifier.name()) + .withColumns( + Arrays.stream(columns) + .map( + column -> { + checkColumn(column.name(), column.defaultValue(), column.autoIncrement()); + return PaimonColumn.builder() + .withName(column.name()) + .withType(column.dataType()) + .withComment(column.comment()) + .withNullable(column.nullable()) + .withAutoIncrement(column.autoIncrement()) + .withDefaultValue(column.defaultValue()) + .build(); + }) + .toArray(PaimonColumn[]::new)) + .withComment(comment) + .withProperties(properties) + .withPartitioning(partitioning) + .withDistribution(distribution) + .withSortOrders(sortOrders) + .withIndexes(indexes) + .withAuditInfo( + AuditInfo.builder().withCreator(currentUser).withCreateTime(Instant.now()).build()) + .build(); + try { + paimonTableOps.createTable(createdTable.toPaimonTable(identifier.toString())); + } catch (Catalog.DatabaseNotExistException e) { + throw new NoSuchSchemaException(e, NO_SUCH_SCHEMA_EXCEPTION, identifier); + } catch (Catalog.TableAlreadyExistException e) { + throw new TableAlreadyExistsException(e, TABLE_ALREADY_EXISTS_EXCEPTION, identifier); + } + LOG.info( + "Created Paimon table: {}.\nCurrent user: {} \nComment: {}.\nMetadata: {}.", + identifier, + currentUser, + comment, + properties); + return createdTable; + } + + /** + * Alters the table with the provided identifier according to the specified {@link TableChange} + * changes. + * + * @param identifier The identifier of the table to alter. + * @param changes The changes to apply to the table. + * @return The altered {@link PaimonTable} instance. + * @throws NoSuchTableException If the table with the provided identifier does not exist. + * @throws IllegalArgumentException This exception will not be thrown in this method. + */ + @Override + public PaimonTable alterTable(NameIdentifier identifier, TableChange... changes) + throws NoSuchTableException, IllegalArgumentException { + Optional renameTableOpt = + Arrays.stream(changes) + .filter(tableChange -> tableChange instanceof RenameTable) + .reduce((a, b) -> b); + if (renameTableOpt.isPresent()) { + String otherChanges = + Arrays.stream(changes) + .filter(tableChange -> !(tableChange instanceof RenameTable)) + .map(String::valueOf) + .collect(Collectors.joining("\n")); + Preconditions.checkArgument( + StringUtils.isEmpty(otherChanges), + String.format( + "The operation to change the table name cannot be performed together with other operations. " + + "The list of operations that you cannot perform includes: \n%s", + otherChanges)); + return renameTable(identifier, (RenameTable) renameTableOpt.get()); + } + return internalAlterTable(identifier, changes); + } + + /** + * Drops the table with the provided identifier. + * + * @param identifier The identifier of the table to drop. + * @return true if the table is successfully dropped, false if the table does not exist. + */ + @Override + public boolean dropTable(NameIdentifier identifier) { + try { + paimonTableOps.dropTable(identifier.toString()); + } catch (Catalog.TableNotExistException e) { + LOG.warn("Paimon table {} does not exist.", identifier); + return false; + } + LOG.info("Dropped Paimon table {}.", identifier); + return true; + } + + /** + * Purges the table with the provided identifier. + * + * @param identifier The identifier of the table to purge. + * @return true if the table is successfully purged, false if the table does not exist. + * @throws UnsupportedOperationException If the table type is EXTERNAL_TABLE, it cannot be purged. + */ + @Override + public boolean purgeTable(NameIdentifier identifier) throws UnsupportedOperationException { + throw new UnsupportedOperationException(); + } + + @Override + public PropertiesMetadata tablePropertiesMetadata() throws UnsupportedOperationException { + return paimonTablePropertiesMetadata; + } + + @Override + public PropertiesMetadata catalogPropertiesMetadata() throws UnsupportedOperationException { + return paimonCatalogPropertiesMetadata; + } + + @Override + public PropertiesMetadata schemaPropertiesMetadata() throws UnsupportedOperationException { + return paimonSchemaPropertiesMetadata; + } + + @Override + public PropertiesMetadata filesetPropertiesMetadata() throws UnsupportedOperationException { + throw new UnsupportedOperationException( + "Paimon catalog does not support fileset related operations."); + } + + @Override + public PropertiesMetadata topicPropertiesMetadata() throws UnsupportedOperationException { + throw new UnsupportedOperationException( + "Paimon catalog does not support topic related operations."); + } + + @Override + public void close() throws IOException { + if (paimonTableOps != null) { + try { + paimonTableOps.close(); + } catch (Exception e) { + throw new IOException(e.getMessage()); + } + } + } + + /** + * Performs rename table change with the provided identifier. + * + * @param identifier The identifier of the table to rename. + * @param renameTable Table Change to modify the table name. + * @return The renamed {@link PaimonTable} instance. + * @throws NoSuchTableException If the table with the provided identifier does not exist. + * @throws IllegalArgumentException This exception will not be thrown in this method. + */ + private PaimonTable renameTable(NameIdentifier identifier, TableChange.RenameTable renameTable) + throws NoSuchTableException, IllegalArgumentException { + try { + paimonTableOps.renameTable( + identifier.toString(), + NameIdentifier.of(identifier.namespace(), renameTable.getNewName()).toString()); + } catch (Catalog.TableNotExistException e) { + throw new NoSuchTableException(e, NO_SUCH_TABLE_EXCEPTION, identifier); + } catch (Catalog.TableAlreadyExistException e) { + throw new TableAlreadyExistsException(e, TABLE_ALREADY_EXISTS_EXCEPTION, identifier); + } + return loadTable(NameIdentifier.of(identifier.namespace(), renameTable.getNewName())); + } + + /** + * Performs alter table changes with the provided identifier according to the specified {@link + * TableChange} changes. + * + * @param identifier The identifier of the table to alter. + * @param changes The changes to apply to the table. + * @return The altered {@link PaimonTable} instance. + * @throws NoSuchTableException If the table with the provided identifier does not exist. + * @throws IllegalArgumentException This exception will not be thrown in this method. + */ + private PaimonTable internalAlterTable(NameIdentifier identifier, TableChange... changes) + throws NoSuchTableException, IllegalArgumentException { + try { + paimonTableOps.alterTable(identifier.toString(), changes); + } catch (Catalog.TableNotExistException e) { + throw new NoSuchTableException(e, NO_SUCH_TABLE_EXCEPTION, identifier); + } catch (Catalog.ColumnNotExistException e) { + throw new NoSuchColumnException(e, NO_SUCH_COLUMN_EXCEPTION, identifier); + } catch (Exception e) { + throw new RuntimeException(e); + } + return loadTable(NameIdentifier.of(identifier.namespace(), identifier.name())); + } + + private static String currentUser() { + return System.getProperty("user.name"); + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonCatalogPropertiesMetadata.java b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonCatalogPropertiesMetadata.java new file mode 100644 index 00000000000..be9aed2baa2 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonCatalogPropertiesMetadata.java @@ -0,0 +1,67 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog.lakehouse.paimon; + +import static com.datastrato.gravitino.connector.PropertyEntry.enumImmutablePropertyEntry; +import static com.datastrato.gravitino.connector.PropertyEntry.stringRequiredPropertyEntry; + +import com.datastrato.gravitino.connector.BaseCatalogPropertiesMetadata; +import com.datastrato.gravitino.connector.PropertiesMetadata; +import com.datastrato.gravitino.connector.PropertyEntry; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Implementation of {@link PropertiesMetadata} that represents Paimon catalog properties metadata. + */ +public class PaimonCatalogPropertiesMetadata extends BaseCatalogPropertiesMetadata { + + public static final String GRAVITINO_CATALOG_BACKEND = "catalog-backend"; + public static final String GRAVITINO_TABLE_TYPE = "table-type"; + public static final String PAIMON_METASTORE = "metastore"; + public static final String PAIMON_TABLE_TYPE = "table.type"; + public static final String WAREHOUSE = "warehouse"; + public static final String URI = "uri"; + + private static final Map> PROPERTIES_METADATA; + + public static final Map CATALOG_CONFIG_MAPPING = + ImmutableMap.of( + GRAVITINO_CATALOG_BACKEND, + PAIMON_METASTORE, + GRAVITINO_TABLE_TYPE, + PAIMON_TABLE_TYPE, + WAREHOUSE, + WAREHOUSE, + URI, + URI); + + static { + List> propertyEntries = + ImmutableList.of( + enumImmutablePropertyEntry( + GRAVITINO_CATALOG_BACKEND, + "Paimon catalog backend type", + true, + PaimonCatalogBackend.class, + null, + false, + false), + stringRequiredPropertyEntry(WAREHOUSE, "Paimon catalog warehouse config", false, false), + stringRequiredPropertyEntry(URI, "Paimon catalog uri config", false, false)); + HashMap> result = Maps.newHashMap(BASIC_CATALOG_PROPERTY_ENTRIES); + result.putAll(Maps.uniqueIndex(propertyEntries, PropertyEntry::getName)); + PROPERTIES_METADATA = ImmutableMap.copyOf(result); + } + + @Override + protected Map> specificPropertyEntries() { + return PROPERTIES_METADATA; + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonColumn.java b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonColumn.java new file mode 100644 index 00000000000..6babdbda394 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonColumn.java @@ -0,0 +1,93 @@ +/* + * Copyright 2023 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog.lakehouse.paimon; + +import static com.datastrato.gravitino.catalog.lakehouse.paimon.utils.TypeUtils.fromPaimonType; +import static com.datastrato.gravitino.catalog.lakehouse.paimon.utils.TypeUtils.toPaimonType; + +import com.datastrato.gravitino.connector.BaseColumn; +import com.datastrato.gravitino.rel.Column; +import java.util.List; +import java.util.stream.Collectors; +import lombok.EqualsAndHashCode; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.RowType; + +/** Implementation of {@link Column} that represents a column in the Paimon column. */ +@EqualsAndHashCode(callSuper = true) +public class PaimonColumn extends BaseColumn { + + private PaimonColumn() {} + + /** + * Converts {@link PaimonColumn} instance to inner column. + * + * @param id The id of inner column. + * @return The converted inner column. + */ + public DataField toPaimonColumn(int id) { + return new DataField(id, name, toPaimonType(dataType), comment); + } + + /** + * Creates new {@link PaimonColumn} instance from inner columns. + * + * @param rowType The {@link RowType} instance of inner column. + * @return New {@link PaimonColumn} instances. + */ + public static List fromPaimonColumn(RowType rowType) { + return rowType.getFields().stream() + .map(PaimonColumn::fromPaimonColumn) + .collect(Collectors.toList()); + } + + /** + * Creates a new {@link PaimonColumn} instance from inner column. + * + * @param dataField The {@link DataField} instance of inner column. + * @return A new {@link PaimonColumn} instance. + */ + public static PaimonColumn fromPaimonColumn(DataField dataField) { + return builder() + .withName(dataField.name()) + .withType(fromPaimonType(dataField.type())) + .withComment(dataField.description()) + .withNullable(dataField.type().isNullable()) + .build(); + } + + /** A builder class for constructing {@link PaimonColumn} instance. */ + public static class Builder extends BaseColumnBuilder { + + /** Creates a new instance of {@link Builder}. */ + private Builder() {} + + /** + * Internal method to build a {@link PaimonColumn} instance using the provided values. + * + * @return A new {@link PaimonColumn} instance with the configured values. + */ + @Override + protected PaimonColumn internalBuild() { + PaimonColumn paimonColumn = new PaimonColumn(); + paimonColumn.name = name; + paimonColumn.comment = comment; + paimonColumn.dataType = dataType; + paimonColumn.nullable = nullable; + paimonColumn.autoIncrement = autoIncrement; + paimonColumn.defaultValue = defaultValue == null ? DEFAULT_VALUE_NOT_SET : defaultValue; + return paimonColumn; + } + } + + /** + * Creates a new instance of {@link Builder}. + * + * @return The new instance. + */ + public static Builder builder() { + return new Builder(); + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonConfig.java b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonConfig.java new file mode 100644 index 00000000000..85e61743c15 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonConfig.java @@ -0,0 +1,72 @@ +/* + * Copyright 2023 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ + +package com.datastrato.gravitino.catalog.lakehouse.paimon; + +import com.datastrato.gravitino.Config; +import com.datastrato.gravitino.config.ConfigBuilder; +import com.datastrato.gravitino.config.ConfigConstants; +import com.datastrato.gravitino.config.ConfigEntry; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.apache.paimon.options.CatalogOptions; + +public class PaimonConfig extends Config { + + public static final ConfigEntry METASTORE = + new ConfigBuilder(PaimonCatalogPropertiesMetadata.PAIMON_METASTORE) + .doc(CatalogOptions.METASTORE.description().toString()) + .version(ConfigConstants.VERSION_0_5_0) + .stringConf() + .createWithDefault(CatalogOptions.METASTORE.defaultValue()); + + public static final ConfigEntry WAREHOUSE = + new ConfigBuilder(PaimonCatalogPropertiesMetadata.WAREHOUSE) + .doc(CatalogOptions.WAREHOUSE.description().toString()) + .version(ConfigConstants.VERSION_0_5_0) + .stringConf() + .checkValue(StringUtils::isNotBlank, ConfigConstants.NOT_BLANK_ERROR_MSG) + .create(); + + public static final ConfigEntry URI = + new ConfigBuilder(PaimonCatalogPropertiesMetadata.URI) + .doc(CatalogOptions.URI.description().toString()) + .version(ConfigConstants.VERSION_0_5_0) + .stringConf() + .checkValue(StringUtils::isNotBlank, ConfigConstants.NOT_BLANK_ERROR_MSG) + .create(); + + public static final ConfigEntry TABLE_TYPE = + new ConfigBuilder(PaimonCatalogPropertiesMetadata.PAIMON_TABLE_TYPE) + .doc(CatalogOptions.TABLE_TYPE.description().toString()) + .version(ConfigConstants.VERSION_0_5_0) + .stringConf() + .createWithDefault(CatalogOptions.TABLE_TYPE.defaultValue().toString()); + + public PaimonConfig(Map properties) { + super(false); + loadFromMap(properties, k -> true); + } + + public PaimonConfig() { + super(false); + } + + public static PaimonConfig loadPaimonConfig(Map properties) { + return new PaimonConfig( + properties.entrySet().stream() + .filter( + property -> + PaimonCatalogPropertiesMetadata.CATALOG_CONFIG_MAPPING.containsKey( + property.getKey())) + .collect( + Collectors.toMap( + property -> + PaimonCatalogPropertiesMetadata.CATALOG_CONFIG_MAPPING.get( + property.getKey()), + Map.Entry::getValue))); + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonSchema.java b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonSchema.java new file mode 100644 index 00000000000..3bea2f48f64 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonSchema.java @@ -0,0 +1,86 @@ +/* + * Copyright 2023 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog.lakehouse.paimon; + +import static com.datastrato.gravitino.meta.AuditInfo.EMPTY; + +import com.datastrato.gravitino.connector.BaseSchema; +import com.datastrato.gravitino.rel.Schema; +import java.util.Map; +import java.util.Optional; +import lombok.ToString; +import org.apache.paimon.utils.Pair; + +/** + * Implementation of {@link Schema} that represents a Paimon Schema (Database) entity in the Paimon + * schema. + */ +@ToString +public class PaimonSchema extends BaseSchema { + + private PaimonSchema() {} + + /** + * Converts {@link PaimonSchema} instance to inner schema. + * + * @return The converted inner schema. + */ + public Pair> toPaimonSchema() { + return Pair.of(name, properties); + } + + /** + * Creates a new {@link PaimonSchema} instance from inner schema. + * + * @param name The name of inner schema. + * @param properties The properties of inner schema. + * @return A new {@link PaimonSchema} instance. + */ + public static PaimonSchema fromPaimonSchema(String name, Map properties) { + return builder() + .withName(name) + .withComment( + Optional.of(properties) + .map(map -> map.get(PaimonSchemaPropertiesMetadata.COMMENT)) + .orElse(null)) + .withProperties(properties) + .withAuditInfo(EMPTY) + .build(); + } + + /** A builder class for constructing {@link PaimonSchema} instance. */ + public static class Builder extends BaseSchemaBuilder { + + /** Creates a new instance of {@link Builder}. */ + private Builder() {} + + /** + * Internal method to build a {@link PaimonSchema} instance using the provided values. + * + * @return A new {@link PaimonSchema} instance with the configured values. + */ + @Override + protected PaimonSchema internalBuild() { + PaimonSchema paimonSchema = new PaimonSchema(); + paimonSchema.name = name; + paimonSchema.comment = + comment == null + ? (properties == null ? null : properties.get(PaimonSchemaPropertiesMetadata.COMMENT)) + : comment; + paimonSchema.properties = properties; + paimonSchema.auditInfo = auditInfo; + return paimonSchema; + } + } + + /** + * Creates a new instance of {@link Builder}. + * + * @return The new instance. + */ + public static Builder builder() { + return new Builder(); + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonSchemaPropertiesMetadata.java b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonSchemaPropertiesMetadata.java new file mode 100644 index 00000000000..9a228ef6f00 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonSchemaPropertiesMetadata.java @@ -0,0 +1,36 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog.lakehouse.paimon; + +import static com.datastrato.gravitino.connector.PropertyEntry.stringReservedPropertyEntry; + +import com.datastrato.gravitino.connector.BasePropertiesMetadata; +import com.datastrato.gravitino.connector.PropertiesMetadata; +import com.datastrato.gravitino.connector.PropertyEntry; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; +import java.util.List; +import java.util.Map; + +/** + * Implementation of {@link PropertiesMetadata} that represents Paimon schema properties metadata. + */ +public class PaimonSchemaPropertiesMetadata extends BasePropertiesMetadata { + + public static final String COMMENT = "comment"; + + private static final Map> PROPERTIES_METADATA; + + static { + List> propertyEntries = + ImmutableList.of(stringReservedPropertyEntry(COMMENT, "Schema comment", true)); + PROPERTIES_METADATA = Maps.uniqueIndex(propertyEntries, PropertyEntry::getName); + } + + @Override + protected Map> specificPropertyEntries() { + return PROPERTIES_METADATA; + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonTable.java b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonTable.java new file mode 100644 index 00000000000..b0415f77d6a --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonTable.java @@ -0,0 +1,93 @@ +/* + * Copyright 2023 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog.lakehouse.paimon; + +import static com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonColumn.fromPaimonColumn; +import static com.datastrato.gravitino.meta.AuditInfo.EMPTY; + +import com.datastrato.gravitino.connector.BaseTable; +import com.datastrato.gravitino.connector.TableOperations; +import com.google.common.collect.Maps; +import lombok.Getter; +import lombok.ToString; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.table.Table; +import org.apache.paimon.types.DataField; +import org.apache.paimon.utils.Pair; + +/** Implementation of {@link Table} that represents a Paimon Table entity in the Paimon table. */ +@ToString +@Getter +public class PaimonTable extends BaseTable { + + private PaimonTable() {} + + @Override + protected TableOperations newOps() { + // TODO: Implement this interface when we have the Paimon table operations. + throw new UnsupportedOperationException("PaimonTable does not support TableOperations."); + } + + /** + * Converts {@link PaimonTable} instance to inner table. + * + * @return The converted inner table. + */ + public Pair toPaimonTable(String tableName) { + Schema.Builder builder = Schema.newBuilder().comment(comment).options(properties); + for (int index = 0; index < columns.length; index++) { + DataField dataField = ((PaimonColumn) columns[index]).toPaimonColumn(index); + builder.column(dataField.name(), dataField.type(), dataField.description()); + } + return Pair.of(tableName, builder.build()); + } + + /** + * Creates a new {@link PaimonTable} instance from inner table. + * + * @param table The {@link Table} instance of inner table. + * @return A new {@link PaimonTable} instance. + */ + public static PaimonTable fromPaimonTable(Table table) { + return builder() + .withName(table.name()) + .withColumns(fromPaimonColumn(table.rowType()).toArray(new PaimonColumn[0])) + .withComment(table.comment().orElse(null)) + .withProperties(table.options()) + .withAuditInfo(EMPTY) + .build(); + } + + /** A builder class for constructing {@link PaimonTable} instance. */ + public static class Builder extends BaseTableBuilder { + + /** Creates a new instance of {@link Builder}. */ + private Builder() {} + + /** + * Internal method to build a {@link PaimonTable} instance using the provided values. + * + * @return A new {@link PaimonTable} instance with the configured values. + */ + @Override + protected PaimonTable internalBuild() { + PaimonTable paimonTable = new PaimonTable(); + paimonTable.name = name; + paimonTable.comment = comment; + paimonTable.columns = columns; + paimonTable.properties = properties == null ? Maps.newHashMap() : Maps.newHashMap(properties); + paimonTable.auditInfo = auditInfo; + return paimonTable; + } + } + /** + * Creates a new instance of {@link Builder}. + * + * @return The new instance. + */ + public static Builder builder() { + return new Builder(); + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonTablePropertiesMetadata.java b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonTablePropertiesMetadata.java new file mode 100644 index 00000000000..5f58adb04ff --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/PaimonTablePropertiesMetadata.java @@ -0,0 +1,39 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog.lakehouse.paimon; + +import static com.datastrato.gravitino.connector.PropertyEntry.stringReservedPropertyEntry; + +import com.datastrato.gravitino.connector.BasePropertiesMetadata; +import com.datastrato.gravitino.connector.PropertiesMetadata; +import com.datastrato.gravitino.connector.PropertyEntry; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; +import java.util.List; +import java.util.Map; + +/** + * Implementation of {@link PropertiesMetadata} that represents Paimon table properties metadata. + */ +public class PaimonTablePropertiesMetadata extends BasePropertiesMetadata { + + public static final String COMMENT = "comment"; + public static final String CREATOR = "creator"; + + private static final Map> PROPERTIES_METADATA; + + static { + List> propertyEntries = + ImmutableList.of( + stringReservedPropertyEntry(COMMENT, "The table comment", true), + stringReservedPropertyEntry(CREATOR, "The table creator", false)); + PROPERTIES_METADATA = Maps.uniqueIndex(propertyEntries, PropertyEntry::getName); + } + + @Override + protected Map> specificPropertyEntries() { + return PROPERTIES_METADATA; + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/ops/PaimonTableOps.java b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/ops/PaimonTableOps.java new file mode 100644 index 00000000000..60ae42c00d4 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/ops/PaimonTableOps.java @@ -0,0 +1,91 @@ +/* + * Copyright 2023 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog.lakehouse.paimon.ops; + +import static com.datastrato.gravitino.catalog.lakehouse.paimon.utils.CatalogUtils.loadCatalogBackend; +import static com.datastrato.gravitino.catalog.lakehouse.paimon.utils.TableOpsUtils.buildSchemaChanges; + +import com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonConfig; +import com.datastrato.gravitino.rel.TableChange; +import java.util.List; +import java.util.Map; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Catalog.ColumnAlreadyExistException; +import org.apache.paimon.catalog.Catalog.ColumnNotExistException; +import org.apache.paimon.catalog.Catalog.DatabaseAlreadyExistException; +import org.apache.paimon.catalog.Catalog.DatabaseNotEmptyException; +import org.apache.paimon.catalog.Catalog.DatabaseNotExistException; +import org.apache.paimon.catalog.Catalog.TableAlreadyExistException; +import org.apache.paimon.catalog.Catalog.TableNotExistException; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.table.Table; +import org.apache.paimon.utils.Pair; + +/** Table operation proxy that handles table operations of an underlying Paimon catalog. */ +public class PaimonTableOps implements AutoCloseable { + + protected Catalog catalog; + + public PaimonTableOps(PaimonConfig paimonConfig) { + catalog = loadCatalogBackend(paimonConfig); + } + + @Override + public void close() throws Exception { + if (catalog != null) { + catalog.close(); + } + } + + public List listDatabases() { + return catalog.listDatabases(); + } + + public Map loadDatabase(String databaseName) throws DatabaseNotExistException { + return catalog.loadDatabaseProperties(databaseName); + } + + public void createDatabase(Pair> database) + throws DatabaseAlreadyExistException { + catalog.createDatabase(database.getKey(), false, database.getRight()); + } + + public void dropDatabase(String databaseName, boolean cascade) + throws DatabaseNotExistException, DatabaseNotEmptyException { + catalog.dropDatabase(databaseName, false, cascade); + } + + public List listTables(String databaseName) throws DatabaseNotExistException { + return catalog.listTables(databaseName); + } + + public Table loadTable(String tableName) throws TableNotExistException { + return catalog.getTable(tableIdentifier(tableName)); + } + + public void createTable(Pair table) + throws TableAlreadyExistException, DatabaseNotExistException { + catalog.createTable(tableIdentifier(table.getKey()), table.getValue(), false); + } + + public void renameTable(String fromTableName, String toTableName) + throws TableNotExistException, TableAlreadyExistException { + catalog.renameTable(tableIdentifier(fromTableName), tableIdentifier(toTableName), false); + } + + public void alterTable(String tableName, TableChange... changes) + throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException { + catalog.alterTable(tableIdentifier(tableName), buildSchemaChanges(changes), false); + } + + public void dropTable(String tableName) throws TableNotExistException { + catalog.dropTable(tableIdentifier(tableName), false); + } + + private Identifier tableIdentifier(String tableName) { + return Identifier.fromString(tableName); + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/CatalogUtils.java b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/CatalogUtils.java new file mode 100644 index 00000000000..1e6e13306c2 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/CatalogUtils.java @@ -0,0 +1,32 @@ +/* + * Copyright 2023 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog.lakehouse.paimon.utils; + +import static com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonConfig.METASTORE; + +import com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonConfig; +import java.util.Locale; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.options.Options; + +/** Utilities of {@link Catalog} to support catalog management. */ +public class CatalogUtils { + + /** + * Loads {@link Catalog} instance with given {@link PaimonConfig}. + * + * @param paimonConfig The Paimon configuration. + * @return The {@link Catalog} instance of catalog backend. + */ + public static Catalog loadCatalogBackend(PaimonConfig paimonConfig) { + String metastore = paimonConfig.get(METASTORE).toLowerCase(Locale.ROOT); + paimonConfig.set(METASTORE, metastore); + CatalogContext catalogContext = + CatalogContext.create(Options.fromMap(paimonConfig.getAllConfig())); + return CatalogFactory.createCatalog(catalogContext); + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TableOpsUtils.java b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TableOpsUtils.java new file mode 100644 index 00000000000..7b189be26c1 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TableOpsUtils.java @@ -0,0 +1,171 @@ +/* + * Copyright 2023 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog.lakehouse.paimon.utils; + +import static com.datastrato.gravitino.catalog.lakehouse.paimon.utils.TypeUtils.toPaimonType; +import static org.apache.paimon.schema.SchemaChange.addColumn; +import static org.apache.paimon.schema.SchemaChange.dropColumn; +import static org.apache.paimon.schema.SchemaChange.removeOption; +import static org.apache.paimon.schema.SchemaChange.renameColumn; +import static org.apache.paimon.schema.SchemaChange.setOption; +import static org.apache.paimon.schema.SchemaChange.updateColumnComment; +import static org.apache.paimon.schema.SchemaChange.updateColumnNullability; +import static org.apache.paimon.schema.SchemaChange.updateColumnPosition; +import static org.apache.paimon.schema.SchemaChange.updateColumnType; +import static org.apache.paimon.schema.SchemaChange.updateComment; + +import com.datastrato.gravitino.catalog.lakehouse.paimon.ops.PaimonTableOps; +import com.datastrato.gravitino.rel.Column; +import com.datastrato.gravitino.rel.TableChange; +import com.datastrato.gravitino.rel.TableChange.AddColumn; +import com.datastrato.gravitino.rel.TableChange.After; +import com.datastrato.gravitino.rel.TableChange.ColumnChange; +import com.datastrato.gravitino.rel.TableChange.ColumnPosition; +import com.datastrato.gravitino.rel.TableChange.Default; +import com.datastrato.gravitino.rel.TableChange.DeleteColumn; +import com.datastrato.gravitino.rel.TableChange.First; +import com.datastrato.gravitino.rel.TableChange.RemoveProperty; +import com.datastrato.gravitino.rel.TableChange.RenameColumn; +import com.datastrato.gravitino.rel.TableChange.SetProperty; +import com.datastrato.gravitino.rel.TableChange.UpdateColumnComment; +import com.datastrato.gravitino.rel.TableChange.UpdateColumnNullability; +import com.datastrato.gravitino.rel.TableChange.UpdateColumnPosition; +import com.datastrato.gravitino.rel.TableChange.UpdateColumnType; +import com.datastrato.gravitino.rel.TableChange.UpdateComment; +import com.datastrato.gravitino.rel.expressions.Expression; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import java.util.ArrayList; +import java.util.List; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.schema.SchemaChange.Move; + +/** Utilities of {@link PaimonTableOps} to support table operation. */ +public class TableOpsUtils { + + public static final Joiner DOT = Joiner.on("."); + + /** + * Builds {@link SchemaChange} schema changes with given {@link TableChange} table changes. + * + * @param tableChanges The {@link TableChange} table changes. + * @return The built {@link SchemaChange} schema changes. + * @throws UnsupportedOperationException If the provided table change does not support. + */ + public static List buildSchemaChanges(TableChange... tableChanges) + throws UnsupportedOperationException { + List schemaChanges = new ArrayList<>(); + for (TableChange tableChange : tableChanges) { + schemaChanges.add(buildSchemaChange(tableChange)); + } + return schemaChanges; + } + + /** + * Builds {@link SchemaChange} schema change with given {@link TableChange} table change. + * + * @param tableChange The {@link TableChange} table change. + * @return The built {@link SchemaChange} schema change. + * @throws UnsupportedOperationException If the provided table change does not support. + */ + public static SchemaChange buildSchemaChange(TableChange tableChange) + throws UnsupportedOperationException { + if (tableChange instanceof ColumnChange) { + String[] fieldNames = ((ColumnChange) tableChange).fieldName(); + Preconditions.checkArgument( + fieldNames.length == 1, + String.format( + "Paimon does not support update non-primitive type column. Illegal column: %s.", + fieldName(fieldNames))); + if (tableChange instanceof AddColumn) { + AddColumn addColumn = (AddColumn) tableChange; + String fieldName = fieldName(addColumn); + checkColumn(fieldName, addColumn.getDefaultValue(), addColumn.isAutoIncrement()); + return addColumn( + fieldName, + toPaimonType(addColumn.getDataType()).copy(addColumn.isNullable()), + addColumn.getComment(), + move(fieldName, addColumn.getPosition())); + } else if (tableChange instanceof DeleteColumn) { + return dropColumn(fieldName((DeleteColumn) tableChange)); + } else if (tableChange instanceof RenameColumn) { + RenameColumn renameColumn = ((RenameColumn) tableChange); + return renameColumn(fieldName(renameColumn), renameColumn.getNewName()); + } else if (tableChange instanceof UpdateColumnComment) { + UpdateColumnComment updateColumnComment = (UpdateColumnComment) tableChange; + return updateColumnComment( + fieldName(updateColumnComment), updateColumnComment.getNewComment()); + } else if (tableChange instanceof UpdateColumnNullability) { + UpdateColumnNullability updateColumnNullability = (UpdateColumnNullability) tableChange; + return updateColumnNullability( + fieldName(updateColumnNullability), updateColumnNullability.nullable()); + } else if (tableChange instanceof UpdateColumnPosition) { + UpdateColumnPosition updateColumnPosition = (UpdateColumnPosition) tableChange; + return updateColumnPosition( + move(fieldName(updateColumnPosition), updateColumnPosition.getPosition())); + } else if (tableChange instanceof UpdateColumnType) { + UpdateColumnType updateColumnType = (UpdateColumnType) tableChange; + return updateColumnType( + fieldName(updateColumnType), toPaimonType(updateColumnType.getNewDataType())); + } + } else if (tableChange instanceof UpdateComment) { + return updateComment(((UpdateComment) tableChange).getNewComment()); + } else if (tableChange instanceof SetProperty) { + SetProperty setProperty = ((SetProperty) tableChange); + return setOption(setProperty.getProperty(), setProperty.getValue()); + } else if (tableChange instanceof RemoveProperty) { + return removeOption(((RemoveProperty) tableChange).getProperty()); + } + throw new UnsupportedOperationException( + String.format( + "Paimon does not support %s table change.", tableChange.getClass().getSimpleName())); + } + + public static void checkColumn(String fieldName, Expression defaultValue, boolean autoIncrement) { + checkColumnDefaultValue(fieldName, defaultValue); + checkColumnAutoIncrement(fieldName, autoIncrement); + } + + public static String fieldName(String[] fieldName) { + return DOT.join(fieldName); + } + + public static String[] fieldName(String fieldName) { + return new String[] {fieldName}; + } + + private static String fieldName(ColumnChange columnChange) { + return fieldName(columnChange.fieldName()); + } + + private static void checkColumnDefaultValue(String fieldName, Expression defaultValue) { + Preconditions.checkArgument( + defaultValue.equals(Column.DEFAULT_VALUE_NOT_SET), + String.format( + "Paimon does not support column default value. Illegal column: %s.", fieldName)); + } + + private static void checkColumnAutoIncrement(String fieldName, boolean autoIncrement) { + Preconditions.checkArgument( + !autoIncrement, + String.format( + "Paimon does not support auto increment column. Illegal column: %s.", fieldName)); + } + + private static Move move(String fieldName, ColumnPosition columnPosition) + throws UnsupportedOperationException { + if (columnPosition instanceof After) { + return Move.after(fieldName, ((After) columnPosition).getColumn()); + } else if (columnPosition instanceof Default) { + return null; + } else if (columnPosition instanceof First) { + return Move.first(fieldName); + } + throw new UnsupportedOperationException( + String.format( + "Paimon does not support %s column position.", + columnPosition.getClass().getSimpleName())); + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TypeUtils.java b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TypeUtils.java new file mode 100644 index 00000000000..c6529e3372d --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TypeUtils.java @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.datastrato.gravitino.catalog.lakehouse.paimon.utils; + +import com.datastrato.gravitino.rel.types.Type; +import com.datastrato.gravitino.rel.types.Types; +import java.util.Arrays; +import java.util.stream.Collectors; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.BigIntType; +import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.BooleanType; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypeDefaultVisitor; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.DateType; +import org.apache.paimon.types.DecimalType; +import org.apache.paimon.types.DoubleType; +import org.apache.paimon.types.FloatType; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.LocalZonedTimestampType; +import org.apache.paimon.types.MapType; +import org.apache.paimon.types.MultisetType; +import org.apache.paimon.types.RowType; +import org.apache.paimon.types.SmallIntType; +import org.apache.paimon.types.TimeType; +import org.apache.paimon.types.TimestampType; +import org.apache.paimon.types.TinyIntType; +import org.apache.paimon.types.VarCharType; + +/** Utilities of {@link Type} to support type conversion. */ +public class TypeUtils { + + private TypeUtils() {} + + /** + * Convert Paimon {@link DataType} data type to Gravitino {@link Type} data type. + * + * @param dataType Paimon {@link DataType} data type. + * @return Gravitino {@link Type} data type. + */ + public static Type fromPaimonType(DataType dataType) { + return dataType.accept(PaimonToGravitinoTypeVisitor.INSTANCE); + } + + /** + * Convert Gravitino {@link Type} data type to Paimon {@link DataType} data type. + * + * @param type Gravitino {@link Type} data type. + * @return Paimon {@link DataType} data type. + */ + public static DataType toPaimonType(Type type) { + return GravitinoToPaimonTypeVisitor.visit(type); + } + + private static class PaimonToGravitinoTypeVisitor extends DataTypeDefaultVisitor { + + private static final PaimonToGravitinoTypeVisitor INSTANCE = new PaimonToGravitinoTypeVisitor(); + + @Override + public Type visit(VarCharType varCharType) { + return Types.VarCharType.of(varCharType.getLength()); + } + + @Override + public Type visit(BooleanType booleanType) { + return Types.BooleanType.get(); + } + + @Override + public Type visit(BinaryType binaryType) { + return Types.BinaryType.get(); + } + + @Override + public Type visit(DecimalType decimalType) { + return Types.DecimalType.of(decimalType.getPrecision(), decimalType.getScale()); + } + + @Override + public Type visit(TinyIntType tinyIntType) { + return Types.ByteType.get(); + } + + @Override + public Type visit(SmallIntType smallIntType) { + return Types.ShortType.get(); + } + + @Override + public Type visit(IntType intType) { + return Types.IntegerType.get(); + } + + @Override + public Type visit(BigIntType bigIntType) { + return Types.LongType.get(); + } + + @Override + public Type visit(FloatType floatType) { + return Types.FloatType.get(); + } + + @Override + public Type visit(DoubleType doubleType) { + return Types.DoubleType.get(); + } + + @Override + public Type visit(DateType dateType) { + return Types.DateType.get(); + } + + @Override + public Type visit(TimeType timeType) { + return Types.TimeType.get(); + } + + @Override + public Type visit(TimestampType timestampType) { + return Types.TimestampType.withoutTimeZone(); + } + + @Override + public Type visit(LocalZonedTimestampType localZonedTimestampType) { + return Types.TimestampType.withTimeZone(); + } + + @Override + public Type visit(ArrayType arrayType) { + return Types.ListType.of( + arrayType.getElementType().accept(this), arrayType.getElementType().isNullable()); + } + + @Override + public Type visit(MultisetType multisetType) { + return Types.MapType.of( + multisetType.getElementType().accept(this), Types.IntegerType.get(), false); + } + + @Override + public Type visit(MapType mapType) { + return Types.MapType.of( + mapType.getKeyType().accept(this), + mapType.getValueType().accept(this), + mapType.getValueType().isNullable()); + } + + @Override + public Type visit(RowType rowType) { + return Types.StructType.of( + rowType.getFields().stream() + .map( + field -> + Types.StructType.Field.of( + field.name(), + field.type().accept(this), + field.type().isNullable(), + field.description())) + .toArray(Types.StructType.Field[]::new)); + } + + @Override + protected Type defaultMethod(DataType dataType) { + return Types.UnparsedType.of(dataType.asSQLString()); + } + } + + private static class GravitinoToPaimonTypeVisitor { + + public static DataType visit(Type type) { + switch (type.name()) { + case BOOLEAN: + return DataTypes.BOOLEAN(); + case BYTE: + return DataTypes.TINYINT(); + case SHORT: + return DataTypes.SMALLINT(); + case INTEGER: + return DataTypes.INT(); + case LONG: + return DataTypes.BIGINT(); + case FLOAT: + return DataTypes.FLOAT(); + case DOUBLE: + return DataTypes.DOUBLE(); + case DECIMAL: + Types.DecimalType decimalType = (Types.DecimalType) type; + return DataTypes.DECIMAL(decimalType.precision(), decimalType.scale()); + case BINARY: + return DataTypes.BINARY(BinaryType.MAX_LENGTH); + case STRING: + return DataTypes.STRING(); + case VARCHAR: + return DataTypes.VARCHAR(((Types.VarCharType) type).length()); + case DATE: + return DataTypes.DATE(); + case TIME: + return DataTypes.TIME(); + case TIMESTAMP: + return ((Types.TimestampType) type).hasTimeZone() + ? DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE() + : DataTypes.TIMESTAMP(); + case LIST: + Types.ListType listType = (Types.ListType) type; + return DataTypes.ARRAY(visit(listType.elementType())); + case MAP: + Types.MapType mapType = (Types.MapType) type; + return DataTypes.MAP(visit(mapType.keyType()), visit(mapType.valueType())); + case STRUCT: + return RowType.builder() + .fields( + Arrays.stream(((Types.StructType) type).fields()) + .map(Types.StructType.Field::type) + .map(GravitinoToPaimonTypeVisitor::visit) + .collect(Collectors.toList())) + .build(); + default: + throw new UnsupportedOperationException( + String.format( + "Paimon does not support Gravitino %s data type.", type.simpleString())); + } + } + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/main/resources/META-INF/services/com.datastrato.gravitino.CatalogProvider b/catalogs/catalog-lakehouse-paimon/src/main/resources/META-INF/services/com.datastrato.gravitino.CatalogProvider new file mode 100644 index 00000000000..ef103c4a9d7 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/resources/META-INF/services/com.datastrato.gravitino.CatalogProvider @@ -0,0 +1,5 @@ +# +# Copyright 2024 Datastrato Pvt Ltd. +# This software is licensed under the Apache License version 2. +# +com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonCatalog diff --git a/catalogs/catalog-lakehouse-paimon/src/main/resources/lakehouse-paimon.conf b/catalogs/catalog-lakehouse-paimon/src/main/resources/lakehouse-paimon.conf new file mode 100644 index 00000000000..231d5b1d72a --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/main/resources/lakehouse-paimon.conf @@ -0,0 +1,8 @@ +# +# Copyright 2024 Datastrato Pvt Ltd. +# This software is licensed under the Apache License version 2. +# + +## This file holds common configurations for lakehouse-paimon catalog. The format of the key is +## 'gravitino.bypass.{paimon-inner-config-key}' and `paimon-inner-config-key` is the +## actual key that pass to lakehouse-paimon catalog. diff --git a/catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/TestPaimonConfig.java b/catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/TestPaimonConfig.java new file mode 100644 index 00000000000..07e73a06315 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/TestPaimonConfig.java @@ -0,0 +1,43 @@ +/* + * Copyright 2023 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ + +package com.datastrato.gravitino.catalog.lakehouse.paimon; + +import static com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonCatalogPropertiesMetadata.GRAVITINO_TABLE_TYPE; +import static com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonCatalogPropertiesMetadata.PAIMON_TABLE_TYPE; +import static com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonConfig.TABLE_TYPE; +import static com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonConfig.loadPaimonConfig; +import static org.junit.jupiter.api.Assertions.assertEquals; + +import com.google.common.collect.ImmutableMap; +import java.util.Map; +import org.apache.paimon.table.TableType; +import org.junit.jupiter.api.Test; + +/** Tests for {@link PaimonConfig}. */ +public class TestPaimonConfig { + + @Test + public void testLoadFromMap() { + Map properties = + ImmutableMap.of(TABLE_TYPE.getKey(), TableType.EXTERNAL.toString()); + PaimonConfig paimonConfig = new PaimonConfig(); + paimonConfig.loadFromMap(properties, k -> k.startsWith("gravitino.")); + assertEquals(TABLE_TYPE.getDefaultValue(), paimonConfig.get(TABLE_TYPE)); + assertEquals(TableType.EXTERNAL.toString(), new PaimonConfig(properties).get(TABLE_TYPE)); + } + + @Test + public void testLoadPaimonConfig() { + assertEquals( + TableType.EXTERNAL.toString(), + loadPaimonConfig(ImmutableMap.of(GRAVITINO_TABLE_TYPE, TableType.EXTERNAL.toString())) + .get(TABLE_TYPE)); + assertEquals( + TABLE_TYPE.getDefaultValue(), + loadPaimonConfig(ImmutableMap.of(PAIMON_TABLE_TYPE, TableType.EXTERNAL.toString())) + .get(TABLE_TYPE)); + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/ops/TestPaimonTableOps.java b/catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/ops/TestPaimonTableOps.java new file mode 100644 index 00000000000..1313f2669cb --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/ops/TestPaimonTableOps.java @@ -0,0 +1,392 @@ +/* + * Copyright 2023 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ + +package com.datastrato.gravitino.catalog.lakehouse.paimon.ops; + +import static com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonConfig.WAREHOUSE; +import static com.datastrato.gravitino.catalog.lakehouse.paimon.utils.TableOpsUtils.fieldName; +import static com.datastrato.gravitino.catalog.lakehouse.paimon.utils.TypeUtils.toPaimonType; +import static com.datastrato.gravitino.rel.TableChange.ColumnPosition.after; +import static com.datastrato.gravitino.rel.TableChange.ColumnPosition.defaultPos; +import static com.datastrato.gravitino.rel.TableChange.ColumnPosition.first; +import static com.datastrato.gravitino.rel.TableChange.addColumn; +import static com.datastrato.gravitino.rel.TableChange.deleteColumn; +import static com.datastrato.gravitino.rel.TableChange.removeProperty; +import static com.datastrato.gravitino.rel.TableChange.renameColumn; +import static com.datastrato.gravitino.rel.TableChange.setProperty; +import static com.datastrato.gravitino.rel.TableChange.updateColumnComment; +import static com.datastrato.gravitino.rel.TableChange.updateColumnNullability; +import static com.datastrato.gravitino.rel.TableChange.updateColumnPosition; +import static com.datastrato.gravitino.rel.TableChange.updateColumnType; +import static com.datastrato.gravitino.rel.TableChange.updateComment; +import static org.apache.paimon.CoreOptions.BUCKET; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrowsExactly; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import com.datastrato.gravitino.NameIdentifier; +import com.datastrato.gravitino.Namespace; +import com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonConfig; +import com.datastrato.gravitino.rel.TableChange; +import com.datastrato.gravitino.rel.TableChange.ColumnPosition; +import com.datastrato.gravitino.rel.TableChange.UpdateColumnComment; +import com.datastrato.gravitino.rel.TableChange.UpdateComment; +import com.datastrato.gravitino.rel.types.Type; +import com.datastrato.gravitino.rel.types.Types; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import java.io.File; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Catalog.ColumnAlreadyExistException; +import org.apache.paimon.catalog.Catalog.ColumnNotExistException; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.schema.SchemaChange.AddColumn; +import org.apache.paimon.table.Table; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.DateType; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.MapType; +import org.apache.paimon.types.RowType; +import org.apache.paimon.types.TimestampType; +import org.apache.paimon.types.VarCharType; +import org.apache.paimon.utils.Pair; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +/** Tests for {@link PaimonTableOps}. */ +public class TestPaimonTableOps { + + private PaimonTableOps paimonTableOps; + @TempDir private File warehouse; + + private static final String DATABASE = "test_table_ops_database"; + private static final String TABLE = "test_table_ops_table"; + private static final String COMMENT = "table_ops_table_comment"; + private static final NameIdentifier IDENTIFIER = NameIdentifier.of(Namespace.of(DATABASE), TABLE); + private static final Map OPTIONS = ImmutableMap.of(BUCKET.key(), "10"); + + @BeforeEach + public void setUp() throws Exception { + paimonTableOps = + new PaimonTableOps( + new PaimonConfig(ImmutableMap.of(WAREHOUSE.getKey(), warehouse.getPath()))); + createDatabase(); + createTable(); + } + + @AfterEach + public void tearDown() throws Exception { + dropTable(); + dropDatabase(); + paimonTableOps.close(); + } + + @Test + void testAddColumn() throws Exception { + // Test AddColumn after column. + assertAddColumn(5, Types.BooleanType.get(), after("col_2"), 2); + // Test AddColumn first column. + assertAddColumn(6, Types.FloatType.get(), first(), 0); + // Test AddColumn last column. + assertAddColumn(7, Types.DoubleType.get(), defaultPos(), 6); + assertAddColumn(8, Types.DoubleType.get(), null, 7); + // Test NullPointerException with AddColumn for after non-existent column. + assertThrowsExactly( + NullPointerException.class, + () -> assertAddColumn(9, Types.LongType.get(), after("col_10"), null)); + } + + @Test + public void testUpdateColumnComment() throws Exception { + assertAlterTable( + table -> { + DataField dataField = table.rowType().getFields().get(0); + assertEquals("col_1", dataField.name()); + assertEquals(UpdateColumnComment.class.getSimpleName(), dataField.description()); + }, + updateColumnComment(fieldName("col_1"), UpdateColumnComment.class.getSimpleName())); + assertColumnNotExist( + updateColumnComment(fieldName("col_5"), UpdateComment.class.getSimpleName())); + } + + @Test + public void testUpdateColumnNullability() throws Exception { + assertAlterTable( + table -> { + DataField dataField = table.rowType().getFields().get(1); + assertEquals("col_2", dataField.name()); + assertFalse(dataField.type().isNullable()); + }, + updateColumnNullability(fieldName("col_2"), false)); + assertColumnNotExist(updateColumnNullability(fieldName("col_5"), true)); + } + + @Test + public void testUpdateColumnPosition() throws Exception { + // Test UpdateColumnPosition after column. + assertUpdateColumnPosition(3, after("col_1"), 0, 2, 1, 3); + // Test UpdateColumnPosition first column. + assertUpdateColumnPosition(4, first(), 1, 3, 2, 0); + // Test NullPointerException with UpdateColumnPosition for non-existent column. + assertThrowsExactly( + NullPointerException.class, () -> assertUpdateColumnPosition(5, defaultPos())); + // Test NullPointerException with UpdateColumnPosition for after non-existent column. + assertThrowsExactly( + NullPointerException.class, () -> assertUpdateColumnPosition(1, after("col_5"))); + } + + @Test + public void testUpdateColumnType() throws Exception { + assertAlterTable( + table -> { + DataField dataField = table.rowType().getFields().get(0); + assertEquals("col_1", dataField.name()); + assertEquals(DataTypes.BIGINT(), dataField.type()); + }, + updateColumnType(fieldName("col_1"), Types.LongType.get())); + assertColumnNotExist(updateColumnType(fieldName("col_5"), Types.ShortType.get())); + assertThrowsExactly( + IllegalStateException.class, + () -> + assertAlterTable( + emptyConsumer(), updateColumnType(fieldName("col_1"), Types.DateType.get()))); + assertThrowsExactly( + IllegalStateException.class, + () -> + assertAlterTable( + emptyConsumer(), updateColumnType(fieldName("col_4"), Types.LongType.get()))); + } + + @Test + public void testRenameColumn() throws Exception { + assertAlterTable( + table -> { + List fieldNames = table.rowType().getFieldNames(); + assertFalse(fieldNames.contains("col2")); + assertEquals("col_5", fieldNames.get(1)); + assertEquals(4, fieldNames.size()); + }, + renameColumn(fieldName("col_2"), "col_5")); + assertColumnNotExist(renameColumn(fieldName("col_6"), "col_7")); + assertThrowsExactly( + ColumnAlreadyExistException.class, + () -> assertAlterTable(emptyConsumer(), renameColumn(fieldName("col_1"), "col_4"))); + } + + @Test + public void testDeleteColumn() throws Exception { + assertAlterTable( + table -> { + List fieldNames = table.rowType().getFieldNames(); + assertFalse(fieldNames.contains("col_2")); + assertEquals("col_3", fieldNames.get(1)); + assertEquals(3, fieldNames.size()); + }, + deleteColumn(fieldName("col_2"), true)); + assertColumnNotExist(deleteColumn(fieldName("col_5"), true)); + assertColumnNotExist(deleteColumn(fieldName("col_5"), false)); + } + + @Test + public void testUpdateComment() throws Exception { + assertAlterTable( + table -> { + assertTrue(table.comment().isPresent()); + assertEquals(UpdateComment.class.getSimpleName(), table.comment().get()); + }, + updateComment(UpdateComment.class.getSimpleName())); + } + + @Test + public void testSetAndRemoveProperty() throws Exception { + String propertyKey = "test_property_key_1"; + assertFalse(paimonTableOps.loadTable(IDENTIFIER.toString()).options().containsKey(propertyKey)); + // Test SetProperty with non-existent property. + String propertyValue = "test_property_value_1"; + assertAlterTable( + table -> { + Map options = table.options(); + assertTrue(options.containsKey(propertyKey)); + assertEquals(propertyValue, options.get(propertyKey)); + }, + setProperty(propertyKey, propertyValue)); + // Test SetProperty with overwrite existing property. + String newPropertyValue = "test_property_value_2"; + assertAlterTable( + table -> { + Map options = table.options(); + assertTrue(options.containsKey(propertyKey)); + assertEquals(newPropertyValue, options.get(propertyKey)); + }, + setProperty(propertyKey, newPropertyValue)); + // Test RemoveProperty with existing property. + assertAlterTable( + table -> assertFalse(table.options().containsKey(propertyKey)), + removeProperty(propertyKey)); + // Test RemoveProperty with non-existent property. + assertDoesNotThrow(() -> assertAlterTable(emptyConsumer(), removeProperty(propertyKey))); + } + + @Test + public void testMultipleAlterTable() throws Exception { + assertAlterTable( + table -> { + List fieldNames = table.rowType().getFieldNames(); + assertEquals("col_5", fieldNames.get(0)); + assertFalse(fieldNames.contains("col_2")); + assertEquals(3, fieldNames.size()); + Map options = table.options(); + assertTrue(options.containsKey("test_property_key")); + assertEquals("test_property_value", options.get("test_property_key")); + }, + renameColumn(fieldName("col_1"), "col_5"), + deleteColumn(fieldName("col_2"), true), + setProperty("test_property_key", "test_property_value")); + } + + public void assertUpdateColumnPosition(int column, ColumnPosition columnPosition, int... fields) + throws Exception { + String columnName = "col_" + column; + assertAlterTable( + table -> { + List fieldNames = table.rowType().getFieldNames(); + assertEquals("col_1", fieldNames.get(fields[0])); + assertEquals("col_2", fieldNames.get(fields[1])); + assertEquals("col_3", fieldNames.get(fields[2])); + assertEquals("col_4", fieldNames.get(fields[3])); + }, + updateColumnPosition(fieldName(columnName), columnPosition)); + } + + public void assertAddColumn(int column, Type type, ColumnPosition columnPosition, Integer field) + throws Exception { + String columnName = "col_" + column; + assertAlterTable( + table -> { + DataField dataField = table.rowType().getFields().get(field); + assertEquals(columnName, dataField.name()); + assertEquals(toPaimonType(type), dataField.type()); + assertEquals(AddColumn.class.getSimpleName(), dataField.description()); + assertTrue(dataField.type().isNullable()); + }, + addColumn( + fieldName(columnName), + type, + SchemaChange.AddColumn.class.getSimpleName(), + columnPosition, + true, + false)); + } + + public void assertColumnNotExist(TableChange tableChange) { + assertThrowsExactly( + ColumnNotExistException.class, () -> assertAlterTable(emptyConsumer(), tableChange)); + } + + public void assertAlterTable(Consumer consumer, TableChange... tableChanges) + throws Exception { + consumer.accept(alterTable(tableChanges)); + } + + private Consumer
emptyConsumer() { + return table -> {}; + } + + private void createDatabase() throws Exception { + paimonTableOps.createDatabase(Pair.of(DATABASE, Maps.newHashMap())); + assertEquals(1, paimonTableOps.listDatabases().size()); + assertNotNull(paimonTableOps.loadDatabase(DATABASE)); + } + + private void createTable() throws Exception { + paimonTableOps.createTable( + Pair.of( + IDENTIFIER.toString(), + Schema.newBuilder() + .column("col_1", DataTypes.INT().notNull(), IntType.class.getSimpleName()) + .column("col_2", DataTypes.STRING(), VarCharType.class.getSimpleName()) + .column("col_3", DataTypes.STRING().notNull(), VarCharType.class.getSimpleName()) + .column( + "col_4", + DataTypes.ARRAY( + RowType.builder() + .field( + "sub_col_1", + DataTypes.DATE(), + RowType.class.getSimpleName() + DateType.class.getSimpleName()) + .field( + "sub_col_2", + DataTypes.MAP(DataTypes.STRING(), DataTypes.INT()), + RowType.class.getSimpleName() + MapType.class.getSimpleName()) + .field( + "sub_col_3", + DataTypes.TIMESTAMP().notNull(), + RowType.class.getSimpleName() + TimestampType.class.getSimpleName()) + .build()), + ArrayType.class.getSimpleName()) + .comment(COMMENT) + .options(OPTIONS) + .build())); + Table table = paimonTableOps.loadTable(IDENTIFIER.toString()); + assertEquals(TABLE, table.name()); + assertTrue(table.comment().isPresent()); + assertEquals( + RowType.builder() + .field("col_1", DataTypes.INT().notNull(), IntType.class.getSimpleName()) + .field("col_2", DataTypes.STRING(), VarCharType.class.getSimpleName()) + .field("col_3", DataTypes.STRING().notNull(), VarCharType.class.getSimpleName()) + .field( + "col_4", + DataTypes.ARRAY( + RowType.builder() + .field( + "sub_col_1", + DataTypes.DATE(), + RowType.class.getSimpleName() + DateType.class.getSimpleName()) + .field( + "sub_col_2", + DataTypes.MAP(DataTypes.STRING(), DataTypes.INT()), + RowType.class.getSimpleName() + MapType.class.getSimpleName()) + .field( + "sub_col_3", + DataTypes.TIMESTAMP().notNull(), + RowType.class.getSimpleName() + TimestampType.class.getSimpleName()) + .build()), + ArrayType.class.getSimpleName()) + .build() + .toString(), + table.rowType().toString()); + assertEquals(COMMENT, table.comment().get()); + assertEquals(OPTIONS.get(BUCKET.key()), table.options().get(BUCKET.key())); + } + + public Table alterTable(TableChange... tableChanges) + throws Catalog.ColumnAlreadyExistException, Catalog.TableNotExistException, + Catalog.ColumnNotExistException { + paimonTableOps.alterTable(IDENTIFIER.toString(), tableChanges); + return paimonTableOps.loadTable(IDENTIFIER.toString()); + } + + private void dropDatabase() throws Exception { + paimonTableOps.dropDatabase(DATABASE, true); + assertTrue(paimonTableOps.listDatabases().isEmpty()); + } + + private void dropTable() throws Exception { + paimonTableOps.dropTable(IDENTIFIER.toString()); + assertTrue(paimonTableOps.listTables(DATABASE).isEmpty()); + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TestCatalogUtils.java b/catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TestCatalogUtils.java new file mode 100644 index 00000000000..7f18305a505 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TestCatalogUtils.java @@ -0,0 +1,124 @@ +/* + * Copyright 2023 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ + +package com.datastrato.gravitino.catalog.lakehouse.paimon.utils; + +import static com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonConfig.METASTORE; +import static com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonConfig.WAREHOUSE; +import static com.datastrato.gravitino.catalog.lakehouse.paimon.utils.CatalogUtils.loadCatalogBackend; +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTORECONNECTURLKEY; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrowsExactly; + +import com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonCatalogBackend; +import com.datastrato.gravitino.catalog.lakehouse.paimon.PaimonConfig; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.LineNumberReader; +import java.io.Reader; +import java.nio.charset.StandardCharsets; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Locale; +import java.util.UUID; +import java.util.function.Consumer; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.FileSystemCatalog; +import org.apache.paimon.factories.FactoryException; +import org.apache.paimon.hive.HiveCatalog; +import org.apache.paimon.options.Options; +import org.apache.paimon.utils.HadoopUtils; +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; +import org.mockito.Mockito; + +/** Tests for {@link CatalogUtils}. */ +public class TestCatalogUtils { + + @Test + void testLoadCatalogBackend() throws Exception { + // Test load FileSystemCatalog for filesystem metastore. + assertCatalog(PaimonCatalogBackend.FILESYSTEM.name(), FileSystemCatalog.class); + // Test load HiveCatalog for hive metastore. + try (MockedStatic mocked = Mockito.mockStatic(HadoopUtils.class)) { + HiveConf hiveConf = new HiveConf(); + hiveConf.setVar(METASTORECONNECTURLKEY, connectionUrl()); + mocked + .when(() -> HadoopUtils.getHadoopConfiguration(Mockito.any(Options.class))) + .thenReturn(hiveConf); + assertCatalog(PaimonCatalogBackend.HIVE.name(), HiveCatalog.class); + } + // Test load catalog exception for other metastore. + assertThrowsExactly(FactoryException.class, () -> assertCatalog("other", catalog -> {})); + } + + private void assertCatalog(String metastore, Class expected) throws Exception { + assertCatalog(metastore, catalog -> assertEquals(expected, catalog.getClass())); + assertCatalog( + metastore.toLowerCase(Locale.ROOT), catalog -> assertEquals(expected, catalog.getClass())); + } + + private void assertCatalog(String metastore, Consumer consumer) throws Exception { + try (Catalog catalog = + loadCatalogBackend( + new PaimonConfig( + ImmutableMap.of( + METASTORE.getKey(), metastore, WAREHOUSE.getKey(), "/warehouse")))) { + consumer.accept(catalog); + } + } + + private String connectionUrl() throws SQLException, IOException { + String connectionUrl = String.format("jdbc:derby:memory:%s;create=true", UUID.randomUUID()); + setupMetastore(connectionUrl); + return connectionUrl; + } + + private void setupMetastore(String connectionUrl) throws SQLException, IOException { + InputStream inputStream = + ClassLoader.getSystemClassLoader().getResourceAsStream("hive-schema-3.1.0.derby.sql"); + if (inputStream != null) { + try (Reader reader = new InputStreamReader(inputStream, StandardCharsets.UTF_8)) { + runScript(DriverManager.getConnection(connectionUrl), reader); + } + } + } + + private void runScript(Connection conn, Reader reader) throws IOException, SQLException { + Statement statement = null; + try { + LineNumberReader lineNumberReader = new LineNumberReader(reader); + String line; + StringBuilder command = null; + while ((line = lineNumberReader.readLine()) != null) { + if (command == null) { + command = new StringBuilder(); + } + String script = line.trim(); + if (!script.isEmpty()) { + if (script.endsWith(";")) { + command.append(line, 0, line.lastIndexOf(";")); + command.append(" "); + statement = conn.createStatement(); + statement.execute(command.toString()); + command = null; + } else if (!script.startsWith("--") && !script.startsWith("//")) { + command.append(line); + command.append(" "); + } + } + } + } finally { + if (statement != null) { + statement.close(); + } + } + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TestTableOpsUtils.java b/catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TestTableOpsUtils.java new file mode 100644 index 00000000000..3534ba8c732 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TestTableOpsUtils.java @@ -0,0 +1,278 @@ +/* + * Copyright 2023 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ + +package com.datastrato.gravitino.catalog.lakehouse.paimon.utils; + +import static com.datastrato.gravitino.catalog.lakehouse.paimon.utils.TableOpsUtils.buildSchemaChange; +import static com.datastrato.gravitino.catalog.lakehouse.paimon.utils.TableOpsUtils.fieldName; +import static com.datastrato.gravitino.rel.TableChange.ColumnPosition.after; +import static com.datastrato.gravitino.rel.TableChange.ColumnPosition.defaultPos; +import static com.datastrato.gravitino.rel.TableChange.addColumn; +import static com.datastrato.gravitino.rel.TableChange.addIndex; +import static com.datastrato.gravitino.rel.TableChange.deleteColumn; +import static com.datastrato.gravitino.rel.TableChange.deleteIndex; +import static com.datastrato.gravitino.rel.TableChange.removeProperty; +import static com.datastrato.gravitino.rel.TableChange.rename; +import static com.datastrato.gravitino.rel.TableChange.renameColumn; +import static com.datastrato.gravitino.rel.TableChange.setProperty; +import static com.datastrato.gravitino.rel.TableChange.updateColumnAutoIncrement; +import static com.datastrato.gravitino.rel.TableChange.updateColumnComment; +import static com.datastrato.gravitino.rel.TableChange.updateColumnDefaultValue; +import static com.datastrato.gravitino.rel.TableChange.updateColumnNullability; +import static com.datastrato.gravitino.rel.TableChange.updateColumnPosition; +import static com.datastrato.gravitino.rel.TableChange.updateColumnType; +import static com.datastrato.gravitino.rel.TableChange.updateComment; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrowsExactly; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import com.datastrato.gravitino.rel.TableChange; +import com.datastrato.gravitino.rel.expressions.literals.Literals; +import com.datastrato.gravitino.rel.indexes.Index.IndexType; +import com.datastrato.gravitino.rel.types.Types; +import com.datastrato.gravitino.rel.types.Types.DoubleType; +import com.datastrato.gravitino.rel.types.Types.FloatType; +import com.datastrato.gravitino.rel.types.Types.IntegerType; +import com.datastrato.gravitino.rel.types.Types.ListType; +import com.datastrato.gravitino.rel.types.Types.MapType; +import com.datastrato.gravitino.rel.types.Types.StringType; +import java.util.Arrays; +import java.util.function.Consumer; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.schema.SchemaChange.AddColumn; +import org.apache.paimon.schema.SchemaChange.DropColumn; +import org.apache.paimon.schema.SchemaChange.Move.MoveType; +import org.apache.paimon.schema.SchemaChange.RemoveOption; +import org.apache.paimon.schema.SchemaChange.RenameColumn; +import org.apache.paimon.schema.SchemaChange.SetOption; +import org.apache.paimon.schema.SchemaChange.UpdateColumnComment; +import org.apache.paimon.schema.SchemaChange.UpdateColumnNullability; +import org.apache.paimon.schema.SchemaChange.UpdateColumnPosition; +import org.apache.paimon.schema.SchemaChange.UpdateColumnType; +import org.apache.paimon.schema.SchemaChange.UpdateComment; +import org.apache.paimon.types.DataTypeRoot; +import org.junit.jupiter.api.Test; + +/** Tests for {@link TableOpsUtils}. */ +public class TestTableOpsUtils { + + @Test + void testBuildSchemaChange() { + // Test supported table changes. + assertTableChange( + addColumn( + fieldName("col_1"), + IntegerType.get(), + AddColumn.class.getSimpleName(), + TableChange.ColumnPosition.first(), + false, + false), + AddColumn.class, + schemaChange -> { + AddColumn addColumn = (AddColumn) schemaChange; + assertEquals("col_1", addColumn.fieldName()); + assertEquals(DataTypeRoot.INTEGER, addColumn.dataType().getTypeRoot()); + assertEquals(AddColumn.class.getSimpleName(), addColumn.description()); + assertNotNull(addColumn.move()); + assertEquals(MoveType.FIRST, addColumn.move().type()); + assertEquals("col_1", addColumn.move().fieldName()); + assertNull(addColumn.move().referenceFieldName()); + assertFalse(addColumn.dataType().isNullable()); + }); + assertTableChange( + addColumn( + fieldName("col_2"), + FloatType.get(), + AddColumn.class.getSimpleName(), + after("col_1"), + true, + false), + AddColumn.class, + schemaChange -> { + AddColumn addColumn = (AddColumn) schemaChange; + assertEquals("col_2", addColumn.fieldName()); + assertEquals(DataTypeRoot.FLOAT, addColumn.dataType().getTypeRoot()); + assertEquals(AddColumn.class.getSimpleName(), addColumn.description()); + assertNotNull(addColumn.move()); + assertEquals(MoveType.AFTER, addColumn.move().type()); + assertEquals("col_2", addColumn.move().fieldName()); + assertEquals("col_1", addColumn.move().referenceFieldName()); + assertTrue(addColumn.dataType().isNullable()); + }); + assertTableChange( + addColumn( + fieldName("col_3"), + ListType.of(StringType.get(), false), + AddColumn.class.getSimpleName(), + defaultPos(), + false, + false), + AddColumn.class, + schemaChange -> { + AddColumn addColumn = (AddColumn) schemaChange; + assertEquals("col_3", addColumn.fieldName()); + assertEquals(DataTypeRoot.ARRAY, addColumn.dataType().getTypeRoot()); + assertEquals(AddColumn.class.getSimpleName(), addColumn.description()); + assertNull(addColumn.move()); + assertFalse(addColumn.dataType().isNullable()); + }); + assertTableChange( + addColumn( + fieldName("col_4"), + MapType.of(StringType.get(), IntegerType.get(), true), + AddColumn.class.getSimpleName(), + null, + false, + false), + AddColumn.class, + schemaChange -> { + AddColumn addColumn = (AddColumn) schemaChange; + assertEquals("col_4", addColumn.fieldName()); + assertEquals(DataTypeRoot.MAP, addColumn.dataType().getTypeRoot()); + assertEquals(AddColumn.class.getSimpleName(), addColumn.description()); + assertNull(addColumn.move()); + assertFalse(addColumn.dataType().isNullable()); + }); + assertTableChange( + updateColumnComment(fieldName("col_1"), UpdateColumnComment.class.getSimpleName()), + UpdateColumnComment.class, + schemaChange -> { + UpdateColumnComment updateColumnComment = (UpdateColumnComment) schemaChange; + assertEquals("col_1", fieldName(updateColumnComment.fieldNames())); + assertEquals( + UpdateColumnComment.class.getSimpleName(), updateColumnComment.newDescription()); + }); + assertTableChange( + updateColumnNullability(fieldName("col_2"), false), + UpdateColumnNullability.class, + schemaChange -> { + UpdateColumnNullability updateColumnNullability = (UpdateColumnNullability) schemaChange; + assertEquals("col_2", fieldName(updateColumnNullability.fieldNames())); + assertFalse(updateColumnNullability.newNullability()); + }); + assertTableChange( + updateColumnPosition(fieldName("col_3"), after("col_1")), + UpdateColumnPosition.class, + schemaChange -> { + UpdateColumnPosition updateColumnPosition = (UpdateColumnPosition) schemaChange; + assertEquals("col_3", updateColumnPosition.move().fieldName()); + assertEquals("col_1", updateColumnPosition.move().referenceFieldName()); + }); + assertTableChange( + updateColumnType(fieldName("col_4"), DoubleType.get()), + UpdateColumnType.class, + schemaChange -> { + UpdateColumnType updateColumnType = (UpdateColumnType) schemaChange; + assertEquals("col_4", updateColumnType.fieldName()); + assertEquals(DataTypeRoot.DOUBLE, updateColumnType.newDataType().getTypeRoot()); + }); + assertTableChange( + renameColumn(fieldName("col_1"), "col_5"), + RenameColumn.class, + schemaChange -> { + RenameColumn renameColumn = (RenameColumn) schemaChange; + assertEquals("col_1", renameColumn.fieldName()); + assertEquals("col_5", renameColumn.newName()); + }); + assertTableChange( + deleteColumn(fieldName("col_2"), true), + DropColumn.class, + schemaChange -> { + DropColumn dropColumn = (DropColumn) schemaChange; + assertEquals("col_2", dropColumn.fieldName()); + }); + assertTableChange( + updateComment(UpdateComment.class.getSimpleName()), + UpdateComment.class, + schemaChange -> { + UpdateComment updateComment = (UpdateComment) schemaChange; + assertEquals(UpdateComment.class.getSimpleName(), updateComment.comment()); + }); + assertTableChange( + setProperty("prop_k1", "prop_v1"), + SetOption.class, + schemaChange -> { + SetOption setOption = (SetOption) schemaChange; + assertEquals("prop_k1", setOption.key()); + assertEquals("prop_v1", setOption.value()); + }); + assertTableChange( + removeProperty("prop_k1"), + RemoveOption.class, + schemaChange -> { + RemoveOption removeOption = (RemoveOption) schemaChange; + assertEquals("prop_k1", removeOption.key()); + }); + // Test UnsupportedOperationException with AddIndex, DeleteIndex, RenameTable, + // UpdateColumnAutoIncrement, UpdateColumnDefaultValue. + Arrays.asList( + addIndex(IndexType.UNIQUE_KEY, "uk", new String[][] {{"col_5"}}), + deleteIndex("uk", true), + rename("tb_1"), + updateColumnAutoIncrement(fieldName("col_5"), true), + updateColumnDefaultValue( + fieldName("col_5"), Literals.of("default", Types.VarCharType.of(255)))) + .forEach(this::assertUnsupportedTableChange); + // Test IllegalArgumentException with AddColumn default value and auto increment. + Arrays.asList( + Pair.of( + addColumn( + new String[] {"col_1", "col_6"}, + IntegerType.get(), + AddColumn.class.getSimpleName(), + TableChange.ColumnPosition.first(), + false, + false), + "Paimon does not support update non-primitive type column. Illegal column: col_1.col_6."), + Pair.of( + addColumn( + fieldName("col_1"), + IntegerType.get(), + AddColumn.class.getSimpleName(), + TableChange.ColumnPosition.first(), + false, + false, + Literals.of("default", Types.StringType.get())), + "Paimon does not support column default value. Illegal column: col_1."), + Pair.of( + addColumn( + fieldName("col_1"), + IntegerType.get(), + AddColumn.class.getSimpleName(), + TableChange.ColumnPosition.first(), + false, + true), + "Paimon does not support auto increment column. Illegal column: col_1.")) + .forEach(this::assertIllegalTableChange); + } + + private void assertTableChange( + TableChange tableChange, Class expected, Consumer consumer) { + SchemaChange schemaChange = buildSchemaChange(tableChange); + assertEquals(expected, schemaChange.getClass()); + consumer.accept(schemaChange); + } + + private void assertUnsupportedTableChange(TableChange tableChange) { + UnsupportedOperationException exception = + assertThrowsExactly( + UnsupportedOperationException.class, () -> buildSchemaChange(tableChange)); + assertEquals( + String.format( + "Paimon does not support %s table change.", tableChange.getClass().getSimpleName()), + exception.getMessage()); + } + + private void assertIllegalTableChange(Pair tableChange) { + IllegalArgumentException exception = + assertThrowsExactly( + IllegalArgumentException.class, () -> buildSchemaChange(tableChange.getKey())); + assertEquals(tableChange.getValue(), exception.getMessage()); + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TestTypeUtils.java b/catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TestTypeUtils.java new file mode 100644 index 00000000000..91ec0430585 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/test/java/com/datastrato/gravitino/catalog/lakehouse/paimon/utils/TestTypeUtils.java @@ -0,0 +1,179 @@ +/* + * Copyright 2023 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ + +package com.datastrato.gravitino.catalog.lakehouse.paimon.utils; + +import static com.datastrato.gravitino.catalog.lakehouse.paimon.utils.TypeUtils.fromPaimonType; +import static com.datastrato.gravitino.catalog.lakehouse.paimon.utils.TypeUtils.toPaimonType; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrowsExactly; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import com.datastrato.gravitino.rel.types.Type; +import com.datastrato.gravitino.rel.types.Type.Name; +import com.datastrato.gravitino.rel.types.Types; +import java.util.Arrays; +import java.util.function.Consumer; +import org.apache.paimon.types.BinaryType; +import org.apache.paimon.types.CharType; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.DecimalType; +import org.apache.paimon.types.MapType; +import org.apache.paimon.types.MultisetType; +import org.apache.paimon.types.RowType; +import org.apache.paimon.types.VarBinaryType; +import org.apache.paimon.types.VarCharType; +import org.junit.jupiter.api.Test; + +/** Tests for {@link TypeUtils}. */ +public class TestTypeUtils { + + @Test + void testFromToPaimonType() { + // Test supported data types. + RowType rowType = + RowType.builder() + .fields( + DataTypes.VARCHAR(10), + DataTypes.STRING(), + DataTypes.BOOLEAN(), + DataTypes.BINARY(BinaryType.MAX_LENGTH), + DataTypes.DECIMAL(8, 3), + DataTypes.TINYINT(), + DataTypes.SMALLINT(), + DataTypes.INT(), + DataTypes.BIGINT(), + DataTypes.FLOAT(), + DataTypes.DOUBLE(), + DataTypes.DATE(), + DataTypes.TIME(), + DataTypes.TIMESTAMP(), + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) + .build(); + assertEquals(rowType, toPaimonType(assertDataType(rowType))); + // Test UnparsedType with CharType, VarBinaryType for fromPaimonType. + Arrays.asList( + DataTypes.CHAR(CharType.MAX_LENGTH), DataTypes.VARBINARY(VarBinaryType.MAX_LENGTH)) + .forEach(this::assertDataType); + // Test UnsupportedOperationException with IntervalYearType, IntervalDayType, FixedCharType, + // UUIDType, FixedType, UnionType, NullType for toPaimonType. + Arrays.asList( + Types.IntervalYearType.get(), + Types.IntervalDayType.get(), + Types.FixedCharType.of(10), + Types.UUIDType.get(), + Types.FixedType.of(20), + Types.UnionType.of(Types.IntegerType.get()), + Types.NullType.get(), + Types.UnparsedType.of("unparsed")) + .forEach(this::assertUnsupportedType); + } + + private Type assertDataType(DataType dataType) { + switch (dataType.getTypeRoot()) { + case VARCHAR: + return assertDataType( + dataType, + Name.VARCHAR, + type -> + assertEquals( + ((VarCharType) dataType).getLength(), ((Types.VarCharType) type).length())); + case BOOLEAN: + return assertDataType(dataType, Name.BOOLEAN); + case BINARY: + return assertDataType(dataType, Name.BINARY); + case DECIMAL: + return assertDataType( + dataType, + Name.DECIMAL, + type -> { + assertEquals( + ((DecimalType) dataType).getPrecision(), ((Types.DecimalType) type).precision()); + assertEquals(((DecimalType) dataType).getScale(), ((Types.DecimalType) type).scale()); + }); + case TINYINT: + return assertDataType(dataType, Name.BYTE); + case SMALLINT: + return assertDataType(dataType, Name.SHORT); + case INTEGER: + return assertDataType(dataType, Name.INTEGER); + case BIGINT: + return assertDataType(dataType, Name.LONG); + case FLOAT: + return assertDataType(dataType, Name.FLOAT); + case DOUBLE: + return assertDataType(dataType, Name.DOUBLE); + case DATE: + return assertDataType(dataType, Name.DATE); + case TIME_WITHOUT_TIME_ZONE: + return assertDataType(dataType, Name.TIME); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return assertDataType( + dataType, + Name.TIMESTAMP, + type -> assertFalse(((Types.TimestampType) type).hasTimeZone())); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return assertDataType( + dataType, + Name.TIMESTAMP, + type -> assertTrue(((Types.TimestampType) type).hasTimeZone())); + case ARRAY: + return assertDataType( + dataType, + Name.LIST, + type -> assertTrue(((Types.ListType) type).elementType() instanceof Types.IntegerType)); + case MULTISET: + return assertDataType( + dataType, + Name.MAP, + type -> { + assertEquals( + fromPaimonType(((MultisetType) dataType).getElementType()), + ((Types.MapType) type).keyType()); + assertTrue(((Types.MapType) type).valueType() instanceof Types.IntegerType); + }); + case MAP: + return assertDataType( + dataType, + Name.MAP, + type -> { + assertEquals( + fromPaimonType(((MapType) dataType).getKeyType()), + ((Types.MapType) type).keyType()); + assertEquals( + fromPaimonType(((MapType) dataType).getValueType()), + ((Types.MapType) type).valueType()); + }); + case ROW: + return assertDataType( + dataType, + Name.STRUCT, + type -> ((RowType) dataType).getFieldTypes().forEach(this::assertDataType)); + default: + return assertDataType(dataType, Name.UNPARSED); + } + } + + private Type assertDataType(DataType dataType, Name expected) { + return assertDataType(dataType, expected, type -> {}); + } + + private Type assertDataType(DataType dataType, Name expected, Consumer consumer) { + Type actual = fromPaimonType(dataType); + assertEquals(expected, actual.name()); + consumer.accept(actual); + return actual; + } + + private void assertUnsupportedType(Type type) { + UnsupportedOperationException exception = + assertThrowsExactly(UnsupportedOperationException.class, () -> toPaimonType(type)); + assertEquals( + String.format("Paimon does not support Gravitino %s data type.", type.simpleString()), + exception.getMessage()); + } +} diff --git a/catalogs/catalog-lakehouse-paimon/src/test/resources/hive-schema-3.1.0.derby.sql b/catalogs/catalog-lakehouse-paimon/src/test/resources/hive-schema-3.1.0.derby.sql new file mode 100644 index 00000000000..55097d6639f --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/test/resources/hive-schema-3.1.0.derby.sql @@ -0,0 +1,726 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- +-- This file was copied from Apache Hive, at: +-- https://github.com/apache/hive/blob/master/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-3.1.0.derby.sql +-- +-- This has been modified slightly for compatibility with older Hive versions. +-- +-- Timestamp: 2011-09-22 15:32:02.024 +-- Source database is: /home/carl/Work/repos/hive1/metastore/scripts/upgrade/derby/mdb +-- Connection URL is: jdbc:derby:/home/carl/Work/repos/hive1/metastore/scripts/upgrade/derby/mdb +-- Specified schema is: APP +-- appendLogs: false + +-- ---------------------------------------------- +-- DDL Statements for functions +-- ---------------------------------------------- + +CREATE FUNCTION "APP"."NUCLEUS_ASCII" (C CHAR(1)) RETURNS INTEGER LANGUAGE JAVA PARAMETER STYLE JAVA READS SQL DATA CALLED ON NULL INPUT EXTERNAL NAME 'org.datanucleus.store.rdbms.adapter.DerbySQLFunction.ascii' ; + +CREATE FUNCTION "APP"."NUCLEUS_MATCHES" (TEXT VARCHAR(8000),PATTERN VARCHAR(8000)) RETURNS INTEGER LANGUAGE JAVA PARAMETER STYLE JAVA READS SQL DATA CALLED ON NULL INPUT EXTERNAL NAME 'org.datanucleus.store.rdbms.adapter.DerbySQLFunction.matches' ; + +-- ---------------------------------------------- +-- DDL Statements for tables +-- ---------------------------------------------- +CREATE TABLE "APP"."DBS" ( + "DB_ID" BIGINT NOT NULL, + "DESC" VARCHAR(4000), + "DB_LOCATION_URI" VARCHAR(4000) NOT NULL, + "NAME" VARCHAR(128), + "OWNER_NAME" VARCHAR(128), + "OWNER_TYPE" VARCHAR(10), + "CTLG_NAME" VARCHAR(256) +); + +CREATE TABLE "APP"."TBL_PRIVS" ("TBL_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "TBL_PRIV" VARCHAR(128), "TBL_ID" BIGINT, "AUTHORIZER" VARCHAR(128)); + +CREATE TABLE "APP"."DATABASE_PARAMS" ("DB_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(180) NOT NULL, "PARAM_VALUE" VARCHAR(4000)); + +CREATE TABLE "APP"."TBL_COL_PRIVS" ("TBL_COLUMN_GRANT_ID" BIGINT NOT NULL, "COLUMN_NAME" VARCHAR(767), "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "TBL_COL_PRIV" VARCHAR(128), "TBL_ID" BIGINT, "AUTHORIZER" VARCHAR(128)); + +CREATE TABLE "APP"."SERDE_PARAMS" ("SERDE_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB); + +CREATE TABLE "APP"."COLUMNS_V2" ("CD_ID" BIGINT NOT NULL, "COMMENT" VARCHAR(4000), "COLUMN_NAME" VARCHAR(767) NOT NULL, "TYPE_NAME" CLOB, "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."SORT_COLS" ("SD_ID" BIGINT NOT NULL, "COLUMN_NAME" VARCHAR(767), "ORDER" INTEGER NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."CDS" ("CD_ID" BIGINT NOT NULL); + +CREATE TABLE "APP"."PARTITION_KEY_VALS" ("PART_ID" BIGINT NOT NULL, "PART_KEY_VAL" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."DB_PRIVS" ("DB_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "DB_PRIV" VARCHAR(128), "AUTHORIZER" VARCHAR(128)); + +CREATE TABLE "APP"."IDXS" ("INDEX_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DEFERRED_REBUILD" CHAR(1) NOT NULL, "INDEX_HANDLER_CLASS" VARCHAR(4000), "INDEX_NAME" VARCHAR(128), "INDEX_TBL_ID" BIGINT, "LAST_ACCESS_TIME" INTEGER NOT NULL, "ORIG_TBL_ID" BIGINT, "SD_ID" BIGINT); + +CREATE TABLE "APP"."INDEX_PARAMS" ("INDEX_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" VARCHAR(4000)); + +CREATE TABLE "APP"."PARTITIONS" ("PART_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "LAST_ACCESS_TIME" INTEGER NOT NULL, "PART_NAME" VARCHAR(767), "SD_ID" BIGINT, "TBL_ID" BIGINT); + +CREATE TABLE "APP"."SERDES" ("SERDE_ID" BIGINT NOT NULL, "NAME" VARCHAR(128), "SLIB" VARCHAR(4000), "DESCRIPTION" VARCHAR(4000), "SERIALIZER_CLASS" VARCHAR(4000), "DESERIALIZER_CLASS" VARCHAR(4000), SERDE_TYPE INTEGER); + +CREATE TABLE "APP"."PART_PRIVS" ("PART_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PART_ID" BIGINT, "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "PART_PRIV" VARCHAR(128), "AUTHORIZER" VARCHAR(128)); + +CREATE TABLE "APP"."ROLE_MAP" ("ROLE_GRANT_ID" BIGINT NOT NULL, "ADD_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "ROLE_ID" BIGINT); + +CREATE TABLE "APP"."TYPES" ("TYPES_ID" BIGINT NOT NULL, "TYPE_NAME" VARCHAR(128), "TYPE1" VARCHAR(767), "TYPE2" VARCHAR(767)); + +CREATE TABLE "APP"."GLOBAL_PRIVS" ("USER_GRANT_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "USER_PRIV" VARCHAR(128), "AUTHORIZER" VARCHAR(128)); + +CREATE TABLE "APP"."PARTITION_PARAMS" ("PART_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" VARCHAR(4000)); + +CREATE TABLE "APP"."PARTITION_EVENTS" ( + "PART_NAME_ID" BIGINT NOT NULL, + "CAT_NAME" VARCHAR(256), + "DB_NAME" VARCHAR(128), + "EVENT_TIME" BIGINT NOT NULL, + "EVENT_TYPE" INTEGER NOT NULL, + "PARTITION_NAME" VARCHAR(767), + "TBL_NAME" VARCHAR(256) +); + +CREATE TABLE "APP"."COLUMNS" ("SD_ID" BIGINT NOT NULL, "COMMENT" VARCHAR(256), "COLUMN_NAME" VARCHAR(128) NOT NULL, "TYPE_NAME" VARCHAR(4000) NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."ROLES" ("ROLE_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "OWNER_NAME" VARCHAR(128), "ROLE_NAME" VARCHAR(128)); + +CREATE TABLE "APP"."TBLS" ("TBL_ID" BIGINT NOT NULL, "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "LAST_ACCESS_TIME" INTEGER NOT NULL, "OWNER" VARCHAR(767), "OWNER_TYPE" VARCHAR(10), "RETENTION" INTEGER NOT NULL, "SD_ID" BIGINT, "TBL_NAME" VARCHAR(256), "TBL_TYPE" VARCHAR(128), "VIEW_EXPANDED_TEXT" LONG VARCHAR, "VIEW_ORIGINAL_TEXT" LONG VARCHAR, "IS_REWRITE_ENABLED" CHAR(1) NOT NULL DEFAULT 'N'); + +CREATE TABLE "APP"."PARTITION_KEYS" ("TBL_ID" BIGINT NOT NULL, "PKEY_COMMENT" VARCHAR(4000), "PKEY_NAME" VARCHAR(128) NOT NULL, "PKEY_TYPE" VARCHAR(767) NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."PART_COL_PRIVS" ("PART_COLUMN_GRANT_ID" BIGINT NOT NULL, "COLUMN_NAME" VARCHAR(767), "CREATE_TIME" INTEGER NOT NULL, "GRANT_OPTION" SMALLINT NOT NULL, "GRANTOR" VARCHAR(128), "GRANTOR_TYPE" VARCHAR(128), "PART_ID" BIGINT, "PRINCIPAL_NAME" VARCHAR(128), "PRINCIPAL_TYPE" VARCHAR(128), "PART_COL_PRIV" VARCHAR(128), "AUTHORIZER" VARCHAR(128)); + +CREATE TABLE "APP"."SDS" ("SD_ID" BIGINT NOT NULL, "INPUT_FORMAT" VARCHAR(4000), "IS_COMPRESSED" CHAR(1) NOT NULL, "LOCATION" VARCHAR(4000), "NUM_BUCKETS" INTEGER NOT NULL, "OUTPUT_FORMAT" VARCHAR(4000), "SERDE_ID" BIGINT, "CD_ID" BIGINT, "IS_STOREDASSUBDIRECTORIES" CHAR(1) NOT NULL); + +CREATE TABLE "APP"."SEQUENCE_TABLE" ("SEQUENCE_NAME" VARCHAR(256) NOT NULL, "NEXT_VAL" BIGINT NOT NULL); + +CREATE TABLE "APP"."TAB_COL_STATS"( + "CAT_NAME" VARCHAR(256) NOT NULL, + "DB_NAME" VARCHAR(128) NOT NULL, + "TABLE_NAME" VARCHAR(256) NOT NULL, + "COLUMN_NAME" VARCHAR(767) NOT NULL, + "COLUMN_TYPE" VARCHAR(128) NOT NULL, + "LONG_LOW_VALUE" BIGINT, + "LONG_HIGH_VALUE" BIGINT, + "DOUBLE_LOW_VALUE" DOUBLE, + "DOUBLE_HIGH_VALUE" DOUBLE, + "BIG_DECIMAL_LOW_VALUE" VARCHAR(4000), + "BIG_DECIMAL_HIGH_VALUE" VARCHAR(4000), + "NUM_DISTINCTS" BIGINT, + "NUM_NULLS" BIGINT NOT NULL, + "AVG_COL_LEN" DOUBLE, + "MAX_COL_LEN" BIGINT, + "NUM_TRUES" BIGINT, + "NUM_FALSES" BIGINT, + "LAST_ANALYZED" BIGINT, + "CS_ID" BIGINT NOT NULL, + "TBL_ID" BIGINT NOT NULL, + "BIT_VECTOR" BLOB +); + +CREATE TABLE "APP"."TABLE_PARAMS" ("TBL_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB); + +CREATE TABLE "APP"."BUCKETING_COLS" ("SD_ID" BIGINT NOT NULL, "BUCKET_COL_NAME" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."TYPE_FIELDS" ("TYPE_NAME" BIGINT NOT NULL, "COMMENT" VARCHAR(256), "FIELD_NAME" VARCHAR(128) NOT NULL, "FIELD_TYPE" VARCHAR(767) NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."NUCLEUS_TABLES" ("CLASS_NAME" VARCHAR(128) NOT NULL, "TABLE_NAME" VARCHAR(128) NOT NULL, "TYPE" VARCHAR(4) NOT NULL, "OWNER" VARCHAR(2) NOT NULL, "VERSION" VARCHAR(20) NOT NULL, "INTERFACE_NAME" VARCHAR(256) DEFAULT NULL); + +CREATE TABLE "APP"."SD_PARAMS" ("SD_ID" BIGINT NOT NULL, "PARAM_KEY" VARCHAR(256) NOT NULL, "PARAM_VALUE" CLOB); + +CREATE TABLE "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID" BIGINT NOT NULL); + +CREATE TABLE "APP"."SKEWED_STRING_LIST_VALUES" ("STRING_LIST_ID" BIGINT NOT NULL, "STRING_LIST_VALUE" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."SKEWED_COL_NAMES" ("SD_ID" BIGINT NOT NULL, "SKEWED_COL_NAME" VARCHAR(256), "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ("SD_ID" BIGINT NOT NULL, "STRING_LIST_ID_KID" BIGINT NOT NULL, "LOCATION" VARCHAR(4000)); + +CREATE TABLE "APP"."SKEWED_VALUES" ("SD_ID_OID" BIGINT NOT NULL, "STRING_LIST_ID_EID" BIGINT NOT NULL, "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."MASTER_KEYS" ("KEY_ID" INTEGER NOT NULL generated always as identity (start with 1), "MASTER_KEY" VARCHAR(767)); + +CREATE TABLE "APP"."DELEGATION_TOKENS" ( "TOKEN_IDENT" VARCHAR(767) NOT NULL, "TOKEN" VARCHAR(767)); + +CREATE TABLE "APP"."PART_COL_STATS"( + "CAT_NAME" VARCHAR(256) NOT NULL, + "DB_NAME" VARCHAR(128) NOT NULL, + "TABLE_NAME" VARCHAR(256) NOT NULL, + "PARTITION_NAME" VARCHAR(767) NOT NULL, + "COLUMN_NAME" VARCHAR(767) NOT NULL, + "COLUMN_TYPE" VARCHAR(128) NOT NULL, + "LONG_LOW_VALUE" BIGINT, + "LONG_HIGH_VALUE" BIGINT, + "DOUBLE_LOW_VALUE" DOUBLE, + "DOUBLE_HIGH_VALUE" DOUBLE, + "BIG_DECIMAL_LOW_VALUE" VARCHAR(4000), + "BIG_DECIMAL_HIGH_VALUE" VARCHAR(4000), + "NUM_DISTINCTS" BIGINT, + "BIT_VECTOR" BLOB, + "NUM_NULLS" BIGINT NOT NULL, + "AVG_COL_LEN" DOUBLE, + "MAX_COL_LEN" BIGINT, + "NUM_TRUES" BIGINT, + "NUM_FALSES" BIGINT, + "LAST_ANALYZED" BIGINT, + "CS_ID" BIGINT NOT NULL, + "PART_ID" BIGINT NOT NULL +); + +CREATE TABLE "APP"."VERSION" ("VER_ID" BIGINT NOT NULL, "SCHEMA_VERSION" VARCHAR(127) NOT NULL, "VERSION_COMMENT" VARCHAR(255)); + +CREATE TABLE "APP"."FUNCS" ("FUNC_ID" BIGINT NOT NULL, "CLASS_NAME" VARCHAR(4000), "CREATE_TIME" INTEGER NOT NULL, "DB_ID" BIGINT, "FUNC_NAME" VARCHAR(128), "FUNC_TYPE" INTEGER NOT NULL, "OWNER_NAME" VARCHAR(128), "OWNER_TYPE" VARCHAR(10)); + +CREATE TABLE "APP"."FUNC_RU" ("FUNC_ID" BIGINT NOT NULL, "RESOURCE_TYPE" INTEGER NOT NULL, "RESOURCE_URI" VARCHAR(4000), "INTEGER_IDX" INTEGER NOT NULL); + +CREATE TABLE "APP"."NOTIFICATION_LOG" ( + "NL_ID" BIGINT NOT NULL, + "CAT_NAME" VARCHAR(256), + "DB_NAME" VARCHAR(128), + "EVENT_ID" BIGINT NOT NULL, + "EVENT_TIME" INTEGER NOT NULL, + "EVENT_TYPE" VARCHAR(32) NOT NULL, + "MESSAGE" CLOB, + "TBL_NAME" VARCHAR(256), + "MESSAGE_FORMAT" VARCHAR(16) +); + +CREATE TABLE "APP"."NOTIFICATION_SEQUENCE" ("NNI_ID" BIGINT NOT NULL, "NEXT_EVENT_ID" BIGINT NOT NULL); + +CREATE TABLE "APP"."KEY_CONSTRAINTS" ("CHILD_CD_ID" BIGINT, "CHILD_INTEGER_IDX" INTEGER, "CHILD_TBL_ID" BIGINT, "PARENT_CD_ID" BIGINT , "PARENT_INTEGER_IDX" INTEGER, "PARENT_TBL_ID" BIGINT NOT NULL, "POSITION" BIGINT NOT NULL, "CONSTRAINT_NAME" VARCHAR(400) NOT NULL, "CONSTRAINT_TYPE" SMALLINT NOT NULL, "UPDATE_RULE" SMALLINT, "DELETE_RULE" SMALLINT, "ENABLE_VALIDATE_RELY" SMALLINT NOT NULL, "DEFAULT_VALUE" VARCHAR(400)); + +CREATE TABLE "APP"."METASTORE_DB_PROPERTIES" ("PROPERTY_KEY" VARCHAR(255) NOT NULL, "PROPERTY_VALUE" VARCHAR(1000) NOT NULL, "DESCRIPTION" VARCHAR(1000)); + +CREATE TABLE "APP"."WM_RESOURCEPLAN" (RP_ID BIGINT NOT NULL, NAME VARCHAR(128) NOT NULL, QUERY_PARALLELISM INTEGER, STATUS VARCHAR(20) NOT NULL, DEFAULT_POOL_ID BIGINT); + +CREATE TABLE "APP"."WM_POOL" (POOL_ID BIGINT NOT NULL, RP_ID BIGINT NOT NULL, PATH VARCHAR(1024) NOT NULL, ALLOC_FRACTION DOUBLE, QUERY_PARALLELISM INTEGER, SCHEDULING_POLICY VARCHAR(1024)); + +CREATE TABLE "APP"."WM_TRIGGER" (TRIGGER_ID BIGINT NOT NULL, RP_ID BIGINT NOT NULL, NAME VARCHAR(128) NOT NULL, TRIGGER_EXPRESSION VARCHAR(1024), ACTION_EXPRESSION VARCHAR(1024), IS_IN_UNMANAGED INTEGER NOT NULL DEFAULT 0); + +CREATE TABLE "APP"."WM_POOL_TO_TRIGGER" (POOL_ID BIGINT NOT NULL, TRIGGER_ID BIGINT NOT NULL); + +CREATE TABLE "APP"."WM_MAPPING" (MAPPING_ID BIGINT NOT NULL, RP_ID BIGINT NOT NULL, ENTITY_TYPE VARCHAR(128) NOT NULL, ENTITY_NAME VARCHAR(128) NOT NULL, POOL_ID BIGINT, ORDERING INTEGER); + +CREATE TABLE "APP"."MV_CREATION_METADATA" ( + "MV_CREATION_METADATA_ID" BIGINT NOT NULL, + "CAT_NAME" VARCHAR(256) NOT NULL, + "DB_NAME" VARCHAR(128) NOT NULL, + "TBL_NAME" VARCHAR(256) NOT NULL, + "TXN_LIST" CLOB, + "MATERIALIZATION_TIME" BIGINT NOT NULL +); + +CREATE TABLE "APP"."MV_TABLES_USED" ( + "MV_CREATION_METADATA_ID" BIGINT NOT NULL, + "TBL_ID" BIGINT NOT NULL +); + +CREATE TABLE "APP"."CTLGS" ( + "CTLG_ID" BIGINT NOT NULL, + "NAME" VARCHAR(256) UNIQUE, + "DESC" VARCHAR(4000), + "LOCATION_URI" VARCHAR(4000) NOT NULL); + +-- ---------------------------------------------- +-- DML Statements +-- ---------------------------------------------- + +INSERT INTO "APP"."NOTIFICATION_SEQUENCE" ("NNI_ID", "NEXT_EVENT_ID") SELECT * FROM (VALUES (1,1)) tmp_table WHERE NOT EXISTS ( SELECT "NEXT_EVENT_ID" FROM "APP"."NOTIFICATION_SEQUENCE"); + +INSERT INTO "APP"."SEQUENCE_TABLE" ("SEQUENCE_NAME", "NEXT_VAL") SELECT * FROM (VALUES ('org.apache.hadoop.hive.metastore.model.MNotificationLog', 1)) tmp_table WHERE NOT EXISTS ( SELECT "NEXT_VAL" FROM "APP"."SEQUENCE_TABLE" WHERE "SEQUENCE_NAME" = 'org.apache.hadoop.hive.metastore.model.MNotificationLog'); + +-- ---------------------------------------------- +-- DDL Statements for indexes +-- ---------------------------------------------- + +CREATE UNIQUE INDEX "APP"."UNIQUEINDEX" ON "APP"."IDXS" ("INDEX_NAME", "ORIG_TBL_ID"); + +CREATE INDEX "APP"."TABLECOLUMNPRIVILEGEINDEX" ON "APP"."TBL_COL_PRIVS" ("AUTHORIZER", "TBL_ID", "COLUMN_NAME", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "TBL_COL_PRIV", "GRANTOR", "GRANTOR_TYPE"); + +CREATE UNIQUE INDEX "APP"."DBPRIVILEGEINDEX" ON "APP"."DB_PRIVS" ("AUTHORIZER", "DB_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "DB_PRIV", "GRANTOR", "GRANTOR_TYPE"); + +CREATE INDEX "APP"."PCS_STATS_IDX" ON "APP"."PART_COL_STATS" ("CAT_NAME", "DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME"); + +CREATE INDEX "APP"."TAB_COL_STATS_IDX" ON "APP"."TAB_COL_STATS" ("CAT_NAME", "DB_NAME", "TABLE_NAME", "COLUMN_NAME"); + +CREATE INDEX "APP"."PARTPRIVILEGEINDEX" ON "APP"."PART_PRIVS" ("AUTHORIZER", "PART_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "PART_PRIV", "GRANTOR", "GRANTOR_TYPE"); + +CREATE UNIQUE INDEX "APP"."ROLEENTITYINDEX" ON "APP"."ROLES" ("ROLE_NAME"); + +CREATE INDEX "APP"."TABLEPRIVILEGEINDEX" ON "APP"."TBL_PRIVS" ("AUTHORIZER", "TBL_ID", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "TBL_PRIV", "GRANTOR", "GRANTOR_TYPE"); + +CREATE UNIQUE INDEX "APP"."UNIQUETABLE" ON "APP"."TBLS" ("TBL_NAME", "DB_ID"); + +CREATE UNIQUE INDEX "APP"."UNIQUE_DATABASE" ON "APP"."DBS" ("NAME", "CTLG_NAME"); + +CREATE UNIQUE INDEX "APP"."USERROLEMAPINDEX" ON "APP"."ROLE_MAP" ("PRINCIPAL_NAME", "ROLE_ID", "GRANTOR", "GRANTOR_TYPE"); + +CREATE UNIQUE INDEX "APP"."GLOBALPRIVILEGEINDEX" ON "APP"."GLOBAL_PRIVS" ("AUTHORIZER", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "USER_PRIV", "GRANTOR", "GRANTOR_TYPE"); + +CREATE UNIQUE INDEX "APP"."UNIQUE_TYPE" ON "APP"."TYPES" ("TYPE_NAME"); + +CREATE INDEX "APP"."PARTITIONCOLUMNPRIVILEGEINDEX" ON "APP"."PART_COL_PRIVS" ("AUTHORIZER", "PART_ID", "COLUMN_NAME", "PRINCIPAL_NAME", "PRINCIPAL_TYPE", "PART_COL_PRIV", "GRANTOR", "GRANTOR_TYPE"); + +CREATE UNIQUE INDEX "APP"."UNIQUEPARTITION" ON "APP"."PARTITIONS" ("PART_NAME", "TBL_ID"); + +CREATE UNIQUE INDEX "APP"."UNIQUEFUNCTION" ON "APP"."FUNCS" ("FUNC_NAME", "DB_ID"); + +CREATE INDEX "APP"."FUNCS_N49" ON "APP"."FUNCS" ("DB_ID"); + +CREATE INDEX "APP"."FUNC_RU_N49" ON "APP"."FUNC_RU" ("FUNC_ID"); + +CREATE INDEX "APP"."CONSTRAINTS_PARENT_TBL_ID_INDEX" ON "APP"."KEY_CONSTRAINTS"("PARENT_TBL_ID"); + +CREATE INDEX "APP"."CONSTRAINTS_CONSTRAINT_TYPE_INDEX" ON "APP"."KEY_CONSTRAINTS"("CONSTRAINT_TYPE"); + +CREATE UNIQUE INDEX "APP"."UNIQUE_WM_RESOURCEPLAN" ON "APP"."WM_RESOURCEPLAN" ("NAME"); + +CREATE UNIQUE INDEX "APP"."UNIQUE_WM_POOL" ON "APP"."WM_POOL" ("RP_ID", "PATH"); + +CREATE UNIQUE INDEX "APP"."UNIQUE_WM_TRIGGER" ON "APP"."WM_TRIGGER" ("RP_ID", "NAME"); + +CREATE UNIQUE INDEX "APP"."UNIQUE_WM_MAPPING" ON "APP"."WM_MAPPING" ("RP_ID", "ENTITY_TYPE", "ENTITY_NAME"); + +CREATE UNIQUE INDEX "APP"."MV_UNIQUE_TABLE" ON "APP"."MV_CREATION_METADATA" ("TBL_NAME", "DB_NAME"); + +CREATE UNIQUE INDEX "APP"."UNIQUE_CATALOG" ON "APP"."CTLGS" ("NAME"); + + +-- ---------------------------------------------- +-- DDL Statements for keys +-- ---------------------------------------------- + +-- primary/unique +ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_PK" PRIMARY KEY ("INDEX_ID"); + +ALTER TABLE "APP"."TBL_COL_PRIVS" ADD CONSTRAINT "TBL_COL_PRIVS_PK" PRIMARY KEY ("TBL_COLUMN_GRANT_ID"); + +ALTER TABLE "APP"."CDS" ADD CONSTRAINT "SQL110922153006460" PRIMARY KEY ("CD_ID"); + +ALTER TABLE "APP"."DB_PRIVS" ADD CONSTRAINT "DB_PRIVS_PK" PRIMARY KEY ("DB_GRANT_ID"); + +ALTER TABLE "APP"."INDEX_PARAMS" ADD CONSTRAINT "INDEX_PARAMS_PK" PRIMARY KEY ("INDEX_ID", "PARAM_KEY"); + +ALTER TABLE "APP"."PARTITION_KEYS" ADD CONSTRAINT "PARTITION_KEY_PK" PRIMARY KEY ("TBL_ID", "PKEY_NAME"); + +ALTER TABLE "APP"."SEQUENCE_TABLE" ADD CONSTRAINT "SEQUENCE_TABLE_PK" PRIMARY KEY ("SEQUENCE_NAME"); + +ALTER TABLE "APP"."PART_PRIVS" ADD CONSTRAINT "PART_PRIVS_PK" PRIMARY KEY ("PART_GRANT_ID"); + +ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SDS_PK" PRIMARY KEY ("SD_ID"); + +ALTER TABLE "APP"."SERDES" ADD CONSTRAINT "SERDES_PK" PRIMARY KEY ("SERDE_ID"); + +ALTER TABLE "APP"."COLUMNS" ADD CONSTRAINT "COLUMNS_PK" PRIMARY KEY ("SD_ID", "COLUMN_NAME"); + +ALTER TABLE "APP"."PARTITION_EVENTS" ADD CONSTRAINT "PARTITION_EVENTS_PK" PRIMARY KEY ("PART_NAME_ID"); + +ALTER TABLE "APP"."TYPE_FIELDS" ADD CONSTRAINT "TYPE_FIELDS_PK" PRIMARY KEY ("TYPE_NAME", "FIELD_NAME"); + +ALTER TABLE "APP"."ROLES" ADD CONSTRAINT "ROLES_PK" PRIMARY KEY ("ROLE_ID"); + +ALTER TABLE "APP"."TBL_PRIVS" ADD CONSTRAINT "TBL_PRIVS_PK" PRIMARY KEY ("TBL_GRANT_ID"); + +ALTER TABLE "APP"."SERDE_PARAMS" ADD CONSTRAINT "SERDE_PARAMS_PK" PRIMARY KEY ("SERDE_ID", "PARAM_KEY"); + +ALTER TABLE "APP"."NUCLEUS_TABLES" ADD CONSTRAINT "NUCLEUS_TABLES_PK" PRIMARY KEY ("CLASS_NAME"); + +ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_PK" PRIMARY KEY ("TBL_ID"); + +ALTER TABLE "APP"."SD_PARAMS" ADD CONSTRAINT "SD_PARAMS_PK" PRIMARY KEY ("SD_ID", "PARAM_KEY"); + +ALTER TABLE "APP"."DATABASE_PARAMS" ADD CONSTRAINT "DATABASE_PARAMS_PK" PRIMARY KEY ("DB_ID", "PARAM_KEY"); + +ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_PK" PRIMARY KEY ("DB_ID"); + +ALTER TABLE "APP"."ROLE_MAP" ADD CONSTRAINT "ROLE_MAP_PK" PRIMARY KEY ("ROLE_GRANT_ID"); + +ALTER TABLE "APP"."GLOBAL_PRIVS" ADD CONSTRAINT "GLOBAL_PRIVS_PK" PRIMARY KEY ("USER_GRANT_ID"); + +ALTER TABLE "APP"."BUCKETING_COLS" ADD CONSTRAINT "BUCKETING_COLS_PK" PRIMARY KEY ("SD_ID", "INTEGER_IDX"); + +ALTER TABLE "APP"."SORT_COLS" ADD CONSTRAINT "SORT_COLS_PK" PRIMARY KEY ("SD_ID", "INTEGER_IDX"); + +ALTER TABLE "APP"."PARTITION_KEY_VALS" ADD CONSTRAINT "PARTITION_KEY_VALS_PK" PRIMARY KEY ("PART_ID", "INTEGER_IDX"); + +ALTER TABLE "APP"."TYPES" ADD CONSTRAINT "TYPES_PK" PRIMARY KEY ("TYPES_ID"); + +ALTER TABLE "APP"."COLUMNS_V2" ADD CONSTRAINT "SQL110922153006740" PRIMARY KEY ("CD_ID", "COLUMN_NAME"); + +ALTER TABLE "APP"."PART_COL_PRIVS" ADD CONSTRAINT "PART_COL_PRIVS_PK" PRIMARY KEY ("PART_COLUMN_GRANT_ID"); + +ALTER TABLE "APP"."PARTITION_PARAMS" ADD CONSTRAINT "PARTITION_PARAMS_PK" PRIMARY KEY ("PART_ID", "PARAM_KEY"); + +ALTER TABLE "APP"."PARTITIONS" ADD CONSTRAINT "PARTITIONS_PK" PRIMARY KEY ("PART_ID"); + +ALTER TABLE "APP"."TABLE_PARAMS" ADD CONSTRAINT "TABLE_PARAMS_PK" PRIMARY KEY ("TBL_ID", "PARAM_KEY"); + +ALTER TABLE "APP"."SKEWED_STRING_LIST" ADD CONSTRAINT "SKEWED_STRING_LIST_PK" PRIMARY KEY ("STRING_LIST_ID"); + +ALTER TABLE "APP"."SKEWED_STRING_LIST_VALUES" ADD CONSTRAINT "SKEWED_STRING_LIST_VALUES_PK" PRIMARY KEY ("STRING_LIST_ID", "INTEGER_IDX"); + +ALTER TABLE "APP"."SKEWED_COL_NAMES" ADD CONSTRAINT "SKEWED_COL_NAMES_PK" PRIMARY KEY ("SD_ID", "INTEGER_IDX"); + +ALTER TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_PK" PRIMARY KEY ("SD_ID", "STRING_LIST_ID_KID"); + +ALTER TABLE "APP"."SKEWED_VALUES" ADD CONSTRAINT "SKEWED_VALUES_PK" PRIMARY KEY ("SD_ID_OID", "INTEGER_IDX"); + +ALTER TABLE "APP"."TAB_COL_STATS" ADD CONSTRAINT "TAB_COL_STATS_PK" PRIMARY KEY ("CS_ID"); + +ALTER TABLE "APP"."PART_COL_STATS" ADD CONSTRAINT "PART_COL_STATS_PK" PRIMARY KEY ("CS_ID"); + +ALTER TABLE "APP"."FUNCS" ADD CONSTRAINT "FUNCS_PK" PRIMARY KEY ("FUNC_ID"); + +ALTER TABLE "APP"."FUNC_RU" ADD CONSTRAINT "FUNC_RU_PK" PRIMARY KEY ("FUNC_ID", "INTEGER_IDX"); + +ALTER TABLE "APP"."NOTIFICATION_LOG" ADD CONSTRAINT "NOTIFICATION_LOG_PK" PRIMARY KEY ("NL_ID"); + +ALTER TABLE "APP"."NOTIFICATION_SEQUENCE" ADD CONSTRAINT "NOTIFICATION_SEQUENCE_PK" PRIMARY KEY ("NNI_ID"); + +ALTER TABLE "APP"."KEY_CONSTRAINTS" ADD CONSTRAINT "CONSTRAINTS_PK" PRIMARY KEY ("CONSTRAINT_NAME", "POSITION"); + +ALTER TABLE "APP"."METASTORE_DB_PROPERTIES" ADD CONSTRAINT "PROPERTY_KEY_PK" PRIMARY KEY ("PROPERTY_KEY"); + +ALTER TABLE "APP"."MV_CREATION_METADATA" ADD CONSTRAINT "MV_CREATION_METADATA_PK" PRIMARY KEY ("MV_CREATION_METADATA_ID"); + +ALTER TABLE "APP"."CTLGS" ADD CONSTRAINT "CTLG_PK" PRIMARY KEY ("CTLG_ID"); + + +-- foreign +ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_FK1" FOREIGN KEY ("ORIG_TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_FK2" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "IDXS_FK3" FOREIGN KEY ("INDEX_TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."TBL_COL_PRIVS" ADD CONSTRAINT "TBL_COL_PRIVS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."DB_PRIVS" ADD CONSTRAINT "DB_PRIVS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."INDEX_PARAMS" ADD CONSTRAINT "INDEX_PARAMS_FK1" FOREIGN KEY ("INDEX_ID") REFERENCES "APP"."IDXS" ("INDEX_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."PARTITION_KEYS" ADD CONSTRAINT "PARTITION_KEYS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."PART_PRIVS" ADD CONSTRAINT "PART_PRIVS_FK1" FOREIGN KEY ("PART_ID") REFERENCES "APP"."PARTITIONS" ("PART_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SDS_FK1" FOREIGN KEY ("SERDE_ID") REFERENCES "APP"."SERDES" ("SERDE_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SDS_FK2" FOREIGN KEY ("CD_ID") REFERENCES "APP"."CDS" ("CD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."COLUMNS" ADD CONSTRAINT "COLUMNS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."TYPE_FIELDS" ADD CONSTRAINT "TYPE_FIELDS_FK1" FOREIGN KEY ("TYPE_NAME") REFERENCES "APP"."TYPES" ("TYPES_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."TBL_PRIVS" ADD CONSTRAINT "TBL_PRIVS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SERDE_PARAMS" ADD CONSTRAINT "SERDE_PARAMS_FK1" FOREIGN KEY ("SERDE_ID") REFERENCES "APP"."SERDES" ("SERDE_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_FK2" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."TBLS" ADD CONSTRAINT "TBLS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_FK1" FOREIGN KEY ("CTLG_NAME") REFERENCES "APP"."CTLGS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SD_PARAMS" ADD CONSTRAINT "SD_PARAMS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."DATABASE_PARAMS" ADD CONSTRAINT "DATABASE_PARAMS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."ROLE_MAP" ADD CONSTRAINT "ROLE_MAP_FK1" FOREIGN KEY ("ROLE_ID") REFERENCES "APP"."ROLES" ("ROLE_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."BUCKETING_COLS" ADD CONSTRAINT "BUCKETING_COLS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SORT_COLS" ADD CONSTRAINT "SORT_COLS_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."PARTITION_KEY_VALS" ADD CONSTRAINT "PARTITION_KEY_VALS_FK1" FOREIGN KEY ("PART_ID") REFERENCES "APP"."PARTITIONS" ("PART_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."COLUMNS_V2" ADD CONSTRAINT "COLUMNS_V2_FK1" FOREIGN KEY ("CD_ID") REFERENCES "APP"."CDS" ("CD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."PART_COL_PRIVS" ADD CONSTRAINT "PART_COL_PRIVS_FK1" FOREIGN KEY ("PART_ID") REFERENCES "APP"."PARTITIONS" ("PART_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."PARTITION_PARAMS" ADD CONSTRAINT "PARTITION_PARAMS_FK1" FOREIGN KEY ("PART_ID") REFERENCES "APP"."PARTITIONS" ("PART_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."PARTITIONS" ADD CONSTRAINT "PARTITIONS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."PARTITIONS" ADD CONSTRAINT "PARTITIONS_FK2" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."TABLE_PARAMS" ADD CONSTRAINT "TABLE_PARAMS_FK1" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_STRING_LIST_VALUES" ADD CONSTRAINT "SKEWED_STRING_LIST_VALUES_FK1" FOREIGN KEY ("STRING_LIST_ID") REFERENCES "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_COL_NAMES" ADD CONSTRAINT "SKEWED_COL_NAMES_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_FK1" FOREIGN KEY ("SD_ID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_COL_VALUE_LOC_MAP" ADD CONSTRAINT "SKEWED_COL_VALUE_LOC_MAP_FK2" FOREIGN KEY ("STRING_LIST_ID_KID") REFERENCES "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_VALUES" ADD CONSTRAINT "SKEWED_VALUES_FK1" FOREIGN KEY ("SD_ID_OID") REFERENCES "APP"."SDS" ("SD_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."SKEWED_VALUES" ADD CONSTRAINT "SKEWED_VALUES_FK2" FOREIGN KEY ("STRING_LIST_ID_EID") REFERENCES "APP"."SKEWED_STRING_LIST" ("STRING_LIST_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."TAB_COL_STATS" ADD CONSTRAINT "TAB_COL_STATS_FK" FOREIGN KEY ("TBL_ID") REFERENCES TBLS("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."PART_COL_STATS" ADD CONSTRAINT "PART_COL_STATS_FK" FOREIGN KEY ("PART_ID") REFERENCES PARTITIONS("PART_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."VERSION" ADD CONSTRAINT "VERSION_PK" PRIMARY KEY ("VER_ID"); + +ALTER TABLE "APP"."FUNCS" ADD CONSTRAINT "FUNCS_FK1" FOREIGN KEY ("DB_ID") REFERENCES "APP"."DBS" ("DB_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."FUNC_RU" ADD CONSTRAINT "FUNC_RU_FK1" FOREIGN KEY ("FUNC_ID") REFERENCES "APP"."FUNCS" ("FUNC_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."WM_RESOURCEPLAN" ADD CONSTRAINT "WM_RESOURCEPLAN_PK" PRIMARY KEY ("RP_ID"); + +ALTER TABLE "APP"."WM_POOL" ADD CONSTRAINT "WM_POOL_PK" PRIMARY KEY ("POOL_ID"); + +ALTER TABLE "APP"."WM_POOL" ADD CONSTRAINT "WM_POOL_FK1" FOREIGN KEY ("RP_ID") REFERENCES "APP"."WM_RESOURCEPLAN" ("RP_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."WM_RESOURCEPLAN" ADD CONSTRAINT "WM_RESOURCEPLAN_FK1" FOREIGN KEY ("DEFAULT_POOL_ID") REFERENCES "APP"."WM_POOL" ("POOL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."WM_TRIGGER" ADD CONSTRAINT "WM_TRIGGER_PK" PRIMARY KEY ("TRIGGER_ID"); + +ALTER TABLE "APP"."WM_TRIGGER" ADD CONSTRAINT "WM_TRIGGER_FK1" FOREIGN KEY ("RP_ID") REFERENCES "APP"."WM_RESOURCEPLAN" ("RP_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."WM_POOL_TO_TRIGGER" ADD CONSTRAINT "WM_POOL_TO_TRIGGER_FK1" FOREIGN KEY ("POOL_ID") REFERENCES "APP"."WM_POOL" ("POOL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."WM_POOL_TO_TRIGGER" ADD CONSTRAINT "WM_POOL_TO_TRIGGER_FK2" FOREIGN KEY ("TRIGGER_ID") REFERENCES "APP"."WM_TRIGGER" ("TRIGGER_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."WM_MAPPING" ADD CONSTRAINT "WM_MAPPING_PK" PRIMARY KEY ("MAPPING_ID"); + +ALTER TABLE "APP"."WM_MAPPING" ADD CONSTRAINT "WM_MAPPING_FK1" FOREIGN KEY ("RP_ID") REFERENCES "APP"."WM_RESOURCEPLAN" ("RP_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."WM_MAPPING" ADD CONSTRAINT "WM_MAPPING_FK2" FOREIGN KEY ("POOL_ID") REFERENCES "APP"."WM_POOL" ("POOL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."MV_TABLES_USED" ADD CONSTRAINT "MV_TABLES_USED_FK1" FOREIGN KEY ("MV_CREATION_METADATA_ID") REFERENCES "APP"."MV_CREATION_METADATA" ("MV_CREATION_METADATA_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."MV_TABLES_USED" ADD CONSTRAINT "MV_TABLES_USED_FK2" FOREIGN KEY ("TBL_ID") REFERENCES "APP"."TBLS" ("TBL_ID") ON DELETE NO ACTION ON UPDATE NO ACTION; + +ALTER TABLE "APP"."DBS" ADD CONSTRAINT "DBS_CTLG_FK" FOREIGN KEY ("CTLG_NAME") REFERENCES "APP"."CTLGS" ("NAME") ON DELETE NO ACTION ON UPDATE NO ACTION; + +-- ---------------------------------------------- +-- DDL Statements for checks +-- ---------------------------------------------- + +ALTER TABLE "APP"."IDXS" ADD CONSTRAINT "SQL110318025504980" CHECK (DEFERRED_REBUILD IN ('Y','N')); + +ALTER TABLE "APP"."SDS" ADD CONSTRAINT "SQL110318025505550" CHECK (IS_COMPRESSED IN ('Y','N')); + +-- ---------------------------- +-- Transaction and Lock Tables +-- ---------------------------- +CREATE TABLE TXNS ( + TXN_ID bigint PRIMARY KEY, + TXN_STATE char(1) NOT NULL, + TXN_STARTED bigint NOT NULL, + TXN_LAST_HEARTBEAT bigint NOT NULL, + TXN_USER varchar(128) NOT NULL, + TXN_HOST varchar(128) NOT NULL, + TXN_AGENT_INFO varchar(128), + TXN_META_INFO varchar(128), + TXN_HEARTBEAT_COUNT integer, + TXN_TYPE integer +); + +CREATE TABLE TXN_COMPONENTS ( + TC_TXNID bigint NOT NULL REFERENCES TXNS (TXN_ID), + TC_DATABASE varchar(128) NOT NULL, + TC_TABLE varchar(128), + TC_PARTITION varchar(767), + TC_OPERATION_TYPE char(1) NOT NULL, + TC_WRITEID bigint +); + +CREATE INDEX TC_TXNID_INDEX ON TXN_COMPONENTS (TC_TXNID); + +CREATE TABLE COMPLETED_TXN_COMPONENTS ( + CTC_TXNID bigint NOT NULL, + CTC_DATABASE varchar(128) NOT NULL, + CTC_TABLE varchar(256), + CTC_PARTITION varchar(767), + CTC_TIMESTAMP timestamp DEFAULT CURRENT_TIMESTAMP NOT NULL, + CTC_WRITEID bigint, + CTC_UPDATE_DELETE char(1) NOT NULL +); + +CREATE INDEX COMPLETED_TXN_COMPONENTS_IDX ON COMPLETED_TXN_COMPONENTS (CTC_DATABASE, CTC_TABLE, CTC_PARTITION); + +CREATE TABLE NEXT_TXN_ID ( + NTXN_NEXT bigint NOT NULL +); +INSERT INTO NEXT_TXN_ID VALUES(1); + +CREATE TABLE HIVE_LOCKS ( + HL_LOCK_EXT_ID bigint NOT NULL, + HL_LOCK_INT_ID bigint NOT NULL, + HL_TXNID bigint NOT NULL, + HL_DB varchar(128) NOT NULL, + HL_TABLE varchar(128), + HL_PARTITION varchar(767), + HL_LOCK_STATE char(1) NOT NULL, + HL_LOCK_TYPE char(1) NOT NULL, + HL_LAST_HEARTBEAT bigint NOT NULL, + HL_ACQUIRED_AT bigint, + HL_USER varchar(128) NOT NULL, + HL_HOST varchar(128) NOT NULL, + HL_HEARTBEAT_COUNT integer, + HL_AGENT_INFO varchar(128), + HL_BLOCKEDBY_EXT_ID bigint, + HL_BLOCKEDBY_INT_ID bigint, + PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID) +); + +CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS (HL_TXNID); + +CREATE TABLE NEXT_LOCK_ID ( + NL_NEXT bigint NOT NULL +); +INSERT INTO NEXT_LOCK_ID VALUES(1); + +CREATE TABLE COMPACTION_QUEUE ( + CQ_ID bigint PRIMARY KEY, + CQ_DATABASE varchar(128) NOT NULL, + CQ_TABLE varchar(128) NOT NULL, + CQ_PARTITION varchar(767), + CQ_STATE char(1) NOT NULL, + CQ_TYPE char(1) NOT NULL, + CQ_TBLPROPERTIES varchar(2048), + CQ_WORKER_ID varchar(128), + CQ_START bigint, + CQ_RUN_AS varchar(128), + CQ_HIGHEST_WRITE_ID bigint, + CQ_META_INFO varchar(2048) for bit data, + CQ_HADOOP_JOB_ID varchar(32) +); + +CREATE TABLE NEXT_COMPACTION_QUEUE_ID ( + NCQ_NEXT bigint NOT NULL +); +INSERT INTO NEXT_COMPACTION_QUEUE_ID VALUES(1); + +CREATE TABLE COMPLETED_COMPACTIONS ( + CC_ID bigint PRIMARY KEY, + CC_DATABASE varchar(128) NOT NULL, + CC_TABLE varchar(128) NOT NULL, + CC_PARTITION varchar(767), + CC_STATE char(1) NOT NULL, + CC_TYPE char(1) NOT NULL, + CC_TBLPROPERTIES varchar(2048), + CC_WORKER_ID varchar(128), + CC_START bigint, + CC_END bigint, + CC_RUN_AS varchar(128), + CC_HIGHEST_WRITE_ID bigint, + CC_META_INFO varchar(2048) for bit data, + CC_HADOOP_JOB_ID varchar(32) +); + +CREATE TABLE AUX_TABLE ( + MT_KEY1 varchar(128) NOT NULL, + MT_KEY2 bigint NOT NULL, + MT_COMMENT varchar(255), + PRIMARY KEY(MT_KEY1, MT_KEY2) +); + +--1st 4 cols make up a PK but since WS_PARTITION is nullable we can't declare such PK +--This is a good candidate for Index orgainzed table +CREATE TABLE WRITE_SET ( + WS_DATABASE varchar(128) NOT NULL, + WS_TABLE varchar(128) NOT NULL, + WS_PARTITION varchar(767), + WS_TXNID bigint NOT NULL, + WS_COMMIT_ID bigint NOT NULL, + WS_OPERATION_TYPE char(1) NOT NULL +); + +CREATE TABLE TXN_TO_WRITE_ID ( + T2W_TXNID bigint NOT NULL, + T2W_DATABASE varchar(128) NOT NULL, + T2W_TABLE varchar(256) NOT NULL, + T2W_WRITEID bigint NOT NULL +); + +CREATE UNIQUE INDEX TBL_TO_TXN_ID_IDX ON TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_TXNID); +CREATE UNIQUE INDEX TBL_TO_WRITE_ID_IDX ON TXN_TO_WRITE_ID (T2W_DATABASE, T2W_TABLE, T2W_WRITEID); + +CREATE TABLE NEXT_WRITE_ID ( + NWI_DATABASE varchar(128) NOT NULL, + NWI_TABLE varchar(256) NOT NULL, + NWI_NEXT bigint NOT NULL +); + +CREATE UNIQUE INDEX NEXT_WRITE_ID_IDX ON NEXT_WRITE_ID (NWI_DATABASE, NWI_TABLE); + +CREATE TABLE MIN_HISTORY_LEVEL ( + MHL_TXNID bigint NOT NULL, + MHL_MIN_OPEN_TXNID bigint NOT NULL, + PRIMARY KEY(MHL_TXNID) +); + +CREATE INDEX MIN_HISTORY_LEVEL_IDX ON MIN_HISTORY_LEVEL (MHL_MIN_OPEN_TXNID); + +CREATE TABLE MATERIALIZATION_REBUILD_LOCKS ( + MRL_TXN_ID BIGINT NOT NULL, + MRL_DB_NAME VARCHAR(128) NOT NULL, + MRL_TBL_NAME VARCHAR(256) NOT NULL, + MRL_LAST_HEARTBEAT BIGINT NOT NULL, + PRIMARY KEY(MRL_TXN_ID) +); + +CREATE TABLE "APP"."I_SCHEMA" ( + "SCHEMA_ID" bigint primary key, + "SCHEMA_TYPE" integer not null, + "NAME" varchar(256) unique, + "DB_ID" bigint references "APP"."DBS" ("DB_ID"), + "COMPATIBILITY" integer not null, + "VALIDATION_LEVEL" integer not null, + "CAN_EVOLVE" char(1) not null, + "SCHEMA_GROUP" varchar(256), + "DESCRIPTION" varchar(4000) +); + +CREATE TABLE "APP"."SCHEMA_VERSION" ( + "SCHEMA_VERSION_ID" bigint primary key, + "SCHEMA_ID" bigint references "APP"."I_SCHEMA" ("SCHEMA_ID"), + "VERSION" integer not null, + "CREATED_AT" bigint not null, + "CD_ID" bigint references "APP"."CDS" ("CD_ID"), + "STATE" integer not null, + "DESCRIPTION" varchar(4000), + "SCHEMA_TEXT" clob, + "FINGERPRINT" varchar(256), + "SCHEMA_VERSION_NAME" varchar(256), + "SERDE_ID" bigint references "APP"."SERDES" ("SERDE_ID") +); + +CREATE UNIQUE INDEX "APP"."UNIQUE_SCHEMA_VERSION" ON "APP"."SCHEMA_VERSION" ("SCHEMA_ID", "VERSION"); + +CREATE TABLE REPL_TXN_MAP ( + RTM_REPL_POLICY varchar(256) NOT NULL, + RTM_SRC_TXN_ID bigint NOT NULL, + RTM_TARGET_TXN_ID bigint NOT NULL, + PRIMARY KEY (RTM_REPL_POLICY, RTM_SRC_TXN_ID) +); + +CREATE TABLE "APP"."RUNTIME_STATS" ( + "RS_ID" bigint primary key, + "CREATE_TIME" integer not null, + "WEIGHT" integer not null, + "PAYLOAD" BLOB +); + +CREATE INDEX IDX_RUNTIME_STATS_CREATE_TIME ON RUNTIME_STATS(CREATE_TIME); + +-- ----------------------------------------------------------------- +-- Record schema version. Should be the last step in the init script +-- ----------------------------------------------------------------- +INSERT INTO "APP"."VERSION" (VER_ID, SCHEMA_VERSION, VERSION_COMMENT) VALUES (1, '3.1.0', 'Hive release version 3.1.0'); diff --git a/catalogs/catalog-lakehouse-paimon/src/test/resources/log4j2.properties b/catalogs/catalog-lakehouse-paimon/src/test/resources/log4j2.properties new file mode 100644 index 00000000000..67cbe8f91d2 --- /dev/null +++ b/catalogs/catalog-lakehouse-paimon/src/test/resources/log4j2.properties @@ -0,0 +1,33 @@ +# +# Copyright 2024 Datastrato Pvt Ltd. +# This software is licensed under the Apache License version 2. +# + +# Set to debug or trace if log4j initialization is failing +status = info + +# Name of the configuration +name = ConsoleLogConfig + +# Console appender configuration +appender.console.type = Console +appender.console.name = consoleLogger +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n + +# Log files location +property.logPath = ${sys:gravitino.log.path:-catalog-lakehouse-paimon/build/paimon-integration-test.log} + +# File appender configuration +appender.file.type = File +appender.file.name = fileLogger +appender.file.fileName = ${logPath} +appender.file.layout.type = PatternLayout +appender.file.layout.pattern = %d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n + +# Root logger level +rootLogger.level = info + +# Root logger referring to console and file appenders +rootLogger.appenderRef.stdout.ref = consoleLogger +rootLogger.appenderRef.file.ref = fileLogger diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index e5dbf1a46fe..e8b2c7ea23b 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,6 +28,7 @@ commons-dbcp2 = "2.11.0" caffeine = "2.9.3" rocksdbjni = "7.10.2" iceberg = '1.3.1' # 1.4.0 causes test to fail +paimon = '0.7.0-incubating' trino = '426' spark = "3.4.1" # 3.5.0 causes tests to fail scala-collection-compat = "2.7.0" @@ -98,6 +99,7 @@ jersey-hk2 = { group = "org.glassfish.jersey.inject", name = "jersey-hk2", versi jersey-test-framework-core = { group = "org.glassfish.jersey.test-framework", name = "jersey-test-framework-core", version.ref = "jersey" } jersey-test-framework-provider-jetty = { group = "org.glassfish.jersey.test-framework.providers", name = "jersey-test-framework-provider-jetty", version.ref = "jersey" } mockito-core = { group = "org.mockito", name = "mockito-core", version.ref = "mockito" } +mockito-inline = { group = "org.mockito", name = "mockito-inline", version.ref = "mockito" } hive2-metastore = { group = "org.apache.hive", name = "hive-metastore", version.ref = "hive2"} hive2-exec = { group = "org.apache.hive", name = "hive-exec", version.ref = "hive2"} hive2-common = { group = "org.apache.hive", name = "hive-common", version.ref = "hive2"} @@ -121,6 +123,9 @@ commons-collections4 = { group = "org.apache.commons", name = "commons-collectio iceberg-core = { group = "org.apache.iceberg", name = "iceberg-core", version.ref = "iceberg" } iceberg-api = { group = "org.apache.iceberg", name = "iceberg-api", version.ref = "iceberg" } iceberg-hive-metastore = { group = "org.apache.iceberg", name = "iceberg-hive-metastore", version.ref = "iceberg" } +paimon-core = { group = "org.apache.paimon", name = "paimon-core", version.ref = "paimon" } +paimon-format = { group = "org.apache.paimon", name = "paimon-format", version.ref = "paimon" } +paimon-hive-catalog = { group = "org.apache.paimon", name = "paimon-hive-catalog", version.ref = "paimon" } trino-spi= { group = "io.trino", name = "trino-spi", version.ref = "trino" } trino-toolkit= { group = "io.trino", name = "trino-plugin-toolkit", version.ref = "trino" } trino-testing= { group = "io.trino", name = "trino-testing", version.ref = "trino" } @@ -170,6 +175,7 @@ log4j = ["slf4j-api", "log4j-slf4j2-impl", "log4j-api", "log4j-core", "log4j-12- jetty = ["jetty-server", "jetty-servlet", "jetty-webapp", "jetty-servlets"] jersey = ["jersey-server", "jersey-container-servlet-core", "jersey-container-jetty-http", "jersey-media-json-jackson", "jersey-hk2"] iceberg = ["iceberg-core", "iceberg-api"] +paimon = ["paimon-core", "paimon-format", "paimon-hive-catalog"] jwt = ["jwt-api", "jwt-impl", "jwt-gson"] metrics = ["metrics-core", "metrics-jersey2", "metrics-jvm", "metrics-jmx", "metrics-servlets"] prometheus = ["prometheus-servlet", "prometheus-dropwizard", "prometheus-client"] diff --git a/settings.gradle.kts b/settings.gradle.kts index be21feb8790..dd5a64fbcb8 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -12,6 +12,7 @@ include("api", "common", "core", "meta", "server", "integration-test", "server-c include("catalogs:bundled-catalog") include("catalogs:catalog-hive") include("catalogs:catalog-lakehouse-iceberg") +include("catalogs:catalog-lakehouse-paimon") include( "catalogs:catalog-jdbc-common", "catalogs:catalog-jdbc-doris",