From 3f516c97bf4584c52f70405be98c1fa2480f8595 Mon Sep 17 00:00:00 2001 From: Qi Yu Date: Fri, 29 Mar 2024 14:28:56 +0800 Subject: [PATCH 1/4] [#2725] Fix(CI): fix CI pipelines do not work as expected (#2723) ### What changes were proposed in this pull request? Fix the pipeline concurrency configuration and change the cancel conditions. ### Why are the changes needed? It's a bug to be fixed. Fix: #2725 ### Does this PR introduce _any_ user-facing change? N/A. ### How was this patch tested? Test manually. --- .github/workflows/backend-integration-test.yml | 4 ++-- .github/workflows/build.yml | 4 ++-- .github/workflows/frontend-integration-test.yml | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/backend-integration-test.yml b/.github/workflows/backend-integration-test.yml index 669bd94643b..69392503485 100644 --- a/.github/workflows/backend-integration-test.yml +++ b/.github/workflows/backend-integration-test.yml @@ -9,8 +9,8 @@ on: branches: [ "main", "branch-*" ] concurrency: - group: ${{ github.worklfow }}-${{ github.event.pull_request.number || github.ref }} - cancel-in-progress: ${{ github.event_name == 'pull_requests' }} + group: ${{ github.workflow }}-${{ github.event.pull_request.number || github.ref }} + cancel-in-progress: true jobs: changes: diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index 9e262a1426c..71facb817f6 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -9,8 +9,8 @@ on: branches: [ "main", "branch-*" ] concurrency: - group: ${{ github.worklfow }}-${{ github.event.pull_request.number || github.ref }} - cancel-in-progress: ${{ github.event_name == 'pull_requests' }} + group: ${{ github.workflow }}-${{ github.event.pull_request.number || github.ref }} + cancel-in-progress: true # A workflow run is made up of one or more jobs that can run sequentially or in parallel jobs: diff --git a/.github/workflows/frontend-integration-test.yml b/.github/workflows/frontend-integration-test.yml index de2062910fd..f05174d33b9 100644 --- a/.github/workflows/frontend-integration-test.yml +++ b/.github/workflows/frontend-integration-test.yml @@ -9,8 +9,8 @@ on: branches: [ "main", "branch-*" ] concurrency: - group: ${{ github.worklfow }}-${{ github.event.pull_request.number || github.ref }} - cancel-in-progress: ${{ github.event_name == 'pull_requests' }} + group: ${{ github.workflow }}-${{ github.event.pull_request.number || github.ref }} + cancel-in-progress: true jobs: changes: From d89c833ea81f5b989f4708ff83d0f5f533eeba23 Mon Sep 17 00:00:00 2001 From: mchades Date: Fri, 29 Mar 2024 16:50:16 +0800 Subject: [PATCH 2/4] [#2468] feat(kafka-catalog): support topic operations for Kafka catalog (#2615) ### What changes were proposed in this pull request? This PR proposes to add the topic operations support for the Kafka catalog. ### Why are the changes needed? This is a part of the work to support messaging management in Gravitino Fix: #2468 ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? Add UTs to cover the codes. --- .../catalog-messaging-kafka/build.gradle.kts | 6 +- .../catalog/kafka/KafkaCatalogOperations.java | 321 +++++++++++++++++- .../gravitino/catalog/kafka/KafkaTopic.java | 42 +++ .../kafka/KafkaTopicPropertiesMetadata.java | 33 +- .../kafka/TestKafkaCatalogOperations.java | 250 +++++++++++++- .../embeddedKafka/KafkaClusterEmbedded.java | 95 ++++++ .../kafka/embeddedKafka/KafkaEmbedded.java | 109 ++++++ .../embeddedKafka/ZooKeeperEmbedded.java | 45 +++ .../gravitino/connector/BaseTopic.java | 156 +++++++++ .../gravitino/connector/PropertyEntry.java | 27 ++ gradle/libs.versions.toml | 5 + 11 files changed, 1063 insertions(+), 26 deletions(-) create mode 100644 catalogs/catalog-messaging-kafka/src/main/java/com/datastrato/gravitino/catalog/kafka/KafkaTopic.java create mode 100644 catalogs/catalog-messaging-kafka/src/test/java/com/datastrato/gravitino/catalog/kafka/embeddedKafka/KafkaClusterEmbedded.java create mode 100644 catalogs/catalog-messaging-kafka/src/test/java/com/datastrato/gravitino/catalog/kafka/embeddedKafka/KafkaEmbedded.java create mode 100644 catalogs/catalog-messaging-kafka/src/test/java/com/datastrato/gravitino/catalog/kafka/embeddedKafka/ZooKeeperEmbedded.java create mode 100644 core/src/main/java/com/datastrato/gravitino/connector/BaseTopic.java diff --git a/catalogs/catalog-messaging-kafka/build.gradle.kts b/catalogs/catalog-messaging-kafka/build.gradle.kts index a992063ee09..0757fec6bac 100644 --- a/catalogs/catalog-messaging-kafka/build.gradle.kts +++ b/catalogs/catalog-messaging-kafka/build.gradle.kts @@ -16,10 +16,14 @@ dependencies { implementation(project(":common")) implementation(libs.guava) + implementation(libs.kafka.clients) + implementation(libs.slf4j.api) + testImplementation(libs.commons.io) + testImplementation(libs.curator.test) testImplementation(libs.junit.jupiter.api) + testImplementation(libs.kafka) testImplementation(libs.mockito.core) - testImplementation(libs.commons.io) testRuntimeOnly(libs.junit.jupiter.engine) } diff --git a/catalogs/catalog-messaging-kafka/src/main/java/com/datastrato/gravitino/catalog/kafka/KafkaCatalogOperations.java b/catalogs/catalog-messaging-kafka/src/main/java/com/datastrato/gravitino/catalog/kafka/KafkaCatalogOperations.java index 39ec5eb108c..24d36598796 100644 --- a/catalogs/catalog-messaging-kafka/src/main/java/com/datastrato/gravitino/catalog/kafka/KafkaCatalogOperations.java +++ b/catalogs/catalog-messaging-kafka/src/main/java/com/datastrato/gravitino/catalog/kafka/KafkaCatalogOperations.java @@ -6,6 +6,8 @@ import static com.datastrato.gravitino.StringIdentifier.ID_KEY; import static com.datastrato.gravitino.catalog.kafka.KafkaCatalogPropertiesMetadata.BOOTSTRAP_SERVERS; +import static com.datastrato.gravitino.catalog.kafka.KafkaTopicPropertiesMetadata.PARTITION_COUNT; +import static com.datastrato.gravitino.catalog.kafka.KafkaTopicPropertiesMetadata.REPLICATION_FACTOR; import static com.datastrato.gravitino.connector.BaseCatalog.CATALOG_BYPASS_PREFIX; import com.datastrato.gravitino.Entity; @@ -35,31 +37,60 @@ import com.datastrato.gravitino.rel.SchemaChange; import com.datastrato.gravitino.rel.SupportsSchemas; import com.datastrato.gravitino.storage.IdGenerator; +import com.datastrato.gravitino.utils.PrincipalUtils; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import java.io.IOException; import java.time.Instant; +import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.AlterConfigOp; +import org.apache.kafka.clients.admin.Config; +import org.apache.kafka.clients.admin.ConfigEntry; +import org.apache.kafka.clients.admin.CreateTopicsResult; +import org.apache.kafka.clients.admin.DescribeConfigsResult; +import org.apache.kafka.clients.admin.DescribeTopicsResult; +import org.apache.kafka.clients.admin.ListTopicsResult; +import org.apache.kafka.clients.admin.NewPartitions; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.errors.InvalidConfigurationException; +import org.apache.kafka.common.errors.InvalidReplicationFactorException; +import org.apache.kafka.common.errors.TopicExistsException; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class KafkaCatalogOperations implements CatalogOperations, SupportsSchemas, TopicCatalog { + private static final Logger LOG = LoggerFactory.getLogger(KafkaCatalogOperations.class); private static final KafkaCatalogPropertiesMetadata CATALOG_PROPERTIES_METADATA = new KafkaCatalogPropertiesMetadata(); private static final KafkaSchemaPropertiesMetadata SCHEMA_PROPERTIES_METADATA = new KafkaSchemaPropertiesMetadata(); private static final KafkaTopicPropertiesMetadata TOPIC_PROPERTIES_METADATA = new KafkaTopicPropertiesMetadata(); + private static final String DEFAULT_SCHEMA_NAME = "default"; + @VisibleForTesting static final String CLIENT_ID_TEMPLATE = "%s-%s.%s"; private final EntityStore store; private final IdGenerator idGenerator; - private final String DEFAULT_SCHEMA_NAME = "default"; @VisibleForTesting NameIdentifier defaultSchemaIdent; @VisibleForTesting Properties adminClientConfig; private CatalogInfo info; + private AdminClient adminClient; @VisibleForTesting KafkaCatalogOperations(EntityStore store, IdGenerator idGenerator) { @@ -92,39 +123,196 @@ public void initialize(Map config, CatalogInfo info) throws Runt e -> e.getKey().substring(CATALOG_BYPASS_PREFIX.length()), Map.Entry::getValue)); adminClientConfig.putAll(bypassConfigs); - adminClientConfig.put(BOOTSTRAP_SERVERS, config.get(BOOTSTRAP_SERVERS)); + adminClientConfig.put( + AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, config.get(BOOTSTRAP_SERVERS)); // use gravitino catalog id as the admin client id - adminClientConfig.put("client.id", config.get(ID_KEY)); + adminClientConfig.put( + AdminClientConfig.CLIENT_ID_CONFIG, + String.format(CLIENT_ID_TEMPLATE, config.get(ID_KEY), info.namespace(), info.name())); createDefaultSchema(); + adminClient = AdminClient.create(adminClientConfig); } @Override public NameIdentifier[] listTopics(Namespace namespace) throws NoSuchSchemaException { - throw new UnsupportedOperationException(); + NameIdentifier schemaIdent = NameIdentifier.of(namespace.levels()); + checkSchemaExists(schemaIdent); + + try { + ListTopicsResult result = adminClient.listTopics(); + Set topicNames = result.names().get(); + return topicNames.stream() + .map(name -> NameIdentifier.of(namespace, name)) + .toArray(NameIdentifier[]::new); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Failed to list topics under the schema " + namespace, e); + } } @Override public Topic loadTopic(NameIdentifier ident) throws NoSuchTopicException { - throw new UnsupportedOperationException(); + NameIdentifier schemaIdent = NameIdentifier.of(ident.namespace().levels()); + checkSchemaExists(schemaIdent); + + DescribeTopicsResult result = adminClient.describeTopics(Collections.singleton(ident.name())); + ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, ident.name()); + DescribeConfigsResult configsResult = + adminClient.describeConfigs(Collections.singleton(configResource)); + int partitions; + int replicationFactor; + Map properties = Maps.newHashMap(); + try { + TopicDescription topicDescription = result.topicNameValues().get(ident.name()).get(); + partitions = topicDescription.partitions().size(); + replicationFactor = topicDescription.partitions().get(0).replicas().size(); + + Config topicConfigs = configsResult.all().get().get(configResource); + topicConfigs.entries().forEach(e -> properties.put(e.name(), e.value())); + properties.put(PARTITION_COUNT, String.valueOf(partitions)); + properties.put(REPLICATION_FACTOR, String.valueOf(replicationFactor)); + } catch (ExecutionException e) { + if (e.getCause() instanceof UnknownTopicOrPartitionException) { + throw new NoSuchTopicException(e, "Topic %s does not exist", ident); + } else { + throw new RuntimeException("Failed to load topic " + ident.name() + " from Kafka", e); + } + } catch (InterruptedException e) { + throw new RuntimeException("Failed to load topic " + ident.name() + " from Kafka", e); + } + + LOG.info("Loaded topic {} from Kafka", ident); + + return KafkaTopic.builder() + .withName(ident.name()) + .withProperties(properties) + .withAuditInfo( + AuditInfo.builder() + .withCreator(PrincipalUtils.getCurrentPrincipal().getName()) + .withCreateTime(Instant.now()) + .build()) + .build(); } @Override public Topic createTopic( NameIdentifier ident, String comment, DataLayout dataLayout, Map properties) throws NoSuchSchemaException, TopicAlreadyExistsException { - throw new UnsupportedOperationException(); + NameIdentifier schemaIdent = NameIdentifier.of(ident.namespace().levels()); + checkSchemaExists(schemaIdent); + + try { + CreateTopicsResult createTopicsResult = + adminClient.createTopics(Collections.singleton(buildNewTopic(ident, properties))); + LOG.info( + "Created topic {} with {} partitions and replication factor {}", + ident, + createTopicsResult.numPartitions(ident.name()).get(), + createTopicsResult.replicationFactor(ident.name()).get()); + } catch (ExecutionException e) { + if (e.getCause() instanceof TopicExistsException) { + throw new TopicAlreadyExistsException(e, "Topic %s already exists", ident); + + } else if (e.getCause() instanceof InvalidReplicationFactorException) { + throw new IllegalArgumentException( + "Invalid replication factor for topic " + ident + e.getCause().getMessage(), e); + + } else if (e.getCause() instanceof InvalidConfigurationException) { + throw new IllegalArgumentException( + "Invalid properties for topic " + ident + e.getCause().getMessage(), e); + + } else { + throw new RuntimeException("Failed to create topic in Kafka" + ident, e); + } + } catch (InterruptedException e) { + throw new RuntimeException("Failed to create topic in Kafka" + ident, e); + } + + return KafkaTopic.builder() + .withName(ident.name()) + .withComment(comment) + .withProperties(properties) + .withAuditInfo( + AuditInfo.builder() + .withCreator(PrincipalUtils.getCurrentPrincipal().getName()) + .withCreateTime(Instant.now()) + .build()) + .build(); } @Override public Topic alterTopic(NameIdentifier ident, TopicChange... changes) throws NoSuchTopicException, IllegalArgumentException { - throw new UnsupportedOperationException(); + NameIdentifier schemaIdent = NameIdentifier.of(ident.namespace().levels()); + checkSchemaExists(schemaIdent); + + KafkaTopic topic = (KafkaTopic) loadTopic(ident); + String newComment = topic.comment(); + int oldPartitionCount = Integer.parseInt(topic.properties().get(PARTITION_COUNT)); + int newPartitionCount = oldPartitionCount; + Map alteredProperties = Maps.newHashMap(topic.properties()); + List alterConfigOps = Lists.newArrayList(); + for (TopicChange change : changes) { + if (change instanceof TopicChange.UpdateTopicComment) { + newComment = ((TopicChange.UpdateTopicComment) change).getNewComment(); + + } else if (change instanceof TopicChange.SetProperty) { + TopicChange.SetProperty setProperty = (TopicChange.SetProperty) change; + if (PARTITION_COUNT.equals(setProperty.getProperty())) { + // alter partition count + newPartitionCount = setPartitionCount(setProperty, newPartitionCount, alteredProperties); + } else { + // alter other properties + setProperty(setProperty, alteredProperties, alterConfigOps); + } + + } else if (change instanceof TopicChange.RemoveProperty) { + removeProperty((TopicChange.RemoveProperty) change, alteredProperties, alterConfigOps); + + } else { + throw new IllegalArgumentException("Unsupported topic change: " + change); + } + } + + if (newPartitionCount != oldPartitionCount) { + doPartitionCountIncrement(ident.name(), newPartitionCount); + } + + if (!alterConfigOps.isEmpty()) { + doAlterTopicConfig(ident.name(), alterConfigOps); + } + + return KafkaTopic.builder() + .withName(ident.name()) + .withComment(newComment) + .withProperties(alteredProperties) + .withAuditInfo( + AuditInfo.builder() + .withCreator(topic.auditInfo().creator()) + .withCreateTime(topic.auditInfo().createTime()) + .withLastModifier(PrincipalUtils.getCurrentPrincipal().getName()) + .withLastModifiedTime(Instant.now()) + .build()) + .build(); } @Override public boolean dropTopic(NameIdentifier ident) throws NoSuchTopicException { - throw new UnsupportedOperationException(); + NameIdentifier schemaIdent = NameIdentifier.of(ident.namespace().levels()); + checkSchemaExists(schemaIdent); + + try { + adminClient.deleteTopics(Collections.singleton(ident.name())).all().get(); + return true; + } catch (ExecutionException e) { + if (e.getCause() instanceof UnknownTopicOrPartitionException) { + return false; + } else { + throw new RuntimeException("Failed to drop topic " + ident.name() + " from Kafka", e); + } + } catch (InterruptedException e) { + throw new RuntimeException("Failed to drop topic " + ident.name() + " from Kafka", e); + } } @Override @@ -205,7 +393,12 @@ public PropertiesMetadata topicPropertiesMetadata() throws UnsupportedOperationE } @Override - public void close() throws IOException {} + public void close() throws IOException { + if (adminClient != null) { + adminClient.close(); + adminClient = null; + } + } @Override public PropertiesMetadata filesetPropertiesMetadata() throws UnsupportedOperationException { @@ -217,6 +410,116 @@ public PropertiesMetadata tablePropertiesMetadata() throws UnsupportedOperationE throw new UnsupportedOperationException("Kafka catalog does not support table operations"); } + /** + * Make sure the schema exists, otherwise throw an exception. + * + * @param ident The schema identifier. + * @throws NoSuchSchemaException If the schema does not exist. + */ + private void checkSchemaExists(NameIdentifier ident) throws NoSuchSchemaException { + if (!schemaExists(ident)) { + LOG.warn("Kafka catalog schema {} does not exist", ident); + throw new NoSuchSchemaException("Schema %s does not exist", ident); + } + } + + /** + * Set the new partition count for the topic if it is greater than the current partition count. + * + * @param setProperty The property change to set the partition count. + * @param currentPartitionCount The current partition count. + * @param properties The properties map to update. + * @return The new partition count. + */ + private int setPartitionCount( + TopicChange.SetProperty setProperty, + int currentPartitionCount, + Map properties) { + Preconditions.checkArgument( + PARTITION_COUNT.equals(setProperty.getProperty()), "Invalid property: %s", setProperty); + + int targetPartitionCount = Integer.parseInt(setProperty.getValue()); + if (targetPartitionCount == currentPartitionCount) { + return currentPartitionCount; + } else if (targetPartitionCount < currentPartitionCount) { + throw new IllegalArgumentException( + "Cannot reduce partition count from " + + currentPartitionCount + + " to " + + targetPartitionCount); + } else { + properties.put(PARTITION_COUNT, setProperty.getValue()); + return targetPartitionCount; + } + } + + private void setProperty( + TopicChange.SetProperty setProperty, + Map alteredProperties, + List alterConfigOps) { + alteredProperties.put(setProperty.getProperty(), setProperty.getValue()); + alterConfigOps.add( + new AlterConfigOp( + new ConfigEntry(setProperty.getProperty(), setProperty.getValue()), + AlterConfigOp.OpType.SET)); + } + + private void removeProperty( + TopicChange.RemoveProperty removeProperty, + Map alteredProperties, + List alterConfigOps) { + Preconditions.checkArgument( + !PARTITION_COUNT.equals(removeProperty.getProperty()), "Cannot remove partition count"); + alteredProperties.remove(removeProperty.getProperty()); + alterConfigOps.add( + new AlterConfigOp( + new ConfigEntry(removeProperty.getProperty(), null), AlterConfigOp.OpType.DELETE)); + } + + private void doPartitionCountIncrement(String topicName, int newPartitionCount) { + try { + adminClient + .createPartitions( + Collections.singletonMap(topicName, NewPartitions.increaseTo(newPartitionCount))) + .all() + .get(); + } catch (Exception e) { + throw new RuntimeException("Failed to increase partition count for topic " + topicName, e); + } + } + + private void doAlterTopicConfig(String topicName, List alterConfigOps) { + ConfigResource topicResource = new ConfigResource(ConfigResource.Type.TOPIC, topicName); + try { + adminClient + .incrementalAlterConfigs(Collections.singletonMap(topicResource, alterConfigOps)) + .all() + .get(); + } catch (UnknownTopicOrPartitionException e) { + throw new NoSuchTopicException(e, "Topic %s does not exist", topicName); + } catch (Exception e) { + throw new RuntimeException("Failed to alter topic properties for topic " + topicName, e); + } + } + + private NewTopic buildNewTopic(NameIdentifier ident, Map properties) { + Optional partitionCount = + Optional.ofNullable( + (int) TOPIC_PROPERTIES_METADATA.getOrDefault(properties, PARTITION_COUNT)); + Optional replicationFactor = + Optional.ofNullable( + (short) TOPIC_PROPERTIES_METADATA.getOrDefault(properties, REPLICATION_FACTOR)); + NewTopic newTopic = new NewTopic(ident.name(), partitionCount, replicationFactor); + return newTopic.configs(buildNewTopicConfigs(properties)); + } + + private Map buildNewTopicConfigs(Map properties) { + Map topicConfigs = Maps.newHashMap(properties); + topicConfigs.remove(PARTITION_COUNT); + topicConfigs.remove(REPLICATION_FACTOR); + return topicConfigs; + } + private void createDefaultSchema() { // If the default schema already exists, do nothing try { diff --git a/catalogs/catalog-messaging-kafka/src/main/java/com/datastrato/gravitino/catalog/kafka/KafkaTopic.java b/catalogs/catalog-messaging-kafka/src/main/java/com/datastrato/gravitino/catalog/kafka/KafkaTopic.java new file mode 100644 index 00000000000..2e43ccf9427 --- /dev/null +++ b/catalogs/catalog-messaging-kafka/src/main/java/com/datastrato/gravitino/catalog/kafka/KafkaTopic.java @@ -0,0 +1,42 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog.kafka; + +import static com.datastrato.gravitino.catalog.kafka.KafkaTopicPropertiesMetadata.PARTITION_COUNT; + +import com.datastrato.gravitino.connector.BaseTopic; +import java.util.Optional; +import org.apache.kafka.clients.admin.NewTopic; + +public class KafkaTopic extends BaseTopic { + + public NewTopic toKafkaTopic(KafkaTopicPropertiesMetadata propertiesMetadata) { + Optional partitionCount = + Optional.ofNullable((int) propertiesMetadata.getOrDefault(properties(), PARTITION_COUNT)); + Optional replicationFactor = + Optional.ofNullable( + (short) + propertiesMetadata.getOrDefault( + properties(), KafkaTopicPropertiesMetadata.REPLICATION_FACTOR)); + return new NewTopic(name, partitionCount, replicationFactor); + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder extends BaseTopicBuilder { + + @Override + protected KafkaTopic internalBuild() { + KafkaTopic topic = new KafkaTopic(); + topic.name = name; + topic.comment = comment; + topic.properties = properties; + topic.auditInfo = auditInfo; + return topic; + } + } +} diff --git a/catalogs/catalog-messaging-kafka/src/main/java/com/datastrato/gravitino/catalog/kafka/KafkaTopicPropertiesMetadata.java b/catalogs/catalog-messaging-kafka/src/main/java/com/datastrato/gravitino/catalog/kafka/KafkaTopicPropertiesMetadata.java index 111e326f8e0..d6a6bfe405e 100644 --- a/catalogs/catalog-messaging-kafka/src/main/java/com/datastrato/gravitino/catalog/kafka/KafkaTopicPropertiesMetadata.java +++ b/catalogs/catalog-messaging-kafka/src/main/java/com/datastrato/gravitino/catalog/kafka/KafkaTopicPropertiesMetadata.java @@ -6,12 +6,41 @@ import com.datastrato.gravitino.connector.BasePropertiesMetadata; import com.datastrato.gravitino.connector.PropertyEntry; -import java.util.Collections; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; +import java.util.List; import java.util.Map; public class KafkaTopicPropertiesMetadata extends BasePropertiesMetadata { + public static final String PARTITION_COUNT = "partition-count"; + public static final String REPLICATION_FACTOR = "replication-factor"; + + private static final Map> PROPERTIES_METADATA; + + static { + List> propertyEntries = + ImmutableList.of( + PropertyEntry.integerOptionalPropertyEntry( + PARTITION_COUNT, + "The number of partitions for the topic, if not specified, " + + "will use the num.partition property in the broker", + false /* immutable */, + null /* default value */, + false /* hidden */), + // TODO: make REPLICATION_FACTOR mutable if needed + PropertyEntry.shortOptionalPropertyEntry( + REPLICATION_FACTOR, + "The number of replications for the topic, if not specified, " + + "will use the default.replication.factor property in the broker", + true /* immutable */, + null /* default value */, + false /* hidden */)); + + PROPERTIES_METADATA = Maps.uniqueIndex(propertyEntries, PropertyEntry::getName); + } + @Override protected Map> specificPropertyEntries() { - return Collections.emptyMap(); + return PROPERTIES_METADATA; } } diff --git a/catalogs/catalog-messaging-kafka/src/test/java/com/datastrato/gravitino/catalog/kafka/TestKafkaCatalogOperations.java b/catalogs/catalog-messaging-kafka/src/test/java/com/datastrato/gravitino/catalog/kafka/TestKafkaCatalogOperations.java index 88f68710655..9dacdd1625d 100644 --- a/catalogs/catalog-messaging-kafka/src/test/java/com/datastrato/gravitino/catalog/kafka/TestKafkaCatalogOperations.java +++ b/catalogs/catalog-messaging-kafka/src/test/java/com/datastrato/gravitino/catalog/kafka/TestKafkaCatalogOperations.java @@ -12,7 +12,10 @@ import static com.datastrato.gravitino.Configs.KV_DELETE_AFTER_TIME; import static com.datastrato.gravitino.Configs.STORE_TRANSACTION_MAX_SKEW_TIME; import static com.datastrato.gravitino.StringIdentifier.ID_KEY; +import static com.datastrato.gravitino.catalog.kafka.KafkaCatalogOperations.CLIENT_ID_TEMPLATE; import static com.datastrato.gravitino.catalog.kafka.KafkaCatalogPropertiesMetadata.BOOTSTRAP_SERVERS; +import static com.datastrato.gravitino.catalog.kafka.KafkaTopicPropertiesMetadata.PARTITION_COUNT; +import static com.datastrato.gravitino.catalog.kafka.KafkaTopicPropertiesMetadata.REPLICATION_FACTOR; import com.datastrato.gravitino.Config; import com.datastrato.gravitino.Configs; @@ -21,7 +24,13 @@ import com.datastrato.gravitino.EntityStoreFactory; import com.datastrato.gravitino.NameIdentifier; import com.datastrato.gravitino.Namespace; +import com.datastrato.gravitino.catalog.kafka.embeddedKafka.KafkaClusterEmbedded; import com.datastrato.gravitino.connector.BasePropertiesMetadata; +import com.datastrato.gravitino.exceptions.NoSuchSchemaException; +import com.datastrato.gravitino.exceptions.NoSuchTopicException; +import com.datastrato.gravitino.exceptions.TopicAlreadyExistsException; +import com.datastrato.gravitino.messaging.Topic; +import com.datastrato.gravitino.messaging.TopicChange; import com.datastrato.gravitino.meta.AuditInfo; import com.datastrato.gravitino.meta.CatalogEntity; import com.datastrato.gravitino.rel.Schema; @@ -32,23 +41,23 @@ import java.io.IOException; import java.time.Instant; import java.util.Map; -import java.util.UUID; import org.apache.commons.io.FileUtils; +import org.apache.kafka.common.config.TopicConfig; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.mockito.Mockito; -public class TestKafkaCatalogOperations { +public class TestKafkaCatalogOperations extends KafkaClusterEmbedded { private static final String ROCKS_DB_STORE_PATH = - "/tmp/gravitino_test_entityStore_" + UUID.randomUUID().toString().replace("-", ""); + "/tmp/gravitino_test_entityStore_" + genRandomString(); private static final String METALAKE_NAME = "metalake"; private static final String CATALOG_NAME = "test_kafka_catalog"; + private static final String DEFAULT_SCHEMA_NAME = "default"; private static final Map MOCK_CATALOG_PROPERTIES = - ImmutableMap.of( - BOOTSTRAP_SERVERS, "localhost:9092", ID_KEY, "gravitino.v1.uid33220758755757000"); + ImmutableMap.of(BOOTSTRAP_SERVERS, brokerList(), ID_KEY, "gravitino.v1.uid33220758755757000"); private static EntityStore store; private static IdGenerator idGenerator; private static CatalogEntity kafkaCatalogEntity; @@ -90,8 +99,10 @@ public static void setUp() { @AfterAll public static void tearDown() throws IOException { - store.close(); - FileUtils.deleteDirectory(FileUtils.getFile(ROCKS_DB_STORE_PATH)); + if (store != null) { + store.close(); + FileUtils.deleteDirectory(FileUtils.getFile(ROCKS_DB_STORE_PATH)); + } } @Test @@ -120,7 +131,12 @@ public void testKafkaCatalogConfiguration() { MOCK_CATALOG_PROPERTIES.get(BOOTSTRAP_SERVERS), ops.adminClientConfig.get(BOOTSTRAP_SERVERS)); Assertions.assertEquals( - MOCK_CATALOG_PROPERTIES.get(ID_KEY), ops.adminClientConfig.get("client.id")); + String.format( + CLIENT_ID_TEMPLATE, + MOCK_CATALOG_PROPERTIES.get(ID_KEY), + catalogEntity.namespace(), + catalogName), + ops.adminClientConfig.get("client.id")); } @Test @@ -143,13 +159,13 @@ public void testInitialization() { ops.initialize(MOCK_CATALOG_PROPERTIES, catalogEntity.toCatalogInfo()); Assertions.assertNotNull(ops.defaultSchemaIdent); - Assertions.assertEquals("default", ops.defaultSchemaIdent.name()); + Assertions.assertEquals(DEFAULT_SCHEMA_NAME, ops.defaultSchemaIdent.name()); Assertions.assertEquals( METALAKE_NAME + "." + catalogName, ops.defaultSchemaIdent.namespace().toString()); Assertions.assertTrue(ops.schemaExists(ops.defaultSchemaIdent)); Schema schema = ops.loadSchema(ops.defaultSchemaIdent); - Assertions.assertEquals("default", schema.name()); + Assertions.assertEquals(DEFAULT_SCHEMA_NAME, schema.name()); } @Test @@ -167,10 +183,10 @@ public void testCreateSchema() { @Test public void testLoadSchema() { - NameIdentifier ident = NameIdentifier.of(METALAKE_NAME, CATALOG_NAME, "default"); + NameIdentifier ident = NameIdentifier.of(METALAKE_NAME, CATALOG_NAME, DEFAULT_SCHEMA_NAME); Schema schema = kafkaCatalogOperations.loadSchema(ident); - Assertions.assertEquals("default", schema.name()); + Assertions.assertEquals(DEFAULT_SCHEMA_NAME, schema.name()); Assertions.assertEquals( "The default schema of Kafka catalog including all topics", schema.comment()); Assertions.assertEquals(2, schema.properties().size()); @@ -186,7 +202,7 @@ public void testAlterSchema() { IllegalArgumentException.class, () -> kafkaCatalogOperations.alterSchema( - NameIdentifier.of(METALAKE_NAME, CATALOG_NAME, "default"), + NameIdentifier.of(METALAKE_NAME, CATALOG_NAME, DEFAULT_SCHEMA_NAME), SchemaChange.removeProperty("key1"))); Assertions.assertEquals("Cannot alter the default schema", exception.getMessage()); @@ -208,7 +224,7 @@ public void testDropSchema() { IllegalArgumentException.class, () -> kafkaCatalogOperations.dropSchema( - NameIdentifier.of(METALAKE_NAME, CATALOG_NAME, "default"), true)); + NameIdentifier.of(METALAKE_NAME, CATALOG_NAME, DEFAULT_SCHEMA_NAME), true)); Assertions.assertEquals("Cannot drop the default schema", exception.getMessage()); NameIdentifier ident = NameIdentifier.of(METALAKE_NAME, CATALOG_NAME, "test_schema"); @@ -219,4 +235,210 @@ public void testDropSchema() { Assertions.assertEquals( "Kafka catalog does not support schema deletion", exception.getMessage()); } + + @Test + public void testCreateTopic() { + NameIdentifier ident = + NameIdentifier.of(METALAKE_NAME, CATALOG_NAME, DEFAULT_SCHEMA_NAME, "test_create_topic"); + String comment = "test comment"; + Map properties = + ImmutableMap.of( + PARTITION_COUNT, + "3", + REPLICATION_FACTOR, + "1", + TopicConfig.COMPRESSION_TYPE_CONFIG, + "producer"); + Topic createdTopic = kafkaCatalogOperations.createTopic(ident, comment, null, properties); + Assertions.assertNotNull(createdTopic); + Assertions.assertEquals(ident.name(), createdTopic.name()); + Assertions.assertEquals("3", createdTopic.properties().get(PARTITION_COUNT)); + Assertions.assertEquals("1", createdTopic.properties().get(REPLICATION_FACTOR)); + Assertions.assertEquals( + "producer", createdTopic.properties().get(TopicConfig.COMPRESSION_TYPE_CONFIG)); + } + + @Test + public void testCreateTopicException() { + Map properties = ImmutableMap.of(PARTITION_COUNT, "3", REPLICATION_FACTOR, "1"); + + // test topic already exists + Exception exception = + Assertions.assertThrows( + TopicAlreadyExistsException.class, + () -> + kafkaCatalogOperations.createTopic( + NameIdentifier.of(METALAKE_NAME, CATALOG_NAME, DEFAULT_SCHEMA_NAME, TOPIC_1), + null, + null, + properties)); + Assertions.assertEquals( + "Topic metalake.test_kafka_catalog.default.kafka-test-topic-1 already exists", + exception.getMessage()); + + // test schema not exists + exception = + Assertions.assertThrows( + NoSuchSchemaException.class, + () -> + kafkaCatalogOperations.createTopic( + NameIdentifier.of(METALAKE_NAME, CATALOG_NAME, "test_schema", "error_topic"), + null, + null, + properties)); + Assertions.assertEquals( + "Schema metalake.test_kafka_catalog.test_schema does not exist", exception.getMessage()); + + Map wrongProperties = + ImmutableMap.of(PARTITION_COUNT, "3", REPLICATION_FACTOR, "3"); + exception = + Assertions.assertThrows( + IllegalArgumentException.class, + () -> + kafkaCatalogOperations.createTopic( + NameIdentifier.of( + METALAKE_NAME, CATALOG_NAME, DEFAULT_SCHEMA_NAME, "error_topic"), + null, + null, + wrongProperties)); + Assertions.assertTrue( + exception.getMessage().contains("Invalid replication factor for topic"), + exception.getMessage()); + } + + @Test + public void testLoadTopic() { + Topic topic = + kafkaCatalogOperations.loadTopic( + NameIdentifier.of(METALAKE_NAME, CATALOG_NAME, DEFAULT_SCHEMA_NAME, TOPIC_1)); + Assertions.assertNotNull(topic); + Assertions.assertEquals(TOPIC_1, topic.name()); + Assertions.assertEquals("1", topic.properties().get(PARTITION_COUNT)); + Assertions.assertEquals("1", topic.properties().get(REPLICATION_FACTOR)); + Assertions.assertTrue(topic.properties().size() > 2); + } + + @Test + public void testLoadTopicException() { + Exception exception = + Assertions.assertThrows( + NoSuchTopicException.class, + () -> + kafkaCatalogOperations.loadTopic( + NameIdentifier.of( + METALAKE_NAME, CATALOG_NAME, DEFAULT_SCHEMA_NAME, "error_topic"))); + Assertions.assertEquals( + "Topic metalake.test_kafka_catalog.default.error_topic does not exist", + exception.getMessage()); + } + + @Test + public void testListTopics() { + NameIdentifier[] topics = + kafkaCatalogOperations.listTopics( + Namespace.of(METALAKE_NAME, CATALOG_NAME, DEFAULT_SCHEMA_NAME)); + Assertions.assertTrue(topics.length > 0); + + Exception exception = + Assertions.assertThrows( + NoSuchSchemaException.class, + () -> + kafkaCatalogOperations.listTopics( + Namespace.of(METALAKE_NAME, CATALOG_NAME, "error_schema"))); + Assertions.assertEquals( + "Schema metalake.test_kafka_catalog.error_schema does not exist", exception.getMessage()); + } + + @Test + public void testDropTopic() { + NameIdentifier ident = + NameIdentifier.of(METALAKE_NAME, CATALOG_NAME, DEFAULT_SCHEMA_NAME, "test_drop_topic"); + Map properties = ImmutableMap.of(PARTITION_COUNT, "3", REPLICATION_FACTOR, "1"); + kafkaCatalogOperations.createTopic(ident, null, null, properties); + Assertions.assertNotNull(kafkaCatalogOperations.loadTopic(ident)); + + Assertions.assertTrue(kafkaCatalogOperations.dropTopic(ident)); + Exception exception = + Assertions.assertThrows( + NoSuchTopicException.class, () -> kafkaCatalogOperations.loadTopic(ident)); + Assertions.assertEquals( + "Topic metalake.test_kafka_catalog.default.test_drop_topic does not exist", + exception.getMessage()); + + Assertions.assertFalse(kafkaCatalogOperations.dropTopic(ident)); + } + + @Test + public void testAlterTopic() { + NameIdentifier ident = + NameIdentifier.of(METALAKE_NAME, CATALOG_NAME, DEFAULT_SCHEMA_NAME, "test_alter_topic"); + Map properties = + ImmutableMap.of( + PARTITION_COUNT, + "2", + REPLICATION_FACTOR, + "1", + TopicConfig.COMPRESSION_TYPE_CONFIG, + "gzip", + TopicConfig.RETENTION_MS_CONFIG, + "43200000"); + Topic createdTopic = kafkaCatalogOperations.createTopic(ident, null, null, properties); + + Topic alteredTopic = + kafkaCatalogOperations.alterTopic( + ident, + TopicChange.updateComment("new comment"), + TopicChange.setProperty(PARTITION_COUNT, "3"), + TopicChange.setProperty(TopicConfig.COMPRESSION_TYPE_CONFIG, "producer"), + TopicChange.removeProperty(TopicConfig.RETENTION_MS_CONFIG)); + Assertions.assertEquals(createdTopic.name(), alteredTopic.name()); + Assertions.assertEquals("new comment", alteredTopic.comment()); + Assertions.assertEquals("3", alteredTopic.properties().get(PARTITION_COUNT)); + Assertions.assertEquals("1", alteredTopic.properties().get(REPLICATION_FACTOR)); + Assertions.assertEquals( + "producer", alteredTopic.properties().get(TopicConfig.COMPRESSION_TYPE_CONFIG)); + Assertions.assertNull(alteredTopic.properties().get(TopicConfig.RETENTION_MS_CONFIG)); + + // reload topic and check if the changes are applied + alteredTopic = kafkaCatalogOperations.loadTopic(ident); + Assertions.assertEquals(createdTopic.name(), alteredTopic.name()); + // comment is null because it is not stored in the topic + Assertions.assertNull(alteredTopic.comment()); + Assertions.assertEquals("3", alteredTopic.properties().get(PARTITION_COUNT)); + Assertions.assertEquals("1", alteredTopic.properties().get(REPLICATION_FACTOR)); + Assertions.assertEquals( + "producer", alteredTopic.properties().get(TopicConfig.COMPRESSION_TYPE_CONFIG)); + // retention.ms overridden was removed, so it should be the default value + Assertions.assertEquals( + "604800000", alteredTopic.properties().get(TopicConfig.RETENTION_MS_CONFIG)); + + // test exception + Exception exception = + Assertions.assertThrows( + NoSuchTopicException.class, + () -> + kafkaCatalogOperations.alterTopic( + NameIdentifier.of( + METALAKE_NAME, CATALOG_NAME, DEFAULT_SCHEMA_NAME, "error_topic"), + TopicChange.updateComment("new comment"))); + Assertions.assertEquals( + "Topic metalake.test_kafka_catalog.default.error_topic does not exist", + exception.getMessage()); + + exception = + Assertions.assertThrows( + IllegalArgumentException.class, + () -> + kafkaCatalogOperations.alterTopic( + ident, TopicChange.removeProperty(PARTITION_COUNT))); + Assertions.assertEquals("Cannot remove partition count", exception.getMessage()); + + exception = + Assertions.assertThrows( + IllegalArgumentException.class, + () -> + kafkaCatalogOperations.alterTopic( + ident, TopicChange.setProperty(PARTITION_COUNT, "1"))); + Assertions.assertEquals("Cannot reduce partition count from 3 to 1", exception.getMessage()); + } } diff --git a/catalogs/catalog-messaging-kafka/src/test/java/com/datastrato/gravitino/catalog/kafka/embeddedKafka/KafkaClusterEmbedded.java b/catalogs/catalog-messaging-kafka/src/test/java/com/datastrato/gravitino/catalog/kafka/embeddedKafka/KafkaClusterEmbedded.java new file mode 100644 index 00000000000..4a4c98e18b0 --- /dev/null +++ b/catalogs/catalog-messaging-kafka/src/test/java/com/datastrato/gravitino/catalog/kafka/embeddedKafka/KafkaClusterEmbedded.java @@ -0,0 +1,95 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog.kafka.embeddedKafka; + +import java.io.IOException; +import java.net.ServerSocket; +import java.util.Properties; +import java.util.UUID; +import kafka.server.KafkaConfig; +import kafka.server.KafkaConfig$; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class KafkaClusterEmbedded { + public static final String TOPIC_1 = "kafka-test-topic-1"; + public static final String TOPIC_2 = "kafka-test-topic-2"; + public static final String TOPIC_3 = "kafka-test-topic-3"; + + private static final Logger LOG = LoggerFactory.getLogger(KafkaClusterEmbedded.class); + private static ZooKeeperEmbedded zookeeper; + private static KafkaEmbedded broker; + + /** Creates and starts the cluster. */ + @BeforeAll + public static void start() throws Exception { + LOG.info("Initiating embedded Kafka cluster startup"); + LOG.info("Starting a ZooKeeper instance..."); + zookeeper = new ZooKeeperEmbedded(); + LOG.info("ZooKeeper instance is running at {}", zookeeper.connectString()); + + Properties brokerConfig = initBrokerConfig(); + LOG.info( + "Starting a Kafka instance on port {} ...", + brokerConfig.getProperty(KafkaConfig.ListenersProp())); + broker = new KafkaEmbedded(brokerConfig); + LOG.info( + "Kafka instance is running at {}, connected to ZooKeeper at {}", + broker.brokerList(), + broker.zookeeperConnect()); + + // Create initial topics + broker.createTopic(TOPIC_1); + broker.createTopic(TOPIC_2); + broker.createTopic(TOPIC_3); + } + + @AfterAll + public static void stop() throws IOException { + LOG.info("Stopping embedded Kafka cluster"); + if (broker != null) { + broker.stop(); + } + + if (zookeeper != null) { + zookeeper.stop(); + } + + LOG.info("Embedded Kafka cluster stopped"); + } + + protected static String genRandomString() { + return UUID.randomUUID().toString().replace("-", ""); + } + + public static String brokerList() { + return broker.brokerList(); + } + + private static Properties initBrokerConfig() { + Properties configs = new Properties(); + configs.put(KafkaConfig$.MODULE$.ZkConnectProp(), zookeeper.connectString()); + configs.put(KafkaConfig$.MODULE$.ZkSessionTimeoutMsProp(), 30 * 1000); + configs.put(KafkaConfig$.MODULE$.ZkConnectionTimeoutMsProp(), 60 * 1000); + configs.put(KafkaConfig$.MODULE$.DeleteTopicEnableProp(), true); + configs.put(KafkaConfig$.MODULE$.LogCleanerDedupeBufferSizeProp(), 2 * 1024 * 1024L); + configs.put(KafkaConfig$.MODULE$.GroupMinSessionTimeoutMsProp(), 0); + configs.put(KafkaConfig$.MODULE$.OffsetsTopicReplicationFactorProp(), (short) 1); + configs.put(KafkaConfig$.MODULE$.OffsetsTopicPartitionsProp(), 1); + configs.put(KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), true); + // Find a random port + try (ServerSocket socket = new ServerSocket(0)) { + socket.setReuseAddress(true); + configs.put( + KafkaConfig.ListenersProp(), + String.format("PLAINTEXT://127.0.0.1:%s", socket.getLocalPort())); + } catch (IOException e) { + throw new RuntimeException("Can't find a port to start embedded Kafka broker", e); + } + return configs; + } +} diff --git a/catalogs/catalog-messaging-kafka/src/test/java/com/datastrato/gravitino/catalog/kafka/embeddedKafka/KafkaEmbedded.java b/catalogs/catalog-messaging-kafka/src/test/java/com/datastrato/gravitino/catalog/kafka/embeddedKafka/KafkaEmbedded.java new file mode 100644 index 00000000000..f9d63abda9f --- /dev/null +++ b/catalogs/catalog-messaging-kafka/src/test/java/com/datastrato/gravitino/catalog/kafka/embeddedKafka/KafkaEmbedded.java @@ -0,0 +1,109 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog.kafka.embeddedKafka; + +import static com.datastrato.gravitino.catalog.kafka.embeddedKafka.KafkaClusterEmbedded.genRandomString; + +import java.io.IOException; +import java.util.Collections; +import java.util.Properties; +import java.util.concurrent.ExecutionException; +import kafka.cluster.EndPoint; +import kafka.server.KafkaConfig; +import kafka.server.KafkaConfig$; +import kafka.server.KafkaServer; +import org.apache.commons.io.FileUtils; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.common.utils.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Option; + +public class KafkaEmbedded { + private static final Logger LOG = LoggerFactory.getLogger(KafkaEmbedded.class); + private static final String LOG_DIR = "/tmp/gravitino_test_embeddedKafka_" + genRandomString(); + + private final Properties effectiveConfig; + private final KafkaServer kafka; + + public KafkaEmbedded(final Properties config) { + effectiveConfig = effectiveConfigFrom(config); + final boolean loggingEnabled = true; + + final KafkaConfig kafkaConfig = new KafkaConfig(effectiveConfig, loggingEnabled); + LOG.info("Starting embedded Kafka broker (with ZK ensemble at {}) ...", zookeeperConnect()); + kafka = new KafkaServer(kafkaConfig, Time.SYSTEM, Option.apply("embedded-kafka-broker"), false); + kafka.startup(); + LOG.info( + "Startup of embedded Kafka broker at {} completed (with ZK ensemble at {}) ...", + brokerList(), + zookeeperConnect()); + } + + public void createTopic(String topic) { + Properties properties = new Properties(); + properties.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList()); + + try (AdminClient adminClient = AdminClient.create(properties)) { + NewTopic newTopic = new NewTopic(topic, 1, (short) 1); + adminClient.createTopics(Collections.singletonList(newTopic)).all().get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException("Failed to create topic " + topic, e); + } + } + + /** + * This broker's `metadata.broker.list` value. Example: `127.0.0.1:9092`. + * + *

You can use this to tell Kafka producers and consumers how to connect to this instance. + */ + public String brokerList() { + final EndPoint endPoint = kafka.advertisedListeners().head(); + final String hostname = endPoint.host() == null ? "" : endPoint.host(); + + return String.join( + ":", + hostname, + Integer.toString( + kafka.boundPort(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)))); + } + + /** The ZooKeeper connection string aka `zookeeper.connect`. */ + public String zookeeperConnect() { + return effectiveConfig.getProperty("zookeeper.connect"); + } + + /** Stop the broker. */ + public void stop() throws IOException { + LOG.info( + "Shutting down embedded Kafka broker at {} (with ZK ensemble at {}) ...", + brokerList(), + zookeeperConnect()); + kafka.shutdown(); + kafka.awaitShutdown(); + FileUtils.deleteDirectory(FileUtils.getFile(LOG_DIR)); + LOG.info( + "Shutdown of embedded Kafka broker at {} completed (with ZK ensemble at {}) ...", + brokerList(), + zookeeperConnect()); + } + + private Properties effectiveConfigFrom(final Properties initialConfig) { + final Properties effectiveConfig = new Properties(); + effectiveConfig.put(KafkaConfig$.MODULE$.BrokerIdProp(), 0); + effectiveConfig.put(KafkaConfig$.MODULE$.NumPartitionsProp(), 1); + effectiveConfig.put(KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), true); + effectiveConfig.put(KafkaConfig$.MODULE$.MessageMaxBytesProp(), 1000000); + effectiveConfig.put(KafkaConfig$.MODULE$.ControlledShutdownEnableProp(), true); + + effectiveConfig.putAll(initialConfig); + effectiveConfig.setProperty(KafkaConfig$.MODULE$.LogDirProp(), LOG_DIR); + return effectiveConfig; + } +} diff --git a/catalogs/catalog-messaging-kafka/src/test/java/com/datastrato/gravitino/catalog/kafka/embeddedKafka/ZooKeeperEmbedded.java b/catalogs/catalog-messaging-kafka/src/test/java/com/datastrato/gravitino/catalog/kafka/embeddedKafka/ZooKeeperEmbedded.java new file mode 100644 index 00000000000..bab31e83f17 --- /dev/null +++ b/catalogs/catalog-messaging-kafka/src/test/java/com/datastrato/gravitino/catalog/kafka/embeddedKafka/ZooKeeperEmbedded.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.kafka.embeddedKafka; + +import java.io.IOException; +import org.apache.curator.test.TestingServer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ZooKeeperEmbedded { + private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperEmbedded.class); + private final TestingServer server; + + /** + * Creates and starts a ZooKeeper instance. + * + * @throws Exception if an error occurs during ZooKeeper startup + */ + public ZooKeeperEmbedded() throws Exception { + LOG.info("Starting embedded ZooKeeper server..."); + this.server = new TestingServer(); + LOG.info( + "Embedded ZooKeeper server at {} uses the temp directory at {}", + server.getConnectString(), + server.getTempDirectory()); + } + + public void stop() throws IOException { + LOG.info("Shutting down embedded ZooKeeper server at {} ...", server.getConnectString()); + server.close(); + LOG.info("Shutdown of embedded ZooKeeper server at {} completed", server.getConnectString()); + } + + /** + * The ZooKeeper connection string aka `zookeeper.connect` in `hostnameOrIp:port` format. Example: + * `127.0.0.1:2181`. + * + *

You can use this to e.g. tell Kafka brokers how to connect to this instance. + */ + public String connectString() { + return server.getConnectString(); + } +} diff --git a/core/src/main/java/com/datastrato/gravitino/connector/BaseTopic.java b/core/src/main/java/com/datastrato/gravitino/connector/BaseTopic.java new file mode 100644 index 00000000000..f68fc233e45 --- /dev/null +++ b/core/src/main/java/com/datastrato/gravitino/connector/BaseTopic.java @@ -0,0 +1,156 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.connector; + +import com.datastrato.gravitino.Audit; +import com.datastrato.gravitino.annotation.Evolving; +import com.datastrato.gravitino.messaging.Topic; +import com.datastrato.gravitino.meta.AuditInfo; +import java.util.Map; +import javax.annotation.Nullable; + +/** An abstract class representing a base topic in a messaging system. */ +@Evolving +public abstract class BaseTopic implements Topic { + + protected String name; + + @Nullable protected String comment; + + @Nullable protected Map properties; + + protected Audit auditInfo; + + /** @return The name of the topic. */ + @Override + public String name() { + return name; + } + + /** @return The comment or description for the topic. */ + @Nullable + @Override + public String comment() { + return comment; + } + + /** @return The associated properties of the topic. */ + @Override + public Map properties() { + return properties; + } + + /** @return The audit information for the topic. */ + @Override + public Audit auditInfo() { + return auditInfo; + } + + /** + * Builder interface for {@link BaseTopic}. + * + * @param The type of the builder. + * @param The type of the topic being built. + */ + interface Builder, T extends BaseTopic> { + + SELF withName(String name); + + SELF withComment(String comment); + + SELF withProperties(Map properties); + + SELF withAuditInfo(AuditInfo auditInfo); + + T build(); + } + + /** + * An abstract class implementing the builder interface for {@link BaseTopic}. This class should + * be extended by the concrete topic builders. + * + * @param The type of the builder. + * @param The type of the topic being built. + */ + public abstract static class BaseTopicBuilder< + SELF extends BaseTopicBuilder, T extends BaseTopic> + implements Builder { + protected String name; + protected String comment; + protected Map properties; + protected AuditInfo auditInfo; + + /** + * Sets the name of the topic. + * + * @param name The name of the topic. + * @return The builder instance. + */ + @Override + public SELF withName(String name) { + this.name = name; + return self(); + } + + /** + * Sets the comment of the topic. + * + * @param comment The comment or description for the topic. + * @return The builder instance. + */ + @Override + public SELF withComment(String comment) { + this.comment = comment; + return self(); + } + + /** + * Sets the associated properties of the topic. + * + * @param properties The associated properties of the topic. + * @return The builder instance. + */ + @Override + public SELF withProperties(Map properties) { + this.properties = properties; + return self(); + } + + /** + * Sets the audit information for the topic. + * + * @param auditInfo The audit information for the topic. + * @return The builder instance. + */ + @Override + public SELF withAuditInfo(AuditInfo auditInfo) { + this.auditInfo = auditInfo; + return self(); + } + + /** + * Builds the topic with the provided attributes. + * + * @return The built topic instance. + */ + @Override + public T build() { + T t = internalBuild(); + return t; + } + + private SELF self() { + return (SELF) this; + } + + /** + * Builds the concrete instance of the topic with the provided attributes. + * + * @return The concrete instance of the topic. + */ + @Evolving + protected abstract T internalBuild(); + } +} diff --git a/core/src/main/java/com/datastrato/gravitino/connector/PropertyEntry.java b/core/src/main/java/com/datastrato/gravitino/connector/PropertyEntry.java index e67ca450be5..577ecd481cb 100644 --- a/core/src/main/java/com/datastrato/gravitino/connector/PropertyEntry.java +++ b/core/src/main/java/com/datastrato/gravitino/connector/PropertyEntry.java @@ -222,6 +222,28 @@ public static PropertyEntry integerPropertyEntry( .build(); } + public static PropertyEntry shortPropertyEntry( + String name, + String description, + boolean required, + boolean immutable, + Short defaultValue, + boolean hidden, + boolean reserved) { + return new Builder() + .withName(name) + .withDescription(description) + .withRequired(required) + .withImmutable(immutable) + .withJavaType(Short.class) + .withDefaultValue(defaultValue) + .withDecoder(Short::parseShort) + .withEncoder(String::valueOf) + .withHidden(hidden) + .withReserved(reserved) + .build(); + } + public static PropertyEntry stringReservedPropertyEntry( String name, String description, boolean hidden) { return stringPropertyEntry(name, description, false, true, null, hidden, true); @@ -264,6 +286,11 @@ public static PropertyEntry stringOptionalPropertyEntry( return stringPropertyEntry(name, description, false, immutable, defaultValue, hidden, false); } + public static PropertyEntry shortOptionalPropertyEntry( + String name, String description, boolean immutable, Short defaultValue, boolean hidden) { + return shortPropertyEntry(name, description, false, immutable, defaultValue, hidden, false); + } + public static PropertyEntry integerOptionalPropertyEntry( String name, String description, boolean immutable, Integer defaultValue, boolean hidden) { return integerPropertyEntry(name, description, false, immutable, defaultValue, hidden, false); diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index acc44b37f8d..39a9459ba57 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -50,6 +50,8 @@ rauschig = "1.2.0" mybatis = "3.5.6" h2db = "1.4.200" kyuubi = "1.8.0" +kafka = "3.4.0" +curator = "2.12.0" protobuf-plugin = "0.9.2" spotless-plugin = '6.11.0' @@ -152,6 +154,9 @@ minikdc = { group = "org.apache.hadoop", name = "hadoop-minikdc", version.ref = immutables-value = { module = "org.immutables:value", version.ref = "immutables-value" } commons-cli = { group = "commons-cli", name = "commons-cli", version.ref = "commons-cli" } sun-activation = { group = "com.sun.activation", name = "javax.activation", version.ref = "sun-activation-version" } +kafka-clients = { group = "org.apache.kafka", name = "kafka-clients", version.ref = "kafka" } +kafka = { group = "org.apache.kafka", name = "kafka_2.12", version.ref = "kafka" } +curator-test = { group = "org.apache.curator", name = "curator-test", version.ref = "curator"} selenium = { group = "org.seleniumhq.selenium", name = "selenium-java", version.ref = "selenium" } rauschig = { group = "org.rauschig", name = "jarchivelib", version.ref = "rauschig" } From c6f08c61e709a22b6f33112d19ad71425cb36e95 Mon Sep 17 00:00:00 2001 From: FANNG Date: Fri, 29 Mar 2024 18:00:46 +0800 Subject: [PATCH 3/4] [#2620] feat(spark-connector): support hive table format properties (#2605) ### What changes were proposed in this pull request? support hive table format properties ```sql CREATE TABLE xxx STORED AS PARQUET CREATE TABLE xxx USING PARQUET CREATE TABLE xxx ROW FORMAT SERDE xx STORED AS INPUTFORMAT xx OUTPUTFORMAT xx CREATE TABLE xxx ROW FORMAT DELIMITED FIELDS TERMINATED xx ``` ### Why are the changes needed? Fix: #2620 ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? UT and IT --- .../hive/HiveTablePropertiesMetadata.java | 11 +- .../jdbc/JdbcTablePropertiesMetadata.java | 7 +- .../mysql/MysqlTablePropertiesMetadata.java | 2 +- .../integration/test/spark/SparkCommonIT.java | 30 +++ .../test/spark/hive/SparkHiveCatalogIT.java | 171 ++++++++++++++++++ .../test/util/spark/SparkTableInfo.java | 9 +- .../util/spark/SparkTableInfoChecker.java | 20 ++ .../test/util/spark/SparkUtilIT.java | 2 +- spark-connector/build.gradle.kts | 1 + .../hive/HivePropertiesConstants.java | 60 ++++++ .../hive/HivePropertiesConverter.java | 110 ++++++++++- .../hive/TestHivePropertiesConverter.java | 101 +++++++++++ 12 files changed, 508 insertions(+), 16 deletions(-) create mode 100644 spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/HivePropertiesConstants.java create mode 100644 spark-connector/src/test/java/com/datastrato/gravitino/spark/connector/hive/TestHivePropertiesConverter.java diff --git a/catalogs/catalog-hive/src/main/java/com/datastrato/gravitino/catalog/hive/HiveTablePropertiesMetadata.java b/catalogs/catalog-hive/src/main/java/com/datastrato/gravitino/catalog/hive/HiveTablePropertiesMetadata.java index 301acc52a84..0be2271a1de 100644 --- a/catalogs/catalog-hive/src/main/java/com/datastrato/gravitino/catalog/hive/HiveTablePropertiesMetadata.java +++ b/catalogs/catalog-hive/src/main/java/com/datastrato/gravitino/catalog/hive/HiveTablePropertiesMetadata.java @@ -57,11 +57,11 @@ public class HiveTablePropertiesMetadata extends BasePropertiesMetadata { @VisibleForTesting public static final String ORC_SERDE_CLASS = "org.apache.hadoop.hive.ql.io.orc.OrcSerde"; - private static final String PARQUET_INPUT_FORMAT_CLASS = + public static final String PARQUET_INPUT_FORMAT_CLASS = "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"; - private static final String PARQUET_OUTPUT_FORMAT_CLASS = + public static final String PARQUET_OUTPUT_FORMAT_CLASS = "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"; - private static final String PARQUET_SERDE_CLASS = + public static final String PARQUET_SERDE_CLASS = "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"; private static final String COLUMNAR_SERDE_CLASS = "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe"; @@ -89,7 +89,10 @@ public enum TableType { VIRTUAL_INDEX, } - enum StorageFormat { + // In embedded test mode, HiveTablePropertiesMetadata will be loaded by spark connector which has + // different classloaders with Hive catalog. If StorageFormat is package scope, it couldn't + // be accessed by Hive catalog related classes in same package, so making it public. + public enum StorageFormat { SEQUENCEFILE( SEQUENCEFILE_INPUT_FORMAT_CLASS, SEQUENCEFILE_OUTPUT_FORMAT_CLASS, LAZY_SIMPLE_SERDE_CLASS), TEXTFILE(TEXT_INPUT_FORMAT_CLASS, IGNORE_KEY_OUTPUT_FORMAT_CLASS, LAZY_SIMPLE_SERDE_CLASS), diff --git a/catalogs/catalog-jdbc-common/src/main/java/com/datastrato/gravitino/catalog/jdbc/JdbcTablePropertiesMetadata.java b/catalogs/catalog-jdbc-common/src/main/java/com/datastrato/gravitino/catalog/jdbc/JdbcTablePropertiesMetadata.java index 11beec8af0a..062023b8393 100644 --- a/catalogs/catalog-jdbc-common/src/main/java/com/datastrato/gravitino/catalog/jdbc/JdbcTablePropertiesMetadata.java +++ b/catalogs/catalog-jdbc-common/src/main/java/com/datastrato/gravitino/catalog/jdbc/JdbcTablePropertiesMetadata.java @@ -6,6 +6,7 @@ import com.datastrato.gravitino.StringIdentifier; import com.datastrato.gravitino.connector.BasePropertiesMetadata; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Maps; import java.util.HashMap; import java.util.Map; @@ -14,13 +15,15 @@ public abstract class JdbcTablePropertiesMetadata extends BasePropertiesMetadata public static final String COMMENT_KEY = "comment"; - protected Map transformToJdbcProperties(Map properties) { + @VisibleForTesting + public Map transformToJdbcProperties(Map properties) { HashMap resultProperties = Maps.newHashMap(properties); resultProperties.remove(StringIdentifier.ID_KEY); return resultProperties; } - protected Map convertFromJdbcProperties(Map properties) { + @VisibleForTesting + public Map convertFromJdbcProperties(Map properties) { return properties; } } diff --git a/catalogs/catalog-jdbc-mysql/src/main/java/com/datastrato/gravitino/catalog/mysql/MysqlTablePropertiesMetadata.java b/catalogs/catalog-jdbc-mysql/src/main/java/com/datastrato/gravitino/catalog/mysql/MysqlTablePropertiesMetadata.java index 48dba023dca..1162f048b2e 100644 --- a/catalogs/catalog-jdbc-mysql/src/main/java/com/datastrato/gravitino/catalog/mysql/MysqlTablePropertiesMetadata.java +++ b/catalogs/catalog-jdbc-mysql/src/main/java/com/datastrato/gravitino/catalog/mysql/MysqlTablePropertiesMetadata.java @@ -105,7 +105,7 @@ public Map transformToJdbcProperties(Map propert } @Override - protected Map convertFromJdbcProperties(Map properties) { + public Map convertFromJdbcProperties(Map properties) { BidiMap mysqlConfigToGravitino = GRAVITINO_CONFIG_TO_MYSQL.inverseBidiMap(); return Collections.unmodifiableMap( new HashMap() { diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/SparkCommonIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/SparkCommonIT.java index a028e5add02..731836370fe 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/SparkCommonIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/SparkCommonIT.java @@ -19,6 +19,7 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.connector.catalog.TableCatalog; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; import org.junit.jupiter.api.Assertions; @@ -116,6 +117,7 @@ void testCreateAndLoadSchema() { @Test void testAlterSchema() { String testDatabaseName = "t_alter"; + dropDatabaseIfExists(testDatabaseName); sql("CREATE DATABASE " + testDatabaseName); Assertions.assertTrue( StringUtils.isBlank(getDatabaseMetadata(testDatabaseName).get("Properties"))); @@ -174,6 +176,7 @@ void testCreateTableWithDatabase() { createDatabaseIfNotExists(databaseName); String tableIdentifier = String.join(".", databaseName, tableName); + dropTableIfExists(tableIdentifier); createSimpleTable(tableIdentifier); SparkTableInfo tableInfo = getTableInfo(tableIdentifier); SparkTableInfoChecker checker = @@ -187,6 +190,7 @@ void testCreateTableWithDatabase() { createDatabaseIfNotExists(databaseName); sql("USE " + databaseName); + dropTableIfExists(tableName); createSimpleTable(tableName); tableInfo = getTableInfo(tableName); checker = @@ -257,6 +261,8 @@ void testRenameTable() { void testListTable() { String table1 = "list1"; String table2 = "list2"; + dropTableIfExists(table1); + dropTableIfExists(table2); createSimpleTable(table1); createSimpleTable(table2); Set tables = listTableNames(); @@ -268,6 +274,8 @@ void testListTable() { String table3 = "list3"; String table4 = "list4"; createDatabaseIfNotExists(database); + dropTableIfExists(String.join(".", database, table3)); + dropTableIfExists(String.join(".", database, table4)); createSimpleTable(String.join(".", database, table3)); createSimpleTable(String.join(".", database, table4)); tables = listTableNames(database); @@ -590,6 +598,23 @@ protected void checkDirExists(Path dir) { } } + @Test + void testTableOptions() { + String tableName = "options_table"; + dropTableIfExists(tableName); + String createTableSql = getCreateSimpleTableString(tableName); + createTableSql += " OPTIONS('a'='b')"; + sql(createTableSql); + SparkTableInfo tableInfo = getTableInfo(tableName); + + SparkTableInfoChecker checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withTableProperties(ImmutableMap.of(TableCatalog.OPTION_PREFIX + "a", "b")); + checker.check(tableInfo); + checkTableReadWrite(tableInfo); + } + protected void checkTableReadWrite(SparkTableInfo table) { String name = table.getTableIdentifier(); boolean isPartitionTable = table.isPartitionTable(); @@ -687,4 +712,9 @@ private String getPartitionExpression(SparkTableInfo table, String delimiter) { .map(column -> column.getName() + "=" + typeConstant.get(column.getType())) .collect(Collectors.joining(delimiter)); } + + protected void checkParquetFile(SparkTableInfo tableInfo) { + String location = tableInfo.getTableLocation(); + Assertions.assertDoesNotThrow(() -> getSparkSession().read().parquet(location).printSchema()); + } } diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/hive/SparkHiveCatalogIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/hive/SparkHiveCatalogIT.java index 65d98f5da64..bc513eafa79 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/hive/SparkHiveCatalogIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/hive/SparkHiveCatalogIT.java @@ -8,10 +8,13 @@ import com.datastrato.gravitino.integration.test.util.spark.SparkTableInfo; import com.datastrato.gravitino.integration.test.util.spark.SparkTableInfo.SparkColumnInfo; import com.datastrato.gravitino.integration.test.util.spark.SparkTableInfoChecker; +import com.datastrato.gravitino.spark.connector.hive.HivePropertiesConstants; +import com.google.common.collect.ImmutableMap; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import org.apache.hadoop.fs.Path; +import org.apache.spark.sql.connector.catalog.TableCatalog; import org.apache.spark.sql.types.DataTypes; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Tag; @@ -150,4 +153,172 @@ public void testInsertHiveFormatPartitionTableAsSelect() { Assertions.assertTrue(tableData.size() == 1); Assertions.assertEquals(expectedData, tableData.get(0)); } + + @Test + void testHiveDefaultFormat() { + String tableName = "hive_default_format_table"; + dropTableIfExists(tableName); + createSimpleTable(tableName); + SparkTableInfo tableInfo = getTableInfo(tableName); + + SparkTableInfoChecker checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withTableProperties( + ImmutableMap.of( + HivePropertiesConstants.SPARK_HIVE_INPUT_FORMAT, + HivePropertiesConstants.TEXT_INPUT_FORMAT_CLASS, + HivePropertiesConstants.SPARK_HIVE_OUTPUT_FORMAT, + HivePropertiesConstants.IGNORE_KEY_OUTPUT_FORMAT_CLASS, + HivePropertiesConstants.SPARK_HIVE_SERDE_LIB, + HivePropertiesConstants.LAZY_SIMPLE_SERDE_CLASS)); + checker.check(tableInfo); + checkTableReadWrite(tableInfo); + } + + @Test + void testHiveFormatWithStoredAs() { + String tableName = "test_hive_format_stored_as_table"; + dropTableIfExists(tableName); + String createTableSql = getCreateSimpleTableString(tableName); + createTableSql += "STORED AS PARQUET"; + sql(createTableSql); + SparkTableInfo tableInfo = getTableInfo(tableName); + + SparkTableInfoChecker checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withTableProperties( + ImmutableMap.of( + HivePropertiesConstants.SPARK_HIVE_INPUT_FORMAT, + HivePropertiesConstants.PARQUET_INPUT_FORMAT_CLASS, + HivePropertiesConstants.SPARK_HIVE_OUTPUT_FORMAT, + HivePropertiesConstants.PARQUET_OUTPUT_FORMAT_CLASS, + HivePropertiesConstants.SPARK_HIVE_SERDE_LIB, + HivePropertiesConstants.PARQUET_SERDE_CLASS)); + checker.check(tableInfo); + checkTableReadWrite(tableInfo); + checkParquetFile(tableInfo); + } + + @Test + void testHiveFormatWithUsing() { + String tableName = "test_hive_format_using_table"; + dropTableIfExists(tableName); + String createTableSql = getCreateSimpleTableString(tableName); + createTableSql += "USING PARQUET"; + sql(createTableSql); + SparkTableInfo tableInfo = getTableInfo(tableName); + + SparkTableInfoChecker checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withTableProperties( + ImmutableMap.of( + HivePropertiesConstants.SPARK_HIVE_INPUT_FORMAT, + HivePropertiesConstants.PARQUET_INPUT_FORMAT_CLASS, + HivePropertiesConstants.SPARK_HIVE_OUTPUT_FORMAT, + HivePropertiesConstants.PARQUET_OUTPUT_FORMAT_CLASS, + HivePropertiesConstants.SPARK_HIVE_SERDE_LIB, + HivePropertiesConstants.PARQUET_SERDE_CLASS)); + checker.check(tableInfo); + checkTableReadWrite(tableInfo); + checkParquetFile(tableInfo); + } + + @Test + void testHivePropertiesWithSerdeRowFormat() { + String tableName = "test_hive_row_serde_table"; + dropTableIfExists(tableName); + String createTableSql = getCreateSimpleTableString(tableName); + createTableSql = + String.format( + "%s ROW FORMAT SERDE '%s' WITH SERDEPROPERTIES ('serialization.format'='@', 'field.delim' = ',') STORED AS INPUTFORMAT '%s' OUTPUTFORMAT '%s'", + createTableSql, + HivePropertiesConstants.PARQUET_SERDE_CLASS, + HivePropertiesConstants.PARQUET_INPUT_FORMAT_CLASS, + HivePropertiesConstants.PARQUET_OUTPUT_FORMAT_CLASS); + sql(createTableSql); + SparkTableInfo tableInfo = getTableInfo(tableName); + + SparkTableInfoChecker checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withTableProperties( + ImmutableMap.of( + TableCatalog.OPTION_PREFIX + "serialization.format", + "@", + TableCatalog.OPTION_PREFIX + "field.delim", + ",", + HivePropertiesConstants.SPARK_HIVE_INPUT_FORMAT, + HivePropertiesConstants.PARQUET_INPUT_FORMAT_CLASS, + HivePropertiesConstants.SPARK_HIVE_OUTPUT_FORMAT, + HivePropertiesConstants.PARQUET_OUTPUT_FORMAT_CLASS, + HivePropertiesConstants.SPARK_HIVE_SERDE_LIB, + HivePropertiesConstants.PARQUET_SERDE_CLASS)); + checker.check(tableInfo); + checkTableReadWrite(tableInfo); + checkParquetFile(tableInfo); + } + + /* + | DELIMITED [ FIELDS TERMINATED BY fields_terminated_char [ ESCAPED BY escaped_char ] ] + [ COLLECTION ITEMS TERMINATED BY collection_items_terminated_char ] + [ MAP KEYS TERMINATED BY map_key_terminated_char ] + [ LINES TERMINATED BY row_terminated_char ] + [ NULL DEFINED AS null_char ] + */ + @Test + void testHivePropertiesWithDelimitedRowFormat() { + String tableName = "test_hive_row_format_table"; + dropTableIfExists(tableName); + String createTableSql = getCreateSimpleTableString(tableName); + createTableSql += + "ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' ESCAPED BY ';' " + + "COLLECTION ITEMS TERMINATED BY '@' " + + "MAP KEYS TERMINATED BY ':' " + + "NULL DEFINED AS 'n' " + + "STORED AS TEXTFILE"; + sql(createTableSql); + SparkTableInfo tableInfo = getTableInfo(tableName); + + SparkTableInfoChecker checker = + SparkTableInfoChecker.create() + .withName(tableName) + .withTableProperties( + ImmutableMap.of( + TableCatalog.OPTION_PREFIX + "field.delim", + ",", + TableCatalog.OPTION_PREFIX + "escape.delim", + ";", + TableCatalog.OPTION_PREFIX + "mapkey.delim", + ":", + TableCatalog.OPTION_PREFIX + "serialization.format", + ",", + TableCatalog.OPTION_PREFIX + "colelction.delim", + "@", + HivePropertiesConstants.SPARK_HIVE_INPUT_FORMAT, + HivePropertiesConstants.TEXT_INPUT_FORMAT_CLASS, + HivePropertiesConstants.SPARK_HIVE_OUTPUT_FORMAT, + HivePropertiesConstants.IGNORE_KEY_OUTPUT_FORMAT_CLASS, + HivePropertiesConstants.SPARK_HIVE_SERDE_LIB, + HivePropertiesConstants.LAZY_SIMPLE_SERDE_CLASS)); + checker.check(tableInfo); + checkTableReadWrite(tableInfo); + + // check it's a text file and field.delim take effects + List rows = + rowsToJava( + getSparkSession() + .read() + .option("delimiter", ",") + .csv(tableInfo.getTableLocation()) + .collectAsList()); + Assertions.assertTrue(rows.size() == 1); + Object[] row = rows.get(0); + Assertions.assertEquals(3, row.length); + Assertions.assertEquals("2", row[0]); + Assertions.assertEquals("gravitino_it_test", (String) row[1]); + Assertions.assertEquals("2", row[2]); + } } diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkTableInfo.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkTableInfo.java index 449237ff157..8d32c8ef1ca 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkTableInfo.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkTableInfo.java @@ -17,6 +17,7 @@ import javax.ws.rs.NotSupportedException; import lombok.Data; import org.apache.commons.lang3.StringUtils; +import org.apache.spark.sql.connector.catalog.TableCatalog; import org.apache.spark.sql.connector.expressions.BucketTransform; import org.apache.spark.sql.connector.expressions.IdentityTransform; import org.apache.spark.sql.connector.expressions.SortedBucketTransform; @@ -43,6 +44,10 @@ public String getTableName() { return tableName; } + public String getTableLocation() { + return tableProperties.get(TableCatalog.PROP_LOCATION); + } + // Include database name and table name public String getTableIdentifier() { if (StringUtils.isNotBlank(database)) { @@ -52,10 +57,6 @@ public String getTableIdentifier() { } } - public String getTableLocation() { - return tableProperties.get(ConnectorConstants.LOCATION); - } - public boolean isPartitionTable() { return partitions.size() > 0; } diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkTableInfoChecker.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkTableInfoChecker.java index d346769281c..c41ccd23213 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkTableInfoChecker.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkTableInfoChecker.java @@ -9,6 +9,7 @@ import com.datastrato.gravitino.spark.connector.SparkTransformConverter; import java.util.ArrayList; import java.util.List; +import java.util.Map; import org.apache.spark.sql.connector.expressions.Expressions; import org.apache.spark.sql.connector.expressions.IdentityTransform; import org.apache.spark.sql.connector.expressions.Transform; @@ -34,6 +35,7 @@ private enum CheckField { PARTITION, BUCKET, COMMENT, + TABLE_PROPERTY, } public SparkTableInfoChecker withName(String name) { @@ -82,6 +84,12 @@ public SparkTableInfoChecker withComment(String comment) { return this; } + public SparkTableInfoChecker withTableProperties(Map properties) { + this.expectedTableInfo.setTableProperties(properties); + this.checkFields.add(CheckField.TABLE_PROPERTY); + return this; + } + public void check(SparkTableInfo realTableInfo) { checkFields.stream() .forEach( @@ -106,6 +114,18 @@ public void check(SparkTableInfo realTableInfo) { Assertions.assertEquals( expectedTableInfo.getComment(), realTableInfo.getComment()); break; + case TABLE_PROPERTY: + Map realTableProperties = realTableInfo.getTableProperties(); + expectedTableInfo + .getTableProperties() + .forEach( + (k, v) -> { + Assertions.assertTrue( + realTableProperties.containsKey(k), + k + " not exits," + realTableProperties); + Assertions.assertEquals(v, realTableProperties.get(k)); + }); + break; default: Assertions.fail(checkField + " not checked"); break; diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkUtilIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkUtilIT.java index 6768f7309dc..6616df7e2c0 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkUtilIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/util/spark/SparkUtilIT.java @@ -143,7 +143,7 @@ private static String getSelectAllSql(String tableName) { return String.format("SELECT * FROM %s", tableName); } - private List rowsToJava(List rows) { + protected List rowsToJava(List rows) { return rows.stream().map(this::toJava).collect(Collectors.toList()); } diff --git a/spark-connector/build.gradle.kts b/spark-connector/build.gradle.kts index 1a03e73f34f..23b5f77317d 100644 --- a/spark-connector/build.gradle.kts +++ b/spark-connector/build.gradle.kts @@ -20,6 +20,7 @@ val scalaJava8CompatVersion: String = libs.versions.scala.java.compat.get() dependencies { implementation(project(":api")) + implementation(project(":catalogs:bundled-catalog", configuration = "shadow")) implementation(project(":clients:client-java-runtime", configuration = "shadow")) implementation(project(":common")) implementation(libs.bundles.log4j) diff --git a/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/HivePropertiesConstants.java b/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/HivePropertiesConstants.java new file mode 100644 index 00000000000..c70e038a19a --- /dev/null +++ b/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/HivePropertiesConstants.java @@ -0,0 +1,60 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ + +package com.datastrato.gravitino.spark.connector.hive; + +import com.datastrato.gravitino.catalog.hive.HiveTablePropertiesMetadata; +import com.datastrato.gravitino.catalog.hive.HiveTablePropertiesMetadata.StorageFormat; +import com.google.common.annotations.VisibleForTesting; + +public class HivePropertiesConstants { + public static final String GRAVITINO_HIVE_FORMAT = HiveTablePropertiesMetadata.FORMAT; + public static final String GRAVITINO_HIVE_INPUT_FORMAT = HiveTablePropertiesMetadata.INPUT_FORMAT; + public static final String GRAVITINO_HIVE_OUTPUT_FORMAT = + HiveTablePropertiesMetadata.OUTPUT_FORMAT; + public static final String GRAVITINO_HIVE_SERDE_LIB = HiveTablePropertiesMetadata.SERDE_LIB; + public static final String GRAVITINO_HIVE_SERDE_PARAMETER_PREFIX = + HiveTablePropertiesMetadata.SERDE_PARAMETER_PREFIX; + + public static final String GRAVITINO_HIVE_FORMAT_PARQUET = StorageFormat.PARQUET.toString(); + public static final String GRAVITINO_HIVE_FORMAT_SEQUENCEFILE = + StorageFormat.SEQUENCEFILE.toString(); + public static final String GRAVITINO_HIVE_FORMAT_ORC = StorageFormat.ORC.toString(); + public static final String GRAVITINO_HIVE_FORMAT_RCFILE = StorageFormat.RCFILE.toString(); + public static final String GRAVITINO_HIVE_FORMAT_TEXTFILE = StorageFormat.TEXTFILE.toString(); + public static final String GRAVITINO_HIVE_FORMAT_AVRO = StorageFormat.AVRO.toString(); + public static final String GRAVITINO_HIVE_FORMAT_JSON = StorageFormat.JSON.toString(); + public static final String GRAVITINO_HIVE_FORMAT_CSV = StorageFormat.CSV.toString(); + + public static final String SPARK_HIVE_STORED_AS = "hive.stored-as"; + public static final String SPARK_HIVE_INPUT_FORMAT = "input-format"; + public static final String SPARK_HIVE_OUTPUT_FORMAT = "output-format"; + public static final String SPARK_HIVE_SERDE_LIB = "serde-lib"; + + @VisibleForTesting + public static final String TEXT_INPUT_FORMAT_CLASS = + HiveTablePropertiesMetadata.TEXT_INPUT_FORMAT_CLASS; + + @VisibleForTesting + public static final String IGNORE_KEY_OUTPUT_FORMAT_CLASS = + HiveTablePropertiesMetadata.IGNORE_KEY_OUTPUT_FORMAT_CLASS; + + @VisibleForTesting + public static final String LAZY_SIMPLE_SERDE_CLASS = + HiveTablePropertiesMetadata.LAZY_SIMPLE_SERDE_CLASS; + + @VisibleForTesting + public static final String PARQUET_INPUT_FORMAT_CLASS = + HiveTablePropertiesMetadata.PARQUET_INPUT_FORMAT_CLASS; + + @VisibleForTesting + public static final String PARQUET_OUTPUT_FORMAT_CLASS = + HiveTablePropertiesMetadata.PARQUET_OUTPUT_FORMAT_CLASS; + + @VisibleForTesting + public static final String PARQUET_SERDE_CLASS = HiveTablePropertiesMetadata.PARQUET_SERDE_CLASS; + + private HivePropertiesConstants() {} +} diff --git a/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/HivePropertiesConverter.java b/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/HivePropertiesConverter.java index 0c816106db2..6958ef89ca4 100644 --- a/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/HivePropertiesConverter.java +++ b/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/hive/HivePropertiesConverter.java @@ -6,19 +6,121 @@ package com.datastrato.gravitino.spark.connector.hive; import com.datastrato.gravitino.spark.connector.PropertiesConverter; -import java.util.HashMap; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableMap; +import java.util.Locale; import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import javax.ws.rs.NotSupportedException; +import org.apache.spark.sql.connector.catalog.TableCatalog; -/** Transform hive catalog properties between Spark and Gravitino. Will implement in another PR. */ +/** Transform hive catalog properties between Spark and Gravitino. */ public class HivePropertiesConverter implements PropertiesConverter { + // Transform Spark hive file format to Gravitino hive file format + static final Map fileFormatMap = + ImmutableMap.of( + "sequencefile", HivePropertiesConstants.GRAVITINO_HIVE_FORMAT_SEQUENCEFILE, + "rcfile", HivePropertiesConstants.GRAVITINO_HIVE_FORMAT_RCFILE, + "orc", HivePropertiesConstants.GRAVITINO_HIVE_FORMAT_ORC, + "parquet", HivePropertiesConstants.GRAVITINO_HIVE_FORMAT_PARQUET, + "textfile", HivePropertiesConstants.GRAVITINO_HIVE_FORMAT_TEXTFILE, + "json", HivePropertiesConstants.GRAVITINO_HIVE_FORMAT_JSON, + "csv", HivePropertiesConstants.GRAVITINO_HIVE_FORMAT_CSV, + "avro", HivePropertiesConstants.GRAVITINO_HIVE_FORMAT_AVRO); + + static final Map sparkToGravitinoPropertyMap = + ImmutableMap.of( + "hive.output-format", + HivePropertiesConstants.GRAVITINO_HIVE_OUTPUT_FORMAT, + "hive.input-format", + HivePropertiesConstants.GRAVITINO_HIVE_INPUT_FORMAT, + "hive.serde", + HivePropertiesConstants.GRAVITINO_HIVE_SERDE_LIB); + + /** + * CREATE TABLE xxx STORED AS PARQUET will save "hive.stored-as" = "PARQUET" in property. + * + *

CREATE TABLE xxx USING PARQUET will save "provider" = "PARQUET" in property. + * + *

CREATE TABLE xxx ROW FORMAT SERDE xx STORED AS INPUTFORMAT xx OUTPUTFORMAT xx will save + * "hive.input-format", "hive.output-format", "hive.serde" in property. + * + *

CREATE TABLE xxx ROW FORMAT DELIMITED FIELDS TERMINATED xx will save "option.field.delim" in + * property. + * + *

Please refer to + * https://github.com/apache/spark/blob/7d87a94dd77f43120701e48a371324a4f5f2064b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala#L397 + * for more details. + */ @Override public Map toGravitinoTableProperties(Map properties) { - return new HashMap<>(properties); + Map gravitinoTableProperties = fromOptionProperties(properties); + String provider = gravitinoTableProperties.get(TableCatalog.PROP_PROVIDER); + String storeAs = gravitinoTableProperties.get(HivePropertiesConstants.SPARK_HIVE_STORED_AS); + String fileFormat = Optional.ofNullable(storeAs).orElse(provider); + if (fileFormat != null) { + String gravitinoFormat = fileFormatMap.get(fileFormat.toLowerCase(Locale.ROOT)); + if (gravitinoFormat != null) { + gravitinoTableProperties.put( + HivePropertiesConstants.GRAVITINO_HIVE_FORMAT, gravitinoFormat); + } else { + throw new NotSupportedException("Doesn't support hive file format: " + fileFormat); + } + } + + sparkToGravitinoPropertyMap.forEach( + (sparkProperty, gravitinoProperty) -> { + if (gravitinoTableProperties.containsKey(sparkProperty)) { + String value = gravitinoTableProperties.remove(sparkProperty); + gravitinoTableProperties.put(gravitinoProperty, value); + } + }); + + return gravitinoTableProperties; } @Override public Map toSparkTableProperties(Map properties) { - return new HashMap<>(properties); + return toOptionProperties(properties); + } + + @VisibleForTesting + static Map toOptionProperties(Map properties) { + return properties.entrySet().stream() + .collect( + Collectors.toMap( + entry -> { + String key = entry.getKey(); + if (key.startsWith( + HivePropertiesConstants.GRAVITINO_HIVE_SERDE_PARAMETER_PREFIX)) { + return TableCatalog.OPTION_PREFIX + + key.substring( + HivePropertiesConstants.GRAVITINO_HIVE_SERDE_PARAMETER_PREFIX.length()); + } else { + return key; + } + }, + entry -> entry.getValue(), + (existingValue, newValue) -> newValue)); + } + + @VisibleForTesting + static Map fromOptionProperties(Map properties) { + return properties.entrySet().stream() + .collect( + Collectors.toMap( + entry -> { + String key = entry.getKey(); + if (key.startsWith(TableCatalog.OPTION_PREFIX)) { + return HivePropertiesConstants.GRAVITINO_HIVE_SERDE_PARAMETER_PREFIX + + key.substring(TableCatalog.OPTION_PREFIX.length()); + } else { + return key; + } + }, + entry -> entry.getValue(), + (existingValue, newValue) -> newValue)); } } diff --git a/spark-connector/src/test/java/com/datastrato/gravitino/spark/connector/hive/TestHivePropertiesConverter.java b/spark-connector/src/test/java/com/datastrato/gravitino/spark/connector/hive/TestHivePropertiesConverter.java new file mode 100644 index 00000000000..2a04915d917 --- /dev/null +++ b/spark-connector/src/test/java/com/datastrato/gravitino/spark/connector/hive/TestHivePropertiesConverter.java @@ -0,0 +1,101 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ + +package com.datastrato.gravitino.spark.connector.hive; + +import com.google.common.collect.ImmutableMap; +import java.util.Map; +import javax.ws.rs.NotSupportedException; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.TestInstance.Lifecycle; + +@TestInstance(Lifecycle.PER_CLASS) +public class TestHivePropertiesConverter { + HivePropertiesConverter hivePropertiesConverter = new HivePropertiesConverter(); + + @Test + void testTableFormat() { + // stored as + Map hiveProperties = + hivePropertiesConverter.toGravitinoTableProperties( + ImmutableMap.of(HivePropertiesConstants.SPARK_HIVE_STORED_AS, "PARQUET")); + Assertions.assertEquals( + hiveProperties.get(HivePropertiesConstants.GRAVITINO_HIVE_FORMAT), "PARQUET"); + Assertions.assertThrowsExactly( + NotSupportedException.class, + () -> + hivePropertiesConverter.toGravitinoTableProperties( + ImmutableMap.of(HivePropertiesConstants.SPARK_HIVE_STORED_AS, "notExists"))); + + // using + hiveProperties = + hivePropertiesConverter.toGravitinoTableProperties( + ImmutableMap.of(TableCatalog.PROP_PROVIDER, "PARQUET")); + Assertions.assertEquals( + hiveProperties.get(HivePropertiesConstants.GRAVITINO_HIVE_FORMAT), "PARQUET"); + Assertions.assertThrowsExactly( + NotSupportedException.class, + () -> + hivePropertiesConverter.toGravitinoTableProperties( + ImmutableMap.of(TableCatalog.PROP_PROVIDER, "notExists"))); + + // row format + hiveProperties = + hivePropertiesConverter.toGravitinoTableProperties( + ImmutableMap.of( + "hive.input-format", "a", "hive.output-format", "b", "hive.serde", "c")); + Assertions.assertEquals( + ImmutableMap.of( + HivePropertiesConstants.GRAVITINO_HIVE_INPUT_FORMAT, + "a", + HivePropertiesConstants.GRAVITINO_HIVE_OUTPUT_FORMAT, + "b", + HivePropertiesConstants.GRAVITINO_HIVE_SERDE_LIB, + "c"), + hiveProperties); + + hiveProperties = + hivePropertiesConverter.toGravitinoTableProperties( + ImmutableMap.of(TableCatalog.OPTION_PREFIX + "a", "a", "b", "b")); + Assertions.assertEquals( + ImmutableMap.of( + HivePropertiesConstants.GRAVITINO_HIVE_SERDE_PARAMETER_PREFIX + "a", "a", "b", "b"), + hiveProperties); + + hiveProperties = + hivePropertiesConverter.toSparkTableProperties( + ImmutableMap.of( + HivePropertiesConstants.GRAVITINO_HIVE_SERDE_PARAMETER_PREFIX + "a", + "a", + "b", + "b")); + Assertions.assertEquals( + ImmutableMap.of(TableCatalog.OPTION_PREFIX + "a", "a", "b", "b"), hiveProperties); + } + + @Test + void testOptionProperties() { + Map properties = + HivePropertiesConverter.fromOptionProperties( + ImmutableMap.of(TableCatalog.OPTION_PREFIX + "a", "1", "b", "2")); + Assertions.assertEquals( + ImmutableMap.of( + HivePropertiesConstants.GRAVITINO_HIVE_SERDE_PARAMETER_PREFIX + "a", "1", "b", "2"), + properties); + + properties = + HivePropertiesConverter.toOptionProperties( + ImmutableMap.of( + HivePropertiesConstants.GRAVITINO_HIVE_SERDE_PARAMETER_PREFIX + "a", + "1", + "b", + "2")); + Assertions.assertEquals( + ImmutableMap.of(TableCatalog.OPTION_PREFIX + "a", "1", "b", "2"), properties); + } +} From 0d36d2f4b3781359242879e5278f575ec62c62fa Mon Sep 17 00:00:00 2001 From: mchades Date: Fri, 29 Mar 2024 18:33:51 +0800 Subject: [PATCH 4/4] [#2727] refactor(core,server): refactor CatalogOperationDispatcher (#2728) ### What changes were proposed in this pull request? Split CatalogOperationDispatcher into separate classes based on different operations. ### Why are the changes needed? Fix: #2727 ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? UTs --- .../datastrato/gravitino/GravitinoEnv.java | 44 +- .../catalog/CatalogOperationDispatcher.java | 946 ------------------ .../catalog/FilesetOperationDispatcher.java | 184 ++++ .../catalog/OperationDispatcher.java | 265 +++++ .../catalog/SchemaOperationDispatcher.java | 315 ++++++ .../catalog/TableOperationDispatcher.java | 354 +++++++ .../TestCatalogOperationDispatcher.java | 693 ------------- .../TestFilesetOperationDispatcher.java | 147 +++ .../catalog/TestOperationDispatcher.java | 132 +++ .../TestSchemaOperationDispatcher.java | 199 ++++ .../catalog/TestTableOperationDispatcher.java | 277 +++++ .../gravitino/server/GravitinoServer.java | 14 +- .../server/web/rest/FilesetOperations.java | 6 +- .../server/web/rest/PartitionOperations.java | 6 +- .../server/web/rest/SchemaOperations.java | 6 +- .../server/web/rest/TableOperations.java | 6 +- .../web/rest/TestFilesetOperations.java | 6 +- .../web/rest/TestPartitionOperations.java | 6 +- .../server/web/rest/TestSchemaOperations.java | 6 +- .../server/web/rest/TestTableOperations.java | 6 +- 20 files changed, 1944 insertions(+), 1674 deletions(-) delete mode 100644 core/src/main/java/com/datastrato/gravitino/catalog/CatalogOperationDispatcher.java create mode 100644 core/src/main/java/com/datastrato/gravitino/catalog/FilesetOperationDispatcher.java create mode 100644 core/src/main/java/com/datastrato/gravitino/catalog/OperationDispatcher.java create mode 100644 core/src/main/java/com/datastrato/gravitino/catalog/SchemaOperationDispatcher.java create mode 100644 core/src/main/java/com/datastrato/gravitino/catalog/TableOperationDispatcher.java delete mode 100644 core/src/test/java/com/datastrato/gravitino/catalog/TestCatalogOperationDispatcher.java create mode 100644 core/src/test/java/com/datastrato/gravitino/catalog/TestFilesetOperationDispatcher.java create mode 100644 core/src/test/java/com/datastrato/gravitino/catalog/TestOperationDispatcher.java create mode 100644 core/src/test/java/com/datastrato/gravitino/catalog/TestSchemaOperationDispatcher.java create mode 100644 core/src/test/java/com/datastrato/gravitino/catalog/TestTableOperationDispatcher.java diff --git a/core/src/main/java/com/datastrato/gravitino/GravitinoEnv.java b/core/src/main/java/com/datastrato/gravitino/GravitinoEnv.java index 7537a1e8d5b..bd23f9bff40 100644 --- a/core/src/main/java/com/datastrato/gravitino/GravitinoEnv.java +++ b/core/src/main/java/com/datastrato/gravitino/GravitinoEnv.java @@ -7,7 +7,9 @@ import com.datastrato.gravitino.authorization.AccessControlManager; import com.datastrato.gravitino.auxiliary.AuxiliaryServiceManager; import com.datastrato.gravitino.catalog.CatalogManager; -import com.datastrato.gravitino.catalog.CatalogOperationDispatcher; +import com.datastrato.gravitino.catalog.FilesetOperationDispatcher; +import com.datastrato.gravitino.catalog.SchemaOperationDispatcher; +import com.datastrato.gravitino.catalog.TableOperationDispatcher; import com.datastrato.gravitino.lock.LockManager; import com.datastrato.gravitino.metalake.MetalakeManager; import com.datastrato.gravitino.metrics.MetricsSystem; @@ -32,7 +34,11 @@ public class GravitinoEnv { private CatalogManager catalogManager; - private CatalogOperationDispatcher catalogOperationDispatcher; + private SchemaOperationDispatcher schemaOperationDispatcher; + + private TableOperationDispatcher tableOperationDispatcher; + + private FilesetOperationDispatcher filesetOperationDispatcher; private MetalakeManager metalakeManager; @@ -97,8 +103,12 @@ public void initialize(Config config) { // Create and initialize Catalog related modules this.catalogManager = new CatalogManager(config, entityStore, idGenerator); - this.catalogOperationDispatcher = - new CatalogOperationDispatcher(catalogManager, entityStore, idGenerator); + this.schemaOperationDispatcher = + new SchemaOperationDispatcher(catalogManager, entityStore, idGenerator); + this.tableOperationDispatcher = + new TableOperationDispatcher(catalogManager, entityStore, idGenerator); + this.filesetOperationDispatcher = + new FilesetOperationDispatcher(catalogManager, entityStore, idGenerator); // Create and initialize access control related modules this.accessControlManager = new AccessControlManager(entityStore, idGenerator); @@ -141,12 +151,30 @@ public CatalogManager catalogManager() { } /** - * Get the CatalogOperationDispatcher associated with the Gravitino environment. + * Get the SchemaOperationDispatcher associated with the Gravitino environment. + * + * @return The SchemaOperationDispatcher instance. + */ + public SchemaOperationDispatcher schemaOperationDispatcher() { + return schemaOperationDispatcher; + } + + /** + * Get the TableOperationDispatcher associated with the Gravitino environment. + * + * @return The TableOperationDispatcher instance. + */ + public TableOperationDispatcher tableOperationDispatcher() { + return tableOperationDispatcher; + } + + /** + * Get the FilesetOperationDispatcher associated with the Gravitino environment. * - * @return The CatalogOperationDispatcher instance. + * @return The FilesetOperationDispatcher instance. */ - public CatalogOperationDispatcher catalogOperationDispatcher() { - return catalogOperationDispatcher; + public FilesetOperationDispatcher filesetOperationDispatcher() { + return filesetOperationDispatcher; } /** diff --git a/core/src/main/java/com/datastrato/gravitino/catalog/CatalogOperationDispatcher.java b/core/src/main/java/com/datastrato/gravitino/catalog/CatalogOperationDispatcher.java deleted file mode 100644 index 5cfa5e18efe..00000000000 --- a/core/src/main/java/com/datastrato/gravitino/catalog/CatalogOperationDispatcher.java +++ /dev/null @@ -1,946 +0,0 @@ -/* - * Copyright 2023 Datastrato Pvt Ltd. - * This software is licensed under the Apache License version 2. - */ -package com.datastrato.gravitino.catalog; - -import static com.datastrato.gravitino.Entity.EntityType.SCHEMA; -import static com.datastrato.gravitino.Entity.EntityType.TABLE; -import static com.datastrato.gravitino.catalog.PropertiesMetadataHelpers.validatePropertyForAlter; -import static com.datastrato.gravitino.catalog.PropertiesMetadataHelpers.validatePropertyForCreate; -import static com.datastrato.gravitino.rel.expressions.transforms.Transforms.EMPTY_TRANSFORM; - -import com.datastrato.gravitino.EntityStore; -import com.datastrato.gravitino.HasIdentifier; -import com.datastrato.gravitino.NameIdentifier; -import com.datastrato.gravitino.Namespace; -import com.datastrato.gravitino.StringIdentifier; -import com.datastrato.gravitino.connector.BasePropertiesMetadata; -import com.datastrato.gravitino.connector.HasPropertyMetadata; -import com.datastrato.gravitino.connector.PropertiesMetadata; -import com.datastrato.gravitino.exceptions.FilesetAlreadyExistsException; -import com.datastrato.gravitino.exceptions.IllegalNameIdentifierException; -import com.datastrato.gravitino.exceptions.NoSuchCatalogException; -import com.datastrato.gravitino.exceptions.NoSuchEntityException; -import com.datastrato.gravitino.exceptions.NoSuchFilesetException; -import com.datastrato.gravitino.exceptions.NoSuchSchemaException; -import com.datastrato.gravitino.exceptions.NoSuchTableException; -import com.datastrato.gravitino.exceptions.NonEmptyEntityException; -import com.datastrato.gravitino.exceptions.NonEmptySchemaException; -import com.datastrato.gravitino.exceptions.SchemaAlreadyExistsException; -import com.datastrato.gravitino.exceptions.TableAlreadyExistsException; -import com.datastrato.gravitino.file.Fileset; -import com.datastrato.gravitino.file.FilesetCatalog; -import com.datastrato.gravitino.file.FilesetChange; -import com.datastrato.gravitino.meta.AuditInfo; -import com.datastrato.gravitino.meta.SchemaEntity; -import com.datastrato.gravitino.meta.TableEntity; -import com.datastrato.gravitino.rel.Column; -import com.datastrato.gravitino.rel.Schema; -import com.datastrato.gravitino.rel.SchemaChange; -import com.datastrato.gravitino.rel.SupportsSchemas; -import com.datastrato.gravitino.rel.Table; -import com.datastrato.gravitino.rel.TableCatalog; -import com.datastrato.gravitino.rel.TableChange; -import com.datastrato.gravitino.rel.expressions.distributions.Distribution; -import com.datastrato.gravitino.rel.expressions.distributions.Distributions; -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.rel.indexes.Indexes; -import com.datastrato.gravitino.storage.IdGenerator; -import com.datastrato.gravitino.utils.PrincipalUtils; -import com.datastrato.gravitino.utils.ThrowableFunction; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Maps; -import java.time.Instant; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A catalog operation dispatcher that dispatches the catalog operations to the underlying catalog - * implementation. - */ -public class CatalogOperationDispatcher implements TableCatalog, FilesetCatalog, SupportsSchemas { - - private static final Logger LOG = LoggerFactory.getLogger(CatalogOperationDispatcher.class); - - private final CatalogManager catalogManager; - - private final EntityStore store; - - private final IdGenerator idGenerator; - - /** - * Creates a new CatalogOperationDispatcher instance. - * - * @param catalogManager The CatalogManager instance to be used for catalog operations. - * @param store The EntityStore instance to be used for catalog operations. - * @param idGenerator The IdGenerator instance to be used for catalog operations. - */ - public CatalogOperationDispatcher( - CatalogManager catalogManager, EntityStore store, IdGenerator idGenerator) { - this.catalogManager = catalogManager; - this.store = store; - this.idGenerator = idGenerator; - } - - /** - * Lists the schemas within the specified namespace. - * - * @param namespace The namespace in which to list schemas. - * @return An array of NameIdentifier objects representing the schemas within the specified - * namespace. - * @throws NoSuchCatalogException If the catalog namespace does not exist. - */ - @Override - public NameIdentifier[] listSchemas(Namespace namespace) throws NoSuchCatalogException { - return doWithCatalog( - getCatalogIdentifier(NameIdentifier.of(namespace.levels())), - c -> c.doWithSchemaOps(s -> s.listSchemas(namespace)), - NoSuchCatalogException.class); - } - - /** - * Creates a new schema. - * - * @param ident The identifier for the schema to be created. - * @param comment The comment for the new schema. - * @param properties Additional properties for the new schema. - * @return The created Schema object. - * @throws NoSuchCatalogException If the catalog corresponding to the provided identifier does not - * exist. - * @throws SchemaAlreadyExistsException If a schema with the same identifier already exists. - */ - @Override - public Schema createSchema(NameIdentifier ident, String comment, Map properties) - throws NoSuchCatalogException, SchemaAlreadyExistsException { - NameIdentifier catalogIdent = getCatalogIdentifier(ident); - doWithCatalog( - catalogIdent, - c -> - c.doWithPropertiesMeta( - p -> { - validatePropertyForCreate(p.schemaPropertiesMetadata(), properties); - return null; - }), - IllegalArgumentException.class); - long uid = idGenerator.nextId(); - // Add StringIdentifier to the properties, the specific catalog will handle this - // StringIdentifier to make sure only when the operation is successful, the related - // SchemaEntity will be visible. - StringIdentifier stringId = StringIdentifier.fromId(uid); - Map updatedProperties = - StringIdentifier.newPropertiesWithId(stringId, properties); - - Schema createdSchema = - doWithCatalog( - catalogIdent, - c -> c.doWithSchemaOps(s -> s.createSchema(ident, comment, updatedProperties)), - NoSuchCatalogException.class, - SchemaAlreadyExistsException.class); - - // If the Schema is maintained by the Gravitino's store, we don't have to store again. - boolean isManagedSchema = isManagedEntity(createdSchema.properties()); - if (isManagedSchema) { - return EntityCombinedSchema.of(createdSchema) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - catalogIdent, - HasPropertyMetadata::schemaPropertiesMetadata, - createdSchema.properties())); - } - - // Retrieve the Schema again to obtain some values generated by underlying catalog - Schema schema = - doWithCatalog( - catalogIdent, - c -> c.doWithSchemaOps(s -> s.loadSchema(ident)), - NoSuchSchemaException.class); - - SchemaEntity schemaEntity = - SchemaEntity.builder() - .withId(uid) - .withName(ident.name()) - .withNamespace(ident.namespace()) - .withAuditInfo( - AuditInfo.builder() - .withCreator(PrincipalUtils.getCurrentPrincipal().getName()) - .withCreateTime(Instant.now()) - .build()) - .build(); - - try { - store.put(schemaEntity, true /* overwrite */); - } catch (Exception e) { - LOG.error(FormattedErrorMessages.STORE_OP_FAILURE, "put", ident, e); - return EntityCombinedSchema.of(schema) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - catalogIdent, - HasPropertyMetadata::schemaPropertiesMetadata, - schema.properties())); - } - - // Merge both the metadata from catalog operation and the metadata from entity store. - return EntityCombinedSchema.of(schema, schemaEntity) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - catalogIdent, HasPropertyMetadata::schemaPropertiesMetadata, schema.properties())); - } - - /** - * Loads and retrieves a schema. - * - * @param ident The identifier of the schema to be loaded. - * @return The loaded Schema object. - * @throws NoSuchSchemaException If the schema does not exist. - */ - @Override - public Schema loadSchema(NameIdentifier ident) throws NoSuchSchemaException { - NameIdentifier catalogIdentifier = getCatalogIdentifier(ident); - Schema schema = - doWithCatalog( - catalogIdentifier, - c -> c.doWithSchemaOps(s -> s.loadSchema(ident)), - NoSuchSchemaException.class); - - // If the Schema is maintained by the Gravitino's store, we don't have to load again. - boolean isManagedSchema = isManagedEntity(schema.properties()); - if (isManagedSchema) { - return EntityCombinedSchema.of(schema) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - catalogIdentifier, - HasPropertyMetadata::schemaPropertiesMetadata, - schema.properties())); - } - - StringIdentifier stringId = getStringIdFromProperties(schema.properties()); - // Case 1: The schema is not created by Gravitino. - if (stringId == null) { - return EntityCombinedSchema.of(schema) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - catalogIdentifier, - HasPropertyMetadata::schemaPropertiesMetadata, - schema.properties())); - } - - SchemaEntity schemaEntity = - operateOnEntity( - ident, - identifier -> store.get(identifier, SCHEMA, SchemaEntity.class), - "GET", - stringId.id()); - return EntityCombinedSchema.of(schema, schemaEntity) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - catalogIdentifier, - HasPropertyMetadata::schemaPropertiesMetadata, - schema.properties())); - } - - /** - * Alters the schema by applying the provided schema changes. - * - * @param ident The identifier of the schema to be altered. - * @param changes The array of SchemaChange objects representing the alterations to apply. - * @return The altered Schema object. - * @throws NoSuchSchemaException If the schema corresponding to the provided identifier does not - * exist. - */ - @Override - public Schema alterSchema(NameIdentifier ident, SchemaChange... changes) - throws NoSuchSchemaException { - validateAlterProperties(ident, HasPropertyMetadata::schemaPropertiesMetadata, changes); - - NameIdentifier catalogIdent = getCatalogIdentifier(ident); - Schema tempAlteredSchema = - doWithCatalog( - catalogIdent, - c -> c.doWithSchemaOps(s -> s.alterSchema(ident, changes)), - NoSuchSchemaException.class); - - // Retrieve the Schema again to obtain some values generated by underlying catalog - Schema alteredSchema = - doWithCatalog( - catalogIdent, - c -> - c.doWithSchemaOps( - s -> - s.loadSchema( - NameIdentifier.of(ident.namespace(), tempAlteredSchema.name()))), - NoSuchSchemaException.class); - - // If the Schema is maintained by the Gravitino's store, we don't have to alter again. - boolean isManagedSchema = isManagedEntity(alteredSchema.properties()); - if (isManagedSchema) { - return EntityCombinedSchema.of(alteredSchema) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - catalogIdent, - HasPropertyMetadata::schemaPropertiesMetadata, - alteredSchema.properties())); - } - - StringIdentifier stringId = getStringIdFromProperties(alteredSchema.properties()); - // Case 1: The schema is not created by Gravitino. - if (stringId == null) { - return EntityCombinedSchema.of(alteredSchema) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - catalogIdent, - HasPropertyMetadata::schemaPropertiesMetadata, - alteredSchema.properties())); - } - - SchemaEntity updatedSchemaEntity = - operateOnEntity( - ident, - id -> - store.update( - id, - SchemaEntity.class, - SCHEMA, - schemaEntity -> - SchemaEntity.builder() - .withId(schemaEntity.id()) - .withName(schemaEntity.name()) - .withNamespace(ident.namespace()) - .withAuditInfo( - AuditInfo.builder() - .withCreator(schemaEntity.auditInfo().creator()) - .withCreateTime(schemaEntity.auditInfo().createTime()) - .withLastModifier( - PrincipalUtils.getCurrentPrincipal().getName()) - .withLastModifiedTime(Instant.now()) - .build()) - .build()), - "UPDATE", - stringId.id()); - return EntityCombinedSchema.of(alteredSchema, updatedSchemaEntity) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - catalogIdent, - HasPropertyMetadata::schemaPropertiesMetadata, - alteredSchema.properties())); - } - - /** - * Drops a schema. - * - * @param ident The identifier of the schema to be dropped. - * @param cascade If true, drops all tables within the schema as well. - * @return True if the schema was successfully dropped, false otherwise. - * @throws NonEmptySchemaException If the schema contains tables and cascade is set to false. - */ - @Override - public boolean dropSchema(NameIdentifier ident, boolean cascade) throws NonEmptySchemaException { - boolean dropped = - doWithCatalog( - getCatalogIdentifier(ident), - c -> c.doWithSchemaOps(s -> s.dropSchema(ident, cascade)), - NonEmptySchemaException.class); - - if (!dropped) { - return false; - } - - try { - return store.delete(ident, SCHEMA, cascade); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - /** - * Lists the tables within a schema. - * - * @param namespace The namespace of the schema containing the tables. - * @return An array of {@link NameIdentifier} objects representing the identifiers of the tables - * in the schema. - * @throws NoSuchSchemaException If the specified schema does not exist. - */ - @Override - public NameIdentifier[] listTables(Namespace namespace) throws NoSuchSchemaException { - return doWithCatalog( - getCatalogIdentifier(NameIdentifier.of(namespace.levels())), - c -> c.doWithTableOps(t -> t.listTables(namespace)), - NoSuchSchemaException.class); - } - - /** - * Loads a table. - * - * @param ident The identifier of the table to load. - * @return The loaded {@link Table} object representing the requested table. - * @throws NoSuchTableException If the specified table does not exist. - */ - @Override - public Table loadTable(NameIdentifier ident) throws NoSuchTableException { - NameIdentifier catalogIdentifier = getCatalogIdentifier(ident); - Table table = - doWithCatalog( - catalogIdentifier, - c -> c.doWithTableOps(t -> t.loadTable(ident)), - NoSuchTableException.class); - - StringIdentifier stringId = getStringIdFromProperties(table.properties()); - // Case 1: The table is not created by Gravitino. - if (stringId == null) { - return EntityCombinedTable.of(table) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - catalogIdentifier, - HasPropertyMetadata::tablePropertiesMetadata, - table.properties())); - } - - TableEntity tableEntity = - operateOnEntity( - ident, - identifier -> store.get(identifier, TABLE, TableEntity.class), - "GET", - stringId.id()); - - return EntityCombinedTable.of(table, tableEntity) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - catalogIdentifier, - HasPropertyMetadata::tablePropertiesMetadata, - table.properties())); - } - - /** - * Creates a new table in a schema. - * - * @param ident The identifier of the table to create. - * @param columns An array of {@link Column} objects representing the columns of the table. - * @param comment A description or comment associated with the table. - * @param properties Additional properties to set for the table. - * @param partitions An array of {@link Transform} objects representing the partitioning of table - * @param indexes An array of {@link Index} objects representing the indexes of the table. - * @return The newly created {@link Table} object. - * @throws NoSuchSchemaException If the schema in which to create the table does not exist. - * @throws TableAlreadyExistsException If a table with the same name already exists in the schema. - */ - @Override - public Table createTable( - NameIdentifier ident, - Column[] columns, - String comment, - Map properties, - Transform[] partitions, - Distribution distribution, - SortOrder[] sortOrders, - Index[] indexes) - throws NoSuchSchemaException, TableAlreadyExistsException { - NameIdentifier catalogIdent = getCatalogIdentifier(ident); - doWithCatalog( - catalogIdent, - c -> - c.doWithPropertiesMeta( - p -> { - validatePropertyForCreate(p.tablePropertiesMetadata(), properties); - return null; - }), - IllegalArgumentException.class); - long uid = idGenerator.nextId(); - // Add StringIdentifier to the properties, the specific catalog will handle this - // StringIdentifier to make sure only when the operation is successful, the related - // TableEntity will be visible. - StringIdentifier stringId = StringIdentifier.fromId(uid); - Map updatedProperties = - StringIdentifier.newPropertiesWithId(stringId, properties); - - doWithCatalog( - catalogIdent, - c -> - c.doWithTableOps( - t -> - t.createTable( - ident, - columns, - comment, - updatedProperties, - partitions == null ? EMPTY_TRANSFORM : partitions, - distribution == null ? Distributions.NONE : distribution, - sortOrders == null ? new SortOrder[0] : sortOrders, - indexes == null ? Indexes.EMPTY_INDEXES : indexes)), - NoSuchSchemaException.class, - TableAlreadyExistsException.class); - - // Retrieve the Table again to obtain some values generated by underlying catalog - Table table = - doWithCatalog( - catalogIdent, - c -> c.doWithTableOps(t -> t.loadTable(ident)), - NoSuchTableException.class); - - TableEntity tableEntity = - TableEntity.builder() - .withId(uid) - .withName(ident.name()) - .withNamespace(ident.namespace()) - .withAuditInfo( - AuditInfo.builder() - .withCreator(PrincipalUtils.getCurrentPrincipal().getName()) - .withCreateTime(Instant.now()) - .build()) - .build(); - - try { - store.put(tableEntity, true /* overwrite */); - } catch (Exception e) { - LOG.error(FormattedErrorMessages.STORE_OP_FAILURE, "put", ident, e); - return EntityCombinedTable.of(table) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - catalogIdent, HasPropertyMetadata::tablePropertiesMetadata, table.properties())); - } - - return EntityCombinedTable.of(table, tableEntity) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - catalogIdent, HasPropertyMetadata::tablePropertiesMetadata, table.properties())); - } - - /** - * Alters an existing table. - * - * @param ident The identifier of the table to alter. - * @param changes An array of {@link TableChange} objects representing the changes to apply to the - * table. - * @return The altered {@link Table} object after applying the changes. - * @throws NoSuchTableException If the table to alter does not exist. - * @throws IllegalArgumentException If an unsupported or invalid change is specified. - */ - @Override - public Table alterTable(NameIdentifier ident, TableChange... changes) - throws NoSuchTableException, IllegalArgumentException { - validateAlterProperties(ident, HasPropertyMetadata::tablePropertiesMetadata, changes); - - NameIdentifier catalogIdent = getCatalogIdentifier(ident); - Table tempAlteredTable = - doWithCatalog( - catalogIdent, - c -> c.doWithTableOps(t -> t.alterTable(ident, changes)), - NoSuchTableException.class, - IllegalArgumentException.class); - - // Retrieve the Table again to obtain some values generated by underlying catalog - Table alteredTable = - doWithCatalog( - catalogIdent, - c -> - c.doWithTableOps( - t -> - t.loadTable(NameIdentifier.of(ident.namespace(), tempAlteredTable.name()))), - NoSuchTableException.class); - - StringIdentifier stringId = getStringIdFromProperties(alteredTable.properties()); - // Case 1: The table is not created by Gravitino. - if (stringId == null) { - return EntityCombinedTable.of(alteredTable) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - getCatalogIdentifier(ident), - HasPropertyMetadata::tablePropertiesMetadata, - alteredTable.properties())); - } - - TableEntity updatedTableEntity = - operateOnEntity( - ident, - id -> - store.update( - id, - TableEntity.class, - TABLE, - tableEntity -> { - String newName = - Arrays.stream(changes) - .filter(c -> c instanceof TableChange.RenameTable) - .map(c -> ((TableChange.RenameTable) c).getNewName()) - .reduce((c1, c2) -> c2) - .orElse(tableEntity.name()); - - return TableEntity.builder() - .withId(tableEntity.id()) - .withName(newName) - .withNamespace(ident.namespace()) - .withAuditInfo( - AuditInfo.builder() - .withCreator(tableEntity.auditInfo().creator()) - .withCreateTime(tableEntity.auditInfo().createTime()) - .withLastModifier(PrincipalUtils.getCurrentPrincipal().getName()) - .withLastModifiedTime(Instant.now()) - .build()) - .build(); - }), - "UPDATE", - stringId.id()); - - return EntityCombinedTable.of(alteredTable, updatedTableEntity) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - getCatalogIdentifier(ident), - HasPropertyMetadata::tablePropertiesMetadata, - alteredTable.properties())); - } - - /** - * Drops a table from the catalog. - * - * @param ident The identifier of the table to drop. - * @return {@code true} if the table was successfully dropped, {@code false} if the table does not - * exist. - * @throws NoSuchTableException If the table to drop does not exist. - */ - @Override - public boolean dropTable(NameIdentifier ident) { - boolean dropped = - doWithCatalog( - getCatalogIdentifier(ident), - c -> c.doWithTableOps(t -> t.dropTable(ident)), - NoSuchTableException.class); - - if (!dropped) { - return false; - } - - try { - store.delete(ident, TABLE); - } catch (Exception e) { - throw new RuntimeException(e); - } - - return true; - } - - @Override - public boolean purgeTable(NameIdentifier ident) throws UnsupportedOperationException { - boolean purged = - doWithCatalog( - getCatalogIdentifier(ident), - c -> c.doWithTableOps(t -> t.purgeTable(ident)), - NoSuchTableException.class, - UnsupportedOperationException.class); - - if (!purged) { - return false; - } - - try { - store.delete(ident, TABLE); - } catch (Exception e) { - throw new RuntimeException(e); - } - - return true; - } - - @Override - public NameIdentifier[] listFilesets(Namespace namespace) throws NoSuchSchemaException { - return doWithCatalog( - getCatalogIdentifier(NameIdentifier.of(namespace.levels())), - c -> c.doWithFilesetOps(f -> f.listFilesets(namespace)), - NoSuchSchemaException.class); - } - - @Override - public Fileset loadFileset(NameIdentifier ident) throws NoSuchFilesetException { - NameIdentifier catalogIdent = getCatalogIdentifier(ident); - Fileset fileset = - doWithCatalog( - catalogIdent, - c -> c.doWithFilesetOps(f -> f.loadFileset(ident)), - NoSuchFilesetException.class); - - // Currently we only support maintaining the Fileset in the Gravitino's store. - return EntityCombinedFileset.of(fileset) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - catalogIdent, - HasPropertyMetadata::filesetPropertiesMetadata, - fileset.properties())); - } - - @Override - public Fileset createFileset( - NameIdentifier ident, - String comment, - Fileset.Type type, - String storageLocation, - Map properties) - throws NoSuchSchemaException, FilesetAlreadyExistsException { - NameIdentifier catalogIdent = getCatalogIdentifier(ident); - doWithCatalog( - catalogIdent, - c -> - c.doWithPropertiesMeta( - p -> { - validatePropertyForCreate(p.filesetPropertiesMetadata(), properties); - return null; - }), - IllegalArgumentException.class); - long uid = idGenerator.nextId(); - StringIdentifier stringId = StringIdentifier.fromId(uid); - Map updatedProperties = - StringIdentifier.newPropertiesWithId(stringId, properties); - - Fileset createdFileset = - doWithCatalog( - catalogIdent, - c -> - c.doWithFilesetOps( - f -> f.createFileset(ident, comment, type, storageLocation, updatedProperties)), - NoSuchSchemaException.class, - FilesetAlreadyExistsException.class); - return EntityCombinedFileset.of(createdFileset) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - catalogIdent, - HasPropertyMetadata::filesetPropertiesMetadata, - createdFileset.properties())); - } - - @Override - public Fileset alterFileset(NameIdentifier ident, FilesetChange... changes) - throws NoSuchFilesetException, IllegalArgumentException { - validateAlterProperties(ident, HasPropertyMetadata::filesetPropertiesMetadata, changes); - - NameIdentifier catalogIdent = getCatalogIdentifier(ident); - Fileset alteredFileset = - doWithCatalog( - catalogIdent, - c -> c.doWithFilesetOps(f -> f.alterFileset(ident, changes)), - NoSuchFilesetException.class, - IllegalArgumentException.class); - return EntityCombinedFileset.of(alteredFileset) - .withHiddenPropertiesSet( - getHiddenPropertyNames( - catalogIdent, - HasPropertyMetadata::filesetPropertiesMetadata, - alteredFileset.properties())); - } - - @Override - public boolean dropFileset(NameIdentifier ident) { - return doWithCatalog( - getCatalogIdentifier(ident), - c -> c.doWithFilesetOps(f -> f.dropFileset(ident)), - NonEmptyEntityException.class); - } - - private R doWithCatalog( - NameIdentifier ident, ThrowableFunction fn, Class ex) - throws E { - try { - CatalogManager.CatalogWrapper c = catalogManager.loadCatalogAndWrap(ident); - return fn.apply(c); - } catch (Throwable throwable) { - if (ex.isInstance(throwable)) { - throw ex.cast(throwable); - } - throw new RuntimeException(throwable); - } - } - - private R doWithCatalog( - NameIdentifier ident, - ThrowableFunction fn, - Class ex1, - Class ex2) - throws E1, E2 { - try { - CatalogManager.CatalogWrapper c = catalogManager.loadCatalogAndWrap(ident); - return fn.apply(c); - } catch (Throwable throwable) { - if (ex1.isInstance(throwable)) { - throw ex1.cast(throwable); - } else if (ex2.isInstance(throwable)) { - throw ex2.cast(throwable); - } - if (RuntimeException.class.isAssignableFrom(throwable.getClass())) { - throw (RuntimeException) throwable; - } - - throw new RuntimeException(throwable); - } - } - - private Set getHiddenPropertyNames( - NameIdentifier catalogIdent, - ThrowableFunction provider, - Map properties) { - return doWithCatalog( - catalogIdent, - c -> - c.doWithPropertiesMeta( - p -> { - PropertiesMetadata propertiesMetadata = provider.apply(p); - return properties.keySet().stream() - .filter(propertiesMetadata::isHiddenProperty) - .collect(Collectors.toSet()); - }), - IllegalArgumentException.class); - } - - private void validateAlterProperties( - NameIdentifier ident, - ThrowableFunction provider, - T... changes) { - doWithCatalog( - getCatalogIdentifier(ident), - c -> - c.doWithPropertiesMeta( - p -> { - Map upserts = getPropertiesForSet(changes); - Map deletes = getPropertiesForDelete(changes); - validatePropertyForAlter(provider.apply(p), upserts, deletes); - return null; - }), - IllegalArgumentException.class); - } - - private Map getPropertiesForSet(T... t) { - Map properties = Maps.newHashMap(); - for (T item : t) { - if (item instanceof TableChange.SetProperty) { - TableChange.SetProperty setProperty = (TableChange.SetProperty) item; - properties.put(setProperty.getProperty(), setProperty.getValue()); - } else if (item instanceof SchemaChange.SetProperty) { - SchemaChange.SetProperty setProperty = (SchemaChange.SetProperty) item; - properties.put(setProperty.getProperty(), setProperty.getValue()); - } else if (item instanceof FilesetChange.SetProperty) { - FilesetChange.SetProperty setProperty = (FilesetChange.SetProperty) item; - properties.put(setProperty.getProperty(), setProperty.getValue()); - } - } - - return properties; - } - - private Map getPropertiesForDelete(T... t) { - Map properties = Maps.newHashMap(); - for (T item : t) { - if (item instanceof TableChange.RemoveProperty) { - TableChange.RemoveProperty removeProperty = (TableChange.RemoveProperty) item; - properties.put(removeProperty.getProperty(), removeProperty.getProperty()); - } else if (item instanceof SchemaChange.RemoveProperty) { - SchemaChange.RemoveProperty removeProperty = (SchemaChange.RemoveProperty) item; - properties.put(removeProperty.getProperty(), removeProperty.getProperty()); - } else if (item instanceof FilesetChange.RemoveProperty) { - FilesetChange.RemoveProperty removeProperty = (FilesetChange.RemoveProperty) item; - properties.put(removeProperty.getProperty(), removeProperty.getProperty()); - } - } - - return properties; - } - - private StringIdentifier getStringIdFromProperties(Map properties) { - try { - StringIdentifier stringId = StringIdentifier.fromProperties(properties); - if (stringId == null) { - LOG.warn(FormattedErrorMessages.STRING_ID_NOT_FOUND); - } - return stringId; - } catch (IllegalArgumentException e) { - LOG.warn(FormattedErrorMessages.STRING_ID_PARSE_ERROR, e.getMessage()); - return null; - } - } - - private R operateOnEntity( - NameIdentifier ident, ThrowableFunction fn, String opName, long id) { - R ret = null; - try { - ret = fn.apply(ident); - } catch (NoSuchEntityException e) { - // Case 2: The table is created by Gravitino, but has no corresponding entity in Gravitino. - LOG.error(FormattedErrorMessages.ENTITY_NOT_FOUND, ident); - } catch (Exception e) { - // Case 3: The table is created by Gravitino, but failed to operate the corresponding entity - // in Gravitino - LOG.error(FormattedErrorMessages.STORE_OP_FAILURE, opName, ident, e); - } - - // Case 4: The table is created by Gravitino, but the uid in the corresponding entity is not - // matched. - if (ret != null && ret.id() != id) { - LOG.error(FormattedErrorMessages.ENTITY_UNMATCHED, ident, ret.id(), id); - ret = null; - } - - return ret; - } - - @VisibleForTesting - // TODO(xun): Remove this method when we implement a better way to get the catalog identifier - // [#257] Add an explicit get catalog functions in NameIdentifier - NameIdentifier getCatalogIdentifier(NameIdentifier ident) { - NameIdentifier.check( - ident.name() != null, "The name variable in the NameIdentifier must have value."); - Namespace.check( - ident.namespace() != null && ident.namespace().length() > 0, - "Catalog namespace must be non-null and have 1 level, the input namespace is %s", - ident.namespace()); - - List allElems = - Stream.concat(Arrays.stream(ident.namespace().levels()), Stream.of(ident.name())) - .collect(Collectors.toList()); - if (allElems.size() < 2) { - throw new IllegalNameIdentifierException( - "Cannot create a catalog NameIdentifier less than two elements."); - } - return NameIdentifier.of(allElems.get(0), allElems.get(1)); - } - - private boolean isManagedEntity(Map properties) { - return Optional.ofNullable(properties) - .map( - p -> - p.getOrDefault( - BasePropertiesMetadata.GRAVITINO_MANAGED_ENTITY, Boolean.FALSE.toString()) - .equals(Boolean.TRUE.toString())) - .orElse(false); - } - - private static final class FormattedErrorMessages { - static final String STORE_OP_FAILURE = - "Failed to {} entity for {} in " - + "Gravitino, with this situation the returned object will not contain the metadata from " - + "Gravitino."; - - static final String STRING_ID_NOT_FOUND = - "String identifier is not set in schema properties, " - + "this is because the schema is not created by Gravitino, or the schema is created by " - + "Gravitino but the string identifier is removed by the user."; - - static final String STRING_ID_PARSE_ERROR = - "Failed to get string identifier from schema " - + "properties: {}, this maybe caused by the same-name string identifier is set by the user " - + "with unsupported format."; - - static final String ENTITY_NOT_FOUND = - "Entity for {} doesn't exist in Gravitino, " - + "this is unexpected if this is created by Gravitino. With this situation the " - + "returned object will not contain the metadata from Gravitino"; - - static final String ENTITY_UNMATCHED = - "Entity {} with uid {} doesn't match the string " - + "identifier in the property {}, this is unexpected if this object is created by " - + "Gravitino. This might be due to some operations that are not performed through Gravitino. " - + "With this situation the returned object will not contain the metadata from Gravitino"; - } -} diff --git a/core/src/main/java/com/datastrato/gravitino/catalog/FilesetOperationDispatcher.java b/core/src/main/java/com/datastrato/gravitino/catalog/FilesetOperationDispatcher.java new file mode 100644 index 00000000000..4d9166c8109 --- /dev/null +++ b/core/src/main/java/com/datastrato/gravitino/catalog/FilesetOperationDispatcher.java @@ -0,0 +1,184 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog; + +import static com.datastrato.gravitino.catalog.PropertiesMetadataHelpers.validatePropertyForCreate; + +import com.datastrato.gravitino.EntityStore; +import com.datastrato.gravitino.NameIdentifier; +import com.datastrato.gravitino.Namespace; +import com.datastrato.gravitino.StringIdentifier; +import com.datastrato.gravitino.connector.HasPropertyMetadata; +import com.datastrato.gravitino.exceptions.FilesetAlreadyExistsException; +import com.datastrato.gravitino.exceptions.NoSuchFilesetException; +import com.datastrato.gravitino.exceptions.NoSuchSchemaException; +import com.datastrato.gravitino.exceptions.NonEmptyEntityException; +import com.datastrato.gravitino.file.Fileset; +import com.datastrato.gravitino.file.FilesetCatalog; +import com.datastrato.gravitino.file.FilesetChange; +import com.datastrato.gravitino.storage.IdGenerator; +import java.util.Map; + +public class FilesetOperationDispatcher extends OperationDispatcher implements FilesetCatalog { + /** + * Creates a new FilesetOperationDispatcher instance. + * + * @param catalogManager The CatalogManager instance to be used for fileset operations. + * @param store The EntityStore instance to be used for fileset operations. + * @param idGenerator The IdGenerator instance to be used for fileset operations. + */ + public FilesetOperationDispatcher( + CatalogManager catalogManager, EntityStore store, IdGenerator idGenerator) { + super(catalogManager, store, idGenerator); + } + + /** + * List the filesets in a schema namespace from the catalog. + * + * @param namespace A schema namespace. + * @return An array of fileset identifiers in the namespace. + * @throws NoSuchSchemaException If the schema does not exist. + */ + @Override + public NameIdentifier[] listFilesets(Namespace namespace) throws NoSuchSchemaException { + return doWithCatalog( + getCatalogIdentifier(NameIdentifier.of(namespace.levels())), + c -> c.doWithFilesetOps(f -> f.listFilesets(namespace)), + NoSuchSchemaException.class); + } + + /** + * Load fileset metadata by {@link NameIdentifier} from the catalog. + * + * @param ident A fileset identifier. + * @return The fileset metadata. + * @throws NoSuchFilesetException If the fileset does not exist. + */ + @Override + public Fileset loadFileset(NameIdentifier ident) throws NoSuchFilesetException { + NameIdentifier catalogIdent = getCatalogIdentifier(ident); + Fileset fileset = + doWithCatalog( + catalogIdent, + c -> c.doWithFilesetOps(f -> f.loadFileset(ident)), + NoSuchFilesetException.class); + + // Currently we only support maintaining the Fileset in the Gravitino's store. + return EntityCombinedFileset.of(fileset) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdent, + HasPropertyMetadata::filesetPropertiesMetadata, + fileset.properties())); + } + + /** + * Create a fileset metadata in the catalog. + * + *

If the type of the fileset object is "MANAGED", the underlying storageLocation can be null, + * and Gravitino will manage the storage location based on the location of the schema. + * + *

If the type of the fileset object is "EXTERNAL", the underlying storageLocation must be set. + * + * @param ident A fileset identifier. + * @param comment The comment of the fileset. + * @param type The type of the fileset. + * @param storageLocation The storage location of the fileset. + * @param properties The properties of the fileset. + * @return The created fileset metadata + * @throws NoSuchSchemaException If the schema does not exist. + * @throws FilesetAlreadyExistsException If the fileset already exists. + */ + @Override + public Fileset createFileset( + NameIdentifier ident, + String comment, + Fileset.Type type, + String storageLocation, + Map properties) + throws NoSuchSchemaException, FilesetAlreadyExistsException { + NameIdentifier catalogIdent = getCatalogIdentifier(ident); + doWithCatalog( + catalogIdent, + c -> + c.doWithPropertiesMeta( + p -> { + validatePropertyForCreate(p.filesetPropertiesMetadata(), properties); + return null; + }), + IllegalArgumentException.class); + long uid = idGenerator.nextId(); + StringIdentifier stringId = StringIdentifier.fromId(uid); + Map updatedProperties = + StringIdentifier.newPropertiesWithId(stringId, properties); + + Fileset createdFileset = + doWithCatalog( + catalogIdent, + c -> + c.doWithFilesetOps( + f -> f.createFileset(ident, comment, type, storageLocation, updatedProperties)), + NoSuchSchemaException.class, + FilesetAlreadyExistsException.class); + return EntityCombinedFileset.of(createdFileset) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdent, + HasPropertyMetadata::filesetPropertiesMetadata, + createdFileset.properties())); + } + + /** + * Apply the {@link FilesetChange change} to a fileset in the catalog. + * + *

Implementation may reject the change. If any change is rejected, no changes should be + * applied to the fileset. + * + *

The {@link FilesetChange.RenameFileset} change will only update the fileset name, the + * underlying storage location for managed fileset will not be renamed. + * + * @param ident A fileset identifier. + * @param changes The changes to apply to the fileset. + * @return The altered fileset metadata. + * @throws NoSuchFilesetException If the fileset does not exist. + * @throws IllegalArgumentException If the change is rejected by the implementation. + */ + @Override + public Fileset alterFileset(NameIdentifier ident, FilesetChange... changes) + throws NoSuchFilesetException, IllegalArgumentException { + validateAlterProperties(ident, HasPropertyMetadata::filesetPropertiesMetadata, changes); + + NameIdentifier catalogIdent = getCatalogIdentifier(ident); + Fileset alteredFileset = + doWithCatalog( + catalogIdent, + c -> c.doWithFilesetOps(f -> f.alterFileset(ident, changes)), + NoSuchFilesetException.class, + IllegalArgumentException.class); + return EntityCombinedFileset.of(alteredFileset) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdent, + HasPropertyMetadata::filesetPropertiesMetadata, + alteredFileset.properties())); + } + + /** + * Drop a fileset from the catalog. + * + *

The underlying files will be deleted if this fileset type is managed, otherwise, only the + * metadata will be dropped. + * + * @param ident A fileset identifier. + * @return true If the fileset is dropped, false the fileset did not exist. + */ + @Override + public boolean dropFileset(NameIdentifier ident) { + return doWithCatalog( + getCatalogIdentifier(ident), + c -> c.doWithFilesetOps(f -> f.dropFileset(ident)), + NonEmptyEntityException.class); + } +} diff --git a/core/src/main/java/com/datastrato/gravitino/catalog/OperationDispatcher.java b/core/src/main/java/com/datastrato/gravitino/catalog/OperationDispatcher.java new file mode 100644 index 00000000000..d4a16ede9d2 --- /dev/null +++ b/core/src/main/java/com/datastrato/gravitino/catalog/OperationDispatcher.java @@ -0,0 +1,265 @@ +/* + * Copyright 2023 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog; + +import static com.datastrato.gravitino.catalog.PropertiesMetadataHelpers.validatePropertyForAlter; + +import com.datastrato.gravitino.EntityStore; +import com.datastrato.gravitino.HasIdentifier; +import com.datastrato.gravitino.NameIdentifier; +import com.datastrato.gravitino.Namespace; +import com.datastrato.gravitino.StringIdentifier; +import com.datastrato.gravitino.connector.BasePropertiesMetadata; +import com.datastrato.gravitino.connector.HasPropertyMetadata; +import com.datastrato.gravitino.connector.PropertiesMetadata; +import com.datastrato.gravitino.exceptions.IllegalNameIdentifierException; +import com.datastrato.gravitino.exceptions.NoSuchEntityException; +import com.datastrato.gravitino.file.FilesetChange; +import com.datastrato.gravitino.rel.SchemaChange; +import com.datastrato.gravitino.rel.TableChange; +import com.datastrato.gravitino.storage.IdGenerator; +import com.datastrato.gravitino.utils.ThrowableFunction; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Maps; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An operation dispatcher that dispatches the operations to the underlying catalog implementation. + */ +public abstract class OperationDispatcher { + + private static final Logger LOG = LoggerFactory.getLogger(OperationDispatcher.class); + + private final CatalogManager catalogManager; + + protected final EntityStore store; + + final IdGenerator idGenerator; + + /** + * Creates a new CatalogOperationDispatcher instance. + * + * @param catalogManager The CatalogManager instance to be used for catalog operations. + * @param store The EntityStore instance to be used for catalog operations. + * @param idGenerator The IdGenerator instance to be used for catalog operations. + */ + public OperationDispatcher( + CatalogManager catalogManager, EntityStore store, IdGenerator idGenerator) { + this.catalogManager = catalogManager; + this.store = store; + this.idGenerator = idGenerator; + } + + R doWithCatalog( + NameIdentifier ident, ThrowableFunction fn, Class ex) + throws E { + try { + CatalogManager.CatalogWrapper c = catalogManager.loadCatalogAndWrap(ident); + return fn.apply(c); + } catch (Throwable throwable) { + if (ex.isInstance(throwable)) { + throw ex.cast(throwable); + } + throw new RuntimeException(throwable); + } + } + + R doWithCatalog( + NameIdentifier ident, + ThrowableFunction fn, + Class ex1, + Class ex2) + throws E1, E2 { + try { + CatalogManager.CatalogWrapper c = catalogManager.loadCatalogAndWrap(ident); + return fn.apply(c); + } catch (Throwable throwable) { + if (ex1.isInstance(throwable)) { + throw ex1.cast(throwable); + } else if (ex2.isInstance(throwable)) { + throw ex2.cast(throwable); + } + if (RuntimeException.class.isAssignableFrom(throwable.getClass())) { + throw (RuntimeException) throwable; + } + + throw new RuntimeException(throwable); + } + } + + Set getHiddenPropertyNames( + NameIdentifier catalogIdent, + ThrowableFunction provider, + Map properties) { + return doWithCatalog( + catalogIdent, + c -> + c.doWithPropertiesMeta( + p -> { + PropertiesMetadata propertiesMetadata = provider.apply(p); + return properties.keySet().stream() + .filter(propertiesMetadata::isHiddenProperty) + .collect(Collectors.toSet()); + }), + IllegalArgumentException.class); + } + + void validateAlterProperties( + NameIdentifier ident, + ThrowableFunction provider, + T... changes) { + doWithCatalog( + getCatalogIdentifier(ident), + c -> + c.doWithPropertiesMeta( + p -> { + Map upserts = getPropertiesForSet(changes); + Map deletes = getPropertiesForDelete(changes); + validatePropertyForAlter(provider.apply(p), upserts, deletes); + return null; + }), + IllegalArgumentException.class); + } + + private Map getPropertiesForSet(T... t) { + Map properties = Maps.newHashMap(); + for (T item : t) { + if (item instanceof TableChange.SetProperty) { + TableChange.SetProperty setProperty = (TableChange.SetProperty) item; + properties.put(setProperty.getProperty(), setProperty.getValue()); + } else if (item instanceof SchemaChange.SetProperty) { + SchemaChange.SetProperty setProperty = (SchemaChange.SetProperty) item; + properties.put(setProperty.getProperty(), setProperty.getValue()); + } else if (item instanceof FilesetChange.SetProperty) { + FilesetChange.SetProperty setProperty = (FilesetChange.SetProperty) item; + properties.put(setProperty.getProperty(), setProperty.getValue()); + } + } + + return properties; + } + + private Map getPropertiesForDelete(T... t) { + Map properties = Maps.newHashMap(); + for (T item : t) { + if (item instanceof TableChange.RemoveProperty) { + TableChange.RemoveProperty removeProperty = (TableChange.RemoveProperty) item; + properties.put(removeProperty.getProperty(), removeProperty.getProperty()); + } else if (item instanceof SchemaChange.RemoveProperty) { + SchemaChange.RemoveProperty removeProperty = (SchemaChange.RemoveProperty) item; + properties.put(removeProperty.getProperty(), removeProperty.getProperty()); + } else if (item instanceof FilesetChange.RemoveProperty) { + FilesetChange.RemoveProperty removeProperty = (FilesetChange.RemoveProperty) item; + properties.put(removeProperty.getProperty(), removeProperty.getProperty()); + } + } + + return properties; + } + + StringIdentifier getStringIdFromProperties(Map properties) { + try { + StringIdentifier stringId = StringIdentifier.fromProperties(properties); + if (stringId == null) { + LOG.warn(FormattedErrorMessages.STRING_ID_NOT_FOUND); + } + return stringId; + } catch (IllegalArgumentException e) { + LOG.warn(FormattedErrorMessages.STRING_ID_PARSE_ERROR, e.getMessage()); + return null; + } + } + + R operateOnEntity( + NameIdentifier ident, ThrowableFunction fn, String opName, long id) { + R ret = null; + try { + ret = fn.apply(ident); + } catch (NoSuchEntityException e) { + // Case 2: The table is created by Gravitino, but has no corresponding entity in Gravitino. + LOG.error(FormattedErrorMessages.ENTITY_NOT_FOUND, ident); + } catch (Exception e) { + // Case 3: The table is created by Gravitino, but failed to operate the corresponding entity + // in Gravitino + LOG.error(FormattedErrorMessages.STORE_OP_FAILURE, opName, ident, e); + } + + // Case 4: The table is created by Gravitino, but the uid in the corresponding entity is not + // matched. + if (ret != null && ret.id() != id) { + LOG.error(FormattedErrorMessages.ENTITY_UNMATCHED, ident, ret.id(), id); + ret = null; + } + + return ret; + } + + @VisibleForTesting + // TODO(xun): Remove this method when we implement a better way to get the catalog identifier + // [#257] Add an explicit get catalog functions in NameIdentifier + NameIdentifier getCatalogIdentifier(NameIdentifier ident) { + NameIdentifier.check( + ident.name() != null, "The name variable in the NameIdentifier must have value."); + Namespace.check( + ident.namespace() != null && ident.namespace().length() > 0, + "Catalog namespace must be non-null and have 1 level, the input namespace is %s", + ident.namespace()); + + List allElems = + Stream.concat(Arrays.stream(ident.namespace().levels()), Stream.of(ident.name())) + .collect(Collectors.toList()); + if (allElems.size() < 2) { + throw new IllegalNameIdentifierException( + "Cannot create a catalog NameIdentifier less than two elements."); + } + return NameIdentifier.of(allElems.get(0), allElems.get(1)); + } + + boolean isManagedEntity(Map properties) { + return Optional.ofNullable(properties) + .map( + p -> + p.getOrDefault( + BasePropertiesMetadata.GRAVITINO_MANAGED_ENTITY, Boolean.FALSE.toString()) + .equals(Boolean.TRUE.toString())) + .orElse(false); + } + + static final class FormattedErrorMessages { + static final String STORE_OP_FAILURE = + "Failed to {} entity for {} in " + + "Gravitino, with this situation the returned object will not contain the metadata from " + + "Gravitino."; + + static final String STRING_ID_NOT_FOUND = + "String identifier is not set in schema properties, " + + "this is because the schema is not created by Gravitino, or the schema is created by " + + "Gravitino but the string identifier is removed by the user."; + + static final String STRING_ID_PARSE_ERROR = + "Failed to get string identifier from schema " + + "properties: {}, this maybe caused by the same-name string identifier is set by the user " + + "with unsupported format."; + + static final String ENTITY_NOT_FOUND = + "Entity for {} doesn't exist in Gravitino, " + + "this is unexpected if this is created by Gravitino. With this situation the " + + "returned object will not contain the metadata from Gravitino"; + + static final String ENTITY_UNMATCHED = + "Entity {} with uid {} doesn't match the string " + + "identifier in the property {}, this is unexpected if this object is created by " + + "Gravitino. This might be due to some operations that are not performed through Gravitino. " + + "With this situation the returned object will not contain the metadata from Gravitino"; + } +} diff --git a/core/src/main/java/com/datastrato/gravitino/catalog/SchemaOperationDispatcher.java b/core/src/main/java/com/datastrato/gravitino/catalog/SchemaOperationDispatcher.java new file mode 100644 index 00000000000..ed750ce976f --- /dev/null +++ b/core/src/main/java/com/datastrato/gravitino/catalog/SchemaOperationDispatcher.java @@ -0,0 +1,315 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog; + +import static com.datastrato.gravitino.Entity.EntityType.SCHEMA; +import static com.datastrato.gravitino.catalog.PropertiesMetadataHelpers.validatePropertyForCreate; + +import com.datastrato.gravitino.EntityStore; +import com.datastrato.gravitino.NameIdentifier; +import com.datastrato.gravitino.Namespace; +import com.datastrato.gravitino.StringIdentifier; +import com.datastrato.gravitino.connector.HasPropertyMetadata; +import com.datastrato.gravitino.exceptions.NoSuchCatalogException; +import com.datastrato.gravitino.exceptions.NoSuchSchemaException; +import com.datastrato.gravitino.exceptions.NonEmptySchemaException; +import com.datastrato.gravitino.exceptions.SchemaAlreadyExistsException; +import com.datastrato.gravitino.meta.AuditInfo; +import com.datastrato.gravitino.meta.SchemaEntity; +import com.datastrato.gravitino.rel.Schema; +import com.datastrato.gravitino.rel.SchemaChange; +import com.datastrato.gravitino.rel.SupportsSchemas; +import com.datastrato.gravitino.storage.IdGenerator; +import com.datastrato.gravitino.utils.PrincipalUtils; +import java.time.Instant; +import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SchemaOperationDispatcher extends OperationDispatcher implements SupportsSchemas { + + private static final Logger LOG = LoggerFactory.getLogger(SchemaOperationDispatcher.class); + + /** + * Creates a new SchemaOperationDispatcher instance. + * + * @param catalogManager The CatalogManager instance to be used for schema operations. + * @param store The EntityStore instance to be used for schema operations. + * @param idGenerator The IdGenerator instance to be used for schema operations. + */ + public SchemaOperationDispatcher( + CatalogManager catalogManager, EntityStore store, IdGenerator idGenerator) { + super(catalogManager, store, idGenerator); + } + + /** + * Lists the schemas within the specified namespace. + * + * @param namespace The namespace in which to list schemas. + * @return An array of NameIdentifier objects representing the schemas within the specified + * namespace. + * @throws NoSuchCatalogException If the catalog namespace does not exist. + */ + @Override + public NameIdentifier[] listSchemas(Namespace namespace) throws NoSuchCatalogException { + return doWithCatalog( + getCatalogIdentifier(NameIdentifier.of(namespace.levels())), + c -> c.doWithSchemaOps(s -> s.listSchemas(namespace)), + NoSuchCatalogException.class); + } + + /** + * Creates a new schema. + * + * @param ident The identifier for the schema to be created. + * @param comment The comment for the new schema. + * @param properties Additional properties for the new schema. + * @return The created Schema object. + * @throws NoSuchCatalogException If the catalog corresponding to the provided identifier does not + * exist. + * @throws SchemaAlreadyExistsException If a schema with the same identifier already exists. + */ + @Override + public Schema createSchema(NameIdentifier ident, String comment, Map properties) + throws NoSuchCatalogException, SchemaAlreadyExistsException { + NameIdentifier catalogIdent = getCatalogIdentifier(ident); + doWithCatalog( + catalogIdent, + c -> + c.doWithPropertiesMeta( + p -> { + validatePropertyForCreate(p.schemaPropertiesMetadata(), properties); + return null; + }), + IllegalArgumentException.class); + long uid = idGenerator.nextId(); + // Add StringIdentifier to the properties, the specific catalog will handle this + // StringIdentifier to make sure only when the operation is successful, the related + // SchemaEntity will be visible. + StringIdentifier stringId = StringIdentifier.fromId(uid); + Map updatedProperties = + StringIdentifier.newPropertiesWithId(stringId, properties); + + Schema createdSchema = + doWithCatalog( + catalogIdent, + c -> c.doWithSchemaOps(s -> s.createSchema(ident, comment, updatedProperties)), + NoSuchCatalogException.class, + SchemaAlreadyExistsException.class); + + // If the Schema is maintained by the Gravitino's store, we don't have to store again. + boolean isManagedSchema = isManagedEntity(createdSchema.properties()); + if (isManagedSchema) { + return EntityCombinedSchema.of(createdSchema) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdent, + HasPropertyMetadata::schemaPropertiesMetadata, + createdSchema.properties())); + } + + // Retrieve the Schema again to obtain some values generated by underlying catalog + Schema schema = + doWithCatalog( + catalogIdent, + c -> c.doWithSchemaOps(s -> s.loadSchema(ident)), + NoSuchSchemaException.class); + + SchemaEntity schemaEntity = + SchemaEntity.builder() + .withId(uid) + .withName(ident.name()) + .withNamespace(ident.namespace()) + .withAuditInfo( + AuditInfo.builder() + .withCreator(PrincipalUtils.getCurrentPrincipal().getName()) + .withCreateTime(Instant.now()) + .build()) + .build(); + + try { + store.put(schemaEntity, true /* overwrite */); + } catch (Exception e) { + LOG.error(FormattedErrorMessages.STORE_OP_FAILURE, "put", ident, e); + return EntityCombinedSchema.of(schema) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdent, + HasPropertyMetadata::schemaPropertiesMetadata, + schema.properties())); + } + + // Merge both the metadata from catalog operation and the metadata from entity store. + return EntityCombinedSchema.of(schema, schemaEntity) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdent, HasPropertyMetadata::schemaPropertiesMetadata, schema.properties())); + } + + /** + * Loads and retrieves a schema. + * + * @param ident The identifier of the schema to be loaded. + * @return The loaded Schema object. + * @throws NoSuchSchemaException If the schema does not exist. + */ + @Override + public Schema loadSchema(NameIdentifier ident) throws NoSuchSchemaException { + NameIdentifier catalogIdentifier = getCatalogIdentifier(ident); + Schema schema = + doWithCatalog( + catalogIdentifier, + c -> c.doWithSchemaOps(s -> s.loadSchema(ident)), + NoSuchSchemaException.class); + + // If the Schema is maintained by the Gravitino's store, we don't have to load again. + boolean isManagedSchema = isManagedEntity(schema.properties()); + if (isManagedSchema) { + return EntityCombinedSchema.of(schema) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdentifier, + HasPropertyMetadata::schemaPropertiesMetadata, + schema.properties())); + } + + StringIdentifier stringId = getStringIdFromProperties(schema.properties()); + // Case 1: The schema is not created by Gravitino. + if (stringId == null) { + return EntityCombinedSchema.of(schema) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdentifier, + HasPropertyMetadata::schemaPropertiesMetadata, + schema.properties())); + } + + SchemaEntity schemaEntity = + operateOnEntity( + ident, + identifier -> store.get(identifier, SCHEMA, SchemaEntity.class), + "GET", + stringId.id()); + return EntityCombinedSchema.of(schema, schemaEntity) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdentifier, + HasPropertyMetadata::schemaPropertiesMetadata, + schema.properties())); + } + + /** + * Alters the schema by applying the provided schema changes. + * + * @param ident The identifier of the schema to be altered. + * @param changes The array of SchemaChange objects representing the alterations to apply. + * @return The altered Schema object. + * @throws NoSuchSchemaException If the schema corresponding to the provided identifier does not + * exist. + */ + @Override + public Schema alterSchema(NameIdentifier ident, SchemaChange... changes) + throws NoSuchSchemaException { + validateAlterProperties(ident, HasPropertyMetadata::schemaPropertiesMetadata, changes); + + NameIdentifier catalogIdent = getCatalogIdentifier(ident); + Schema tempAlteredSchema = + doWithCatalog( + catalogIdent, + c -> c.doWithSchemaOps(s -> s.alterSchema(ident, changes)), + NoSuchSchemaException.class); + + // Retrieve the Schema again to obtain some values generated by underlying catalog + Schema alteredSchema = + doWithCatalog( + catalogIdent, + c -> + c.doWithSchemaOps( + s -> + s.loadSchema( + NameIdentifier.of(ident.namespace(), tempAlteredSchema.name()))), + NoSuchSchemaException.class); + + // If the Schema is maintained by the Gravitino's store, we don't have to alter again. + boolean isManagedSchema = isManagedEntity(alteredSchema.properties()); + if (isManagedSchema) { + return EntityCombinedSchema.of(alteredSchema) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdent, + HasPropertyMetadata::schemaPropertiesMetadata, + alteredSchema.properties())); + } + + StringIdentifier stringId = getStringIdFromProperties(alteredSchema.properties()); + // Case 1: The schema is not created by Gravitino. + if (stringId == null) { + return EntityCombinedSchema.of(alteredSchema) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdent, + HasPropertyMetadata::schemaPropertiesMetadata, + alteredSchema.properties())); + } + + SchemaEntity updatedSchemaEntity = + operateOnEntity( + ident, + id -> + store.update( + id, + SchemaEntity.class, + SCHEMA, + schemaEntity -> + SchemaEntity.builder() + .withId(schemaEntity.id()) + .withName(schemaEntity.name()) + .withNamespace(ident.namespace()) + .withAuditInfo( + AuditInfo.builder() + .withCreator(schemaEntity.auditInfo().creator()) + .withCreateTime(schemaEntity.auditInfo().createTime()) + .withLastModifier( + PrincipalUtils.getCurrentPrincipal().getName()) + .withLastModifiedTime(Instant.now()) + .build()) + .build()), + "UPDATE", + stringId.id()); + return EntityCombinedSchema.of(alteredSchema, updatedSchemaEntity) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdent, + HasPropertyMetadata::schemaPropertiesMetadata, + alteredSchema.properties())); + } + + /** + * Drops a schema. + * + * @param ident The identifier of the schema to be dropped. + * @param cascade If true, drops all tables within the schema as well. + * @return True if the schema was successfully dropped, false otherwise. + * @throws NonEmptySchemaException If the schema contains tables and cascade is set to false. + */ + @Override + public boolean dropSchema(NameIdentifier ident, boolean cascade) throws NonEmptySchemaException { + boolean dropped = + doWithCatalog( + getCatalogIdentifier(ident), + c -> c.doWithSchemaOps(s -> s.dropSchema(ident, cascade)), + NonEmptySchemaException.class); + + if (!dropped) { + return false; + } + + try { + return store.delete(ident, SCHEMA, cascade); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/core/src/main/java/com/datastrato/gravitino/catalog/TableOperationDispatcher.java b/core/src/main/java/com/datastrato/gravitino/catalog/TableOperationDispatcher.java new file mode 100644 index 00000000000..2947ca3180a --- /dev/null +++ b/core/src/main/java/com/datastrato/gravitino/catalog/TableOperationDispatcher.java @@ -0,0 +1,354 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog; + +import static com.datastrato.gravitino.Entity.EntityType.TABLE; +import static com.datastrato.gravitino.catalog.PropertiesMetadataHelpers.validatePropertyForCreate; +import static com.datastrato.gravitino.rel.expressions.transforms.Transforms.EMPTY_TRANSFORM; + +import com.datastrato.gravitino.EntityStore; +import com.datastrato.gravitino.NameIdentifier; +import com.datastrato.gravitino.Namespace; +import com.datastrato.gravitino.StringIdentifier; +import com.datastrato.gravitino.connector.HasPropertyMetadata; +import com.datastrato.gravitino.exceptions.NoSuchSchemaException; +import com.datastrato.gravitino.exceptions.NoSuchTableException; +import com.datastrato.gravitino.exceptions.TableAlreadyExistsException; +import com.datastrato.gravitino.meta.AuditInfo; +import com.datastrato.gravitino.meta.TableEntity; +import com.datastrato.gravitino.rel.Column; +import com.datastrato.gravitino.rel.Table; +import com.datastrato.gravitino.rel.TableCatalog; +import com.datastrato.gravitino.rel.TableChange; +import com.datastrato.gravitino.rel.expressions.distributions.Distribution; +import com.datastrato.gravitino.rel.expressions.distributions.Distributions; +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.rel.indexes.Indexes; +import com.datastrato.gravitino.storage.IdGenerator; +import com.datastrato.gravitino.utils.PrincipalUtils; +import java.time.Instant; +import java.util.Arrays; +import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TableOperationDispatcher extends OperationDispatcher implements TableCatalog { + + private static final Logger LOG = LoggerFactory.getLogger(TableOperationDispatcher.class); + + /** + * Creates a new TableOperationDispatcher instance. + * + * @param catalogManager The CatalogManager instance to be used for table operations. + * @param store The EntityStore instance to be used for table operations. + * @param idGenerator The IdGenerator instance to be used for table operations. + */ + public TableOperationDispatcher( + CatalogManager catalogManager, EntityStore store, IdGenerator idGenerator) { + super(catalogManager, store, idGenerator); + } + + /** + * Lists the tables within a schema. + * + * @param namespace The namespace of the schema containing the tables. + * @return An array of {@link NameIdentifier} objects representing the identifiers of the tables + * in the schema. + * @throws NoSuchSchemaException If the specified schema does not exist. + */ + @Override + public NameIdentifier[] listTables(Namespace namespace) throws NoSuchSchemaException { + return doWithCatalog( + getCatalogIdentifier(NameIdentifier.of(namespace.levels())), + c -> c.doWithTableOps(t -> t.listTables(namespace)), + NoSuchSchemaException.class); + } + + /** + * Loads a table. + * + * @param ident The identifier of the table to load. + * @return The loaded {@link Table} object representing the requested table. + * @throws NoSuchTableException If the specified table does not exist. + */ + @Override + public Table loadTable(NameIdentifier ident) throws NoSuchTableException { + NameIdentifier catalogIdentifier = getCatalogIdentifier(ident); + Table table = + doWithCatalog( + catalogIdentifier, + c -> c.doWithTableOps(t -> t.loadTable(ident)), + NoSuchTableException.class); + + StringIdentifier stringId = getStringIdFromProperties(table.properties()); + // Case 1: The table is not created by Gravitino. + if (stringId == null) { + return EntityCombinedTable.of(table) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdentifier, + HasPropertyMetadata::tablePropertiesMetadata, + table.properties())); + } + + TableEntity tableEntity = + operateOnEntity( + ident, + identifier -> store.get(identifier, TABLE, TableEntity.class), + "GET", + stringId.id()); + + return EntityCombinedTable.of(table, tableEntity) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdentifier, + HasPropertyMetadata::tablePropertiesMetadata, + table.properties())); + } + + /** + * Creates a new table in a schema. + * + * @param ident The identifier of the table to create. + * @param columns An array of {@link Column} objects representing the columns of the table. + * @param comment A description or comment associated with the table. + * @param properties Additional properties to set for the table. + * @param partitions An array of {@link Transform} objects representing the partitioning of table + * @param indexes An array of {@link Index} objects representing the indexes of the table. + * @return The newly created {@link Table} object. + * @throws NoSuchSchemaException If the schema in which to create the table does not exist. + * @throws TableAlreadyExistsException If a table with the same name already exists in the schema. + */ + @Override + public Table createTable( + NameIdentifier ident, + Column[] columns, + String comment, + Map properties, + Transform[] partitions, + Distribution distribution, + SortOrder[] sortOrders, + Index[] indexes) + throws NoSuchSchemaException, TableAlreadyExistsException { + NameIdentifier catalogIdent = getCatalogIdentifier(ident); + doWithCatalog( + catalogIdent, + c -> + c.doWithPropertiesMeta( + p -> { + validatePropertyForCreate(p.tablePropertiesMetadata(), properties); + return null; + }), + IllegalArgumentException.class); + long uid = idGenerator.nextId(); + // Add StringIdentifier to the properties, the specific catalog will handle this + // StringIdentifier to make sure only when the operation is successful, the related + // TableEntity will be visible. + StringIdentifier stringId = StringIdentifier.fromId(uid); + Map updatedProperties = + StringIdentifier.newPropertiesWithId(stringId, properties); + + doWithCatalog( + catalogIdent, + c -> + c.doWithTableOps( + t -> + t.createTable( + ident, + columns, + comment, + updatedProperties, + partitions == null ? EMPTY_TRANSFORM : partitions, + distribution == null ? Distributions.NONE : distribution, + sortOrders == null ? new SortOrder[0] : sortOrders, + indexes == null ? Indexes.EMPTY_INDEXES : indexes)), + NoSuchSchemaException.class, + TableAlreadyExistsException.class); + + // Retrieve the Table again to obtain some values generated by underlying catalog + Table table = + doWithCatalog( + catalogIdent, + c -> c.doWithTableOps(t -> t.loadTable(ident)), + NoSuchTableException.class); + + TableEntity tableEntity = + TableEntity.builder() + .withId(uid) + .withName(ident.name()) + .withNamespace(ident.namespace()) + .withAuditInfo( + AuditInfo.builder() + .withCreator(PrincipalUtils.getCurrentPrincipal().getName()) + .withCreateTime(Instant.now()) + .build()) + .build(); + + try { + store.put(tableEntity, true /* overwrite */); + } catch (Exception e) { + LOG.error(FormattedErrorMessages.STORE_OP_FAILURE, "put", ident, e); + return EntityCombinedTable.of(table) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdent, HasPropertyMetadata::tablePropertiesMetadata, table.properties())); + } + + return EntityCombinedTable.of(table, tableEntity) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdent, HasPropertyMetadata::tablePropertiesMetadata, table.properties())); + } + + /** + * Alters an existing table. + * + * @param ident The identifier of the table to alter. + * @param changes An array of {@link TableChange} objects representing the changes to apply to the + * table. + * @return The altered {@link Table} object after applying the changes. + * @throws NoSuchTableException If the table to alter does not exist. + * @throws IllegalArgumentException If an unsupported or invalid change is specified. + */ + @Override + public Table alterTable(NameIdentifier ident, TableChange... changes) + throws NoSuchTableException, IllegalArgumentException { + validateAlterProperties(ident, HasPropertyMetadata::tablePropertiesMetadata, changes); + + NameIdentifier catalogIdent = getCatalogIdentifier(ident); + Table tempAlteredTable = + doWithCatalog( + catalogIdent, + c -> c.doWithTableOps(t -> t.alterTable(ident, changes)), + NoSuchTableException.class, + IllegalArgumentException.class); + + // Retrieve the Table again to obtain some values generated by underlying catalog + Table alteredTable = + doWithCatalog( + catalogIdent, + c -> + c.doWithTableOps( + t -> + t.loadTable(NameIdentifier.of(ident.namespace(), tempAlteredTable.name()))), + NoSuchTableException.class); + + StringIdentifier stringId = getStringIdFromProperties(alteredTable.properties()); + // Case 1: The table is not created by Gravitino. + if (stringId == null) { + return EntityCombinedTable.of(alteredTable) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + getCatalogIdentifier(ident), + HasPropertyMetadata::tablePropertiesMetadata, + alteredTable.properties())); + } + + TableEntity updatedTableEntity = + operateOnEntity( + ident, + id -> + store.update( + id, + TableEntity.class, + TABLE, + tableEntity -> { + String newName = + Arrays.stream(changes) + .filter(c -> c instanceof TableChange.RenameTable) + .map(c -> ((TableChange.RenameTable) c).getNewName()) + .reduce((c1, c2) -> c2) + .orElse(tableEntity.name()); + + return TableEntity.builder() + .withId(tableEntity.id()) + .withName(newName) + .withNamespace(ident.namespace()) + .withAuditInfo( + AuditInfo.builder() + .withCreator(tableEntity.auditInfo().creator()) + .withCreateTime(tableEntity.auditInfo().createTime()) + .withLastModifier(PrincipalUtils.getCurrentPrincipal().getName()) + .withLastModifiedTime(Instant.now()) + .build()) + .build(); + }), + "UPDATE", + stringId.id()); + + return EntityCombinedTable.of(alteredTable, updatedTableEntity) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + getCatalogIdentifier(ident), + HasPropertyMetadata::tablePropertiesMetadata, + alteredTable.properties())); + } + + /** + * Drops a table from the catalog. + * + * @param ident The identifier of the table to drop. + * @return {@code true} if the table was successfully dropped, {@code false} if the table does not + * exist. + * @throws NoSuchTableException If the table to drop does not exist. + */ + @Override + public boolean dropTable(NameIdentifier ident) { + boolean dropped = + doWithCatalog( + getCatalogIdentifier(ident), + c -> c.doWithTableOps(t -> t.dropTable(ident)), + NoSuchTableException.class); + + if (!dropped) { + return false; + } + + try { + store.delete(ident, TABLE); + } catch (Exception e) { + throw new RuntimeException(e); + } + + return true; + } + + /** + * Drop a table from the catalog and completely remove its data. Removes both the metadata and the + * directory associated with the table completely and skipping trash. If the table is an external + * table or the catalogs don't support purge table, {@link UnsupportedOperationException} is + * thrown. + * + *

If the catalog supports to purge a table, this method should be overridden. The default + * implementation throws an {@link UnsupportedOperationException}. + * + * @param ident A table identifier. + * @return True if the table was purged, false if the table did not exist. + * @throws UnsupportedOperationException If the catalog does not support to purge a table. + */ + @Override + public boolean purgeTable(NameIdentifier ident) throws UnsupportedOperationException { + boolean purged = + doWithCatalog( + getCatalogIdentifier(ident), + c -> c.doWithTableOps(t -> t.purgeTable(ident)), + NoSuchTableException.class, + UnsupportedOperationException.class); + + if (!purged) { + return false; + } + + try { + store.delete(ident, TABLE); + } catch (Exception e) { + throw new RuntimeException(e); + } + + return true; + } +} diff --git a/core/src/test/java/com/datastrato/gravitino/catalog/TestCatalogOperationDispatcher.java b/core/src/test/java/com/datastrato/gravitino/catalog/TestCatalogOperationDispatcher.java deleted file mode 100644 index 6c165365408..00000000000 --- a/core/src/test/java/com/datastrato/gravitino/catalog/TestCatalogOperationDispatcher.java +++ /dev/null @@ -1,693 +0,0 @@ -/* - * Copyright 2023 Datastrato Pvt Ltd. - * This software is licensed under the Apache License version 2. - */ -package com.datastrato.gravitino.catalog; - -import static com.datastrato.gravitino.Entity.EntityType.SCHEMA; -import static com.datastrato.gravitino.Entity.EntityType.TABLE; -import static com.datastrato.gravitino.StringIdentifier.ID_KEY; -import static com.datastrato.gravitino.TestBasePropertiesMetadata.COMMENT_KEY; -import static com.datastrato.gravitino.TestFilesetPropertiesMetadata.TEST_FILESET_HIDDEN_KEY; -import static com.datastrato.gravitino.connector.BasePropertiesMetadata.GRAVITINO_MANAGED_ENTITY; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.reset; -import static org.mockito.Mockito.spy; - -import com.datastrato.gravitino.Catalog; -import com.datastrato.gravitino.Config; -import com.datastrato.gravitino.Configs; -import com.datastrato.gravitino.EntityStore; -import com.datastrato.gravitino.NameIdentifier; -import com.datastrato.gravitino.Namespace; -import com.datastrato.gravitino.StringIdentifier; -import com.datastrato.gravitino.TestColumn; -import com.datastrato.gravitino.auth.AuthConstants; -import com.datastrato.gravitino.exceptions.IllegalNamespaceException; -import com.datastrato.gravitino.exceptions.NoSuchEntityException; -import com.datastrato.gravitino.file.Fileset; -import com.datastrato.gravitino.file.FilesetChange; -import com.datastrato.gravitino.meta.AuditInfo; -import com.datastrato.gravitino.meta.BaseMetalake; -import com.datastrato.gravitino.meta.SchemaEntity; -import com.datastrato.gravitino.meta.SchemaVersion; -import com.datastrato.gravitino.meta.TableEntity; -import com.datastrato.gravitino.rel.Column; -import com.datastrato.gravitino.rel.Schema; -import com.datastrato.gravitino.rel.SchemaChange; -import com.datastrato.gravitino.rel.Table; -import com.datastrato.gravitino.rel.TableChange; -import com.datastrato.gravitino.rel.expressions.transforms.Transform; -import com.datastrato.gravitino.rel.types.Types; -import com.datastrato.gravitino.storage.IdGenerator; -import com.datastrato.gravitino.storage.RandomIdGenerator; -import com.datastrato.gravitino.storage.memory.TestMemoryEntityStore; -import com.google.common.collect.ImmutableMap; -import java.io.IOException; -import java.time.Instant; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.function.Executable; - -public class TestCatalogOperationDispatcher { - - private static EntityStore entityStore; - - private static final IdGenerator idGenerator = new RandomIdGenerator(); - - private static final String metalake = "metalake"; - - private static final String catalog = "catalog"; - - private static CatalogManager catalogManager; - - private static Config config; - - private static CatalogOperationDispatcher dispatcher; - - @BeforeAll - public static void setUp() throws IOException { - config = new Config(false) {}; - config.set(Configs.CATALOG_LOAD_ISOLATED, false); - - entityStore = spy(new TestMemoryEntityStore.InMemoryEntityStore()); - entityStore.initialize(config); - entityStore.setSerDe(null); - - BaseMetalake metalakeEntity = - BaseMetalake.builder() - .withId(1L) - .withName(metalake) - .withAuditInfo( - AuditInfo.builder().withCreator("test").withCreateTime(Instant.now()).build()) - .withVersion(SchemaVersion.V_0_1) - .build(); - entityStore.put(metalakeEntity, true); - - catalogManager = new CatalogManager(config, entityStore, idGenerator); - dispatcher = new CatalogOperationDispatcher(catalogManager, entityStore, idGenerator); - - NameIdentifier ident = NameIdentifier.of(metalake, catalog); - Map props = ImmutableMap.of(); - catalogManager.createCatalog(ident, Catalog.Type.RELATIONAL, "test", "comment", props); - } - - @AfterAll - public static void tearDown() throws IOException { - if (entityStore != null) { - entityStore.close(); - entityStore = null; - } - - if (catalogManager != null) { - catalogManager.close(); - catalogManager = null; - } - } - - @BeforeEach - public void beforeStart() throws IOException { - reset(entityStore); - } - - @Test - public void testCreateAndListSchemas() throws IOException { - Namespace ns = Namespace.of(metalake, catalog); - - NameIdentifier schemaIdent = NameIdentifier.of(ns, "schema1"); - Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); - Schema schema = dispatcher.createSchema(schemaIdent, "comment", props); - - // Check if the created Schema's field values are correct - Assertions.assertEquals("schema1", schema.name()); - Assertions.assertEquals("comment", schema.comment()); - testProperties(props, schema.properties()); - - // Test required table properties exception - Map illegalTableProperties = - new HashMap() { - { - put("k2", "v2"); - } - }; - - testPropertyException( - () -> dispatcher.createSchema(schemaIdent, "comment", illegalTableProperties), - "Properties are required and must be set"); - - // Test reserved table properties exception - illegalTableProperties.put(COMMENT_KEY, "table comment"); - illegalTableProperties.put(ID_KEY, "gravitino.v1.uidfdsafdsa"); - testPropertyException( - () -> dispatcher.createSchema(schemaIdent, "comment", illegalTableProperties), - "Properties are reserved and cannot be set", - "comment", - "gravitino.identifier"); - - // Check if the Schema entity is stored in the EntityStore - SchemaEntity schemaEntity = entityStore.get(schemaIdent, SCHEMA, SchemaEntity.class); - Assertions.assertNotNull(schemaEntity); - Assertions.assertEquals("schema1", schemaEntity.name()); - Assertions.assertNotNull(schemaEntity.id()); - - Optional ident1 = - Arrays.stream(dispatcher.listSchemas(ns)) - .filter(s -> s.name().equals("schema1")) - .findFirst(); - Assertions.assertTrue(ident1.isPresent()); - - // Test when the entity store failed to put the schema entity - doThrow(new IOException()).when(entityStore).put(any(), anyBoolean()); - NameIdentifier schemaIdent2 = NameIdentifier.of(ns, "schema2"); - Schema schema2 = dispatcher.createSchema(schemaIdent2, "comment", props); - - // Check if the created Schema's field values are correct - Assertions.assertEquals("schema2", schema2.name()); - Assertions.assertEquals("comment", schema2.comment()); - testProperties(props, schema2.properties()); - - // Check if the Schema entity is stored in the EntityStore - Assertions.assertFalse(entityStore.exists(schemaIdent2, SCHEMA)); - Assertions.assertThrows( - NoSuchEntityException.class, - () -> entityStore.get(schemaIdent2, SCHEMA, SchemaEntity.class)); - - // Audit info is gotten from the catalog, not from the entity store - Assertions.assertEquals("test", schema2.auditInfo().creator()); - } - - @Test - public void testCreateAndLoadSchema() throws IOException { - NameIdentifier schemaIdent = NameIdentifier.of(metalake, catalog, "schema11"); - Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); - Schema schema = dispatcher.createSchema(schemaIdent, "comment", props); - - Schema loadedSchema = dispatcher.loadSchema(schemaIdent); - Assertions.assertEquals(schema.name(), loadedSchema.name()); - Assertions.assertEquals(schema.comment(), loadedSchema.comment()); - testProperties(schema.properties(), loadedSchema.properties()); - // Audit info is gotten from entity store - Assertions.assertEquals(AuthConstants.ANONYMOUS_USER, loadedSchema.auditInfo().creator()); - - // Case 2: Test if the schema is not found in entity store - doThrow(new NoSuchEntityException("mock error")).when(entityStore).get(any(), any(), any()); - Schema loadedSchema1 = dispatcher.loadSchema(schemaIdent); - // Audit info is gotten from the catalog, not from the entity store - Assertions.assertEquals("test", loadedSchema1.auditInfo().creator()); - - // Case 3: Test if entity store is failed to get the schema entity - reset(entityStore); - doThrow(new IOException()).when(entityStore).get(any(), any(), any()); - Schema loadedSchema2 = dispatcher.loadSchema(schemaIdent); - // Audit info is gotten from the catalog, not from the entity store - Assertions.assertEquals("test", loadedSchema2.auditInfo().creator()); - - // Case 4: Test if the fetched schema entity is matched. - reset(entityStore); - SchemaEntity unmatchedEntity = - SchemaEntity.builder() - .withId(1L) - .withName("schema11") - .withNamespace(Namespace.of(metalake, catalog)) - .withAuditInfo( - AuditInfo.builder() - .withCreator(AuthConstants.ANONYMOUS_USER) - .withCreateTime(Instant.now()) - .build()) - .build(); - doReturn(unmatchedEntity).when(entityStore).get(any(), any(), any()); - dispatcher.loadSchema(schemaIdent); - // Audit info is gotten from the catalog, not from the entity store - Assertions.assertEquals("test", loadedSchema2.auditInfo().creator()); - } - - @Test - public void testCreateAndAlterSchema() throws IOException { - NameIdentifier schemaIdent = NameIdentifier.of(metalake, catalog, "schema21"); - Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); - Schema schema = dispatcher.createSchema(schemaIdent, "comment", props); - - // Test immutable schema properties - SchemaChange[] illegalChange = - new SchemaChange[] {SchemaChange.setProperty(COMMENT_KEY, "new comment")}; - testPropertyException( - () -> dispatcher.alterSchema(schemaIdent, illegalChange), - "Property comment is immutable or reserved, cannot be set"); - - SchemaChange[] changes = - new SchemaChange[] { - SchemaChange.setProperty("k3", "v3"), SchemaChange.removeProperty("k1") - }; - - Schema alteredSchema = dispatcher.alterSchema(schemaIdent, changes); - Assertions.assertEquals(schema.name(), alteredSchema.name()); - Assertions.assertEquals(schema.comment(), alteredSchema.comment()); - Map expectedProps = ImmutableMap.of("k2", "v2", "k3", "v3"); - testProperties(expectedProps, alteredSchema.properties()); - // Audit info is gotten from gravitino entity store. - Assertions.assertEquals(AuthConstants.ANONYMOUS_USER, alteredSchema.auditInfo().creator()); - Assertions.assertEquals(AuthConstants.ANONYMOUS_USER, alteredSchema.auditInfo().lastModifier()); - - // Case 2: Test if the schema is not found in entity store - doThrow(new NoSuchEntityException("mock error")) - .when(entityStore) - .update(any(), any(), any(), any()); - Schema alteredSchema1 = dispatcher.alterSchema(schemaIdent, changes); - Assertions.assertEquals(schema.name(), alteredSchema1.name()); - Assertions.assertEquals(schema.comment(), alteredSchema1.comment()); - testProperties(expectedProps, alteredSchema1.properties()); - // Audit info is gotten from catalog, not from the entity store - Assertions.assertEquals("test", alteredSchema1.auditInfo().creator()); - Assertions.assertEquals("test", alteredSchema1.auditInfo().lastModifier()); - - // Case 3: Test if entity store is failed to get the schema entity - reset(entityStore); - doThrow(new IOException()).when(entityStore).update(any(), any(), any(), any()); - Schema alteredSchema2 = dispatcher.alterSchema(schemaIdent, changes); - Assertions.assertEquals(schema.name(), alteredSchema2.name()); - Assertions.assertEquals(schema.comment(), alteredSchema2.comment()); - testProperties(expectedProps, alteredSchema2.properties()); - // Audit info is gotten from catalog, not from the entity store - Assertions.assertEquals("test", alteredSchema2.auditInfo().creator()); - Assertions.assertEquals("test", alteredSchema1.auditInfo().lastModifier()); - - // Case 4: Test if the fetched schema entity is matched. - reset(entityStore); - SchemaEntity unmatchedEntity = - SchemaEntity.builder() - .withId(1L) - .withName("schema21") - .withNamespace(Namespace.of(metalake, catalog)) - .withAuditInfo( - AuditInfo.builder() - .withCreator(AuthConstants.ANONYMOUS_USER) - .withCreateTime(Instant.now()) - .build()) - .build(); - doReturn(unmatchedEntity).when(entityStore).update(any(), any(), any(), any()); - Schema alteredSchema3 = dispatcher.alterSchema(schemaIdent, changes); - Assertions.assertEquals(schema.name(), alteredSchema3.name()); - Assertions.assertEquals(schema.comment(), alteredSchema3.comment()); - testProperties(expectedProps, alteredSchema3.properties()); - // Audit info is gotten from catalog, not from the entity store - Assertions.assertEquals("test", alteredSchema3.auditInfo().creator()); - Assertions.assertEquals("test", alteredSchema1.auditInfo().lastModifier()); - } - - @Test - public void testCreateAndDropSchema() throws IOException { - NameIdentifier schemaIdent = NameIdentifier.of(metalake, catalog, "schema31"); - Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); - - dispatcher.createSchema(schemaIdent, "comment", props); - - boolean dropped = dispatcher.dropSchema(schemaIdent, false); - Assertions.assertTrue(dropped); - - // Test if entity store is failed to drop the schema entity - dispatcher.createSchema(schemaIdent, "comment", props); - doThrow(new IOException()).when(entityStore).delete(any(), any(), anyBoolean()); - Assertions.assertThrows( - RuntimeException.class, () -> dispatcher.dropSchema(schemaIdent, false)); - } - - @Test - public void testCreateAndListTables() throws IOException { - Namespace tableNs = Namespace.of(metalake, catalog, "schema41"); - Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); - dispatcher.createSchema(NameIdentifier.of(tableNs.levels()), "comment", props); - - NameIdentifier tableIdent1 = NameIdentifier.of(tableNs, "table1"); - Column[] columns = - new Column[] { - TestColumn.builder().withName("col1").withType(Types.StringType.get()).build(), - TestColumn.builder().withName("col2").withType(Types.StringType.get()).build() - }; - - Table table1 = dispatcher.createTable(tableIdent1, columns, "comment", props, new Transform[0]); - Assertions.assertEquals("table1", table1.name()); - Assertions.assertEquals("comment", table1.comment()); - testProperties(props, table1.properties()); - Assertions.assertEquals(0, table1.partitioning().length); - Assertions.assertArrayEquals(columns, table1.columns()); - - // Test required table properties exception - Map illegalTableProperties = - new HashMap() { - { - put("k2", "v2"); - } - }; - testPropertyException( - () -> - dispatcher.createTable( - tableIdent1, columns, "comment", illegalTableProperties, new Transform[0]), - "Properties are required and must be set"); - - // Test reserved table properties exception - illegalTableProperties.put(COMMENT_KEY, "table comment"); - illegalTableProperties.put(ID_KEY, "gravitino.v1.uidfdsafdsa"); - testPropertyException( - () -> - dispatcher.createTable( - tableIdent1, columns, "comment", illegalTableProperties, new Transform[0]), - "Properties are reserved and cannot be set", - "comment", - "gravitino.identifier"); - - // Check if the Table entity is stored in the EntityStore - TableEntity tableEntity = entityStore.get(tableIdent1, TABLE, TableEntity.class); - Assertions.assertNotNull(tableEntity); - Assertions.assertEquals("table1", tableEntity.name()); - - Assertions.assertFalse(table1.properties().containsKey(ID_KEY)); - - Optional ident1 = - Arrays.stream(dispatcher.listTables(tableNs)) - .filter(s -> s.name().equals("table1")) - .findFirst(); - Assertions.assertTrue(ident1.isPresent()); - - // Test when the entity store failed to put the table entity - doThrow(new IOException()).when(entityStore).put(any(), anyBoolean()); - NameIdentifier tableIdent2 = NameIdentifier.of(tableNs, "table2"); - Table table2 = dispatcher.createTable(tableIdent2, columns, "comment", props, new Transform[0]); - - // Check if the created Schema's field values are correct - Assertions.assertEquals("table2", table2.name()); - Assertions.assertEquals("comment", table2.comment()); - testProperties(props, table2.properties()); - - // Check if the Table entity is stored in the EntityStore - Assertions.assertFalse(entityStore.exists(tableIdent2, TABLE)); - Assertions.assertThrows( - NoSuchEntityException.class, () -> entityStore.get(tableIdent2, TABLE, TableEntity.class)); - - // Audit info is gotten from the catalog, not from the entity store - Assertions.assertEquals("test", table2.auditInfo().creator()); - } - - @Test - public void testCreateAndLoadTable() throws IOException { - Namespace tableNs = Namespace.of(metalake, catalog, "schema51"); - Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); - dispatcher.createSchema(NameIdentifier.of(tableNs.levels()), "comment", props); - - NameIdentifier tableIdent1 = NameIdentifier.of(tableNs, "table11"); - Column[] columns = - new Column[] { - TestColumn.builder().withName("col1").withType(Types.StringType.get()).build(), - TestColumn.builder().withName("col2").withType(Types.StringType.get()).build() - }; - - Table table1 = dispatcher.createTable(tableIdent1, columns, "comment", props, new Transform[0]); - Table loadedTable1 = dispatcher.loadTable(tableIdent1); - Assertions.assertEquals(table1.name(), loadedTable1.name()); - Assertions.assertEquals(table1.comment(), loadedTable1.comment()); - testProperties(table1.properties(), loadedTable1.properties()); - Assertions.assertEquals(0, loadedTable1.partitioning().length); - Assertions.assertArrayEquals(table1.columns(), loadedTable1.columns()); - // Audit info is gotten from the entity store - Assertions.assertEquals(AuthConstants.ANONYMOUS_USER, loadedTable1.auditInfo().creator()); - - // Case 2: Test if the table entity is not found in the entity store - reset(entityStore); - doThrow(new NoSuchEntityException("")).when(entityStore).get(any(), any(), any()); - Table loadedTable2 = dispatcher.loadTable(tableIdent1); - // Audit info is gotten from the catalog, not from the entity store - Assertions.assertEquals("test", loadedTable2.auditInfo().creator()); - - // Case 3: Test if the entity store is failed to get the table entity - reset(entityStore); - doThrow(new IOException()).when(entityStore).get(any(), any(), any()); - Table loadedTable3 = dispatcher.loadTable(tableIdent1); - // Audit info is gotten from the catalog, not from the entity store - Assertions.assertEquals("test", loadedTable3.auditInfo().creator()); - - // Case 4: Test if the table entity is not matched - reset(entityStore); - TableEntity tableEntity = - TableEntity.builder() - .withId(1L) - .withName("table11") - .withNamespace(tableNs) - .withAuditInfo( - AuditInfo.builder().withCreator("gravitino").withCreateTime(Instant.now()).build()) - .build(); - doReturn(tableEntity).when(entityStore).get(any(), any(), any()); - Table loadedTable4 = dispatcher.loadTable(tableIdent1); - // Audit info is gotten from the catalog, not from the entity store - Assertions.assertEquals("test", loadedTable4.auditInfo().creator()); - } - - @Test - public void testCreateAndAlterTable() throws IOException { - Namespace tableNs = Namespace.of(metalake, catalog, "schema61"); - Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); - dispatcher.createSchema(NameIdentifier.of(tableNs.levels()), "comment", props); - - NameIdentifier tableIdent = NameIdentifier.of(tableNs, "table21"); - Column[] columns = - new Column[] { - TestColumn.builder().withName("col1").withType(Types.StringType.get()).build(), - TestColumn.builder().withName("col2").withType(Types.StringType.get()).build() - }; - - Table table = dispatcher.createTable(tableIdent, columns, "comment", props, new Transform[0]); - - // Test immutable table properties - TableChange[] illegalChange = - new TableChange[] {TableChange.setProperty(COMMENT_KEY, "new comment")}; - testPropertyException( - () -> dispatcher.alterTable(tableIdent, illegalChange), - "Property comment is immutable or reserved, cannot be set"); - - TableChange[] changes = - new TableChange[] {TableChange.setProperty("k3", "v3"), TableChange.removeProperty("k1")}; - - Table alteredTable = dispatcher.alterTable(tableIdent, changes); - Assertions.assertEquals(table.name(), alteredTable.name()); - Assertions.assertEquals(table.comment(), alteredTable.comment()); - Map expectedProps = ImmutableMap.of("k2", "v2", "k3", "v3"); - testProperties(expectedProps, alteredTable.properties()); - // Audit info is gotten from gravitino entity store - Assertions.assertEquals(AuthConstants.ANONYMOUS_USER, alteredTable.auditInfo().creator()); - Assertions.assertEquals(AuthConstants.ANONYMOUS_USER, alteredTable.auditInfo().lastModifier()); - - // Case 2: Test if the table entity is not found in the entity store - reset(entityStore); - doThrow(new NoSuchEntityException("")).when(entityStore).update(any(), any(), any(), any()); - Table alteredTable2 = dispatcher.alterTable(tableIdent, changes); - // Audit info is gotten from the catalog, not from the entity store - Assertions.assertEquals("test", alteredTable2.auditInfo().creator()); - Assertions.assertEquals("test", alteredTable2.auditInfo().lastModifier()); - - // Case 3: Test if the entity store is failed to update the table entity - reset(entityStore); - doThrow(new IOException()).when(entityStore).update(any(), any(), any(), any()); - Table alteredTable3 = dispatcher.alterTable(tableIdent, changes); - // Audit info is gotten from the catalog, not from the entity store - Assertions.assertEquals("test", alteredTable3.auditInfo().creator()); - Assertions.assertEquals("test", alteredTable3.auditInfo().lastModifier()); - - // Case 4: Test if the table entity is not matched - reset(entityStore); - TableEntity unmatchedEntity = - TableEntity.builder() - .withId(1L) - .withName("table21") - .withNamespace(tableNs) - .withAuditInfo( - AuditInfo.builder().withCreator("gravitino").withCreateTime(Instant.now()).build()) - .build(); - doReturn(unmatchedEntity).when(entityStore).update(any(), any(), any(), any()); - Table alteredTable4 = dispatcher.alterTable(tableIdent, changes); - // Audit info is gotten from the catalog, not from the entity store - Assertions.assertEquals("test", alteredTable4.auditInfo().creator()); - Assertions.assertEquals("test", alteredTable4.auditInfo().lastModifier()); - } - - @Test - public void testCreateAndDropTable() throws IOException { - NameIdentifier tableIdent = NameIdentifier.of(metalake, catalog, "schema71", "table31"); - Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); - Column[] columns = - new Column[] { - TestColumn.builder().withName("col1").withType(Types.StringType.get()).build(), - TestColumn.builder().withName("col2").withType(Types.StringType.get()).build() - }; - - dispatcher.createTable(tableIdent, columns, "comment", props, new Transform[0]); - - boolean dropped = dispatcher.dropTable(tableIdent); - Assertions.assertTrue(dropped); - - // Test if the entity store is failed to drop the table entity - dispatcher.createTable(tableIdent, columns, "comment", props, new Transform[0]); - reset(entityStore); - doThrow(new IOException()).when(entityStore).delete(any(), any(), anyBoolean()); - Assertions.assertThrows(RuntimeException.class, () -> dispatcher.dropTable(tableIdent)); - } - - @Test - public void testCreateAndListFilesets() { - Namespace filesetNs = Namespace.of(metalake, catalog, "schema81"); - Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); - dispatcher.createSchema(NameIdentifier.of(filesetNs.levels()), "comment", props); - - NameIdentifier filesetIdent1 = NameIdentifier.of(filesetNs, "fileset1"); - Fileset fileset1 = - dispatcher.createFileset(filesetIdent1, "comment", Fileset.Type.MANAGED, "test", props); - Assertions.assertEquals("fileset1", fileset1.name()); - Assertions.assertEquals("comment", fileset1.comment()); - testProperties(props, fileset1.properties()); - Assertions.assertEquals(Fileset.Type.MANAGED, fileset1.type()); - Assertions.assertEquals("test", fileset1.storageLocation()); - - NameIdentifier[] idents = dispatcher.listFilesets(filesetNs); - Assertions.assertEquals(1, idents.length); - Assertions.assertEquals(filesetIdent1, idents[0]); - - Map illegalProps = ImmutableMap.of("k2", "v2"); - testPropertyException( - () -> - dispatcher.createFileset( - filesetIdent1, "comment", Fileset.Type.MANAGED, "test", illegalProps), - "Properties are required and must be set"); - - Map illegalProps2 = ImmutableMap.of("k1", "v1", ID_KEY, "test"); - testPropertyException( - () -> - dispatcher.createFileset( - filesetIdent1, "comment", Fileset.Type.MANAGED, "test", illegalProps2), - "Properties are reserved and cannot be set", - "gravitino.identifier"); - } - - @Test - public void testCreateAndLoadFileset() { - Namespace filesetNs = Namespace.of(metalake, catalog, "schema91"); - Map props = ImmutableMap.of("k1", "v1", "location", "schema91"); - dispatcher.createSchema(NameIdentifier.of(filesetNs.levels()), "comment", props); - - NameIdentifier filesetIdent1 = NameIdentifier.of(filesetNs, "fileset11"); - Fileset fileset1 = - dispatcher.createFileset(filesetIdent1, "comment", Fileset.Type.MANAGED, null, props); - Assertions.assertEquals("fileset11", fileset1.name()); - Assertions.assertEquals("comment", fileset1.comment()); - testProperties(props, fileset1.properties()); - Assertions.assertEquals(Fileset.Type.MANAGED, fileset1.type()); - Assertions.assertNull(fileset1.storageLocation()); - - Fileset loadedFileset1 = dispatcher.loadFileset(filesetIdent1); - Assertions.assertEquals(fileset1.name(), loadedFileset1.name()); - Assertions.assertEquals(fileset1.comment(), loadedFileset1.comment()); - testProperties(props, loadedFileset1.properties()); - Assertions.assertEquals(fileset1.type(), loadedFileset1.type()); - Assertions.assertEquals(fileset1.storageLocation(), loadedFileset1.storageLocation()); - } - - @Test - public void testCreateAndAlterFileset() { - Namespace filesetNs = Namespace.of(metalake, catalog, "schema101"); - Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); - dispatcher.createSchema(NameIdentifier.of(filesetNs.levels()), "comment", props); - - NameIdentifier filesetIdent1 = NameIdentifier.of(filesetNs, "fileset21"); - Fileset fileset1 = - dispatcher.createFileset( - filesetIdent1, "comment", Fileset.Type.MANAGED, "fileset21", props); - Assertions.assertEquals("fileset21", fileset1.name()); - Assertions.assertEquals("comment", fileset1.comment()); - testProperties(props, fileset1.properties()); - Assertions.assertEquals(Fileset.Type.MANAGED, fileset1.type()); - Assertions.assertEquals("fileset21", fileset1.storageLocation()); - - FilesetChange[] changes = - new FilesetChange[] { - FilesetChange.setProperty("k3", "v3"), FilesetChange.removeProperty("k1") - }; - - Fileset alteredFileset = dispatcher.alterFileset(filesetIdent1, changes); - Assertions.assertEquals(fileset1.name(), alteredFileset.name()); - Assertions.assertEquals(fileset1.comment(), alteredFileset.comment()); - Map expectedProps = ImmutableMap.of("k2", "v2", "k3", "v3"); - testProperties(expectedProps, alteredFileset.properties()); - - // Test immutable fileset properties - FilesetChange[] illegalChange = - new FilesetChange[] {FilesetChange.setProperty(GRAVITINO_MANAGED_ENTITY, "test")}; - testPropertyException( - () -> dispatcher.alterFileset(filesetIdent1, illegalChange), - "Property gravitino.managed.entity is immutable or reserved, cannot be set"); - } - - @Test - public void testCreateAndDropFileset() { - Namespace filesetNs = Namespace.of(metalake, catalog, "schema111"); - Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); - dispatcher.createSchema(NameIdentifier.of(filesetNs.levels()), "comment", props); - - NameIdentifier filesetIdent1 = NameIdentifier.of(filesetNs, "fileset31"); - Fileset fileset1 = - dispatcher.createFileset( - filesetIdent1, "comment", Fileset.Type.MANAGED, "fileset31", props); - Assertions.assertEquals("fileset31", fileset1.name()); - Assertions.assertEquals("comment", fileset1.comment()); - testProperties(props, fileset1.properties()); - Assertions.assertEquals(Fileset.Type.MANAGED, fileset1.type()); - Assertions.assertEquals("fileset31", fileset1.storageLocation()); - - boolean dropped = dispatcher.dropFileset(filesetIdent1); - Assertions.assertTrue(dropped); - } - - @Test - public void testGetCatalogIdentifier() { - CatalogOperationDispatcher dispatcher = new CatalogOperationDispatcher(null, null, null); - - NameIdentifier id1 = NameIdentifier.of("a"); - assertThrows(IllegalNamespaceException.class, () -> dispatcher.getCatalogIdentifier(id1)); - - NameIdentifier id2 = NameIdentifier.of("a", "b"); - assertEquals(dispatcher.getCatalogIdentifier(id2), NameIdentifier.of("a", "b")); - - NameIdentifier id3 = NameIdentifier.of("a", "b", "c"); - assertEquals(dispatcher.getCatalogIdentifier(id3), NameIdentifier.of("a", "b")); - - NameIdentifier id4 = NameIdentifier.of("a", "b", "c", "d"); - assertEquals(dispatcher.getCatalogIdentifier(id4), NameIdentifier.of("a", "b")); - - NameIdentifier id5 = NameIdentifier.of("a", "b", "c", "d", "e"); - assertEquals(dispatcher.getCatalogIdentifier(id5), NameIdentifier.of("a", "b")); - } - - private void testProperties(Map expectedProps, Map testProps) { - expectedProps.forEach( - (k, v) -> { - Assertions.assertEquals(v, testProps.get(k)); - }); - Assertions.assertFalse(testProps.containsKey(StringIdentifier.ID_KEY)); - Assertions.assertFalse(testProps.containsKey(GRAVITINO_MANAGED_ENTITY)); - Assertions.assertFalse(testProps.containsKey(TEST_FILESET_HIDDEN_KEY)); - } - - private void testPropertyException(Executable operation, String... errorMessage) { - IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, operation); - for (String msg : errorMessage) { - Assertions.assertTrue(exception.getMessage().contains(msg)); - } - } -} diff --git a/core/src/test/java/com/datastrato/gravitino/catalog/TestFilesetOperationDispatcher.java b/core/src/test/java/com/datastrato/gravitino/catalog/TestFilesetOperationDispatcher.java new file mode 100644 index 00000000000..3bcd906f330 --- /dev/null +++ b/core/src/test/java/com/datastrato/gravitino/catalog/TestFilesetOperationDispatcher.java @@ -0,0 +1,147 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog; + +import static com.datastrato.gravitino.StringIdentifier.ID_KEY; +import static com.datastrato.gravitino.connector.BasePropertiesMetadata.GRAVITINO_MANAGED_ENTITY; + +import com.datastrato.gravitino.NameIdentifier; +import com.datastrato.gravitino.Namespace; +import com.datastrato.gravitino.file.Fileset; +import com.datastrato.gravitino.file.FilesetChange; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.util.Map; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +public class TestFilesetOperationDispatcher extends TestOperationDispatcher { + private static FilesetOperationDispatcher filesetOperationDispatcher; + private static SchemaOperationDispatcher schemaOperationDispatcher; + + @BeforeAll + public static void initialize() throws IOException { + schemaOperationDispatcher = + new SchemaOperationDispatcher(catalogManager, entityStore, idGenerator); + filesetOperationDispatcher = + new FilesetOperationDispatcher(catalogManager, entityStore, idGenerator); + } + + @Test + public void testCreateAndListFilesets() { + Namespace filesetNs = Namespace.of(metalake, catalog, "schema81"); + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + schemaOperationDispatcher.createSchema(NameIdentifier.of(filesetNs.levels()), "comment", props); + + NameIdentifier filesetIdent1 = NameIdentifier.of(filesetNs, "fileset1"); + Fileset fileset1 = + filesetOperationDispatcher.createFileset( + filesetIdent1, "comment", Fileset.Type.MANAGED, "test", props); + Assertions.assertEquals("fileset1", fileset1.name()); + Assertions.assertEquals("comment", fileset1.comment()); + testProperties(props, fileset1.properties()); + Assertions.assertEquals(Fileset.Type.MANAGED, fileset1.type()); + Assertions.assertEquals("test", fileset1.storageLocation()); + + NameIdentifier[] idents = filesetOperationDispatcher.listFilesets(filesetNs); + Assertions.assertEquals(1, idents.length); + Assertions.assertEquals(filesetIdent1, idents[0]); + + Map illegalProps = ImmutableMap.of("k2", "v2"); + testPropertyException( + () -> + filesetOperationDispatcher.createFileset( + filesetIdent1, "comment", Fileset.Type.MANAGED, "test", illegalProps), + "Properties are required and must be set"); + + Map illegalProps2 = ImmutableMap.of("k1", "v1", ID_KEY, "test"); + testPropertyException( + () -> + filesetOperationDispatcher.createFileset( + filesetIdent1, "comment", Fileset.Type.MANAGED, "test", illegalProps2), + "Properties are reserved and cannot be set", + "gravitino.identifier"); + } + + @Test + public void testCreateAndLoadFileset() { + Namespace filesetNs = Namespace.of(metalake, catalog, "schema91"); + Map props = ImmutableMap.of("k1", "v1", "location", "schema91"); + schemaOperationDispatcher.createSchema(NameIdentifier.of(filesetNs.levels()), "comment", props); + + NameIdentifier filesetIdent1 = NameIdentifier.of(filesetNs, "fileset11"); + Fileset fileset1 = + filesetOperationDispatcher.createFileset( + filesetIdent1, "comment", Fileset.Type.MANAGED, null, props); + Assertions.assertEquals("fileset11", fileset1.name()); + Assertions.assertEquals("comment", fileset1.comment()); + testProperties(props, fileset1.properties()); + Assertions.assertEquals(Fileset.Type.MANAGED, fileset1.type()); + Assertions.assertNull(fileset1.storageLocation()); + + Fileset loadedFileset1 = filesetOperationDispatcher.loadFileset(filesetIdent1); + Assertions.assertEquals(fileset1.name(), loadedFileset1.name()); + Assertions.assertEquals(fileset1.comment(), loadedFileset1.comment()); + testProperties(props, loadedFileset1.properties()); + Assertions.assertEquals(fileset1.type(), loadedFileset1.type()); + Assertions.assertEquals(fileset1.storageLocation(), loadedFileset1.storageLocation()); + } + + @Test + public void testCreateAndAlterFileset() { + Namespace filesetNs = Namespace.of(metalake, catalog, "schema101"); + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + schemaOperationDispatcher.createSchema(NameIdentifier.of(filesetNs.levels()), "comment", props); + + NameIdentifier filesetIdent1 = NameIdentifier.of(filesetNs, "fileset21"); + Fileset fileset1 = + filesetOperationDispatcher.createFileset( + filesetIdent1, "comment", Fileset.Type.MANAGED, "fileset21", props); + Assertions.assertEquals("fileset21", fileset1.name()); + Assertions.assertEquals("comment", fileset1.comment()); + testProperties(props, fileset1.properties()); + Assertions.assertEquals(Fileset.Type.MANAGED, fileset1.type()); + Assertions.assertEquals("fileset21", fileset1.storageLocation()); + + FilesetChange[] changes = + new FilesetChange[] { + FilesetChange.setProperty("k3", "v3"), FilesetChange.removeProperty("k1") + }; + + Fileset alteredFileset = filesetOperationDispatcher.alterFileset(filesetIdent1, changes); + Assertions.assertEquals(fileset1.name(), alteredFileset.name()); + Assertions.assertEquals(fileset1.comment(), alteredFileset.comment()); + Map expectedProps = ImmutableMap.of("k2", "v2", "k3", "v3"); + testProperties(expectedProps, alteredFileset.properties()); + + // Test immutable fileset properties + FilesetChange[] illegalChange = + new FilesetChange[] {FilesetChange.setProperty(GRAVITINO_MANAGED_ENTITY, "test")}; + testPropertyException( + () -> filesetOperationDispatcher.alterFileset(filesetIdent1, illegalChange), + "Property gravitino.managed.entity is immutable or reserved, cannot be set"); + } + + @Test + public void testCreateAndDropFileset() { + Namespace filesetNs = Namespace.of(metalake, catalog, "schema111"); + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + schemaOperationDispatcher.createSchema(NameIdentifier.of(filesetNs.levels()), "comment", props); + + NameIdentifier filesetIdent1 = NameIdentifier.of(filesetNs, "fileset31"); + Fileset fileset1 = + filesetOperationDispatcher.createFileset( + filesetIdent1, "comment", Fileset.Type.MANAGED, "fileset31", props); + Assertions.assertEquals("fileset31", fileset1.name()); + Assertions.assertEquals("comment", fileset1.comment()); + testProperties(props, fileset1.properties()); + Assertions.assertEquals(Fileset.Type.MANAGED, fileset1.type()); + Assertions.assertEquals("fileset31", fileset1.storageLocation()); + + boolean dropped = filesetOperationDispatcher.dropFileset(filesetIdent1); + Assertions.assertTrue(dropped); + } +} diff --git a/core/src/test/java/com/datastrato/gravitino/catalog/TestOperationDispatcher.java b/core/src/test/java/com/datastrato/gravitino/catalog/TestOperationDispatcher.java new file mode 100644 index 00000000000..f2f57ac1418 --- /dev/null +++ b/core/src/test/java/com/datastrato/gravitino/catalog/TestOperationDispatcher.java @@ -0,0 +1,132 @@ +/* + * Copyright 2023 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog; + +import static com.datastrato.gravitino.TestFilesetPropertiesMetadata.TEST_FILESET_HIDDEN_KEY; +import static com.datastrato.gravitino.connector.BasePropertiesMetadata.GRAVITINO_MANAGED_ENTITY; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.spy; + +import com.datastrato.gravitino.Catalog; +import com.datastrato.gravitino.Config; +import com.datastrato.gravitino.Configs; +import com.datastrato.gravitino.EntityStore; +import com.datastrato.gravitino.NameIdentifier; +import com.datastrato.gravitino.StringIdentifier; +import com.datastrato.gravitino.exceptions.IllegalNamespaceException; +import com.datastrato.gravitino.meta.AuditInfo; +import com.datastrato.gravitino.meta.BaseMetalake; +import com.datastrato.gravitino.meta.SchemaVersion; +import com.datastrato.gravitino.storage.IdGenerator; +import com.datastrato.gravitino.storage.RandomIdGenerator; +import com.datastrato.gravitino.storage.memory.TestMemoryEntityStore; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.time.Instant; +import java.util.Map; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.function.Executable; + +public abstract class TestOperationDispatcher { + + protected static EntityStore entityStore; + + protected static final IdGenerator idGenerator = new RandomIdGenerator(); + + protected static final String metalake = "metalake"; + + protected static final String catalog = "catalog"; + + protected static CatalogManager catalogManager; + + private static Config config; + + @BeforeAll + public static void setUp() throws IOException { + config = new Config(false) {}; + config.set(Configs.CATALOG_LOAD_ISOLATED, false); + + entityStore = spy(new TestMemoryEntityStore.InMemoryEntityStore()); + entityStore.initialize(config); + entityStore.setSerDe(null); + + BaseMetalake metalakeEntity = + BaseMetalake.builder() + .withId(1L) + .withName(metalake) + .withAuditInfo( + AuditInfo.builder().withCreator("test").withCreateTime(Instant.now()).build()) + .withVersion(SchemaVersion.V_0_1) + .build(); + entityStore.put(metalakeEntity, true); + + catalogManager = new CatalogManager(config, entityStore, idGenerator); + + NameIdentifier ident = NameIdentifier.of(metalake, catalog); + Map props = ImmutableMap.of(); + catalogManager.createCatalog(ident, Catalog.Type.RELATIONAL, "test", "comment", props); + } + + @AfterAll + public static void tearDown() throws IOException { + if (entityStore != null) { + entityStore.close(); + entityStore = null; + } + + if (catalogManager != null) { + catalogManager.close(); + catalogManager = null; + } + } + + @BeforeEach + public void beforeStart() throws IOException { + reset(entityStore); + } + + @Test + public void testGetCatalogIdentifier() { + OperationDispatcher dispatcher = new OperationDispatcher(null, null, null) {}; + + NameIdentifier id1 = NameIdentifier.of("a"); + assertThrows(IllegalNamespaceException.class, () -> dispatcher.getCatalogIdentifier(id1)); + + NameIdentifier id2 = NameIdentifier.of("a", "b"); + assertEquals(dispatcher.getCatalogIdentifier(id2), NameIdentifier.of("a", "b")); + + NameIdentifier id3 = NameIdentifier.of("a", "b", "c"); + assertEquals(dispatcher.getCatalogIdentifier(id3), NameIdentifier.of("a", "b")); + + NameIdentifier id4 = NameIdentifier.of("a", "b", "c", "d"); + assertEquals(dispatcher.getCatalogIdentifier(id4), NameIdentifier.of("a", "b")); + + NameIdentifier id5 = NameIdentifier.of("a", "b", "c", "d", "e"); + assertEquals(dispatcher.getCatalogIdentifier(id5), NameIdentifier.of("a", "b")); + } + + void testProperties(Map expectedProps, Map testProps) { + expectedProps.forEach( + (k, v) -> { + Assertions.assertEquals(v, testProps.get(k)); + }); + Assertions.assertFalse(testProps.containsKey(StringIdentifier.ID_KEY)); + Assertions.assertFalse(testProps.containsKey(GRAVITINO_MANAGED_ENTITY)); + Assertions.assertFalse(testProps.containsKey(TEST_FILESET_HIDDEN_KEY)); + } + + void testPropertyException(Executable operation, String... errorMessage) { + IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, operation); + for (String msg : errorMessage) { + Assertions.assertTrue(exception.getMessage().contains(msg)); + } + } +} diff --git a/core/src/test/java/com/datastrato/gravitino/catalog/TestSchemaOperationDispatcher.java b/core/src/test/java/com/datastrato/gravitino/catalog/TestSchemaOperationDispatcher.java new file mode 100644 index 00000000000..d44295364a3 --- /dev/null +++ b/core/src/test/java/com/datastrato/gravitino/catalog/TestSchemaOperationDispatcher.java @@ -0,0 +1,199 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog; + +import static com.datastrato.gravitino.Entity.EntityType.SCHEMA; +import static com.datastrato.gravitino.StringIdentifier.ID_KEY; +import static com.datastrato.gravitino.TestBasePropertiesMetadata.COMMENT_KEY; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.reset; + +import com.datastrato.gravitino.NameIdentifier; +import com.datastrato.gravitino.Namespace; +import com.datastrato.gravitino.auth.AuthConstants; +import com.datastrato.gravitino.exceptions.NoSuchEntityException; +import com.datastrato.gravitino.meta.AuditInfo; +import com.datastrato.gravitino.meta.SchemaEntity; +import com.datastrato.gravitino.rel.Schema; +import com.datastrato.gravitino.rel.SchemaChange; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.time.Instant; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +public class TestSchemaOperationDispatcher extends TestOperationDispatcher { + + private static SchemaOperationDispatcher dispatcher; + + @BeforeAll + public static void initialize() throws IOException { + dispatcher = new SchemaOperationDispatcher(catalogManager, entityStore, idGenerator); + } + + @Test + public void testCreateAndListSchemas() throws IOException { + Namespace ns = Namespace.of(metalake, catalog); + + NameIdentifier schemaIdent = NameIdentifier.of(ns, "schema1"); + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + Schema schema = dispatcher.createSchema(schemaIdent, "comment", props); + + // Check if the created Schema's field values are correct + Assertions.assertEquals("schema1", schema.name()); + Assertions.assertEquals("comment", schema.comment()); + testProperties(props, schema.properties()); + + // Test required table properties exception + Map illegalTableProperties = + new HashMap() { + { + put("k2", "v2"); + } + }; + + testPropertyException( + () -> dispatcher.createSchema(schemaIdent, "comment", illegalTableProperties), + "Properties are required and must be set"); + + // Test reserved table properties exception + illegalTableProperties.put(COMMENT_KEY, "table comment"); + illegalTableProperties.put(ID_KEY, "gravitino.v1.uidfdsafdsa"); + testPropertyException( + () -> dispatcher.createSchema(schemaIdent, "comment", illegalTableProperties), + "Properties are reserved and cannot be set", + "comment", + "gravitino.identifier"); + + // Check if the Schema entity is stored in the EntityStore + SchemaEntity schemaEntity = entityStore.get(schemaIdent, SCHEMA, SchemaEntity.class); + Assertions.assertNotNull(schemaEntity); + Assertions.assertEquals("schema1", schemaEntity.name()); + Assertions.assertNotNull(schemaEntity.id()); + + Optional ident1 = + Arrays.stream(dispatcher.listSchemas(ns)) + .filter(s -> s.name().equals("schema1")) + .findFirst(); + Assertions.assertTrue(ident1.isPresent()); + + // Test when the entity store failed to put the schema entity + doThrow(new IOException()).when(entityStore).put(any(), anyBoolean()); + NameIdentifier schemaIdent2 = NameIdentifier.of(ns, "schema2"); + Schema schema2 = dispatcher.createSchema(schemaIdent2, "comment", props); + + // Check if the created Schema's field values are correct + Assertions.assertEquals("schema2", schema2.name()); + Assertions.assertEquals("comment", schema2.comment()); + testProperties(props, schema2.properties()); + + // Check if the Schema entity is stored in the EntityStore + Assertions.assertFalse(entityStore.exists(schemaIdent2, SCHEMA)); + Assertions.assertThrows( + NoSuchEntityException.class, + () -> entityStore.get(schemaIdent2, SCHEMA, SchemaEntity.class)); + + // Audit info is gotten from the catalog, not from the entity store + Assertions.assertEquals("test", schema2.auditInfo().creator()); + } + + @Test + public void testCreateAndAlterSchema() throws IOException { + NameIdentifier schemaIdent = NameIdentifier.of(metalake, catalog, "schema21"); + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + Schema schema = dispatcher.createSchema(schemaIdent, "comment", props); + + // Test immutable schema properties + SchemaChange[] illegalChange = + new SchemaChange[] {SchemaChange.setProperty(COMMENT_KEY, "new comment")}; + testPropertyException( + () -> dispatcher.alterSchema(schemaIdent, illegalChange), + "Property comment is immutable or reserved, cannot be set"); + + SchemaChange[] changes = + new SchemaChange[] { + SchemaChange.setProperty("k3", "v3"), SchemaChange.removeProperty("k1") + }; + + Schema alteredSchema = dispatcher.alterSchema(schemaIdent, changes); + Assertions.assertEquals(schema.name(), alteredSchema.name()); + Assertions.assertEquals(schema.comment(), alteredSchema.comment()); + Map expectedProps = ImmutableMap.of("k2", "v2", "k3", "v3"); + testProperties(expectedProps, alteredSchema.properties()); + // Audit info is gotten from gravitino entity store. + Assertions.assertEquals(AuthConstants.ANONYMOUS_USER, alteredSchema.auditInfo().creator()); + Assertions.assertEquals(AuthConstants.ANONYMOUS_USER, alteredSchema.auditInfo().lastModifier()); + + // Case 2: Test if the schema is not found in entity store + doThrow(new NoSuchEntityException("mock error")) + .when(entityStore) + .update(any(), any(), any(), any()); + Schema alteredSchema1 = dispatcher.alterSchema(schemaIdent, changes); + Assertions.assertEquals(schema.name(), alteredSchema1.name()); + Assertions.assertEquals(schema.comment(), alteredSchema1.comment()); + testProperties(expectedProps, alteredSchema1.properties()); + // Audit info is gotten from catalog, not from the entity store + Assertions.assertEquals("test", alteredSchema1.auditInfo().creator()); + Assertions.assertEquals("test", alteredSchema1.auditInfo().lastModifier()); + + // Case 3: Test if entity store is failed to get the schema entity + reset(entityStore); + doThrow(new IOException()).when(entityStore).update(any(), any(), any(), any()); + Schema alteredSchema2 = dispatcher.alterSchema(schemaIdent, changes); + Assertions.assertEquals(schema.name(), alteredSchema2.name()); + Assertions.assertEquals(schema.comment(), alteredSchema2.comment()); + testProperties(expectedProps, alteredSchema2.properties()); + // Audit info is gotten from catalog, not from the entity store + Assertions.assertEquals("test", alteredSchema2.auditInfo().creator()); + Assertions.assertEquals("test", alteredSchema1.auditInfo().lastModifier()); + + // Case 4: Test if the fetched schema entity is matched. + reset(entityStore); + SchemaEntity unmatchedEntity = + SchemaEntity.builder() + .withId(1L) + .withName("schema21") + .withNamespace(Namespace.of(metalake, catalog)) + .withAuditInfo( + AuditInfo.builder() + .withCreator(AuthConstants.ANONYMOUS_USER) + .withCreateTime(Instant.now()) + .build()) + .build(); + doReturn(unmatchedEntity).when(entityStore).update(any(), any(), any(), any()); + Schema alteredSchema3 = dispatcher.alterSchema(schemaIdent, changes); + Assertions.assertEquals(schema.name(), alteredSchema3.name()); + Assertions.assertEquals(schema.comment(), alteredSchema3.comment()); + testProperties(expectedProps, alteredSchema3.properties()); + // Audit info is gotten from catalog, not from the entity store + Assertions.assertEquals("test", alteredSchema3.auditInfo().creator()); + Assertions.assertEquals("test", alteredSchema1.auditInfo().lastModifier()); + } + + @Test + public void testCreateAndDropSchema() throws IOException { + NameIdentifier schemaIdent = NameIdentifier.of(metalake, catalog, "schema31"); + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + + dispatcher.createSchema(schemaIdent, "comment", props); + + boolean dropped = dispatcher.dropSchema(schemaIdent, false); + Assertions.assertTrue(dropped); + + // Test if entity store is failed to drop the schema entity + dispatcher.createSchema(schemaIdent, "comment", props); + doThrow(new IOException()).when(entityStore).delete(any(), any(), anyBoolean()); + Assertions.assertThrows( + RuntimeException.class, () -> dispatcher.dropSchema(schemaIdent, false)); + } +} diff --git a/core/src/test/java/com/datastrato/gravitino/catalog/TestTableOperationDispatcher.java b/core/src/test/java/com/datastrato/gravitino/catalog/TestTableOperationDispatcher.java new file mode 100644 index 00000000000..4dc1edab7e8 --- /dev/null +++ b/core/src/test/java/com/datastrato/gravitino/catalog/TestTableOperationDispatcher.java @@ -0,0 +1,277 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog; + +import static com.datastrato.gravitino.Entity.EntityType.TABLE; +import static com.datastrato.gravitino.StringIdentifier.ID_KEY; +import static com.datastrato.gravitino.TestBasePropertiesMetadata.COMMENT_KEY; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.reset; + +import com.datastrato.gravitino.NameIdentifier; +import com.datastrato.gravitino.Namespace; +import com.datastrato.gravitino.TestColumn; +import com.datastrato.gravitino.auth.AuthConstants; +import com.datastrato.gravitino.exceptions.NoSuchEntityException; +import com.datastrato.gravitino.meta.AuditInfo; +import com.datastrato.gravitino.meta.TableEntity; +import com.datastrato.gravitino.rel.Column; +import com.datastrato.gravitino.rel.Table; +import com.datastrato.gravitino.rel.TableChange; +import com.datastrato.gravitino.rel.expressions.transforms.Transform; +import com.datastrato.gravitino.rel.types.Types; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.time.Instant; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +public class TestTableOperationDispatcher extends TestOperationDispatcher { + private static TableOperationDispatcher tableOperationDispatcher; + private static SchemaOperationDispatcher schemaOperationDispatcher; + + @BeforeAll + public static void initialize() throws IOException { + schemaOperationDispatcher = + new SchemaOperationDispatcher(catalogManager, entityStore, idGenerator); + tableOperationDispatcher = + new TableOperationDispatcher(catalogManager, entityStore, idGenerator); + } + + @Test + public void testCreateAndListTables() throws IOException { + Namespace tableNs = Namespace.of(metalake, catalog, "schema41"); + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + schemaOperationDispatcher.createSchema(NameIdentifier.of(tableNs.levels()), "comment", props); + + NameIdentifier tableIdent1 = NameIdentifier.of(tableNs, "table1"); + Column[] columns = + new Column[] { + TestColumn.builder().withName("col1").withType(Types.StringType.get()).build(), + TestColumn.builder().withName("col2").withType(Types.StringType.get()).build() + }; + + Table table1 = + tableOperationDispatcher.createTable( + tableIdent1, columns, "comment", props, new Transform[0]); + Assertions.assertEquals("table1", table1.name()); + Assertions.assertEquals("comment", table1.comment()); + testProperties(props, table1.properties()); + Assertions.assertEquals(0, table1.partitioning().length); + Assertions.assertArrayEquals(columns, table1.columns()); + + // Test required table properties exception + Map illegalTableProperties = + new HashMap() { + { + put("k2", "v2"); + } + }; + testPropertyException( + () -> + tableOperationDispatcher.createTable( + tableIdent1, columns, "comment", illegalTableProperties, new Transform[0]), + "Properties are required and must be set"); + + // Test reserved table properties exception + illegalTableProperties.put(COMMENT_KEY, "table comment"); + illegalTableProperties.put(ID_KEY, "gravitino.v1.uidfdsafdsa"); + testPropertyException( + () -> + tableOperationDispatcher.createTable( + tableIdent1, columns, "comment", illegalTableProperties, new Transform[0]), + "Properties are reserved and cannot be set", + "comment", + "gravitino.identifier"); + + // Check if the Table entity is stored in the EntityStore + TableEntity tableEntity = entityStore.get(tableIdent1, TABLE, TableEntity.class); + Assertions.assertNotNull(tableEntity); + Assertions.assertEquals("table1", tableEntity.name()); + + Assertions.assertFalse(table1.properties().containsKey(ID_KEY)); + + Optional ident1 = + Arrays.stream(tableOperationDispatcher.listTables(tableNs)) + .filter(s -> s.name().equals("table1")) + .findFirst(); + Assertions.assertTrue(ident1.isPresent()); + + // Test when the entity store failed to put the table entity + doThrow(new IOException()).when(entityStore).put(any(), anyBoolean()); + NameIdentifier tableIdent2 = NameIdentifier.of(tableNs, "table2"); + Table table2 = + tableOperationDispatcher.createTable( + tableIdent2, columns, "comment", props, new Transform[0]); + + // Check if the created Schema's field values are correct + Assertions.assertEquals("table2", table2.name()); + Assertions.assertEquals("comment", table2.comment()); + testProperties(props, table2.properties()); + + // Check if the Table entity is stored in the EntityStore + Assertions.assertFalse(entityStore.exists(tableIdent2, TABLE)); + Assertions.assertThrows( + NoSuchEntityException.class, () -> entityStore.get(tableIdent2, TABLE, TableEntity.class)); + + // Audit info is gotten from the catalog, not from the entity store + Assertions.assertEquals("test", table2.auditInfo().creator()); + } + + @Test + public void testCreateAndLoadTable() throws IOException { + Namespace tableNs = Namespace.of(metalake, catalog, "schema51"); + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + schemaOperationDispatcher.createSchema(NameIdentifier.of(tableNs.levels()), "comment", props); + + NameIdentifier tableIdent1 = NameIdentifier.of(tableNs, "table11"); + Column[] columns = + new Column[] { + TestColumn.builder().withName("col1").withType(Types.StringType.get()).build(), + TestColumn.builder().withName("col2").withType(Types.StringType.get()).build() + }; + + Table table1 = + tableOperationDispatcher.createTable( + tableIdent1, columns, "comment", props, new Transform[0]); + Table loadedTable1 = tableOperationDispatcher.loadTable(tableIdent1); + Assertions.assertEquals(table1.name(), loadedTable1.name()); + Assertions.assertEquals(table1.comment(), loadedTable1.comment()); + testProperties(table1.properties(), loadedTable1.properties()); + Assertions.assertEquals(0, loadedTable1.partitioning().length); + Assertions.assertArrayEquals(table1.columns(), loadedTable1.columns()); + // Audit info is gotten from the entity store + Assertions.assertEquals(AuthConstants.ANONYMOUS_USER, loadedTable1.auditInfo().creator()); + + // Case 2: Test if the table entity is not found in the entity store + reset(entityStore); + doThrow(new NoSuchEntityException("")).when(entityStore).get(any(), any(), any()); + Table loadedTable2 = tableOperationDispatcher.loadTable(tableIdent1); + // Audit info is gotten from the catalog, not from the entity store + Assertions.assertEquals("test", loadedTable2.auditInfo().creator()); + + // Case 3: Test if the entity store is failed to get the table entity + reset(entityStore); + doThrow(new IOException()).when(entityStore).get(any(), any(), any()); + Table loadedTable3 = tableOperationDispatcher.loadTable(tableIdent1); + // Audit info is gotten from the catalog, not from the entity store + Assertions.assertEquals("test", loadedTable3.auditInfo().creator()); + + // Case 4: Test if the table entity is not matched + reset(entityStore); + TableEntity tableEntity = + TableEntity.builder() + .withId(1L) + .withName("table11") + .withNamespace(tableNs) + .withAuditInfo( + AuditInfo.builder().withCreator("gravitino").withCreateTime(Instant.now()).build()) + .build(); + doReturn(tableEntity).when(entityStore).get(any(), any(), any()); + Table loadedTable4 = tableOperationDispatcher.loadTable(tableIdent1); + // Audit info is gotten from the catalog, not from the entity store + Assertions.assertEquals("test", loadedTable4.auditInfo().creator()); + } + + @Test + public void testCreateAndAlterTable() throws IOException { + Namespace tableNs = Namespace.of(metalake, catalog, "schema61"); + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + schemaOperationDispatcher.createSchema(NameIdentifier.of(tableNs.levels()), "comment", props); + + NameIdentifier tableIdent = NameIdentifier.of(tableNs, "table21"); + Column[] columns = + new Column[] { + TestColumn.builder().withName("col1").withType(Types.StringType.get()).build(), + TestColumn.builder().withName("col2").withType(Types.StringType.get()).build() + }; + + Table table = + tableOperationDispatcher.createTable( + tableIdent, columns, "comment", props, new Transform[0]); + + // Test immutable table properties + TableChange[] illegalChange = + new TableChange[] {TableChange.setProperty(COMMENT_KEY, "new comment")}; + testPropertyException( + () -> tableOperationDispatcher.alterTable(tableIdent, illegalChange), + "Property comment is immutable or reserved, cannot be set"); + + TableChange[] changes = + new TableChange[] {TableChange.setProperty("k3", "v3"), TableChange.removeProperty("k1")}; + + Table alteredTable = tableOperationDispatcher.alterTable(tableIdent, changes); + Assertions.assertEquals(table.name(), alteredTable.name()); + Assertions.assertEquals(table.comment(), alteredTable.comment()); + Map expectedProps = ImmutableMap.of("k2", "v2", "k3", "v3"); + testProperties(expectedProps, alteredTable.properties()); + // Audit info is gotten from gravitino entity store + Assertions.assertEquals(AuthConstants.ANONYMOUS_USER, alteredTable.auditInfo().creator()); + Assertions.assertEquals(AuthConstants.ANONYMOUS_USER, alteredTable.auditInfo().lastModifier()); + + // Case 2: Test if the table entity is not found in the entity store + reset(entityStore); + doThrow(new NoSuchEntityException("")).when(entityStore).update(any(), any(), any(), any()); + Table alteredTable2 = tableOperationDispatcher.alterTable(tableIdent, changes); + // Audit info is gotten from the catalog, not from the entity store + Assertions.assertEquals("test", alteredTable2.auditInfo().creator()); + Assertions.assertEquals("test", alteredTable2.auditInfo().lastModifier()); + + // Case 3: Test if the entity store is failed to update the table entity + reset(entityStore); + doThrow(new IOException()).when(entityStore).update(any(), any(), any(), any()); + Table alteredTable3 = tableOperationDispatcher.alterTable(tableIdent, changes); + // Audit info is gotten from the catalog, not from the entity store + Assertions.assertEquals("test", alteredTable3.auditInfo().creator()); + Assertions.assertEquals("test", alteredTable3.auditInfo().lastModifier()); + + // Case 4: Test if the table entity is not matched + reset(entityStore); + TableEntity unmatchedEntity = + TableEntity.builder() + .withId(1L) + .withName("table21") + .withNamespace(tableNs) + .withAuditInfo( + AuditInfo.builder().withCreator("gravitino").withCreateTime(Instant.now()).build()) + .build(); + doReturn(unmatchedEntity).when(entityStore).update(any(), any(), any(), any()); + Table alteredTable4 = tableOperationDispatcher.alterTable(tableIdent, changes); + // Audit info is gotten from the catalog, not from the entity store + Assertions.assertEquals("test", alteredTable4.auditInfo().creator()); + Assertions.assertEquals("test", alteredTable4.auditInfo().lastModifier()); + } + + @Test + public void testCreateAndDropTable() throws IOException { + NameIdentifier tableIdent = NameIdentifier.of(metalake, catalog, "schema71", "table31"); + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + Column[] columns = + new Column[] { + TestColumn.builder().withName("col1").withType(Types.StringType.get()).build(), + TestColumn.builder().withName("col2").withType(Types.StringType.get()).build() + }; + + tableOperationDispatcher.createTable(tableIdent, columns, "comment", props, new Transform[0]); + + boolean dropped = tableOperationDispatcher.dropTable(tableIdent); + Assertions.assertTrue(dropped); + + // Test if the entity store is failed to drop the table entity + tableOperationDispatcher.createTable(tableIdent, columns, "comment", props, new Transform[0]); + reset(entityStore); + doThrow(new IOException()).when(entityStore).delete(any(), any(), anyBoolean()); + Assertions.assertThrows( + RuntimeException.class, () -> tableOperationDispatcher.dropTable(tableIdent)); + } +} diff --git a/server/src/main/java/com/datastrato/gravitino/server/GravitinoServer.java b/server/src/main/java/com/datastrato/gravitino/server/GravitinoServer.java index 1651f493f6a..83533364a07 100644 --- a/server/src/main/java/com/datastrato/gravitino/server/GravitinoServer.java +++ b/server/src/main/java/com/datastrato/gravitino/server/GravitinoServer.java @@ -6,7 +6,9 @@ import com.datastrato.gravitino.GravitinoEnv; import com.datastrato.gravitino.catalog.CatalogManager; -import com.datastrato.gravitino.catalog.CatalogOperationDispatcher; +import com.datastrato.gravitino.catalog.FilesetOperationDispatcher; +import com.datastrato.gravitino.catalog.SchemaOperationDispatcher; +import com.datastrato.gravitino.catalog.TableOperationDispatcher; import com.datastrato.gravitino.metalake.MetalakeManager; import com.datastrato.gravitino.metrics.MetricsSystem; import com.datastrato.gravitino.metrics.source.MetricsSource; @@ -73,8 +75,14 @@ private void initializeRestApi() { protected void configure() { bind(gravitinoEnv.metalakesManager()).to(MetalakeManager.class).ranked(1); bind(gravitinoEnv.catalogManager()).to(CatalogManager.class).ranked(1); - bind(gravitinoEnv.catalogOperationDispatcher()) - .to(CatalogOperationDispatcher.class) + bind(gravitinoEnv.schemaOperationDispatcher()) + .to(SchemaOperationDispatcher.class) + .ranked(1); + bind(gravitinoEnv.tableOperationDispatcher()) + .to(TableOperationDispatcher.class) + .ranked(1); + bind(gravitinoEnv.filesetOperationDispatcher()) + .to(FilesetOperationDispatcher.class) .ranked(1); } }); diff --git a/server/src/main/java/com/datastrato/gravitino/server/web/rest/FilesetOperations.java b/server/src/main/java/com/datastrato/gravitino/server/web/rest/FilesetOperations.java index 8e2b52e8cd7..1e24834e48d 100644 --- a/server/src/main/java/com/datastrato/gravitino/server/web/rest/FilesetOperations.java +++ b/server/src/main/java/com/datastrato/gravitino/server/web/rest/FilesetOperations.java @@ -8,7 +8,7 @@ import com.codahale.metrics.annotation.Timed; import com.datastrato.gravitino.NameIdentifier; import com.datastrato.gravitino.Namespace; -import com.datastrato.gravitino.catalog.CatalogOperationDispatcher; +import com.datastrato.gravitino.catalog.FilesetOperationDispatcher; import com.datastrato.gravitino.dto.requests.FilesetCreateRequest; import com.datastrato.gravitino.dto.requests.FilesetUpdateRequest; import com.datastrato.gravitino.dto.requests.FilesetUpdatesRequest; @@ -42,12 +42,12 @@ public class FilesetOperations { private static final Logger LOG = LoggerFactory.getLogger(FilesetOperations.class); - private final CatalogOperationDispatcher dispatcher; + private final FilesetOperationDispatcher dispatcher; @Context private HttpServletRequest httpRequest; @Inject - public FilesetOperations(CatalogOperationDispatcher dispatcher) { + public FilesetOperations(FilesetOperationDispatcher dispatcher) { this.dispatcher = dispatcher; } diff --git a/server/src/main/java/com/datastrato/gravitino/server/web/rest/PartitionOperations.java b/server/src/main/java/com/datastrato/gravitino/server/web/rest/PartitionOperations.java index 118d1e63a8a..691ea2aafbd 100644 --- a/server/src/main/java/com/datastrato/gravitino/server/web/rest/PartitionOperations.java +++ b/server/src/main/java/com/datastrato/gravitino/server/web/rest/PartitionOperations.java @@ -10,7 +10,7 @@ import com.codahale.metrics.annotation.ResponseMetered; import com.codahale.metrics.annotation.Timed; import com.datastrato.gravitino.NameIdentifier; -import com.datastrato.gravitino.catalog.CatalogOperationDispatcher; +import com.datastrato.gravitino.catalog.TableOperationDispatcher; import com.datastrato.gravitino.dto.rel.partitions.PartitionDTO; import com.datastrato.gravitino.dto.requests.AddPartitionsRequest; import com.datastrato.gravitino.dto.responses.PartitionListResponse; @@ -39,11 +39,11 @@ @Path("/metalakes/{metalake}/catalogs/{catalog}/schemas/{schema}/tables/{table}/partitions") public class PartitionOperations { - private final CatalogOperationDispatcher dispatcher; + private final TableOperationDispatcher dispatcher; @Context private HttpServletRequest httpRequest; @Inject - public PartitionOperations(CatalogOperationDispatcher dispatcher) { + public PartitionOperations(TableOperationDispatcher dispatcher) { this.dispatcher = dispatcher; } diff --git a/server/src/main/java/com/datastrato/gravitino/server/web/rest/SchemaOperations.java b/server/src/main/java/com/datastrato/gravitino/server/web/rest/SchemaOperations.java index 399f6001a19..bc4dcc009d9 100644 --- a/server/src/main/java/com/datastrato/gravitino/server/web/rest/SchemaOperations.java +++ b/server/src/main/java/com/datastrato/gravitino/server/web/rest/SchemaOperations.java @@ -8,7 +8,7 @@ import com.codahale.metrics.annotation.Timed; import com.datastrato.gravitino.NameIdentifier; import com.datastrato.gravitino.Namespace; -import com.datastrato.gravitino.catalog.CatalogOperationDispatcher; +import com.datastrato.gravitino.catalog.SchemaOperationDispatcher; import com.datastrato.gravitino.dto.requests.SchemaCreateRequest; import com.datastrato.gravitino.dto.requests.SchemaUpdateRequest; import com.datastrato.gravitino.dto.requests.SchemaUpdatesRequest; @@ -47,12 +47,12 @@ public class SchemaOperations { private static final Logger LOG = LoggerFactory.getLogger(SchemaOperations.class); - private final CatalogOperationDispatcher dispatcher; + private final SchemaOperationDispatcher dispatcher; @Context private HttpServletRequest httpRequest; @Inject - public SchemaOperations(CatalogOperationDispatcher dispatcher) { + public SchemaOperations(SchemaOperationDispatcher dispatcher) { this.dispatcher = dispatcher; } diff --git a/server/src/main/java/com/datastrato/gravitino/server/web/rest/TableOperations.java b/server/src/main/java/com/datastrato/gravitino/server/web/rest/TableOperations.java index f2bcd9026d3..e4992700872 100644 --- a/server/src/main/java/com/datastrato/gravitino/server/web/rest/TableOperations.java +++ b/server/src/main/java/com/datastrato/gravitino/server/web/rest/TableOperations.java @@ -11,7 +11,7 @@ import com.codahale.metrics.annotation.Timed; import com.datastrato.gravitino.NameIdentifier; import com.datastrato.gravitino.Namespace; -import com.datastrato.gravitino.catalog.CatalogOperationDispatcher; +import com.datastrato.gravitino.catalog.TableOperationDispatcher; import com.datastrato.gravitino.dto.requests.TableCreateRequest; import com.datastrato.gravitino.dto.requests.TableUpdateRequest; import com.datastrato.gravitino.dto.requests.TableUpdatesRequest; @@ -46,12 +46,12 @@ public class TableOperations { private static final Logger LOG = LoggerFactory.getLogger(TableOperations.class); - private final CatalogOperationDispatcher dispatcher; + private final TableOperationDispatcher dispatcher; @Context private HttpServletRequest httpRequest; @Inject - public TableOperations(CatalogOperationDispatcher dispatcher) { + public TableOperations(TableOperationDispatcher dispatcher) { this.dispatcher = dispatcher; } diff --git a/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestFilesetOperations.java b/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestFilesetOperations.java index 728d8e20780..0d1112cda09 100644 --- a/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestFilesetOperations.java +++ b/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestFilesetOperations.java @@ -16,7 +16,7 @@ import com.datastrato.gravitino.Config; import com.datastrato.gravitino.GravitinoEnv; import com.datastrato.gravitino.NameIdentifier; -import com.datastrato.gravitino.catalog.CatalogOperationDispatcher; +import com.datastrato.gravitino.catalog.FilesetOperationDispatcher; import com.datastrato.gravitino.dto.file.FilesetDTO; import com.datastrato.gravitino.dto.requests.FilesetCreateRequest; import com.datastrato.gravitino.dto.requests.FilesetUpdateRequest; @@ -62,7 +62,7 @@ public HttpServletRequest get() { } } - private CatalogOperationDispatcher dispatcher = mock(CatalogOperationDispatcher.class); + private FilesetOperationDispatcher dispatcher = mock(FilesetOperationDispatcher.class); private final String metalake = "metalake1"; @@ -94,7 +94,7 @@ protected Application configure() { new AbstractBinder() { @Override protected void configure() { - bind(dispatcher).to(CatalogOperationDispatcher.class).ranked(2); + bind(dispatcher).to(FilesetOperationDispatcher.class).ranked(2); bindFactory(MockServletRequestFactory.class).to(HttpServletRequest.class); } }); diff --git a/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestPartitionOperations.java b/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestPartitionOperations.java index 1a5803bb7c1..1f5d990330c 100644 --- a/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestPartitionOperations.java +++ b/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestPartitionOperations.java @@ -16,7 +16,7 @@ import com.datastrato.gravitino.Config; import com.datastrato.gravitino.GravitinoEnv; -import com.datastrato.gravitino.catalog.CatalogOperationDispatcher; +import com.datastrato.gravitino.catalog.TableOperationDispatcher; import com.datastrato.gravitino.dto.rel.partitions.PartitionDTO; import com.datastrato.gravitino.dto.requests.AddPartitionsRequest; import com.datastrato.gravitino.dto.responses.ErrorConstants; @@ -86,7 +86,7 @@ public HttpServletRequest get() { } } - private CatalogOperationDispatcher dispatcher = mock(CatalogOperationDispatcher.class); + private TableOperationDispatcher dispatcher = mock(TableOperationDispatcher.class); private final String metalake = "metalake1"; private final String catalog = "catalog1"; private final String schema = "schema1"; @@ -116,7 +116,7 @@ protected Application configure() { new AbstractBinder() { @Override protected void configure() { - bind(dispatcher).to(CatalogOperationDispatcher.class).ranked(2); + bind(dispatcher).to(TableOperationDispatcher.class).ranked(2); bindFactory(MockServletRequestFactory.class).to(HttpServletRequest.class); } }); diff --git a/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestSchemaOperations.java b/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestSchemaOperations.java index d2d5e3041e2..90dec8989ff 100644 --- a/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestSchemaOperations.java +++ b/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestSchemaOperations.java @@ -17,7 +17,7 @@ import com.datastrato.gravitino.Config; import com.datastrato.gravitino.GravitinoEnv; import com.datastrato.gravitino.NameIdentifier; -import com.datastrato.gravitino.catalog.CatalogOperationDispatcher; +import com.datastrato.gravitino.catalog.SchemaOperationDispatcher; import com.datastrato.gravitino.dto.rel.SchemaDTO; import com.datastrato.gravitino.dto.requests.SchemaCreateRequest; import com.datastrato.gravitino.dto.requests.SchemaUpdateRequest; @@ -63,7 +63,7 @@ public HttpServletRequest get() { } } - private CatalogOperationDispatcher dispatcher = mock(CatalogOperationDispatcher.class); + private SchemaOperationDispatcher dispatcher = mock(SchemaOperationDispatcher.class); private final String metalake = "metalake1"; @@ -93,7 +93,7 @@ protected Application configure() { new AbstractBinder() { @Override protected void configure() { - bind(dispatcher).to(CatalogOperationDispatcher.class).ranked(2); + bind(dispatcher).to(SchemaOperationDispatcher.class).ranked(2); bindFactory(MockServletRequestFactory.class).to(HttpServletRequest.class); } }); diff --git a/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestTableOperations.java b/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestTableOperations.java index eccd92e3d34..185f40dc4be 100644 --- a/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestTableOperations.java +++ b/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestTableOperations.java @@ -17,7 +17,7 @@ import com.datastrato.gravitino.Config; import com.datastrato.gravitino.GravitinoEnv; import com.datastrato.gravitino.NameIdentifier; -import com.datastrato.gravitino.catalog.CatalogOperationDispatcher; +import com.datastrato.gravitino.catalog.TableOperationDispatcher; import com.datastrato.gravitino.dto.rel.ColumnDTO; import com.datastrato.gravitino.dto.rel.DistributionDTO; import com.datastrato.gravitino.dto.rel.SortOrderDTO; @@ -87,7 +87,7 @@ public HttpServletRequest get() { } } - private CatalogOperationDispatcher dispatcher = mock(CatalogOperationDispatcher.class); + private TableOperationDispatcher dispatcher = mock(TableOperationDispatcher.class); private final String metalake = "metalake1"; @@ -119,7 +119,7 @@ protected Application configure() { new AbstractBinder() { @Override protected void configure() { - bind(dispatcher).to(CatalogOperationDispatcher.class).ranked(2); + bind(dispatcher).to(TableOperationDispatcher.class).ranked(2); bindFactory(MockServletRequestFactory.class).to(HttpServletRequest.class); } });