From d89c833ea81f5b989f4708ff83d0f5f533eeba23 Mon Sep 17 00:00:00 2001 From: mchades Date: Fri, 29 Mar 2024 16:50:16 +0800 Subject: [PATCH 01/20] [#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 02/20] [#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 03/20] [#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); } }); From 2dc8c5458727cb7c7db088528de2e185d6f24866 Mon Sep 17 00:00:00 2001 From: YongXing Date: Sat, 30 Mar 2024 09:59:35 +0800 Subject: [PATCH 04/20] [#2591] refactor: refactor garbage collector config and check its value validity (#2592) ### What changes were proposed in this pull request? - refactor `KV_DELETE_AFTER_TIME` to `STORE_DELETE_AFTER_TIME` for various store backend - check config value validity by `Configs` - define a new config `VERSION_RETENTION_COUNT` for the count of versions allowed to be retained, which will be used to delete old versions data. ### Why are the changes needed? Fix: #2591 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By hand. --------- Co-authored-by: YxAc Co-authored-by: YxAc --- .../hadoop/TestHadoopCatalogOperations.java | 4 +- .../kafka/TestKafkaCatalogOperations.java | 4 +- .../java/com/datastrato/gravitino/Config.java | 12 ++-- .../gravitino/config/TestConfigEntry.java | 4 +- .../com/datastrato/gravitino/Configs.java | 70 ++++++++++++++++++- .../storage/kv/KvGarbageCollector.java | 20 +----- .../gravitino/storage/TestEntityStorage.java | 4 +- .../storage/kv/TestEntityKeyEncoding.java | 4 +- .../storage/kv/TestKvEntityStorage.java | 19 +++-- .../storage/kv/TestKvGarbageCollector.java | 48 +++++-------- .../storage/kv/TestKvNameMappingService.java | 4 +- .../storage/kv/TestStorageVersion.java | 4 +- docs/gravitino-server-config.md | 27 +++---- 13 files changed, 131 insertions(+), 93 deletions(-) diff --git a/catalogs/catalog-hadoop/src/test/java/com/datastrato/gravitino/catalog/hadoop/TestHadoopCatalogOperations.java b/catalogs/catalog-hadoop/src/test/java/com/datastrato/gravitino/catalog/hadoop/TestHadoopCatalogOperations.java index e7d1f2b57ef..1eca47aba44 100644 --- a/catalogs/catalog-hadoop/src/test/java/com/datastrato/gravitino/catalog/hadoop/TestHadoopCatalogOperations.java +++ b/catalogs/catalog-hadoop/src/test/java/com/datastrato/gravitino/catalog/hadoop/TestHadoopCatalogOperations.java @@ -8,7 +8,7 @@ import static com.datastrato.gravitino.Configs.ENTITY_KV_STORE; import static com.datastrato.gravitino.Configs.ENTITY_STORE; import static com.datastrato.gravitino.Configs.ENTRY_KV_ROCKSDB_BACKEND_PATH; -import static com.datastrato.gravitino.Configs.KV_DELETE_AFTER_TIME; +import static com.datastrato.gravitino.Configs.STORE_DELETE_AFTER_TIME; import static com.datastrato.gravitino.Configs.STORE_TRANSACTION_MAX_SKEW_TIME; import static com.datastrato.gravitino.connector.BaseCatalog.CATALOG_BYPASS_PREFIX; @@ -78,7 +78,7 @@ public static void setUp() { Assertions.assertEquals(ROCKS_DB_STORE_PATH, config.get(ENTRY_KV_ROCKSDB_BACKEND_PATH)); Mockito.when(config.get(STORE_TRANSACTION_MAX_SKEW_TIME)).thenReturn(1000L); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); + Mockito.when(config.get(STORE_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); store = EntityStoreFactory.createEntityStore(config); store.initialize(config); 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 9dacdd1625d..fe752a1b4f0 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 @@ -9,7 +9,7 @@ import static com.datastrato.gravitino.Configs.ENTITY_KV_STORE; import static com.datastrato.gravitino.Configs.ENTITY_STORE; import static com.datastrato.gravitino.Configs.ENTRY_KV_ROCKSDB_BACKEND_PATH; -import static com.datastrato.gravitino.Configs.KV_DELETE_AFTER_TIME; +import static com.datastrato.gravitino.Configs.STORE_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; @@ -73,7 +73,7 @@ public static void setUp() { Assertions.assertEquals(ROCKS_DB_STORE_PATH, config.get(ENTRY_KV_ROCKSDB_BACKEND_PATH)); Mockito.when(config.get(STORE_TRANSACTION_MAX_SKEW_TIME)).thenReturn(1000L); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); + Mockito.when(config.get(STORE_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); store = EntityStoreFactory.createEntityStore(config); store.initialize(config); diff --git a/common/src/main/java/com/datastrato/gravitino/Config.java b/common/src/main/java/com/datastrato/gravitino/Config.java index cc00c8c7670..95b438d5517 100644 --- a/common/src/main/java/com/datastrato/gravitino/Config.java +++ b/common/src/main/java/com/datastrato/gravitino/Config.java @@ -33,13 +33,13 @@ public abstract class Config { private final ConcurrentMap configMap; private final Map deprecatedConfigMap; - // Constant Array to hold all deprecated configurations + // Constant Array to hold all deprecated configuration keys, when a configuration is deprecated, + // we should add it here. private final DeprecatedConfig[] deprecatedConfigs = { - // Example deprecated configuration - // new DeprecatedConfig( - // "gravitino.test.string", - // "1.0", - // "Please use gravitino.test.string1 instead."), + new DeprecatedConfig( + "gravitino.entity.store.kv.deleteAfterTimeMs", + "0.5.0", + "Please use gravitino.entity.store.deleteAfterTimeMs instead."), }; /** diff --git a/common/src/test/java/com/datastrato/gravitino/config/TestConfigEntry.java b/common/src/test/java/com/datastrato/gravitino/config/TestConfigEntry.java index 4d8eb2ac6bd..2eb40c418d9 100644 --- a/common/src/test/java/com/datastrato/gravitino/config/TestConfigEntry.java +++ b/common/src/test/java/com/datastrato/gravitino/config/TestConfigEntry.java @@ -91,7 +91,7 @@ public void testConfWithAlternatives() { ConfigEntry testConf = new ConfigBuilder("gravitino.test.string") .alternatives( - Lists.newArrayList("gravitino.test.string.alt1", "gravitino.test.string.alt1")) + Lists.newArrayList("gravitino.test.string.alt1", "gravitino.test.string.alt2")) .stringConf() .createWithDefault("test"); @@ -101,7 +101,7 @@ public void testConfWithAlternatives() { ConfigEntry testConf1 = new ConfigBuilder("gravitino.test.string.no-exist") .alternatives( - Lists.newArrayList("gravitino.test.string.alt1", "gravitino.test.string.alt1")) + Lists.newArrayList("gravitino.test.string.alt1", "gravitino.test.string.alt2")) .stringConf() .createWithDefault("test"); diff --git a/core/src/main/java/com/datastrato/gravitino/Configs.java b/core/src/main/java/com/datastrato/gravitino/Configs.java index 384f47628bf..95f9780d725 100644 --- a/core/src/main/java/com/datastrato/gravitino/Configs.java +++ b/core/src/main/java/com/datastrato/gravitino/Configs.java @@ -7,6 +7,7 @@ import com.datastrato.gravitino.config.ConfigBuilder; import com.datastrato.gravitino.config.ConfigConstants; import com.datastrato.gravitino.config.ConfigEntry; +import com.google.common.collect.Lists; import java.io.File; import org.apache.commons.lang3.StringUtils; @@ -33,6 +34,24 @@ public interface Configs { Long DEFAULT_KV_DELETE_AFTER_TIME = 604800000L; // 7 days String KV_DELETE_AFTER_TIME_KEY = "gravitino.entity.store.kv.deleteAfterTimeMs"; + // Config for data keep time after soft deletion, in milliseconds. + String STORE_DELETE_AFTER_TIME_KEY = "gravitino.entity.store.deleteAfterTimeMs"; + // using the fallback default value + Long DEFAULT_STORE_DELETE_AFTER_TIME = DEFAULT_KV_DELETE_AFTER_TIME; + // The maximum allowed keep time for data deletion, in milliseconds. Equivalent to 30 days. + Long MAX_DELETE_TIME_ALLOW = 1000 * 60 * 60 * 24 * 30L; + // The minimum allowed keep time for data deletion, in milliseconds. Equivalent to 10 minutes. + Long MIN_DELETE_TIME_ALLOW = 1000 * 60 * 10L; + + // Count of versions allowed to be retained, including the current version, used to delete old + // versions data. + String VERSION_RETENTION_COUNT_KEY = "gravitino.entity.store.versionRetentionCount"; + Long DEFAULT_VERSION_RETENTION_COUNT = 1L; + // The maximum allowed count of versions to be retained + Long MAX_VERSION_RETENTION_COUNT = 10L; + // The minimum allowed count of versions to be retained + Long MIN_VERSION_RETENTION_COUNT = 1L; + // Default path for RocksDB backend is "${GRAVITINO_HOME}/data/rocksdb" String DEFAULT_KV_ROCKSDB_BACKEND_PATH = String.join(File.separator, System.getenv("GRAVITINO_HOME"), "data", "rocksdb"); @@ -146,11 +165,58 @@ public interface Configs { ConfigEntry KV_DELETE_AFTER_TIME = new ConfigBuilder(KV_DELETE_AFTER_TIME_KEY) .doc( - "The maximum time in milliseconds that the deleted data and old version data is kept") - .version(ConfigConstants.VERSION_0_3_0) + String.format( + "The maximum time in milliseconds that the deleted data and old version data is kept, " + + "max delete time allow is %s ms(30 days), " + + "min delete time allow is %s ms(10 minutes)", + MAX_DELETE_TIME_ALLOW, MIN_DELETE_TIME_ALLOW)) + .version(ConfigConstants.VERSION_0_5_0) + .deprecated() .longConf() + .checkValue( + v -> v >= MIN_DELETE_TIME_ALLOW && v <= MAX_DELETE_TIME_ALLOW, + String.format( + "The value of %s is out of range, which must be between %s and %s", + KV_DELETE_AFTER_TIME_KEY, MIN_DELETE_TIME_ALLOW, MAX_DELETE_TIME_ALLOW)) .createWithDefault(DEFAULT_KV_DELETE_AFTER_TIME); + ConfigEntry STORE_DELETE_AFTER_TIME = + new ConfigBuilder(STORE_DELETE_AFTER_TIME_KEY) + .doc( + String.format( + "The maximum time in milliseconds that the deleted data and old version data is kept, " + + "max delete time allow is %s ms(30 days), " + + "min delete time allow is %s ms(10 minutes)", + MAX_DELETE_TIME_ALLOW, MIN_DELETE_TIME_ALLOW)) + .version(ConfigConstants.VERSION_0_5_0) + .alternatives(Lists.newArrayList(KV_DELETE_AFTER_TIME_KEY)) + .longConf() + .checkValue( + v -> v >= MIN_DELETE_TIME_ALLOW && v <= MAX_DELETE_TIME_ALLOW, + String.format( + "The value of %s is out of range, which must be between %s and %s", + STORE_DELETE_AFTER_TIME_KEY, MIN_DELETE_TIME_ALLOW, MAX_DELETE_TIME_ALLOW)) + .createWithDefault(DEFAULT_STORE_DELETE_AFTER_TIME); + + ConfigEntry VERSION_RETENTION_COUNT = + new ConfigBuilder(VERSION_RETENTION_COUNT_KEY) + .doc( + String.format( + "The count of versions allowed to be retained, including the current version, " + + "max version retention count is %s, " + + "min version retention count is %s", + MAX_VERSION_RETENTION_COUNT, MIN_VERSION_RETENTION_COUNT)) + .version(ConfigConstants.VERSION_0_5_0) + .longConf() + .checkValue( + v -> v >= MIN_VERSION_RETENTION_COUNT && v <= MAX_VERSION_RETENTION_COUNT, + String.format( + "The value of %s is out of range, which must be between %s and %s", + VERSION_RETENTION_COUNT_KEY, + MIN_VERSION_RETENTION_COUNT, + MAX_VERSION_RETENTION_COUNT)) + .createWithDefault(DEFAULT_VERSION_RETENTION_COUNT); + // The followings are configurations for tree lock ConfigEntry TREE_LOCK_MAX_NODE_IN_MEMORY = diff --git a/core/src/main/java/com/datastrato/gravitino/storage/kv/KvGarbageCollector.java b/core/src/main/java/com/datastrato/gravitino/storage/kv/KvGarbageCollector.java index fc8fc5c7bf4..a2f39c361d3 100644 --- a/core/src/main/java/com/datastrato/gravitino/storage/kv/KvGarbageCollector.java +++ b/core/src/main/java/com/datastrato/gravitino/storage/kv/KvGarbageCollector.java @@ -5,7 +5,7 @@ package com.datastrato.gravitino.storage.kv; -import static com.datastrato.gravitino.Configs.KV_DELETE_AFTER_TIME; +import static com.datastrato.gravitino.Configs.STORE_DELETE_AFTER_TIME; import static com.datastrato.gravitino.storage.kv.KvNameMappingService.GENERAL_NAME_MAPPING_PREFIX; import static com.datastrato.gravitino.storage.kv.TransactionalKvBackendImpl.endOfTransactionId; import static com.datastrato.gravitino.storage.kv.TransactionalKvBackendImpl.generateCommitKey; @@ -44,9 +44,6 @@ public final class KvGarbageCollector implements Closeable { private final Config config; private final EntityKeyEncoder entityKeyEncoder; - private static final long MAX_DELETE_TIME_ALLOW = 1000 * 60 * 60 * 24 * 30L; // 30 days - private static final long MIN_DELETE_TIME_ALLOW = 1000 * 60 * 10L; // 10 minutes - private static final String TIME_STAMP_FORMAT = "yyyy-MM-dd HH:mm:ss.SSS"; @VisibleForTesting @@ -65,21 +62,10 @@ public KvGarbageCollector( this.kvBackend = kvBackend; this.config = config; this.entityKeyEncoder = entityKeyEncoder; - - long deleteTimeLine = config.get(KV_DELETE_AFTER_TIME); - if (deleteTimeLine > MAX_DELETE_TIME_ALLOW || deleteTimeLine < MIN_DELETE_TIME_ALLOW) { - throw new IllegalArgumentException( - String.format( - "The delete time line is too long or too short, " - + "please check it. The delete time line is %s ms," - + "max delete time allow is %s ms(30 days)," - + "min delete time allow is %s ms(10 minutes)", - deleteTimeLine, MAX_DELETE_TIME_ALLOW, MIN_DELETE_TIME_ALLOW)); - } } public void start() { - long dateTimeLineMinute = config.get(KV_DELETE_AFTER_TIME) / 1000 / 60; + long dateTimeLineMinute = config.get(STORE_DELETE_AFTER_TIME) / 1000 / 60; // We will collect garbage every 10 minutes at least. If the dateTimeLineMinute is larger than // 100 minutes, we would collect garbage every dateTimeLineMinute/10 minutes. @@ -133,7 +119,7 @@ private void collectAndRemoveUncommittedData() throws IOException { } private void collectAndRemoveOldVersionData() throws IOException { - long deleteTimeLine = System.currentTimeMillis() - config.get(KV_DELETE_AFTER_TIME); + long deleteTimeLine = System.currentTimeMillis() - config.get(STORE_DELETE_AFTER_TIME); // Why should we leave shift 18 bits? please refer to TransactionIdGeneratorImpl#nextId // We can delete the data which is older than deleteTimeLine.(old data with transaction id that // is smaller than transactionIdToDelete) diff --git a/core/src/test/java/com/datastrato/gravitino/storage/TestEntityStorage.java b/core/src/test/java/com/datastrato/gravitino/storage/TestEntityStorage.java index 3aa0bef0dac..6666cd0453a 100644 --- a/core/src/test/java/com/datastrato/gravitino/storage/TestEntityStorage.java +++ b/core/src/test/java/com/datastrato/gravitino/storage/TestEntityStorage.java @@ -15,8 +15,8 @@ import static com.datastrato.gravitino.Configs.ENTITY_RELATIONAL_STORE; import static com.datastrato.gravitino.Configs.ENTITY_STORE; import static com.datastrato.gravitino.Configs.ENTRY_KV_ROCKSDB_BACKEND_PATH; -import static com.datastrato.gravitino.Configs.KV_DELETE_AFTER_TIME; import static com.datastrato.gravitino.Configs.RELATIONAL_ENTITY_STORE; +import static com.datastrato.gravitino.Configs.STORE_DELETE_AFTER_TIME; import static com.datastrato.gravitino.Configs.STORE_TRANSACTION_MAX_SKEW_TIME; import com.datastrato.gravitino.Catalog; @@ -92,7 +92,7 @@ private void init(String type, Config config) { Assertions.assertEquals(KV_STORE_PATH, config.get(ENTRY_KV_ROCKSDB_BACKEND_PATH)); Mockito.when(config.get(STORE_TRANSACTION_MAX_SKEW_TIME)).thenReturn(1000L); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); + Mockito.when(config.get(STORE_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); } else if (type.equals(Configs.RELATIONAL_ENTITY_STORE)) { File dir = new File(DB_DIR); if (dir.exists() || !dir.isDirectory()) { diff --git a/core/src/test/java/com/datastrato/gravitino/storage/kv/TestEntityKeyEncoding.java b/core/src/test/java/com/datastrato/gravitino/storage/kv/TestEntityKeyEncoding.java index 00504a54769..b23b8b88cbf 100644 --- a/core/src/test/java/com/datastrato/gravitino/storage/kv/TestEntityKeyEncoding.java +++ b/core/src/test/java/com/datastrato/gravitino/storage/kv/TestEntityKeyEncoding.java @@ -9,7 +9,7 @@ import static com.datastrato.gravitino.Configs.ENTITY_KV_STORE; import static com.datastrato.gravitino.Configs.ENTITY_STORE; import static com.datastrato.gravitino.Configs.ENTRY_KV_ROCKSDB_BACKEND_PATH; -import static com.datastrato.gravitino.Configs.KV_DELETE_AFTER_TIME; +import static com.datastrato.gravitino.Configs.STORE_DELETE_AFTER_TIME; import static com.datastrato.gravitino.Configs.STORE_TRANSACTION_MAX_SKEW_TIME; import static com.datastrato.gravitino.storage.kv.BinaryEntityKeyEncoder.BYTABLE_NAMESPACE_SEPARATOR; import static com.datastrato.gravitino.storage.kv.BinaryEntityKeyEncoder.WILD_CARD; @@ -52,7 +52,7 @@ private Config getConfig() throws IOException { Mockito.when(config.get(STORE_TRANSACTION_MAX_SKEW_TIME)).thenReturn(3000L); Mockito.when(config.get(ENTITY_STORE)).thenReturn("kv"); Mockito.when(config.get(ENTITY_KV_STORE)).thenReturn(DEFAULT_ENTITY_KV_STORE); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); + Mockito.when(config.get(STORE_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); return config; } diff --git a/core/src/test/java/com/datastrato/gravitino/storage/kv/TestKvEntityStorage.java b/core/src/test/java/com/datastrato/gravitino/storage/kv/TestKvEntityStorage.java index c937400a1ce..df7c6953ace 100644 --- a/core/src/test/java/com/datastrato/gravitino/storage/kv/TestKvEntityStorage.java +++ b/core/src/test/java/com/datastrato/gravitino/storage/kv/TestKvEntityStorage.java @@ -9,7 +9,7 @@ import static com.datastrato.gravitino.Configs.ENTITY_KV_STORE; import static com.datastrato.gravitino.Configs.ENTITY_STORE; import static com.datastrato.gravitino.Configs.ENTRY_KV_ROCKSDB_BACKEND_PATH; -import static com.datastrato.gravitino.Configs.KV_DELETE_AFTER_TIME; +import static com.datastrato.gravitino.Configs.STORE_DELETE_AFTER_TIME; import static com.datastrato.gravitino.Configs.STORE_TRANSACTION_MAX_SKEW_TIME; import com.datastrato.gravitino.Config; @@ -60,16 +60,20 @@ public void cleanEnv() { } } - @Test - void testCreateKvEntityStore() throws IOException { + public Config getConfig() throws IOException { Config config = Mockito.mock(Config.class); Mockito.when(config.get(ENTITY_STORE)).thenReturn("kv"); Mockito.when(config.get(ENTITY_KV_STORE)).thenReturn(DEFAULT_ENTITY_KV_STORE); Mockito.when(config.get(Configs.ENTITY_SERDE)).thenReturn("proto"); Mockito.when(config.get(ENTRY_KV_ROCKSDB_BACKEND_PATH)).thenReturn("/tmp/gravitino"); Mockito.when(config.get(STORE_TRANSACTION_MAX_SKEW_TIME)).thenReturn(1000L); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); + Mockito.when(config.get(STORE_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); + return config; + } + @Test + void testCreateKvEntityStore() throws IOException { + Config config = getConfig(); FileUtils.deleteDirectory(FileUtils.getFile("/tmp/gravitino")); try (EntityStore store = EntityStoreFactory.createEntityStore(config)) { @@ -173,16 +177,11 @@ void testCreateKvEntityStore() throws IOException { @Test @Disabled("KvEntityStore is not thread safe after issue #780") void testConcurrentIssues() throws IOException, ExecutionException, InterruptedException { - Config config = Mockito.mock(Config.class); + Config config = getConfig(); File baseDir = new File(System.getProperty("java.io.tmpdir")); File file = Files.createTempDirectory(baseDir.toPath(), "test").toFile(); file.deleteOnExit(); - Mockito.when(config.get(ENTITY_STORE)).thenReturn("kv"); - Mockito.when(config.get(ENTITY_KV_STORE)).thenReturn(DEFAULT_ENTITY_KV_STORE); - Mockito.when(config.get(Configs.ENTITY_SERDE)).thenReturn("proto"); Mockito.when(config.get(ENTRY_KV_ROCKSDB_BACKEND_PATH)).thenReturn(file.getAbsolutePath()); - Mockito.when(config.get(STORE_TRANSACTION_MAX_SKEW_TIME)).thenReturn(1000L); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); ThreadPoolExecutor threadPoolExecutor = new ThreadPoolExecutor( diff --git a/core/src/test/java/com/datastrato/gravitino/storage/kv/TestKvGarbageCollector.java b/core/src/test/java/com/datastrato/gravitino/storage/kv/TestKvGarbageCollector.java index fbc29af1786..815dbfff3f9 100644 --- a/core/src/test/java/com/datastrato/gravitino/storage/kv/TestKvGarbageCollector.java +++ b/core/src/test/java/com/datastrato/gravitino/storage/kv/TestKvGarbageCollector.java @@ -9,7 +9,7 @@ import static com.datastrato.gravitino.Configs.ENTITY_KV_STORE; import static com.datastrato.gravitino.Configs.ENTITY_STORE; import static com.datastrato.gravitino.Configs.ENTRY_KV_ROCKSDB_BACKEND_PATH; -import static com.datastrato.gravitino.Configs.KV_DELETE_AFTER_TIME; +import static com.datastrato.gravitino.Configs.STORE_DELETE_AFTER_TIME; import static com.datastrato.gravitino.Configs.STORE_TRANSACTION_MAX_SKEW_TIME; import static com.datastrato.gravitino.storage.kv.TestKvEntityStorage.createBaseMakeLake; import static com.datastrato.gravitino.storage.kv.TestKvEntityStorage.createCatalog; @@ -56,7 +56,7 @@ public Config getConfig() throws IOException { Mockito.when(config.get(Configs.ENTITY_SERDE)).thenReturn("proto"); Mockito.when(config.get(ENTRY_KV_ROCKSDB_BACKEND_PATH)).thenReturn(file.getAbsolutePath()); Mockito.when(config.get(STORE_TRANSACTION_MAX_SKEW_TIME)).thenReturn(3L); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); + Mockito.when(config.get(STORE_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); return config; } @@ -69,16 +69,17 @@ private KvBackend getKvBackEnd(Config config) throws IOException { @Test void testScheduler() throws IOException { Config config = getConfig(); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); // 20 minutes - long dateTimeLineMinute = config.get(KV_DELETE_AFTER_TIME) / 1000 / 60; + Mockito.when(config.get(STORE_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); // 20 minutes + long dateTimeLineMinute = config.get(STORE_DELETE_AFTER_TIME) / 1000 / 60; Assertions.assertEquals(10, Math.max(dateTimeLineMinute / 10, 10)); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(2 * 60 * 60 * 1000L); // 2 hours - dateTimeLineMinute = config.get(KV_DELETE_AFTER_TIME) / 1000 / 60; + Mockito.when(config.get(STORE_DELETE_AFTER_TIME)).thenReturn(2 * 60 * 60 * 1000L); // 2 hours + dateTimeLineMinute = config.get(STORE_DELETE_AFTER_TIME) / 1000 / 60; Assertions.assertEquals(12, Math.max(dateTimeLineMinute / 10, 10)); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(2 * 60 * 60 * 24 * 1000L); // 2 days - dateTimeLineMinute = config.get(KV_DELETE_AFTER_TIME) / 1000 / 60; + Mockito.when(config.get(STORE_DELETE_AFTER_TIME)) + .thenReturn(2 * 60 * 60 * 24 * 1000L); // 2 days + dateTimeLineMinute = config.get(STORE_DELETE_AFTER_TIME) / 1000 / 60; Assertions.assertEquals(288, Math.max(dateTimeLineMinute / 10, 10)); } @@ -127,8 +128,9 @@ void testCollectGarbage() throws IOException, InterruptedException { // 0x1E, last_timestamp can be seen as they have not been stored to the backend. Assertions.assertEquals(7, allData.size()); + // Set the TTL to 2 seconds before the kvGarbageCollector is created + Mockito.doReturn(2000L).when(config).get(STORE_DELETE_AFTER_TIME); KvGarbageCollector kvGarbageCollector = new KvGarbageCollector(kvBackend, config, null); - Mockito.doReturn(2000L).when(config).get(KV_DELETE_AFTER_TIME); // Wait TTL time to make sure the data is expired, please see ENTITY_KV_TTL Thread.sleep(3000); @@ -153,16 +155,8 @@ void testCollectGarbage() throws IOException, InterruptedException { @Test void testRemoveWithGCCollector1() throws IOException, InterruptedException { - Config config = Mockito.mock(Config.class); - File baseDir = new File(System.getProperty("java.io.tmpdir")); - File file = Files.createTempDirectory(baseDir.toPath(), "test").toFile(); - file.deleteOnExit(); - Mockito.when(config.get(ENTITY_STORE)).thenReturn("kv"); - Mockito.when(config.get(ENTITY_KV_STORE)).thenReturn(DEFAULT_ENTITY_KV_STORE); - Mockito.when(config.get(Configs.ENTITY_SERDE)).thenReturn("proto"); - Mockito.when(config.get(ENTRY_KV_ROCKSDB_BACKEND_PATH)).thenReturn(file.getAbsolutePath()); + Config config = getConfig(); Mockito.when(config.get(STORE_TRANSACTION_MAX_SKEW_TIME)).thenReturn(1000L); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); try (EntityStore store = EntityStoreFactory.createEntityStore(config)) { store.initialize(config); @@ -239,16 +233,8 @@ void testRemoveWithGCCollector1() throws IOException, InterruptedException { @Test void testRemoveWithGCCollector2() throws IOException, InterruptedException { - Config config = Mockito.mock(Config.class); - File baseDir = new File(System.getProperty("java.io.tmpdir")); - File file = Files.createTempDirectory(baseDir.toPath(), "test").toFile(); - file.deleteOnExit(); - Mockito.when(config.get(ENTITY_STORE)).thenReturn("kv"); - Mockito.when(config.get(ENTITY_KV_STORE)).thenReturn(DEFAULT_ENTITY_KV_STORE); - Mockito.when(config.get(Configs.ENTITY_SERDE)).thenReturn("proto"); - Mockito.when(config.get(ENTRY_KV_ROCKSDB_BACKEND_PATH)).thenReturn(file.getAbsolutePath()); + Config config = getConfig(); Mockito.when(config.get(STORE_TRANSACTION_MAX_SKEW_TIME)).thenReturn(1000L); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); try (EntityStore store = EntityStoreFactory.createEntityStore(config)) { store.initialize(config); @@ -278,7 +264,7 @@ void testRemoveWithGCCollector2() throws IOException, InterruptedException { store.put(metalake2); store.put(metalake3); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(1000L); + Mockito.when(config.get(STORE_DELETE_AFTER_TIME)).thenReturn(1000L); Thread.sleep(1500); kvEntityStore.kvGarbageCollector.collectAndClean(); @@ -309,7 +295,7 @@ void testRemoveWithGCCollector2() throws IOException, InterruptedException { store.put(catalog1); store.put(catalog2); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(1000L); + Mockito.when(config.get(STORE_DELETE_AFTER_TIME)).thenReturn(1000L); Thread.sleep(1500); kvEntityStore.kvGarbageCollector.collectAndClean(); @@ -342,7 +328,7 @@ void testRemoveWithGCCollector2() throws IOException, InterruptedException { store.put(schema1); store.put(schema2); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(1000L); + Mockito.when(config.get(STORE_DELETE_AFTER_TIME)).thenReturn(1000L); Thread.sleep(1500); kvEntityStore.kvGarbageCollector.collectAndClean(); @@ -378,7 +364,7 @@ void testRemoveWithGCCollector2() throws IOException, InterruptedException { store.put(table1); store.put(table2); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(1000L); + Mockito.when(config.get(STORE_DELETE_AFTER_TIME)).thenReturn(1000L); Thread.sleep(1500); kvEntityStore.kvGarbageCollector.collectAndClean(); diff --git a/core/src/test/java/com/datastrato/gravitino/storage/kv/TestKvNameMappingService.java b/core/src/test/java/com/datastrato/gravitino/storage/kv/TestKvNameMappingService.java index 0954726289c..3b02aa95d40 100644 --- a/core/src/test/java/com/datastrato/gravitino/storage/kv/TestKvNameMappingService.java +++ b/core/src/test/java/com/datastrato/gravitino/storage/kv/TestKvNameMappingService.java @@ -9,7 +9,7 @@ import static com.datastrato.gravitino.Configs.ENTITY_KV_STORE; import static com.datastrato.gravitino.Configs.ENTITY_STORE; import static com.datastrato.gravitino.Configs.ENTRY_KV_ROCKSDB_BACKEND_PATH; -import static com.datastrato.gravitino.Configs.KV_DELETE_AFTER_TIME; +import static com.datastrato.gravitino.Configs.STORE_DELETE_AFTER_TIME; import static com.datastrato.gravitino.Configs.STORE_TRANSACTION_MAX_SKEW_TIME; import com.datastrato.gravitino.Config; @@ -37,7 +37,7 @@ private Config getConfig() throws IOException { Mockito.when(config.get(STORE_TRANSACTION_MAX_SKEW_TIME)).thenReturn(3000L); Mockito.when(config.get(ENTITY_STORE)).thenReturn("kv"); Mockito.when(config.get(ENTITY_KV_STORE)).thenReturn(DEFAULT_ENTITY_KV_STORE); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); + Mockito.when(config.get(STORE_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); return config; } diff --git a/core/src/test/java/com/datastrato/gravitino/storage/kv/TestStorageVersion.java b/core/src/test/java/com/datastrato/gravitino/storage/kv/TestStorageVersion.java index 87a75ee77f9..82348de0ccd 100644 --- a/core/src/test/java/com/datastrato/gravitino/storage/kv/TestStorageVersion.java +++ b/core/src/test/java/com/datastrato/gravitino/storage/kv/TestStorageVersion.java @@ -9,7 +9,7 @@ import static com.datastrato.gravitino.Configs.ENTITY_KV_STORE; import static com.datastrato.gravitino.Configs.ENTITY_STORE; import static com.datastrato.gravitino.Configs.ENTRY_KV_ROCKSDB_BACKEND_PATH; -import static com.datastrato.gravitino.Configs.KV_DELETE_AFTER_TIME; +import static com.datastrato.gravitino.Configs.STORE_DELETE_AFTER_TIME; import static com.datastrato.gravitino.Configs.STORE_TRANSACTION_MAX_SKEW_TIME; import com.datastrato.gravitino.Config; @@ -48,7 +48,7 @@ void testStorageLayoutVersion() throws IOException { Mockito.when(config.get(Configs.ENTITY_SERDE)).thenReturn("proto"); Mockito.when(config.get(ENTRY_KV_ROCKSDB_BACKEND_PATH)).thenReturn(file.getAbsolutePath()); Mockito.when(config.get(STORE_TRANSACTION_MAX_SKEW_TIME)).thenReturn(1000L); - Mockito.when(config.get(KV_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); + Mockito.when(config.get(STORE_DELETE_AFTER_TIME)).thenReturn(20 * 60 * 1000L); // First time create entity store, the storage layout version should be DEFAULT_LAYOUT_VERSION try (EntityStore store = EntityStoreFactory.createEntityStore(config)) { diff --git a/docs/gravitino-server-config.md b/docs/gravitino-server-config.md index 7150c7fa8d7..315d7844eea 100644 --- a/docs/gravitino-server-config.md +++ b/docs/gravitino-server-config.md @@ -43,19 +43,20 @@ You can also specify filter parameters by setting configuration entries of the f ### Storage configuration -| Configuration item | Description | Default value | Required | Since version | -|---------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|----------------------------------|----------------------------------|---------------| -| `gravitino.entity.store` | Which storage implementation to use. Key-value pair storage and relational storage are currently supported, the default value is `kv`, and the optional value is `relational`. | `kv` | No | 0.1.0 | -| `gravitino.entity.store.kv` | Detailed implementation of KV storage. `RocksDB` storage is currently supported, and the implementation is `RocksDBKvBackend`. | `RocksDBKvBackend` | No | 0.1.0 | -| `gravitino.entity.store.kv.rocksdbPath` | The storage path for RocksDB storage implementation. It supports both absolute and relative path, if the value is a relative path, the final path is `${GRAVITINO_HOME}/${PATH_YOU_HAVA_SET}`, default value is `${GRAVITINO_HOME}/data/rocksdb` | `${GRAVITINO_HOME}/data/rocksdb` | No | 0.1.0 | -| `graivitino.entity.serde` | The serialization/deserialization class used to support entity storage. `proto' is currently supported. | `proto` | No | 0.1.0 | -| `gravitino.entity.store.maxTransactionSkewTimeMs` | The maximum skew time of transactions in milliseconds. | `2000` | No | 0.3.0 | -| `gravitino.entity.store.kv.deleteAfterTimeMs` | The maximum time in milliseconds that deleted and old-version data is kept. Set to at least 10 minutes and no longer than 30 days. | `604800000`(7 days) | No | 0.3.0 | -| `gravitino.entity.store.relational` | Detailed implementation of Relational storage. `MySQL` is currently supported, and the implementation is `JDBCBackend`. | `JDBCBackend` | No | 0.5.0 | -| `gravitino.entity.store.relational.jdbcUrl` | The database url that the `JDBCBackend` needs to connect to. If you use `MySQL`, you should firstly initialize the database tables yourself by executing the file named `mysql_init.sql` in the `src/main/resources/mysql` directory of the `core` module. | (none) | Yes if you use `JdbcBackend` | 0.5.0 | -| `gravitino.entity.store.relational.jdbcDriver` | The jdbc driver name that the `JDBCBackend` needs to use. You should place the driver Jar package in the `${GRAVITINO_HOME}/libs/` directory. | (none) | Yes if you use `JdbcBackend` | 0.5.0 | -| `gravitino.entity.store.relational.jdbcUser` | The username that the `JDBCBackend` needs to use when connecting the database. It is required for `MySQL`. | (none) | Yes if you use `JdbcBackend` | 0.5.0 | -| `gravitino.entity.store.relational.jdbcPassword` | The password that the `JDBCBackend` needs to use when connecting the database. It is required for `MySQL`. | (none) | Yes if you use `JdbcBackend` | 0.5.0 | +| Configuration item | Description | Default value | Required | Since version | +|------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------|----------------------------------|---------------| +| `gravitino.entity.store` | Which storage implementation to use. Key-value pair storage and relational storage are currently supported, the default value is `kv`, and the optional value is `relational`. | `kv` | No | 0.1.0 | +| `gravitino.entity.store.kv` | Detailed implementation of KV storage. `RocksDB` storage is currently supported, and the implementation is `RocksDBKvBackend`. | `RocksDBKvBackend` | No | 0.1.0 | +| `gravitino.entity.store.kv.rocksdbPath` | The storage path for RocksDB storage implementation. It supports both absolute and relative path, if the value is a relative path, the final path is `${GRAVITINO_HOME}/${PATH_YOU_HAVA_SET}`, default value is `${GRAVITINO_HOME}/data/rocksdb` | `${GRAVITINO_HOME}/data/rocksdb` | No | 0.1.0 | +| `graivitino.entity.serde` | The serialization/deserialization class used to support entity storage. `proto' is currently supported. | `proto` | No | 0.1.0 | +| `gravitino.entity.store.maxTransactionSkewTimeMs` | The maximum skew time of transactions in milliseconds. | `2000` | No | 0.3.0 | +| `gravitino.entity.store.deleteAfterTimeMs` | The maximum time in milliseconds that deleted and old-version data is kept. Set to at least 10 minutes and no longer than 30 days. | `604800000`(7 days) | No | 0.5.0 | +| `gravitino.entity.store.versionRetentionCount` | The Count of versions allowed to be retained, including the current version, used to delete old versions data. Set to at least 1 and no greater than 10. | `1` | No | 0.5.0 | +| `gravitino.entity.store.relational` | Detailed implementation of Relational storage. `MySQL` is currently supported, and the implementation is `JDBCBackend`. | `JDBCBackend` | No | 0.5.0 | +| `gravitino.entity.store.relational.jdbcUrl` | The database url that the `JDBCBackend` needs to connect to. If you use `MySQL`, you should firstly initialize the database tables yourself by executing the file named `mysql_init.sql` in the `src/main/resources/mysql` directory of the `core` module. | (none) | Yes if you use `JdbcBackend` | 0.5.0 | +| `gravitino.entity.store.relational.jdbcDriver` | The jdbc driver name that the `JDBCBackend` needs to use. You should place the driver Jar package in the `${GRAVITINO_HOME}/libs/` directory. | (none) | Yes if you use `JdbcBackend` | 0.5.0 | +| `gravitino.entity.store.relational.jdbcUser` | The username that the `JDBCBackend` needs to use when connecting the database. It is required for `MySQL`. | (none) | Yes if you use `JdbcBackend` | 0.5.0 | +| `gravitino.entity.store.relational.jdbcPassword` | The password that the `JDBCBackend` needs to use when connecting the database. It is required for `MySQL`. | (none) | Yes if you use `JdbcBackend` | 0.5.0 | :::caution We strongly recommend that you change the default value of `gravitino.entity.store.kv.rocksdbPath`, as it's under the deployment directory and future version upgrades may remove it. From 5b4eb40b75f0202accaae4468207661848a4a652 Mon Sep 17 00:00:00 2001 From: qqqttt123 <148952220+qqqttt123@users.noreply.github.com> Date: Sat, 30 Mar 2024 10:38:29 +0800 Subject: [PATCH 05/20] [#2236] feat(core): Add the support of Group entity (#2735) ### What changes were proposed in this pull request? Add the support of Group entity ### Why are the changes needed? Fix: #2236 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Add a new UT. --------- Co-authored-by: Heng Qin --- .../gravitino/authorization/Group.java | 28 +++ .../GroupAlreadyExistsException.java | 36 +++ .../exceptions/NoSuchGroupException.java | 35 +++ .../java/com/datastrato/gravitino/Entity.java | 1 + .../authorization/AccessControlManager.java | 50 ++++- ...UserManager.java => UserGroupManager.java} | 107 ++++++++- .../gravitino/meta/GroupEntity.java | 210 ++++++++++++++++++ .../gravitino/proto/GroupEntitySerDe.java | 45 ++++ .../gravitino/proto/ProtoEntitySerDe.java | 7 +- .../storage/kv/BinaryEntityKeyEncoder.java | 5 +- .../gravitino/storage/kv/KvEntityStore.java | 24 +- .../TestAccessControlManager.java | 58 ++++- .../datastrato/gravitino/meta/TestEntity.java | 25 +++ .../gravitino/proto/TestEntityProtoSerDe.java | 24 ++ .../gravitino/storage/TestEntityStorage.java | 27 ++- meta/src/main/proto/gravitino_meta.proto | 7 + 16 files changed, 661 insertions(+), 28 deletions(-) create mode 100644 api/src/main/java/com/datastrato/gravitino/authorization/Group.java create mode 100644 api/src/main/java/com/datastrato/gravitino/exceptions/GroupAlreadyExistsException.java create mode 100644 api/src/main/java/com/datastrato/gravitino/exceptions/NoSuchGroupException.java rename core/src/main/java/com/datastrato/gravitino/authorization/{UserManager.java => UserGroupManager.java} (52%) create mode 100644 core/src/main/java/com/datastrato/gravitino/meta/GroupEntity.java create mode 100644 core/src/main/java/com/datastrato/gravitino/proto/GroupEntitySerDe.java diff --git a/api/src/main/java/com/datastrato/gravitino/authorization/Group.java b/api/src/main/java/com/datastrato/gravitino/authorization/Group.java new file mode 100644 index 00000000000..ff3e5ce7f42 --- /dev/null +++ b/api/src/main/java/com/datastrato/gravitino/authorization/Group.java @@ -0,0 +1,28 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.authorization; + +import com.datastrato.gravitino.Auditable; +import com.datastrato.gravitino.annotation.Evolving; +import java.util.List; + +/** The interface of a Group. The Group is the entity which contains users. */ +@Evolving +public interface Group extends Auditable { + + /** + * The name of the group. + * + * @return The name of the group. + */ + String name(); + + /** + * The roles of the group. + * + * @return The roles of the group. + */ + List roles(); +} diff --git a/api/src/main/java/com/datastrato/gravitino/exceptions/GroupAlreadyExistsException.java b/api/src/main/java/com/datastrato/gravitino/exceptions/GroupAlreadyExistsException.java new file mode 100644 index 00000000000..edeb10b257d --- /dev/null +++ b/api/src/main/java/com/datastrato/gravitino/exceptions/GroupAlreadyExistsException.java @@ -0,0 +1,36 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ + +package com.datastrato.gravitino.exceptions; + +import com.google.errorprone.annotations.FormatMethod; +import com.google.errorprone.annotations.FormatString; + +/** An exception thrown when a group already exists. */ +public class GroupAlreadyExistsException extends AlreadyExistsException { + + /** + * Constructs a new exception with the specified detail message. + * + * @param message the detail message. + * @param args the arguments to the message. + */ + @FormatMethod + public GroupAlreadyExistsException(@FormatString String message, Object... args) { + super(message, args); + } + + /** + * Constructs a new exception with the specified detail message and cause. + * + * @param cause the cause. + * @param message the detail message. + * @param args the arguments to the message. + */ + @FormatMethod + public GroupAlreadyExistsException(Throwable cause, String message, Object... args) { + super(cause, message, args); + } +} diff --git a/api/src/main/java/com/datastrato/gravitino/exceptions/NoSuchGroupException.java b/api/src/main/java/com/datastrato/gravitino/exceptions/NoSuchGroupException.java new file mode 100644 index 00000000000..a9089233003 --- /dev/null +++ b/api/src/main/java/com/datastrato/gravitino/exceptions/NoSuchGroupException.java @@ -0,0 +1,35 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ + +package com.datastrato.gravitino.exceptions; + +import com.google.errorprone.annotations.FormatMethod; +import com.google.errorprone.annotations.FormatString; + +/** Exception thrown when a group with specified name is not existed. */ +public class NoSuchGroupException extends NotFoundException { + /** + * Constructs a new exception with the specified detail message. + * + * @param message the detail message. + * @param args the arguments to the message. + */ + @FormatMethod + public NoSuchGroupException(@FormatString String message, Object... args) { + super(message, args); + } + + /** + * Constructs a new exception with the specified detail message and cause. + * + * @param cause the cause. + * @param message the detail message. + * @param args the arguments to the message. + */ + @FormatMethod + public NoSuchGroupException(Throwable cause, String message, Object... args) { + super(cause, message, args); + } +} diff --git a/core/src/main/java/com/datastrato/gravitino/Entity.java b/core/src/main/java/com/datastrato/gravitino/Entity.java index 95492996ee0..754c3641229 100644 --- a/core/src/main/java/com/datastrato/gravitino/Entity.java +++ b/core/src/main/java/com/datastrato/gravitino/Entity.java @@ -22,6 +22,7 @@ enum EntityType { FILESET("fi", 5), TOPIC("to", 6), USER("us", 7), + GROUP("gr", 8), AUDIT("au", 65534); diff --git a/core/src/main/java/com/datastrato/gravitino/authorization/AccessControlManager.java b/core/src/main/java/com/datastrato/gravitino/authorization/AccessControlManager.java index 0581df4ad7b..749f8adcb29 100644 --- a/core/src/main/java/com/datastrato/gravitino/authorization/AccessControlManager.java +++ b/core/src/main/java/com/datastrato/gravitino/authorization/AccessControlManager.java @@ -5,6 +5,8 @@ package com.datastrato.gravitino.authorization; import com.datastrato.gravitino.EntityStore; +import com.datastrato.gravitino.exceptions.GroupAlreadyExistsException; +import com.datastrato.gravitino.exceptions.NoSuchGroupException; import com.datastrato.gravitino.exceptions.NoSuchUserException; import com.datastrato.gravitino.exceptions.UserAlreadyExistsException; import com.datastrato.gravitino.storage.IdGenerator; @@ -15,10 +17,10 @@ */ public class AccessControlManager { - private final UserManager userManager; + private final UserGroupManager userGroupManager; public AccessControlManager(EntityStore store, IdGenerator idGenerator) { - this.userManager = new UserManager(store, idGenerator); + this.userGroupManager = new UserGroupManager(store, idGenerator); } /** @@ -31,7 +33,7 @@ public AccessControlManager(EntityStore store, IdGenerator idGenerator) { * @throws RuntimeException If adding the User encounters storage issues. */ public User addUser(String metalake, String name) throws UserAlreadyExistsException { - return userManager.addUser(metalake, name); + return userGroupManager.addUser(metalake, name); } /** @@ -43,7 +45,7 @@ public User addUser(String metalake, String name) throws UserAlreadyExistsExcept * @throws RuntimeException If removing the User encounters storage issues. */ public boolean removeUser(String metalake, String user) { - return userManager.removeUser(metalake, user); + return userGroupManager.removeUser(metalake, user); } /** @@ -56,6 +58,44 @@ public boolean removeUser(String metalake, String user) { * @throws RuntimeException If getting the User encounters storage issues. */ public User getUser(String metalake, String user) throws NoSuchUserException { - return userManager.getUser(metalake, user); + return userGroupManager.getUser(metalake, user); + } + + /** + * Adds a new Group. + * + * @param metalake The Metalake of the Group. + * @param group The name of the Group. + * @return The Added Group instance. + * @throws GroupAlreadyExistsException If a Group with the same identifier already exists. + * @throws RuntimeException If adding the Group encounters storage issues. + */ + public Group addGroup(String metalake, String group) throws GroupAlreadyExistsException { + return userGroupManager.addGroup(metalake, group); + } + + /** + * Removes a Group. + * + * @param metalake The Metalake of the Group. + * @param group THe name of the Group. + * @return `true` if the Group was successfully removed, `false` otherwise. + * @throws RuntimeException If removing the Group encounters storage issues. + */ + public boolean removeGroup(String metalake, String group) { + return userGroupManager.removeGroup(metalake, group); + } + + /** + * Gets a Group. + * + * @param metalake The Metalake of the Group. + * @param group THe name of the Group. + * @return The getting Group instance. + * @throws NoSuchGroupException If the Group with the given identifier does not exist. + * @throws RuntimeException If getting the Group encounters storage issues. + */ + public Group getGroup(String metalake, String group) throws NoSuchGroupException { + return userGroupManager.getGroup(metalake, group); } } diff --git a/core/src/main/java/com/datastrato/gravitino/authorization/UserManager.java b/core/src/main/java/com/datastrato/gravitino/authorization/UserGroupManager.java similarity index 52% rename from core/src/main/java/com/datastrato/gravitino/authorization/UserManager.java rename to core/src/main/java/com/datastrato/gravitino/authorization/UserGroupManager.java index a0fa164b33d..4233b4ec99a 100644 --- a/core/src/main/java/com/datastrato/gravitino/authorization/UserManager.java +++ b/core/src/main/java/com/datastrato/gravitino/authorization/UserGroupManager.java @@ -9,34 +9,42 @@ import com.datastrato.gravitino.EntityStore; import com.datastrato.gravitino.NameIdentifier; import com.datastrato.gravitino.Namespace; +import com.datastrato.gravitino.exceptions.GroupAlreadyExistsException; import com.datastrato.gravitino.exceptions.NoSuchEntityException; +import com.datastrato.gravitino.exceptions.NoSuchGroupException; import com.datastrato.gravitino.exceptions.NoSuchUserException; import com.datastrato.gravitino.exceptions.UserAlreadyExistsException; import com.datastrato.gravitino.meta.AuditInfo; import com.datastrato.gravitino.meta.CatalogEntity; +import com.datastrato.gravitino.meta.GroupEntity; import com.datastrato.gravitino.meta.UserEntity; import com.datastrato.gravitino.storage.IdGenerator; import com.datastrato.gravitino.utils.PrincipalUtils; import com.google.common.collect.Lists; import java.io.IOException; import java.time.Instant; +import java.util.Collections; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * UserManager is used for add, remove and get users from one metalake. UserManager doesn't manage - * users, just sets up the relationship between the metalake and the user. Metalake is like a - * concept of the organization. `AddUser` means that a user enter an organization. + * UserGroupManager is used for add, remove and get users or roles from one metalake. + * UserGroupManager doesn't manage users or groups, just sets up the relationship between the + * metalake and the user or group. Metalake is like a concept of the organization. `AddUser` or + * `AddGroup` means that a role or user enters an organization. */ -public class UserManager { +public class UserGroupManager { - private static final Logger LOG = LoggerFactory.getLogger(UserManager.class); + private static final Logger LOG = LoggerFactory.getLogger(UserGroupManager.class); private static final String USER_DOES_NOT_EXIST_MSG = "User %s does not exist in th metalake %s"; + private static final String GROUP_DOES_NOT_EXIST_MSG = + "Group %s does not exist in th metalake %s"; + private final EntityStore store; private final IdGenerator idGenerator; - public UserManager(EntityStore store, IdGenerator idGenerator) { + public UserGroupManager(EntityStore store, IdGenerator idGenerator) { this.store = store; this.idGenerator = idGenerator; } @@ -121,8 +129,95 @@ public User getUser(String metalake, String user) throws NoSuchUserException { } } + /** + * Adds a new Group. + * + * @param metalake The Metalake of the Group. + * @param group The name of the Group. + * @return The Added Group instance. + * @throws GroupAlreadyExistsException If a Group with the same identifier already exists. + * @throws RuntimeException If adding the Group encounters storage issues. + */ + public Group addGroup(String metalake, String group) throws GroupAlreadyExistsException { + GroupEntity groupEntity = + GroupEntity.builder() + .withId(idGenerator.nextId()) + .withName(group) + .withNamespace( + Namespace.of( + metalake, + CatalogEntity.SYSTEM_CATALOG_RESERVED_NAME, + GroupEntity.GROUP_SCHEMA_NAME)) + .withRoles(Collections.emptyList()) + .withAuditInfo( + AuditInfo.builder() + .withCreator(PrincipalUtils.getCurrentPrincipal().getName()) + .withCreateTime(Instant.now()) + .build()) + .build(); + try { + store.put(groupEntity, false /* overwritten */); + return groupEntity; + } catch (EntityAlreadyExistsException e) { + LOG.warn("Group {} in the metalake {} already exists", group, metalake, e); + throw new GroupAlreadyExistsException( + "Group %s in the metalake %s already exists", group, metalake); + } catch (IOException ioe) { + LOG.error( + "Adding group {} failed in the metalake {} due to storage issues", group, metalake, ioe); + throw new RuntimeException(ioe); + } + } + + /** + * Removes a Group. + * + * @param metalake The Metalake of the Group. + * @param group THe name of the Group. + * @return `true` if the Group was successfully removed, `false` otherwise. + * @throws RuntimeException If removing the Group encounters storage issues. + */ + public boolean removeGroup(String metalake, String group) { + try { + return store.delete(ofGroup(metalake, group), Entity.EntityType.GROUP); + } catch (IOException ioe) { + LOG.error( + "Removing group {} in the metalake {} failed due to storage issues", + group, + metalake, + ioe); + throw new RuntimeException(ioe); + } + } + + /** + * Gets a Group. + * + * @param metalake The Metalake of the Group. + * @param group THe name of the Group. + * @return The getting Group instance. + * @throws NoSuchGroupException If the Group with the given identifier does not exist. + * @throws RuntimeException If getting the Group encounters storage issues. + */ + public Group getGroup(String metalake, String group) { + try { + return store.get(ofGroup(metalake, group), Entity.EntityType.GROUP, GroupEntity.class); + } catch (NoSuchEntityException e) { + LOG.warn("Group {} does not exist in the metalake {}", group, metalake, e); + throw new NoSuchGroupException(GROUP_DOES_NOT_EXIST_MSG, group, metalake); + } catch (IOException ioe) { + LOG.error("Getting group {} failed due to storage issues", group, ioe); + throw new RuntimeException(ioe); + } + } + private NameIdentifier ofUser(String metalake, String user) { return NameIdentifier.of( metalake, CatalogEntity.SYSTEM_CATALOG_RESERVED_NAME, UserEntity.USER_SCHEMA_NAME, user); } + + private NameIdentifier ofGroup(String metalake, String group) { + return NameIdentifier.of( + metalake, CatalogEntity.SYSTEM_CATALOG_RESERVED_NAME, GroupEntity.GROUP_SCHEMA_NAME, group); + } } diff --git a/core/src/main/java/com/datastrato/gravitino/meta/GroupEntity.java b/core/src/main/java/com/datastrato/gravitino/meta/GroupEntity.java new file mode 100644 index 00000000000..7d198b79471 --- /dev/null +++ b/core/src/main/java/com/datastrato/gravitino/meta/GroupEntity.java @@ -0,0 +1,210 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.meta; + +import com.datastrato.gravitino.Auditable; +import com.datastrato.gravitino.Entity; +import com.datastrato.gravitino.Field; +import com.datastrato.gravitino.HasIdentifier; +import com.datastrato.gravitino.Namespace; +import com.datastrato.gravitino.authorization.Group; +import com.google.common.collect.Maps; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class GroupEntity implements Group, Entity, Auditable, HasIdentifier { + + public static final String GROUP_SCHEMA_NAME = "group"; + + public static final Field ID = + Field.required("id", Long.class, " The unique id of the group entity."); + + public static final Field NAME = + Field.required("name", String.class, "The name of the group entity."); + + public static final Field ROLES = + Field.optional("roles", List.class, "The roles of the group entity."); + + public static final Field AUDIT_INFO = + Field.required("audit_info", AuditInfo.class, "The audit details of the group entity."); + + private Long id; + private String name; + private AuditInfo auditInfo; + private List roles; + private Namespace namespace; + + private GroupEntity() {} + + /** + * Returns a map of fields and their corresponding values for this group entity. + * + * @return An unmodifiable map of the fields and values. + */ + @Override + public Map fields() { + Map fields = Maps.newHashMap(); + fields.put(ID, id); + fields.put(NAME, name); + fields.put(AUDIT_INFO, auditInfo); + fields.put(ROLES, roles); + + return Collections.unmodifiableMap(fields); + } + + /** + * Returns the name of the group. + * + * @return The name of the group. + */ + @Override + public String name() { + return name; + } + + /** + * Returns the unique id of the group. + * + * @return The unique id of the group. + */ + @Override + public Long id() { + return id; + } + + /** + * Returns the type of the entity. + * + * @return The type of the entity. + */ + @Override + public EntityType type() { + return EntityType.GROUP; + } + + /** + * Returns the namespace of the group. + * + * @return The namespace of the group. + */ + @Override + public Namespace namespace() { + return namespace; + } + + /** + * Returns the audit details of the group. + * + * @return The audit details of the group. + */ + public AuditInfo auditInfo() { + return auditInfo; + } + + /** + * Returns the roles of the group entity. + * + * @return The roles of the group entity. + */ + public List roles() { + return roles; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof GroupEntity)) return false; + + GroupEntity that = (GroupEntity) o; + return Objects.equals(id, that.id) + && Objects.equals(name, that.name) + && Objects.equals(auditInfo, that.auditInfo) + && Objects.equals(roles, that.roles); + } + + @Override + public int hashCode() { + return Objects.hash(id, name, auditInfo, roles); + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + private final GroupEntity groupEntity; + + private Builder() { + this.groupEntity = new GroupEntity(); + } + + /** + * Sets the unique id of the group entity. + * + * @param id The unique id of the group entity. + * @return The builder instance. + */ + public Builder withId(Long id) { + groupEntity.id = id; + return this; + } + + /** + * Sets the name of the group entity. + * + * @param name The name of the group entity. + * @return The builder instance. + */ + public Builder withName(String name) { + groupEntity.name = name; + return this; + } + + /** + * Sets the audit details of the group entity. + * + * @param auditInfo The audit details of the group entity. + * @return The builder instance. + */ + public Builder withAuditInfo(AuditInfo auditInfo) { + groupEntity.auditInfo = auditInfo; + return this; + } + + /** + * Sets the roles of the group entity. + * + * @param roles The roles of the group entity. + * @return The builder instance. + */ + public Builder withRoles(List roles) { + groupEntity.roles = roles; + return this; + } + + /** + * Sets the namespace of the group entity. + * + * @param namespace The namespace of the group entity. + * @return The builder instance. + */ + public Builder withNamespace(Namespace namespace) { + groupEntity.namespace = namespace; + return this; + } + + /** + * Builds the group entity. + * + * @return The built group entity. + */ + public GroupEntity build() { + groupEntity.validate(); + return groupEntity; + } + } +} diff --git a/core/src/main/java/com/datastrato/gravitino/proto/GroupEntitySerDe.java b/core/src/main/java/com/datastrato/gravitino/proto/GroupEntitySerDe.java new file mode 100644 index 00000000000..083b69029d8 --- /dev/null +++ b/core/src/main/java/com/datastrato/gravitino/proto/GroupEntitySerDe.java @@ -0,0 +1,45 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.proto; + +import com.datastrato.gravitino.meta.GroupEntity; +import java.util.Collection; + +public class GroupEntitySerDe implements ProtoSerDe { + + @Override + public Group serialize(GroupEntity groupEntity) { + Group.Builder builder = + Group.newBuilder() + .setId(groupEntity.id()) + .setName(groupEntity.name()) + .setAuditInfo(new AuditInfoSerDe().serialize(groupEntity.auditInfo())); + + if (isCollectionNotEmpty(groupEntity.roles())) { + builder.addAllRoles(groupEntity.roles()); + } + + return builder.build(); + } + + @Override + public GroupEntity deserialize(Group group) { + GroupEntity.Builder builder = + GroupEntity.builder() + .withId(group.getId()) + .withName(group.getName()) + .withAuditInfo(new AuditInfoSerDe().deserialize(group.getAuditInfo())); + + if (group.getRolesCount() > 0) { + builder.withRoles(group.getRolesList()); + } + + return builder.build(); + } + + private boolean isCollectionNotEmpty(Collection collection) { + return collection != null && !collection.isEmpty(); + } +} diff --git a/core/src/main/java/com/datastrato/gravitino/proto/ProtoEntitySerDe.java b/core/src/main/java/com/datastrato/gravitino/proto/ProtoEntitySerDe.java index a346a5cd9d4..7da49a85da4 100644 --- a/core/src/main/java/com/datastrato/gravitino/proto/ProtoEntitySerDe.java +++ b/core/src/main/java/com/datastrato/gravitino/proto/ProtoEntitySerDe.java @@ -43,6 +43,9 @@ public class ProtoEntitySerDe implements EntitySerDe { .put( "com.datastrato.gravitino.meta.UserEntity", "com.datastrato.gravitino.proto.UserEntitySerDe") + .put( + "com.datastrato.gravitino.meta.GroupEntity", + "com.datastrato.gravitino.proto.GroupEntitySerDe") .build(); private static final Map ENTITY_TO_PROTO = @@ -62,7 +65,9 @@ public class ProtoEntitySerDe implements EntitySerDe { "com.datastrato.gravitino.meta.TopicEntity", "com.datastrato.gravitino.proto.Topic", "com.datastrato.gravitino.meta.UserEntity", - "com.datastrato.gravitino.proto.User"); + "com.datastrato.gravitino.proto.User", + "com.datastrato.gravitino.meta.GroupEntity", + "com.datastrato.gravitino.proto.Group"); private final Map, ProtoSerDe> entityToSerDe; diff --git a/core/src/main/java/com/datastrato/gravitino/storage/kv/BinaryEntityKeyEncoder.java b/core/src/main/java/com/datastrato/gravitino/storage/kv/BinaryEntityKeyEncoder.java index e6096af3720..9882928d9b9 100644 --- a/core/src/main/java/com/datastrato/gravitino/storage/kv/BinaryEntityKeyEncoder.java +++ b/core/src/main/java/com/datastrato/gravitino/storage/kv/BinaryEntityKeyEncoder.java @@ -6,6 +6,7 @@ import static com.datastrato.gravitino.Entity.EntityType.CATALOG; import static com.datastrato.gravitino.Entity.EntityType.FILESET; +import static com.datastrato.gravitino.Entity.EntityType.GROUP; import static com.datastrato.gravitino.Entity.EntityType.METALAKE; import static com.datastrato.gravitino.Entity.EntityType.SCHEMA; import static com.datastrato.gravitino.Entity.EntityType.TABLE; @@ -78,7 +79,9 @@ public class BinaryEntityKeyEncoder implements EntityKeyEncoder { FILESET, new String[] {FILESET.getShortName() + "/", "/", "/", "/"}, USER, - new String[] {USER.getShortName() + "/", "/", "/", "/"}); + new String[] {USER.getShortName() + "/", "/", "/", "/"}, + GROUP, + new String[] {GROUP.getShortName() + "/", "/", "/", "/"}); @VisibleForTesting final NameMappingService nameMappingService; diff --git a/core/src/main/java/com/datastrato/gravitino/storage/kv/KvEntityStore.java b/core/src/main/java/com/datastrato/gravitino/storage/kv/KvEntityStore.java index e79d679269e..c241211bde0 100644 --- a/core/src/main/java/com/datastrato/gravitino/storage/kv/KvEntityStore.java +++ b/core/src/main/java/com/datastrato/gravitino/storage/kv/KvEntityStore.java @@ -8,6 +8,7 @@ import static com.datastrato.gravitino.Configs.ENTITY_KV_STORE; import static com.datastrato.gravitino.Entity.EntityType.CATALOG; import static com.datastrato.gravitino.Entity.EntityType.FILESET; +import static com.datastrato.gravitino.Entity.EntityType.GROUP; import static com.datastrato.gravitino.Entity.EntityType.METALAKE; import static com.datastrato.gravitino.Entity.EntityType.SCHEMA; import static com.datastrato.gravitino.Entity.EntityType.TABLE; @@ -334,18 +335,23 @@ private List getSubEntitiesPrefix(NameIdentifier ident, EntityType type) return prefixes; } - void deleteUserEntitiesIfNecessary(NameIdentifier ident, EntityType type) throws IOException { + void deleteAuthorizationEntitiesIfNecessary(NameIdentifier ident, EntityType type) + throws IOException { if (type != METALAKE) { return; } byte[] encode = entityKeyEncoder.encode(ident, type, true); - byte[] prefix = replacePrefixTypeInfo(encode, USER.getShortName()); - transactionalKvBackend.deleteRange( - new KvRange.KvRangeBuilder() - .start(prefix) - .startInclusive(true) - .end(Bytes.increment(Bytes.wrap(prefix)).get()) - .build()); + + String[] entityShortNames = new String[] {USER.getShortName(), GROUP.getShortName()}; + for (String name : entityShortNames) { + byte[] prefix = replacePrefixTypeInfo(encode, name); + transactionalKvBackend.deleteRange( + new KvRange.KvRangeBuilder() + .start(prefix) + .startInclusive(true) + .end(Bytes.increment(Bytes.wrap(prefix)).get()) + .build()); + } } private byte[] replacePrefixTypeInfo(byte[] encode, String subTypePrefix) { @@ -406,7 +412,7 @@ public boolean delete(NameIdentifier ident, EntityType entityType, boolean casca .build()); } - deleteUserEntitiesIfNecessary(ident, entityType); + deleteAuthorizationEntitiesIfNecessary(ident, entityType); return transactionalKvBackend.delete(dataKey); }); } diff --git a/core/src/test/java/com/datastrato/gravitino/authorization/TestAccessControlManager.java b/core/src/test/java/com/datastrato/gravitino/authorization/TestAccessControlManager.java index eaa0081fcf7..2a1e023d6cd 100644 --- a/core/src/test/java/com/datastrato/gravitino/authorization/TestAccessControlManager.java +++ b/core/src/test/java/com/datastrato/gravitino/authorization/TestAccessControlManager.java @@ -6,6 +6,8 @@ import com.datastrato.gravitino.Config; import com.datastrato.gravitino.EntityStore; +import com.datastrato.gravitino.exceptions.GroupAlreadyExistsException; +import com.datastrato.gravitino.exceptions.NoSuchGroupException; import com.datastrato.gravitino.exceptions.NoSuchUserException; import com.datastrato.gravitino.exceptions.UserAlreadyExistsException; import com.datastrato.gravitino.meta.AuditInfo; @@ -99,11 +101,59 @@ public void testRemoveUser() { accessControlManager.addUser("metalake", "testRemove"); // Test to remove user - boolean dropped = accessControlManager.removeUser("metalake", "testRemove"); - Assertions.assertTrue(dropped); + boolean removed = accessControlManager.removeUser("metalake", "testRemove"); + Assertions.assertTrue(removed); // Test to remove non-existed user - boolean dropped1 = accessControlManager.removeUser("metalake", "no-exist"); - Assertions.assertFalse(dropped1); + boolean removed1 = accessControlManager.removeUser("metalake", "no-exist"); + Assertions.assertFalse(removed1); + } + + @Test + public void testAddGroup() { + + Group group = accessControlManager.addGroup("metalake", "testAdd"); + Assertions.assertEquals("testAdd", group.name()); + Assertions.assertTrue(group.roles().isEmpty()); + + group = accessControlManager.addGroup("metalake", "testAddWithOptionalField"); + + Assertions.assertEquals("testAddWithOptionalField", group.name()); + Assertions.assertTrue(group.roles().isEmpty()); + + // Test with GroupAlreadyExistsException + Assertions.assertThrows( + GroupAlreadyExistsException.class, + () -> accessControlManager.addGroup("metalake", "testAdd")); + } + + @Test + public void testGetGroup() { + + accessControlManager.addGroup("metalake", "testGet"); + + Group group = accessControlManager.getGroup("metalake", "testGet"); + Assertions.assertEquals("testGet", group.name()); + + // Test to get non-existed group + Throwable exception = + Assertions.assertThrows( + NoSuchGroupException.class, + () -> accessControlManager.getGroup("metalake", "not-exist")); + Assertions.assertTrue(exception.getMessage().contains("Group not-exist does not exist")); + } + + @Test + public void testRemoveGroup() { + + accessControlManager.addGroup("metalake", "testRemove"); + + // Test to remove group + boolean removed = accessControlManager.removeGroup("metalake", "testRemove"); + Assertions.assertTrue(removed); + + // Test to remove non-existed group + boolean removed1 = accessControlManager.removeUser("metalake", "no-exist"); + Assertions.assertFalse(removed1); } } diff --git a/core/src/test/java/com/datastrato/gravitino/meta/TestEntity.java b/core/src/test/java/com/datastrato/gravitino/meta/TestEntity.java index 94c6b7fd964..6158574a57d 100644 --- a/core/src/test/java/com/datastrato/gravitino/meta/TestEntity.java +++ b/core/src/test/java/com/datastrato/gravitino/meta/TestEntity.java @@ -52,6 +52,10 @@ public class TestEntity { private final Long userId = 1L; private final String userName = "testUser"; + // Group test data + private final Long groupId = 1L; + private final String groupName = "testGroup"; + @Test public void testMetalake() { BaseMetalake metalake = @@ -227,4 +231,25 @@ public void testUser() { Assertions.assertNull(testUserEntityWithoutFields.roles()); } + + @Test + public void testGroup() { + GroupEntity group = + GroupEntity.builder() + .withId(groupId) + .withName(groupName) + .withAuditInfo(auditInfo) + .withRoles(Lists.newArrayList("role")) + .build(); + Map fields = group.fields(); + Assertions.assertEquals(groupId, fields.get(GroupEntity.ID)); + Assertions.assertEquals(groupName, fields.get(GroupEntity.NAME)); + Assertions.assertEquals(auditInfo, fields.get(GroupEntity.AUDIT_INFO)); + Assertions.assertEquals(Lists.newArrayList("role"), fields.get(GroupEntity.ROLES)); + + GroupEntity groupWithoutFields = + GroupEntity.builder().withId(userId).withName(userName).withAuditInfo(auditInfo).build(); + + Assertions.assertNull(groupWithoutFields.roles()); + } } diff --git a/core/src/test/java/com/datastrato/gravitino/proto/TestEntityProtoSerDe.java b/core/src/test/java/com/datastrato/gravitino/proto/TestEntityProtoSerDe.java index 8c8a50d4544..83e0f41ca48 100644 --- a/core/src/test/java/com/datastrato/gravitino/proto/TestEntityProtoSerDe.java +++ b/core/src/test/java/com/datastrato/gravitino/proto/TestEntityProtoSerDe.java @@ -6,6 +6,7 @@ import com.datastrato.gravitino.EntitySerDe; import com.datastrato.gravitino.EntitySerDeFactory; +import com.datastrato.gravitino.meta.GroupEntity; import com.datastrato.gravitino.meta.SchemaVersion; import com.datastrato.gravitino.meta.UserEntity; import com.google.common.collect.ImmutableMap; @@ -296,5 +297,28 @@ public void testEntitiesSerDe() throws IOException { userBytes = protoEntitySerDe.serialize(userEntityWithoutFields); userEntityFromBytes = protoEntitySerDe.deserialize(userBytes, UserEntity.class); Assertions.assertEquals(userEntityWithoutFields, userEntityFromBytes); + Assertions.assertNull(userEntityWithoutFields.roles()); + + // Test GroupEntity + Long groupId = 1L; + String groupName = "group"; + + GroupEntity group = + GroupEntity.builder() + .withId(groupId) + .withName(groupName) + .withAuditInfo(auditInfo) + .withRoles(Lists.newArrayList("role")) + .build(); + byte[] groupBytes = protoEntitySerDe.serialize(group); + GroupEntity groupFromBytes = protoEntitySerDe.deserialize(groupBytes, GroupEntity.class); + Assertions.assertEquals(group, groupFromBytes); + + GroupEntity groupWithoutFields = + GroupEntity.builder().withId(groupId).withName(groupName).withAuditInfo(auditInfo).build(); + groupBytes = protoEntitySerDe.serialize(groupWithoutFields); + groupFromBytes = protoEntitySerDe.deserialize(groupBytes, GroupEntity.class); + Assertions.assertEquals(groupWithoutFields, groupFromBytes); + Assertions.assertNull(groupWithoutFields.roles()); } } diff --git a/core/src/test/java/com/datastrato/gravitino/storage/TestEntityStorage.java b/core/src/test/java/com/datastrato/gravitino/storage/TestEntityStorage.java index 6666cd0453a..e311fc4a0e6 100644 --- a/core/src/test/java/com/datastrato/gravitino/storage/TestEntityStorage.java +++ b/core/src/test/java/com/datastrato/gravitino/storage/TestEntityStorage.java @@ -35,6 +35,7 @@ import com.datastrato.gravitino.meta.BaseMetalake; import com.datastrato.gravitino.meta.CatalogEntity; import com.datastrato.gravitino.meta.FilesetEntity; +import com.datastrato.gravitino.meta.GroupEntity; import com.datastrato.gravitino.meta.SchemaEntity; import com.datastrato.gravitino.meta.SchemaVersion; import com.datastrato.gravitino.meta.TableEntity; @@ -382,8 +383,8 @@ void testEntityUpdate(String type) throws Exception { @ParameterizedTest @MethodSource("storageProvider") - public void testUserEntityDelete(String type) throws IOException { - // User entity only supports kv store. + public void testAuthorizationEntityDelete(String type) throws IOException { + // User and Group entity only support kv store. Assumptions.assumeTrue(Configs.DEFAULT_ENTITY_STORE.equals(type)); Config config = Mockito.mock(Config.class); init(type, config); @@ -400,11 +401,19 @@ public void testUserEntityDelete(String type) throws IOException { store.put(oneUser); UserEntity anotherUser = createUser("metalake", "anotherUser", auditInfo); store.put(anotherUser); + GroupEntity oneGroup = createGroup("metalake", "oneGroup", auditInfo); + store.put(oneGroup); + GroupEntity anotherGroup = createGroup("metalake", "anotherGroup", auditInfo); + store.put(anotherGroup); Assertions.assertTrue(store.exists(oneUser.nameIdentifier(), Entity.EntityType.USER)); Assertions.assertTrue(store.exists(anotherUser.nameIdentifier(), Entity.EntityType.USER)); + Assertions.assertTrue(store.exists(oneGroup.nameIdentifier(), Entity.EntityType.GROUP)); + Assertions.assertTrue(store.exists(anotherGroup.nameIdentifier(), Entity.EntityType.GROUP)); store.delete(metalake.nameIdentifier(), Entity.EntityType.METALAKE); Assertions.assertFalse(store.exists(oneUser.nameIdentifier(), Entity.EntityType.USER)); Assertions.assertFalse(store.exists(anotherUser.nameIdentifier(), Entity.EntityType.USER)); + Assertions.assertFalse(store.exists(oneGroup.nameIdentifier(), Entity.EntityType.GROUP)); + Assertions.assertFalse(store.exists(anotherGroup.nameIdentifier(), Entity.EntityType.GROUP)); } } @@ -872,6 +881,20 @@ private static UserEntity createUser(String metalake, String name, AuditInfo aud .build(); } + private static GroupEntity createGroup(String metalake, String name, AuditInfo auditInfo) { + return GroupEntity.builder() + .withId(1L) + .withNamespace( + Namespace.of( + metalake, + CatalogEntity.SYSTEM_CATALOG_RESERVED_NAME, + GroupEntity.GROUP_SCHEMA_NAME)) + .withName(name) + .withAuditInfo(auditInfo) + .withRoles(Lists.newArrayList()) + .build(); + } + private void validateDeleteFilesetCascade(EntityStore store, FilesetEntity fileset1) throws IOException { // Delete the fileset 'metalake.catalog.schema1.fileset1' diff --git a/meta/src/main/proto/gravitino_meta.proto b/meta/src/main/proto/gravitino_meta.proto index 5468cfab255..5ced47063e3 100644 --- a/meta/src/main/proto/gravitino_meta.proto +++ b/meta/src/main/proto/gravitino_meta.proto @@ -108,3 +108,10 @@ message User { repeated string roles = 3; AuditInfo audit_info = 4; } + +message Group { + uint64 id = 1; + string name = 2; + repeated string roles = 3; + AuditInfo audit_info = 4; +} \ No newline at end of file From 7ea8222a0d33a4035ebd9a23718aad6bf94c8522 Mon Sep 17 00:00:00 2001 From: Ziva Li Date: Sun, 31 Mar 2024 17:42:44 +0800 Subject: [PATCH 06/20] [#2539] Improvement: Use environment variable to control showing or hiding web page in test (#2555) ### What changes were proposed in this pull request? Use an environment variable to control whether to show web page or not during integration tests. If we would like to see a pop-up page, we can set `param.environment("DISPLAY_WEBPAGE_IN_TESTING", true)` in build.gradle.kts. ### Why are the changes needed? Fix: #2539 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? MetalakePageTest.homePage() --- docs/how-to-test.md | 7 +++++++ .../test/web/ui/utils/ChromeWebDriverProvider.java | 8 +++++++- 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/docs/how-to-test.md b/docs/how-to-test.md index fd5d517b23e..affbcff8161 100644 --- a/docs/how-to-test.md +++ b/docs/how-to-test.md @@ -74,6 +74,13 @@ To deploy the Gravitino server locally to run the integration tests, follow thes * Skip web frontend integration tests by using the `./gradlew build -PskipWebITs` command. * Skip both unit tests and integration tests by using the `./gradlew build -x test` or `./gradlew build -PskipTests -PskipITs` commands. +## Configuring parameters for integration tests +### `DISPLAY_WEBPAGE_IN_TESTING` +By default, the Gravitino web frontend page will not pop up when running integration tests. +If you wish to display the web frontend page during integrations test, you can set the `DISPLAY_WEBPAGE_IN_TESTING` environment variable in `setIntegrationTestEnvironment` in file build.gradle.kts. +For example: +```param.environment("DISPLAY_WEBPAGE_IN_TESTING", true)``` + ## Docker test environment Some integration test cases depend on the Gravitino CI Docker image. diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/web/ui/utils/ChromeWebDriverProvider.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/web/ui/utils/ChromeWebDriverProvider.java index a39a98fc1e1..dbbd588ba0d 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/web/ui/utils/ChromeWebDriverProvider.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/web/ui/utils/ChromeWebDriverProvider.java @@ -11,6 +11,7 @@ import java.time.Instant; import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.SystemUtils; +import org.apache.logging.log4j.util.Strings; import org.openqa.selenium.WebDriver; import org.openqa.selenium.chrome.ChromeDriver; import org.openqa.selenium.chrome.ChromeOptions; @@ -86,12 +87,17 @@ public WebDriver createWebDriver() { System.setProperty( "webdriver.chrome.driver", ITUtils.joinPath(downLoadDir, chromeDriverBinName)); ChromeOptions chromeOptions = new ChromeOptions(); + + // Display the web interface during testing + if (Strings.isEmpty(System.getenv("DISPLAY_WEBPAGE_IN_TESTING"))) { + chromeOptions.addArguments("--headless"); + } + if (SystemUtils.IS_OS_MAC_OSX) { chromeOptions.setBinary( ITUtils.joinPath(downLoadDir, chromeBinName, "Contents", "MacOS", "Chromium")); } else { chromeOptions.setBinary(ITUtils.joinPath(downLoadDir, chromeBinName)); - chromeOptions.addArguments("--headless"); } return new ChromeDriver(chromeOptions); From ababbb177112683d3dbccfb2028a30a7de0baf52 Mon Sep 17 00:00:00 2001 From: mchades Date: Mon, 1 Apr 2024 11:30:06 +0800 Subject: [PATCH 07/20] [#2610] feat(kafka-catalog, core): Adapt the Kafka catalog to CatalogOperationsDispatcher (#2694) ### What changes were proposed in this pull request? This PR add the Kafka catalog adaption to the CatalogOperationsDispatcher ### Why are the changes needed? Part of Kafka catalog support work Fix: #2610 ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? UTs --- .../gravitino/messaging/TopicCatalog.java | 5 +- .../catalog/kafka/KafkaCatalogOperations.java | 44 ++- .../kafka/TestKafkaCatalogOperations.java | 3 + .../datastrato/gravitino/GravitinoEnv.java | 14 + .../gravitino/catalog/CatalogManager.java | 15 + .../catalog/EntityCombinedTopic.java | 76 +++++ .../catalog/OperationDispatcher.java | 7 + .../catalog/TopicOperationDispatcher.java | 267 ++++++++++++++++++ .../storage/kv/BinaryEntityKeyEncoder.java | 5 +- .../storage/relational/JDBCBackend.java | 1 + .../gravitino/TestCatalogOperations.java | 106 ++++++- .../com/datastrato/gravitino/TestTopic.java | 34 +++ .../catalog/TestTopicOperationDispatcher.java | 204 +++++++++++++ 13 files changed, 762 insertions(+), 19 deletions(-) create mode 100644 core/src/main/java/com/datastrato/gravitino/catalog/EntityCombinedTopic.java create mode 100644 core/src/main/java/com/datastrato/gravitino/catalog/TopicOperationDispatcher.java create mode 100644 core/src/test/java/com/datastrato/gravitino/TestTopic.java create mode 100644 core/src/test/java/com/datastrato/gravitino/catalog/TestTopicOperationDispatcher.java diff --git a/api/src/main/java/com/datastrato/gravitino/messaging/TopicCatalog.java b/api/src/main/java/com/datastrato/gravitino/messaging/TopicCatalog.java index de10613623c..b2979697f6e 100644 --- a/api/src/main/java/com/datastrato/gravitino/messaging/TopicCatalog.java +++ b/api/src/main/java/com/datastrato/gravitino/messaging/TopicCatalog.java @@ -85,8 +85,7 @@ Topic alterTopic(NameIdentifier ident, TopicChange... changes) * Drop a topic from the catalog. * * @param ident A topic identifier. - * @return true If the topic is dropped, false otherwise. - * @throws NoSuchTopicException If the topic does not exist. + * @return true If the topic is dropped, false if the topic does not exist. */ - boolean dropTopic(NameIdentifier ident) throws NoSuchTopicException; + boolean dropTopic(NameIdentifier ident); } 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 24d36598796..96c3fde17b9 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 @@ -5,10 +5,12 @@ package com.datastrato.gravitino.catalog.kafka; import static com.datastrato.gravitino.StringIdentifier.ID_KEY; +import static com.datastrato.gravitino.StringIdentifier.newPropertiesWithId; 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 static com.datastrato.gravitino.storage.RandomIdGenerator.MAX_ID; import com.datastrato.gravitino.Entity; import com.datastrato.gravitino.EntityStore; @@ -65,6 +67,7 @@ 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.Uuid; import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.errors.InvalidReplicationFactorException; @@ -159,13 +162,16 @@ public Topic loadTopic(NameIdentifier ident) throws NoSuchTopicException { ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, ident.name()); DescribeConfigsResult configsResult = adminClient.describeConfigs(Collections.singleton(configResource)); + int partitions; int replicationFactor; + Uuid topicId; Map properties = Maps.newHashMap(); try { TopicDescription topicDescription = result.topicNameValues().get(ident.name()).get(); partitions = topicDescription.partitions().size(); replicationFactor = topicDescription.partitions().get(0).replicas().size(); + topicId = topicDescription.topicId(); Config topicConfigs = configsResult.all().get().get(configResource); topicConfigs.entries().forEach(e -> properties.put(e.name(), e.value())); @@ -185,7 +191,9 @@ public Topic loadTopic(NameIdentifier ident) throws NoSuchTopicException { return KafkaTopic.builder() .withName(ident.name()) - .withProperties(properties) + // Because there is no way to store the Gravitino ID in Kafka, therefor we use the topic ID + // as the Gravitino ID + .withProperties(newPropertiesWithId(convertToGravitinoId(topicId), properties)) .withAuditInfo( AuditInfo.builder() .withCreator(PrincipalUtils.getCurrentPrincipal().getName()) @@ -204,11 +212,26 @@ public Topic createTopic( try { CreateTopicsResult createTopicsResult = adminClient.createTopics(Collections.singleton(buildNewTopic(ident, properties))); + Uuid topicId = createTopicsResult.topicId(ident.name()).get(); LOG.info( - "Created topic {} with {} partitions and replication factor {}", + "Created topic {}[id: {}] with {} partitions and replication factor {}", ident, + topicId.toString(), createTopicsResult.numPartitions(ident.name()).get(), createTopicsResult.replicationFactor(ident.name()).get()); + + return KafkaTopic.builder() + .withName(ident.name()) + .withComment(comment) + // Because there is no way to store the Gravitino ID in Kafka, therefor we use the topic + // ID as the Gravitino ID + .withProperties(newPropertiesWithId(convertToGravitinoId(topicId), properties)) + .withAuditInfo( + AuditInfo.builder() + .withCreator(PrincipalUtils.getCurrentPrincipal().getName()) + .withCreateTime(Instant.now()) + .build()) + .build(); } catch (ExecutionException e) { if (e.getCause() instanceof TopicExistsException) { throw new TopicAlreadyExistsException(e, "Topic %s already exists", ident); @@ -227,17 +250,6 @@ public Topic createTopic( } 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 @@ -297,7 +309,7 @@ public Topic alterTopic(NameIdentifier ident, TopicChange... changes) } @Override - public boolean dropTopic(NameIdentifier ident) throws NoSuchTopicException { + public boolean dropTopic(NameIdentifier ident) { NameIdentifier schemaIdent = NameIdentifier.of(ident.namespace().levels()); checkSchemaExists(schemaIdent); @@ -502,6 +514,10 @@ private void doAlterTopicConfig(String topicName, List alterConfi } } + private StringIdentifier convertToGravitinoId(Uuid topicId) { + return StringIdentifier.fromId(topicId.getLeastSignificantBits() & MAX_ID); + } + private NewTopic buildNewTopic(NameIdentifier ident, Map properties) { Optional partitionCount = Optional.ofNullable( 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 fe752a1b4f0..c61d70edbfe 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 @@ -256,6 +256,7 @@ public void testCreateTopic() { Assertions.assertEquals("1", createdTopic.properties().get(REPLICATION_FACTOR)); Assertions.assertEquals( "producer", createdTopic.properties().get(TopicConfig.COMPRESSION_TYPE_CONFIG)); + Assertions.assertNotNull(createdTopic.properties().get(ID_KEY)); } @Test @@ -315,6 +316,7 @@ public void testLoadTopic() { Assertions.assertEquals(TOPIC_1, topic.name()); Assertions.assertEquals("1", topic.properties().get(PARTITION_COUNT)); Assertions.assertEquals("1", topic.properties().get(REPLICATION_FACTOR)); + Assertions.assertNotNull(topic.properties().get(ID_KEY)); Assertions.assertTrue(topic.properties().size() > 2); } @@ -406,6 +408,7 @@ public void testAlterTopic() { Assertions.assertNull(alteredTopic.comment()); Assertions.assertEquals("3", alteredTopic.properties().get(PARTITION_COUNT)); Assertions.assertEquals("1", alteredTopic.properties().get(REPLICATION_FACTOR)); + Assertions.assertNotNull(alteredTopic.properties().get(ID_KEY)); Assertions.assertEquals( "producer", alteredTopic.properties().get(TopicConfig.COMPRESSION_TYPE_CONFIG)); // retention.ms overridden was removed, so it should be the default value diff --git a/core/src/main/java/com/datastrato/gravitino/GravitinoEnv.java b/core/src/main/java/com/datastrato/gravitino/GravitinoEnv.java index bd23f9bff40..2eba11037df 100644 --- a/core/src/main/java/com/datastrato/gravitino/GravitinoEnv.java +++ b/core/src/main/java/com/datastrato/gravitino/GravitinoEnv.java @@ -10,6 +10,7 @@ import com.datastrato.gravitino.catalog.FilesetOperationDispatcher; import com.datastrato.gravitino.catalog.SchemaOperationDispatcher; import com.datastrato.gravitino.catalog.TableOperationDispatcher; +import com.datastrato.gravitino.catalog.TopicOperationDispatcher; import com.datastrato.gravitino.lock.LockManager; import com.datastrato.gravitino.metalake.MetalakeManager; import com.datastrato.gravitino.metrics.MetricsSystem; @@ -40,6 +41,8 @@ public class GravitinoEnv { private FilesetOperationDispatcher filesetOperationDispatcher; + private TopicOperationDispatcher topicOperationDispatcher; + private MetalakeManager metalakeManager; private AccessControlManager accessControlManager; @@ -109,6 +112,8 @@ public void initialize(Config config) { new TableOperationDispatcher(catalogManager, entityStore, idGenerator); this.filesetOperationDispatcher = new FilesetOperationDispatcher(catalogManager, entityStore, idGenerator); + this.topicOperationDispatcher = + new TopicOperationDispatcher(catalogManager, entityStore, idGenerator); // Create and initialize access control related modules this.accessControlManager = new AccessControlManager(entityStore, idGenerator); @@ -177,6 +182,15 @@ public FilesetOperationDispatcher filesetOperationDispatcher() { return filesetOperationDispatcher; } + /** + * Get the TopicOperationDispatcher associated with the Gravitino environment. + * + * @return The TopicOperationDispatcher instance. + */ + public TopicOperationDispatcher topicOperationDispatcher() { + return topicOperationDispatcher; + } + /** * Get the MetalakeManager associated with the Gravitino environment. * diff --git a/core/src/main/java/com/datastrato/gravitino/catalog/CatalogManager.java b/core/src/main/java/com/datastrato/gravitino/catalog/CatalogManager.java index 7f7a718e5e1..9f2447640eb 100644 --- a/core/src/main/java/com/datastrato/gravitino/catalog/CatalogManager.java +++ b/core/src/main/java/com/datastrato/gravitino/catalog/CatalogManager.java @@ -29,6 +29,7 @@ import com.datastrato.gravitino.exceptions.NoSuchEntityException; import com.datastrato.gravitino.exceptions.NoSuchMetalakeException; import com.datastrato.gravitino.file.FilesetCatalog; +import com.datastrato.gravitino.messaging.TopicCatalog; import com.datastrato.gravitino.meta.AuditInfo; import com.datastrato.gravitino.meta.CatalogEntity; import com.datastrato.gravitino.rel.SupportsSchemas; @@ -117,6 +118,16 @@ public R doWithFilesetOps(ThrowableFunction fn) throws Ex }); } + public R doWithTopicOps(ThrowableFunction fn) throws Exception { + return classLoader.withClassLoader( + cl -> { + if (asTopics() == null) { + throw new UnsupportedOperationException("Catalog does not support topic operations"); + } + return fn.apply(asTopics()); + }); + } + public R doWithPropertiesMeta(ThrowableFunction fn) throws Exception { return classLoader.withClassLoader(cl -> fn.apply(catalog.ops())); @@ -150,6 +161,10 @@ private TableCatalog asTables() { private FilesetCatalog asFilesets() { return catalog.ops() instanceof FilesetCatalog ? (FilesetCatalog) catalog.ops() : null; } + + private TopicCatalog asTopics() { + return catalog.ops() instanceof TopicCatalog ? (TopicCatalog) catalog.ops() : null; + } } private final Config config; diff --git a/core/src/main/java/com/datastrato/gravitino/catalog/EntityCombinedTopic.java b/core/src/main/java/com/datastrato/gravitino/catalog/EntityCombinedTopic.java new file mode 100644 index 00000000000..695de359650 --- /dev/null +++ b/core/src/main/java/com/datastrato/gravitino/catalog/EntityCombinedTopic.java @@ -0,0 +1,76 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.catalog; + +import com.datastrato.gravitino.Audit; +import com.datastrato.gravitino.messaging.Topic; +import com.datastrato.gravitino.meta.AuditInfo; +import com.datastrato.gravitino.meta.TopicEntity; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * A Topic class to represent a topic metadata object that combines the metadata both from {@link + * Topic} and {@link TopicEntity}. + */ +public class EntityCombinedTopic implements Topic { + + private final Topic topic; + private final TopicEntity topicEntity; + + // Sets of properties that should be hidden from the user. + private Set hiddenProperties; + + private EntityCombinedTopic(Topic topic, TopicEntity topicEntity) { + this.topic = topic; + this.topicEntity = topicEntity; + } + + public static EntityCombinedTopic of(Topic topic, TopicEntity topicEntity) { + return new EntityCombinedTopic(topic, topicEntity); + } + + public static EntityCombinedTopic of(Topic topic) { + return new EntityCombinedTopic(topic, null); + } + + public EntityCombinedTopic withHiddenPropertiesSet(Set hiddenProperties) { + this.hiddenProperties = hiddenProperties; + return this; + } + + @Override + public String name() { + return topic.name(); + } + + @Override + public String comment() { + return topic.comment(); + } + + @Override + public Map properties() { + return topic.properties().entrySet().stream() + .filter(p -> !hiddenProperties.contains(p.getKey())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + @Override + public Audit auditInfo() { + AuditInfo mergedAudit = + AuditInfo.builder() + .withCreator(topic.auditInfo().creator()) + .withCreateTime(topic.auditInfo().createTime()) + .withLastModifier(topic.auditInfo().lastModifier()) + .withLastModifiedTime(topic.auditInfo().lastModifiedTime()) + .build(); + + return topicEntity == null + ? topic.auditInfo() + : mergedAudit.merge(topicEntity.auditInfo(), true /* overwrite */); + } +} diff --git a/core/src/main/java/com/datastrato/gravitino/catalog/OperationDispatcher.java b/core/src/main/java/com/datastrato/gravitino/catalog/OperationDispatcher.java index d4a16ede9d2..d0d091186eb 100644 --- a/core/src/main/java/com/datastrato/gravitino/catalog/OperationDispatcher.java +++ b/core/src/main/java/com/datastrato/gravitino/catalog/OperationDispatcher.java @@ -17,6 +17,7 @@ import com.datastrato.gravitino.exceptions.IllegalNameIdentifierException; import com.datastrato.gravitino.exceptions.NoSuchEntityException; import com.datastrato.gravitino.file.FilesetChange; +import com.datastrato.gravitino.messaging.TopicChange; import com.datastrato.gravitino.rel.SchemaChange; import com.datastrato.gravitino.rel.TableChange; import com.datastrato.gravitino.storage.IdGenerator; @@ -143,6 +144,9 @@ private Map getPropertiesForSet(T... t) { } else if (item instanceof FilesetChange.SetProperty) { FilesetChange.SetProperty setProperty = (FilesetChange.SetProperty) item; properties.put(setProperty.getProperty(), setProperty.getValue()); + } else if (item instanceof TopicChange.SetProperty) { + TopicChange.SetProperty setProperty = (TopicChange.SetProperty) item; + properties.put(setProperty.getProperty(), setProperty.getValue()); } } @@ -161,6 +165,9 @@ private Map getPropertiesForDelete(T... t) { } else if (item instanceof FilesetChange.RemoveProperty) { FilesetChange.RemoveProperty removeProperty = (FilesetChange.RemoveProperty) item; properties.put(removeProperty.getProperty(), removeProperty.getProperty()); + } else if (item instanceof TopicChange.RemoveProperty) { + TopicChange.RemoveProperty removeProperty = (TopicChange.RemoveProperty) item; + properties.put(removeProperty.getProperty(), removeProperty.getProperty()); } } diff --git a/core/src/main/java/com/datastrato/gravitino/catalog/TopicOperationDispatcher.java b/core/src/main/java/com/datastrato/gravitino/catalog/TopicOperationDispatcher.java new file mode 100644 index 00000000000..a64bd0af23a --- /dev/null +++ b/core/src/main/java/com/datastrato/gravitino/catalog/TopicOperationDispatcher.java @@ -0,0 +1,267 @@ +/* + * 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.TOPIC; +import static com.datastrato.gravitino.StringIdentifier.fromProperties; +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.NoSuchSchemaException; +import com.datastrato.gravitino.exceptions.NoSuchTopicException; +import com.datastrato.gravitino.exceptions.TopicAlreadyExistsException; +import com.datastrato.gravitino.messaging.DataLayout; +import com.datastrato.gravitino.messaging.Topic; +import com.datastrato.gravitino.messaging.TopicCatalog; +import com.datastrato.gravitino.messaging.TopicChange; +import com.datastrato.gravitino.meta.AuditInfo; +import com.datastrato.gravitino.meta.TopicEntity; +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 TopicOperationDispatcher extends OperationDispatcher implements TopicCatalog { + private static final Logger LOG = LoggerFactory.getLogger(TopicOperationDispatcher.class); + + /** + * Creates a new TopicOperationDispatcher 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 TopicOperationDispatcher( + CatalogManager catalogManager, EntityStore store, IdGenerator idGenerator) { + super(catalogManager, store, idGenerator); + } + + /** + * List the topics in a schema namespace from the catalog. + * + * @param namespace A schema namespace. + * @return An array of topic identifiers in the namespace. + * @throws NoSuchSchemaException If the schema does not exist. + */ + @Override + public NameIdentifier[] listTopics(Namespace namespace) throws NoSuchSchemaException { + return doWithCatalog( + getCatalogIdentifier(NameIdentifier.of(namespace.levels())), + c -> c.doWithTopicOps(t -> t.listTopics(namespace)), + NoSuchSchemaException.class); + } + + /** + * Load topic metadata by {@link NameIdentifier} from the catalog. + * + * @param ident A topic identifier. + * @return The topic metadata. + * @throws NoSuchTopicException If the topic does not exist. + */ + @Override + public Topic loadTopic(NameIdentifier ident) throws NoSuchTopicException { + NameIdentifier catalogIdent = getCatalogIdentifier(ident); + Topic topic = + doWithCatalog( + catalogIdent, + c -> c.doWithTopicOps(t -> t.loadTopic(ident)), + NoSuchTopicException.class); + + StringIdentifier stringId = getStringIdFromProperties(topic.properties()); + // Case 1: The topic is not created by Gravitino. + // Note: for Kafka catalog, stringId will not be null. Because there is no way to store the + // Gravitino + // ID in Kafka, therefor we use the topic ID as the Gravitino ID + if (stringId == null) { + return EntityCombinedTopic.of(topic) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdent, HasPropertyMetadata::topicPropertiesMetadata, topic.properties())); + } + + TopicEntity topicEntity = + operateOnEntity( + ident, + identifier -> store.get(identifier, TOPIC, TopicEntity.class), + "GET", + getStringIdFromProperties(topic.properties()).id()); + + return EntityCombinedTopic.of(topic, topicEntity) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdent, HasPropertyMetadata::topicPropertiesMetadata, topic.properties())); + } + + /** + * Create a topic in the catalog. + * + * @param ident A topic identifier. + * @param comment The comment of the topic object. Null is set if no comment is specified. + * @param dataLayout The message schema of the topic object. Always null because it's not + * supported yet. + * @param properties The properties of the topic object. Empty map is set if no properties are + * specified. + * @return The topic metadata. + * @throws NoSuchSchemaException If the schema does not exist. + * @throws TopicAlreadyExistsException If the topic already exists. + */ + @Override + public Topic createTopic( + NameIdentifier ident, String comment, DataLayout dataLayout, Map properties) + throws NoSuchSchemaException, TopicAlreadyExistsException { + NameIdentifier catalogIdent = getCatalogIdentifier(ident); + doWithCatalog( + catalogIdent, + c -> + c.doWithPropertiesMeta( + p -> { + validatePropertyForCreate(p.topicPropertiesMetadata(), properties); + return null; + }), + IllegalArgumentException.class); + Long uid = idGenerator.nextId(); + StringIdentifier stringId = StringIdentifier.fromId(uid); + Map updatedProperties = + StringIdentifier.newPropertiesWithId(stringId, properties); + + doWithCatalog( + catalogIdent, + c -> c.doWithTopicOps(t -> t.createTopic(ident, comment, dataLayout, updatedProperties)), + NoSuchSchemaException.class, + TopicAlreadyExistsException.class); + + // Retrieve the Topic again to obtain some values generated by underlying catalog + Topic topic = + doWithCatalog( + catalogIdent, + c -> c.doWithTopicOps(t -> t.loadTopic(ident)), + NoSuchTopicException.class); + + TopicEntity topicEntity = + TopicEntity.builder() + .withId(fromProperties(topic.properties()).id()) + .withName(ident.name()) + .withNamespace(ident.namespace()) + .withAuditInfo( + AuditInfo.builder() + .withCreator(PrincipalUtils.getCurrentPrincipal().getName()) + .withCreateTime(Instant.now()) + .build()) + .build(); + + try { + store.put(topicEntity, true /* overwrite */); + } catch (Exception e) { + LOG.error(OperationDispatcher.FormattedErrorMessages.STORE_OP_FAILURE, "put", ident, e); + return EntityCombinedTopic.of(topic) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdent, HasPropertyMetadata::topicPropertiesMetadata, topic.properties())); + } + + return EntityCombinedTopic.of(topic, topicEntity) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdent, HasPropertyMetadata::topicPropertiesMetadata, topic.properties())); + } + + /** + * Apply the {@link TopicChange changes} to a topic in the catalog. + * + * @param ident A topic identifier. + * @param changes The changes to apply to the topic. + * @return The altered topic metadata. + * @throws NoSuchTopicException If the topic does not exist. + * @throws IllegalArgumentException If the changes is rejected by the implementation. + */ + @Override + public Topic alterTopic(NameIdentifier ident, TopicChange... changes) + throws NoSuchTopicException, IllegalArgumentException { + validateAlterProperties(ident, HasPropertyMetadata::topicPropertiesMetadata, changes); + + NameIdentifier catalogIdent = getCatalogIdentifier(ident); + Topic tempAlteredTable = + doWithCatalog( + catalogIdent, + c -> c.doWithTopicOps(t -> t.alterTopic(ident, changes)), + NoSuchTopicException.class, + IllegalArgumentException.class); + + // Retrieve the Topic again to obtain some values generated by underlying catalog + Topic alteredTopic = + doWithCatalog( + catalogIdent, + c -> + c.doWithTopicOps( + t -> + t.loadTopic(NameIdentifier.of(ident.namespace(), tempAlteredTable.name()))), + NoSuchTopicException.class); + + TopicEntity updatedTopicEntity = + operateOnEntity( + ident, + id -> + store.update( + id, + TopicEntity.class, + TOPIC, + topicEntity -> + TopicEntity.builder() + .withId(topicEntity.id()) + .withName(topicEntity.name()) + .withNamespace(ident.namespace()) + .withAuditInfo( + AuditInfo.builder() + .withCreator(topicEntity.auditInfo().creator()) + .withCreateTime(topicEntity.auditInfo().createTime()) + .withLastModifier( + PrincipalUtils.getCurrentPrincipal().getName()) + .withLastModifiedTime(Instant.now()) + .build()) + .build()), + "UPDATE", + getStringIdFromProperties(alteredTopic.properties()).id()); + + return EntityCombinedTopic.of(alteredTopic, updatedTopicEntity) + .withHiddenPropertiesSet( + getHiddenPropertyNames( + catalogIdent, + HasPropertyMetadata::topicPropertiesMetadata, + alteredTopic.properties())); + } + + /** + * Drop a topic from the catalog. + * + * @param ident A topic identifier. + * @return true If the topic is dropped, false if the topic does not exist. + */ + @Override + public boolean dropTopic(NameIdentifier ident) { + boolean dropped = + doWithCatalog( + getCatalogIdentifier(ident), + c -> c.doWithTopicOps(t -> t.dropTopic(ident)), + NoSuchTopicException.class); + + if (!dropped) { + return false; + } + + try { + store.delete(ident, TOPIC); + } catch (Exception e) { + throw new RuntimeException(e); + } + + return true; + } +} diff --git a/core/src/main/java/com/datastrato/gravitino/storage/kv/BinaryEntityKeyEncoder.java b/core/src/main/java/com/datastrato/gravitino/storage/kv/BinaryEntityKeyEncoder.java index 9882928d9b9..dfac508507b 100644 --- a/core/src/main/java/com/datastrato/gravitino/storage/kv/BinaryEntityKeyEncoder.java +++ b/core/src/main/java/com/datastrato/gravitino/storage/kv/BinaryEntityKeyEncoder.java @@ -10,6 +10,7 @@ import static com.datastrato.gravitino.Entity.EntityType.METALAKE; import static com.datastrato.gravitino.Entity.EntityType.SCHEMA; import static com.datastrato.gravitino.Entity.EntityType.TABLE; +import static com.datastrato.gravitino.Entity.EntityType.TOPIC; import static com.datastrato.gravitino.Entity.EntityType.USER; import com.datastrato.gravitino.Entity.EntityType; @@ -81,7 +82,9 @@ public class BinaryEntityKeyEncoder implements EntityKeyEncoder { USER, new String[] {USER.getShortName() + "/", "/", "/", "/"}, GROUP, - new String[] {GROUP.getShortName() + "/", "/", "/", "/"}); + new String[] {GROUP.getShortName() + "/", "/", "/", "/"}, + TOPIC, + new String[] {TOPIC.getShortName() + "/", "/", "/", "/"}); @VisibleForTesting final NameMappingService nameMappingService; diff --git a/core/src/main/java/com/datastrato/gravitino/storage/relational/JDBCBackend.java b/core/src/main/java/com/datastrato/gravitino/storage/relational/JDBCBackend.java index 6457c67e606..0de8c5a7477 100644 --- a/core/src/main/java/com/datastrato/gravitino/storage/relational/JDBCBackend.java +++ b/core/src/main/java/com/datastrato/gravitino/storage/relational/JDBCBackend.java @@ -88,6 +88,7 @@ public void insert(E e, boolean overwritten) } else if (e instanceof FilesetEntity) { FilesetMetaService.getInstance().insertFileset((FilesetEntity) e, overwritten); } else { + // TODO: Add support for TopicEntity throw new UnsupportedEntityTypeException( "Unsupported entity type: %s for insert operation", e.getClass()); } diff --git a/core/src/test/java/com/datastrato/gravitino/TestCatalogOperations.java b/core/src/test/java/com/datastrato/gravitino/TestCatalogOperations.java index 44b1ef5c976..dba2b03d307 100644 --- a/core/src/test/java/com/datastrato/gravitino/TestCatalogOperations.java +++ b/core/src/test/java/com/datastrato/gravitino/TestCatalogOperations.java @@ -14,12 +14,18 @@ import com.datastrato.gravitino.exceptions.NoSuchFilesetException; import com.datastrato.gravitino.exceptions.NoSuchSchemaException; import com.datastrato.gravitino.exceptions.NoSuchTableException; +import com.datastrato.gravitino.exceptions.NoSuchTopicException; import com.datastrato.gravitino.exceptions.NonEmptySchemaException; import com.datastrato.gravitino.exceptions.SchemaAlreadyExistsException; import com.datastrato.gravitino.exceptions.TableAlreadyExistsException; +import com.datastrato.gravitino.exceptions.TopicAlreadyExistsException; import com.datastrato.gravitino.file.Fileset; import com.datastrato.gravitino.file.FilesetCatalog; import com.datastrato.gravitino.file.FilesetChange; +import com.datastrato.gravitino.messaging.DataLayout; +import com.datastrato.gravitino.messaging.Topic; +import com.datastrato.gravitino.messaging.TopicCatalog; +import com.datastrato.gravitino.messaging.TopicChange; import com.datastrato.gravitino.meta.AuditInfo; import com.datastrato.gravitino.rel.Column; import com.datastrato.gravitino.rel.Schema; @@ -40,7 +46,7 @@ import java.util.Map; public class TestCatalogOperations - implements CatalogOperations, TableCatalog, FilesetCatalog, SupportsSchemas { + implements CatalogOperations, TableCatalog, FilesetCatalog, TopicCatalog, SupportsSchemas { private final Map tables; @@ -48,6 +54,8 @@ public class TestCatalogOperations private final Map filesets; + private final Map topics; + private final BasePropertiesMetadata tablePropertiesMetadata; private final BasePropertiesMetadata schemaPropertiesMetadata; @@ -64,6 +72,7 @@ public TestCatalogOperations(Map config) { tables = Maps.newHashMap(); schemas = Maps.newHashMap(); filesets = Maps.newHashMap(); + topics = Maps.newHashMap(); tablePropertiesMetadata = new TestBasePropertiesMetadata(); schemaPropertiesMetadata = new TestBasePropertiesMetadata(); filesetPropertiesMetadata = new TestFilesetPropertiesMetadata(); @@ -493,4 +502,99 @@ public boolean dropFileset(NameIdentifier ident) { return false; } } + + @Override + public NameIdentifier[] listTopics(Namespace namespace) throws NoSuchSchemaException { + return topics.keySet().stream() + .filter(ident -> ident.namespace().equals(namespace)) + .toArray(NameIdentifier[]::new); + } + + @Override + public Topic loadTopic(NameIdentifier ident) throws NoSuchTopicException { + if (topics.containsKey(ident)) { + return topics.get(ident); + } else { + throw new NoSuchTopicException("Topic %s does not exist", ident); + } + } + + @Override + public Topic createTopic( + NameIdentifier ident, String comment, DataLayout dataLayout, Map properties) + throws NoSuchSchemaException, TopicAlreadyExistsException { + AuditInfo auditInfo = + AuditInfo.builder().withCreator("test").withCreateTime(Instant.now()).build(); + TestTopic topic = + TestTopic.builder() + .withName(ident.name()) + .withComment(comment) + .withProperties(properties) + .withAuditInfo(auditInfo) + .build(); + + if (topics.containsKey(ident)) { + throw new TopicAlreadyExistsException("Topic %s already exists", ident); + } else { + topics.put(ident, topic); + } + + return topic; + } + + @Override + public Topic alterTopic(NameIdentifier ident, TopicChange... changes) + throws NoSuchTopicException, IllegalArgumentException { + if (!topics.containsKey(ident)) { + throw new NoSuchTopicException("Topic %s does not exist", ident); + } + + AuditInfo updatedAuditInfo = + AuditInfo.builder() + .withCreator("test") + .withCreateTime(Instant.now()) + .withLastModifier("test") + .withLastModifiedTime(Instant.now()) + .build(); + + TestTopic topic = topics.get(ident); + Map newProps = + topic.properties() != null ? Maps.newHashMap(topic.properties()) : Maps.newHashMap(); + String newComment = topic.comment(); + + for (TopicChange change : changes) { + if (change instanceof TopicChange.SetProperty) { + newProps.put( + ((TopicChange.SetProperty) change).getProperty(), + ((TopicChange.SetProperty) change).getValue()); + } else if (change instanceof TopicChange.RemoveProperty) { + newProps.remove(((TopicChange.RemoveProperty) change).getProperty()); + } else if (change instanceof TopicChange.UpdateTopicComment) { + newComment = ((TopicChange.UpdateTopicComment) change).getNewComment(); + } else { + throw new IllegalArgumentException("Unsupported topic change: " + change); + } + } + + TestTopic updatedTopic = + TestTopic.builder() + .withName(ident.name()) + .withComment(newComment) + .withProperties(newProps) + .withAuditInfo(updatedAuditInfo) + .build(); + + topics.put(ident, updatedTopic); + return updatedTopic; + } + + @Override + public boolean dropTopic(NameIdentifier ident) throws NoSuchTopicException { + if (topics.containsKey(ident)) { + topics.remove(ident); + return true; + } else { + return false; + } + } } diff --git a/core/src/test/java/com/datastrato/gravitino/TestTopic.java b/core/src/test/java/com/datastrato/gravitino/TestTopic.java new file mode 100644 index 00000000000..c5478c84351 --- /dev/null +++ b/core/src/test/java/com/datastrato/gravitino/TestTopic.java @@ -0,0 +1,34 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino; + +import com.datastrato.gravitino.connector.BaseTopic; + +public class TestTopic extends BaseTopic { + + public static class Builder extends BaseTopicBuilder { + /** Creates a new instance of {@link Builder}. */ + private Builder() {} + + @Override + protected TestTopic internalBuild() { + TestTopic topic = new TestTopic(); + topic.name = name; + topic.comment = comment; + topic.properties = properties; + topic.auditInfo = auditInfo; + return topic; + } + } + + /** + * Creates a new instance of {@link Builder}. + * + * @return The new instance. + */ + public static Builder builder() { + return new Builder(); + } +} diff --git a/core/src/test/java/com/datastrato/gravitino/catalog/TestTopicOperationDispatcher.java b/core/src/test/java/com/datastrato/gravitino/catalog/TestTopicOperationDispatcher.java new file mode 100644 index 00000000000..22dee4baa4a --- /dev/null +++ b/core/src/test/java/com/datastrato/gravitino/catalog/TestTopicOperationDispatcher.java @@ -0,0 +1,204 @@ +/* + * 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.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.messaging.Topic; +import com.datastrato.gravitino.messaging.TopicChange; +import com.datastrato.gravitino.meta.AuditInfo; +import com.datastrato.gravitino.meta.TopicEntity; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.time.Instant; +import java.util.Map; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +public class TestTopicOperationDispatcher extends TestOperationDispatcher { + + private static SchemaOperationDispatcher schemaOperationDispatcher; + private static TopicOperationDispatcher topicOperationDispatcher; + + @BeforeAll + public static void initialize() throws IOException { + schemaOperationDispatcher = + new SchemaOperationDispatcher(catalogManager, entityStore, idGenerator); + topicOperationDispatcher = + new TopicOperationDispatcher(catalogManager, entityStore, idGenerator); + } + + @Test + public void testCreateAndListTopics() { + Namespace topicNs = Namespace.of(metalake, catalog, "schema121"); + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + schemaOperationDispatcher.createSchema(NameIdentifier.of(topicNs.levels()), "comment", props); + + NameIdentifier topicIdent1 = NameIdentifier.of(topicNs, "topic1"); + Topic topic1 = topicOperationDispatcher.createTopic(topicIdent1, "comment", null, props); + Assertions.assertEquals("topic1", topic1.name()); + Assertions.assertEquals("comment", topic1.comment()); + testProperties(props, topic1.properties()); + + NameIdentifier[] idents = topicOperationDispatcher.listTopics(topicNs); + Assertions.assertEquals(1, idents.length); + Assertions.assertEquals(topicIdent1, idents[0]); + + Map illegalProps = ImmutableMap.of("k2", "v2"); + testPropertyException( + () -> topicOperationDispatcher.createTopic(topicIdent1, "comment", null, illegalProps), + "Properties are required and must be set"); + + Map illegalProps2 = ImmutableMap.of("k1", "v1", ID_KEY, "test"); + testPropertyException( + () -> topicOperationDispatcher.createTopic(topicIdent1, "comment", null, illegalProps2), + "Properties are reserved and cannot be set", + "gravitino.identifier"); + } + + @Test + public void testCreateAndLoadTopic() throws IOException { + Namespace topicNs = Namespace.of(metalake, catalog, "schema131"); + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + schemaOperationDispatcher.createSchema(NameIdentifier.of(topicNs.levels()), "comment", props); + + NameIdentifier topicIdent1 = NameIdentifier.of(topicNs, "topic11"); + Topic topic1 = topicOperationDispatcher.createTopic(topicIdent1, "comment", null, props); + Assertions.assertEquals("topic11", topic1.name()); + Assertions.assertEquals("comment", topic1.comment()); + testProperties(props, topic1.properties()); + + Topic loadedTopic1 = topicOperationDispatcher.loadTopic(topicIdent1); + Assertions.assertEquals(topic1.name(), loadedTopic1.name()); + Assertions.assertEquals(topic1.comment(), loadedTopic1.comment()); + testProperties(props, loadedTopic1.properties()); + // Audit info is gotten from the entity store + Assertions.assertEquals(AuthConstants.ANONYMOUS_USER, loadedTopic1.auditInfo().creator()); + + // Case 2: Test if the topic entity is not found in the entity store + reset(entityStore); + doThrow(new NoSuchEntityException("")).when(entityStore).get(any(), any(), any()); + Topic loadedTopic2 = topicOperationDispatcher.loadTopic(topicIdent1); + // Audit info is gotten from the catalog, not from the entity store + Assertions.assertEquals("test", loadedTopic2.auditInfo().creator()); + + // Case 3: Test if the entity store is failed to get the topic entity + reset(entityStore); + doThrow(new IOException()).when(entityStore).get(any(), any(), any()); + Topic loadedTopic3 = topicOperationDispatcher.loadTopic(topicIdent1); + // Audit info is gotten from the catalog, not from the entity store + Assertions.assertEquals("test", loadedTopic3.auditInfo().creator()); + + // Case 4: Test if the topic entity is not matched + reset(entityStore); + TopicEntity unmatchedEntity = + TopicEntity.builder() + .withId(1L) + .withName("topic11") + .withNamespace(topicNs) + .withAuditInfo( + AuditInfo.builder().withCreator("gravitino").withCreateTime(Instant.now()).build()) + .build(); + doReturn(unmatchedEntity).when(entityStore).get(any(), any(), any()); + Topic loadedTopic4 = topicOperationDispatcher.loadTopic(topicIdent1); + // Audit info is gotten from the catalog, not from the entity store + Assertions.assertEquals("test", loadedTopic4.auditInfo().creator()); + } + + @Test + public void testCreateAndAlterTopic() throws IOException { + Namespace topicNs = Namespace.of(metalake, catalog, "schema141"); + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + schemaOperationDispatcher.createSchema(NameIdentifier.of(topicNs.levels()), "comment", props); + + NameIdentifier topicIdent = NameIdentifier.of(topicNs, "topic21"); + Topic topic = topicOperationDispatcher.createTopic(topicIdent, "comment", null, props); + + TopicChange[] changes = + new TopicChange[] {TopicChange.setProperty("k3", "v3"), TopicChange.removeProperty("k1")}; + + Topic alteredTopic = topicOperationDispatcher.alterTopic(topicIdent, changes); + Assertions.assertEquals(topic.name(), alteredTopic.name()); + Assertions.assertEquals(topic.comment(), alteredTopic.comment()); + Map expectedProps = ImmutableMap.of("k2", "v2", "k3", "v3"); + testProperties(expectedProps, alteredTopic.properties()); + // Audit info is gotten from gravitino entity store + Assertions.assertEquals(AuthConstants.ANONYMOUS_USER, alteredTopic.auditInfo().creator()); + Assertions.assertEquals(AuthConstants.ANONYMOUS_USER, alteredTopic.auditInfo().lastModifier()); + + // Case 2: Test if the topic entity is not found in the entity store + reset(entityStore); + doThrow(new NoSuchEntityException("")).when(entityStore).update(any(), any(), any(), any()); + Topic alteredTopic2 = topicOperationDispatcher.alterTopic(topicIdent, changes); + // Audit info is gotten from the catalog, not from the entity store + Assertions.assertEquals("test", alteredTopic2.auditInfo().creator()); + Assertions.assertEquals("test", alteredTopic2.auditInfo().lastModifier()); + + // Case 3: Test if the entity store is failed to update the topic entity + reset(entityStore); + doThrow(new IOException()).when(entityStore).update(any(), any(), any(), any()); + Topic alteredTopic3 = topicOperationDispatcher.alterTopic(topicIdent, changes); + // Audit info is gotten from the catalog, not from the entity store + Assertions.assertEquals("test", alteredTopic3.auditInfo().creator()); + Assertions.assertEquals("test", alteredTopic3.auditInfo().lastModifier()); + + // Case 4: Test if the topic entity is not matched + reset(entityStore); + TopicEntity unmatchedEntity = + TopicEntity.builder() + .withId(1L) + .withName("topic21") + .withNamespace(topicNs) + .withAuditInfo( + AuditInfo.builder().withCreator("gravitino").withCreateTime(Instant.now()).build()) + .build(); + doReturn(unmatchedEntity).when(entityStore).update(any(), any(), any(), any()); + Topic alteredTopic4 = topicOperationDispatcher.alterTopic(topicIdent, changes); + // Audit info is gotten from the catalog, not from the entity store + Assertions.assertEquals("test", alteredTopic4.auditInfo().creator()); + Assertions.assertEquals("test", alteredTopic4.auditInfo().lastModifier()); + + // Test immutable topic properties + TopicChange[] illegalChange = + new TopicChange[] {TopicChange.setProperty(COMMENT_KEY, "new comment")}; + testPropertyException( + () -> topicOperationDispatcher.alterTopic(topicIdent, illegalChange), + "Property comment is immutable or reserved, cannot be set"); + } + + @Test + public void testCreateAndDropTopic() throws IOException { + Namespace topicNs = Namespace.of(metalake, catalog, "schema151"); + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + schemaOperationDispatcher.createSchema(NameIdentifier.of(topicNs.levels()), "comment", props); + + NameIdentifier topicIdent = NameIdentifier.of(topicNs, "topic31"); + Topic topic = topicOperationDispatcher.createTopic(topicIdent, "comment", null, props); + Assertions.assertEquals("topic31", topic.name()); + Assertions.assertEquals("comment", topic.comment()); + testProperties(props, topic.properties()); + + boolean dropped = topicOperationDispatcher.dropTopic(topicIdent); + Assertions.assertTrue(dropped); + + // Test if the entity store is failed to drop the topic entity + topicOperationDispatcher.createTopic(topicIdent, "comment", null, props); + reset(entityStore); + doThrow(new IOException()).when(entityStore).delete(any(), any(), anyBoolean()); + Assertions.assertThrows( + RuntimeException.class, () -> topicOperationDispatcher.dropTopic(topicIdent)); + } +} From db3633244bb6fc7dc7de12b59e0d0607318fedcd Mon Sep 17 00:00:00 2001 From: Qian Xia Date: Mon, 1 Apr 2024 12:00:18 +0800 Subject: [PATCH 08/20] [#2292]feat(fileset): add web ui support for fileset catalog (#2721) ### What changes were proposed in this pull request? Web ui support for fileset catalog Some try catch which helpful for debugging among extraReducers functions ### Why are the changes needed? image Fix: #2292 ### Does this PR introduce _any_ user-facing change? N/A ### How was this patch tested? run catalog e2e test successfully ![catalog_e2e_test](https://github.com/datastrato/gravitino/assets/9210625/6933cbc0-9ed2-44c4-ae09-a9dfcc05e7b2) --------- Co-authored-by: CHEYNE --- .../test/web/ui/CatalogsPageTest.java | 8 +- .../test/web/ui/pages/CatalogsPage.java | 24 +- .../app/metalakes/metalake/MetalakeTree.js | 22 +- .../app/metalakes/metalake/MetalakeView.js | 34 +- .../rightContent/CreateCatalogDialog.js | 31 +- .../metalake/rightContent/MetalakePath.js | 26 +- .../rightContent/tabsContent/TabsContent.js | 42 ++- .../tabsContent/tableView/TableView.js | 10 +- web/src/lib/api/catalogs/index.js | 2 +- web/src/lib/api/filesets/index.js | 24 ++ web/src/lib/store/metalakes/index.js | 314 ++++++++++++++---- web/src/lib/utils/initial.js | 5 + 12 files changed, 436 insertions(+), 106 deletions(-) create mode 100644 web/src/lib/api/filesets/index.js diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/web/ui/CatalogsPageTest.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/web/ui/CatalogsPageTest.java index 749c605c783..cb1a1e88210 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/web/ui/CatalogsPageTest.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/web/ui/CatalogsPageTest.java @@ -30,6 +30,7 @@ public class CatalogsPageTest extends AbstractWebIT { private static final String metalakeName = "metalake_name"; private static final String metalakeSelectName = "metalake_select_name"; String catalogName = "catalog_name"; + String catalogType = "relational"; String modifiedCatalogName = catalogName + "_edited"; String schemaName = "default"; String tableName = "employee"; @@ -131,21 +132,22 @@ public void testEditCatalog() throws InterruptedException { @Test @Order(6) public void testClickCatalogLink() { - catalogsPage.clickCatalogLink(metalakeName, modifiedCatalogName); + catalogsPage.clickCatalogLink(metalakeName, modifiedCatalogName, catalogType); Assertions.assertTrue(catalogsPage.verifyShowTableTitle("Schemas")); } @Test @Order(7) public void testClickSchemaLink() { - catalogsPage.clickSchemaLink(metalakeName, modifiedCatalogName, schemaName); + catalogsPage.clickSchemaLink(metalakeName, modifiedCatalogName, catalogType, schemaName); Assertions.assertTrue(catalogsPage.verifyShowTableTitle("Tables")); } @Test @Order(8) public void testClickTableLink() { - catalogsPage.clickTableLink(metalakeName, modifiedCatalogName, schemaName, tableName); + catalogsPage.clickTableLink( + metalakeName, modifiedCatalogName, catalogType, schemaName, tableName); Assertions.assertTrue(catalogsPage.verifyShowTableTitle("Columns")); Assertions.assertTrue(catalogsPage.verifyTableColumns()); } diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/web/ui/pages/CatalogsPage.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/web/ui/pages/CatalogsPage.java index 4df998edf22..1921f561a05 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/web/ui/pages/CatalogsPage.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/web/ui/pages/CatalogsPage.java @@ -188,9 +188,16 @@ public void clickDeleteCatalogBtn(String name) { } } - public void clickCatalogLink(String metalakeName, String catalogName) { + public void clickCatalogLink(String metalakeName, String catalogName, String catalogType) { try { - String xpath = "//a[@href='?metalake=" + metalakeName + "&catalog=" + catalogName + "']"; + String xpath = + "//a[@href='?metalake=" + + metalakeName + + "&catalog=" + + catalogName + + "&type=" + + catalogType + + "']"; WebElement link = tableGrid.findElement(By.xpath(xpath)); WebDriverWait wait = new WebDriverWait(driver, MAX_TIMEOUT); wait.until(ExpectedConditions.elementToBeClickable(By.xpath(xpath))); @@ -200,13 +207,16 @@ public void clickCatalogLink(String metalakeName, String catalogName) { } } - public void clickSchemaLink(String metalakeName, String catalogName, String schemaName) { + public void clickSchemaLink( + String metalakeName, String catalogName, String catalogType, String schemaName) { try { String xpath = "//a[@href='?metalake=" + metalakeName + "&catalog=" + catalogName + + "&type=" + + catalogType + "&schema=" + schemaName + "']"; @@ -220,13 +230,19 @@ public void clickSchemaLink(String metalakeName, String catalogName, String sche } public void clickTableLink( - String metalakeName, String catalogName, String schemaName, String tableName) { + String metalakeName, + String catalogName, + String catalogType, + String schemaName, + String tableName) { try { String xpath = "//a[@href='?metalake=" + metalakeName + "&catalog=" + catalogName + + "&type=" + + catalogType + "&schema=" + schemaName + "&table=" diff --git a/web/src/app/metalakes/metalake/MetalakeTree.js b/web/src/app/metalakes/metalake/MetalakeTree.js index f2642e9a8d1..4b807eaf15c 100644 --- a/web/src/app/metalakes/metalake/MetalakeTree.js +++ b/web/src/app/metalakes/metalake/MetalakeTree.js @@ -22,7 +22,8 @@ import { removeExpandedNode, setSelectedNodes, setLoadedNodes, - getTableDetails + getTableDetails, + getFilesetDetails } from '@/lib/store/metalakes' import { extractPlaceholder } from '@/lib/utils' @@ -47,6 +48,12 @@ const MetalakeTree = props => { const [metalake, catalog, schema, table] = pathArr dispatch(getTableDetails({ init: true, metalake, catalog, schema, table })) } + } else if (nodeProps.data.node === 'fileset') { + if (store.selectedNodes.includes(nodeProps.data.key)) { + const pathArr = extractPlaceholder(nodeProps.data.key) + const [metalake, catalog, schema, fileset] = pathArr + dispatch(getFilesetDetails({ init: true, metalake, catalog, schema, fileset })) + } } else { dispatch(setIntoTreeNodeWithFetch({ key: nodeProps.data.key })) } @@ -143,6 +150,19 @@ const MetalakeTree = props => { ) + case 'fileset': + return ( + handleClickIcon(e, nodeProps)} + onMouseEnter={e => onMouseEnter(e, nodeProps)} + onMouseLeave={e => onMouseLeave(e, nodeProps)} + > + + + ) default: return <> diff --git a/web/src/app/metalakes/metalake/MetalakeView.js b/web/src/app/metalakes/metalake/MetalakeView.js index 116c0428367..e700ff145fc 100644 --- a/web/src/app/metalakes/metalake/MetalakeView.js +++ b/web/src/app/metalakes/metalake/MetalakeView.js @@ -18,10 +18,12 @@ import { fetchCatalogs, fetchSchemas, fetchTables, + fetchFilesets, getMetalakeDetails, getCatalogDetails, getSchemaDetails, getTableDetails, + getFilesetDetails, setSelectedNodes } from '@/lib/store/metalakes' @@ -35,39 +37,51 @@ const MetalakeView = () => { const routeParams = { metalake: searchParams.get('metalake'), catalog: searchParams.get('catalog'), + type: searchParams.get('type'), schema: searchParams.get('schema'), - table: searchParams.get('table') + table: searchParams.get('table'), + fileset: searchParams.get('fileset') } if ([...searchParams.keys()].length) { - const { metalake, catalog, schema, table } = routeParams + const { metalake, catalog, type, schema, table, fileset } = routeParams if (paramsSize === 1 && metalake) { dispatch(fetchCatalogs({ init: true, page: 'metalakes', metalake })) dispatch(getMetalakeDetails({ metalake })) } - if (paramsSize === 2 && catalog) { - dispatch(fetchSchemas({ init: true, page: 'catalogs', metalake, catalog })) - dispatch(getCatalogDetails({ metalake, catalog })) + if (paramsSize === 3 && catalog) { + dispatch(fetchSchemas({ init: true, page: 'catalogs', metalake, catalog, type })) + dispatch(getCatalogDetails({ metalake, catalog, type })) } - if (paramsSize === 3 && catalog && schema) { - dispatch(fetchTables({ init: true, page: 'schemas', metalake, catalog, schema })) + if (paramsSize === 4 && catalog && type && schema) { + if (type === 'fileset') { + dispatch(fetchFilesets({ init: true, page: 'schemas', metalake, catalog, schema })) + } else { + dispatch(fetchTables({ init: true, page: 'schemas', metalake, catalog, schema })) + } dispatch(getSchemaDetails({ metalake, catalog, schema })) } - if (paramsSize === 4 && catalog && schema && table) { + if (paramsSize === 5 && catalog && schema && table) { dispatch(getTableDetails({ init: true, metalake, catalog, schema, table })) } + + if (paramsSize === 5 && catalog && schema && fileset) { + dispatch(getFilesetDetails({ init: true, metalake, catalog, schema, fileset })) + } } dispatch( setSelectedNodes( routeParams.catalog ? [ - `{{${routeParams.metalake}}}{{${routeParams.catalog}}}${ + `{{${routeParams.metalake}}}{{${routeParams.catalog}}}{{${routeParams.type}}}${ routeParams.schema ? `{{${routeParams.schema}}}` : '' - }${routeParams.table ? `{{${routeParams.table}}}` : ''}` + }${routeParams.table ? `{{${routeParams.table}}}` : ''}${ + routeParams.fileset ? `{{${routeParams.fileset}}}` : '' + }` ] : [] ) diff --git a/web/src/app/metalakes/metalake/rightContent/CreateCatalogDialog.js b/web/src/app/metalakes/metalake/rightContent/CreateCatalogDialog.js index 6580ce89aa3..63963eb1e0d 100644 --- a/web/src/app/metalakes/metalake/rightContent/CreateCatalogDialog.js +++ b/web/src/app/metalakes/metalake/rightContent/CreateCatalogDialog.js @@ -43,7 +43,7 @@ import { useSearchParams } from 'next/navigation' const defaultValues = { name: '', type: 'relational', - provider: 'hive', + provider: '', comment: '', propItems: providers[0].defaultProps } @@ -58,7 +58,7 @@ const schema = yup.object().shape({ nameRegex, 'This field must start with a letter or underscore, and can only contain letters, numbers, and underscores' ), - type: yup.mixed().oneOf(['relational']).required(), + type: yup.mixed().oneOf(['relational', 'fileset']).required(), provider: yup.mixed().oneOf(providerTypeValues).required(), propItems: yup.array().of( yup.object().shape({ @@ -87,6 +87,8 @@ const CreateCatalogDialog = props => { const [cacheData, setCacheData] = useState() + const [providerTypes, setProviderTypes] = useState(providers) + const { control, reset, @@ -103,6 +105,7 @@ const CreateCatalogDialog = props => { }) const providerSelect = watch('provider') + const typeSelect = watch('type') const handleFormChange = ({ index, event }) => { let data = [...innerProps] @@ -280,6 +283,18 @@ const CreateCatalogDialog = props => { console.error('fields error', errors) } + useEffect(() => { + if (typeSelect === 'fileset') { + setProviderTypes(providers.filter(p => p.value === 'hadoop')) + setValue('provider', 'hadoop') + } else { + setProviderTypes(providers.filter(p => p.value !== 'hadoop')) + setValue('provider', 'hive') + } + + // eslint-disable-next-line react-hooks/exhaustive-deps + }, [typeSelect, open]) + useEffect(() => { let defaultProps = [] @@ -410,6 +425,7 @@ const CreateCatalogDialog = props => { disabled={type === 'update'} > relational + fileset )} /> @@ -436,10 +452,13 @@ const CreateCatalogDialog = props => { labelId='select-catalog-provider' disabled={type === 'update'} > - hive - iceberg - mysql - postgresql + {providerTypes.map(item => { + return ( + + {item.label} + + ) + })} )} /> diff --git a/web/src/app/metalakes/metalake/rightContent/MetalakePath.js b/web/src/app/metalakes/metalake/rightContent/MetalakePath.js index 58d6e3a1f17..4f7a400535a 100644 --- a/web/src/app/metalakes/metalake/rightContent/MetalakePath.js +++ b/web/src/app/metalakes/metalake/rightContent/MetalakePath.js @@ -24,16 +24,19 @@ const MetalakePath = props => { const routeParams = { metalake: searchParams.get('metalake'), catalog: searchParams.get('catalog'), + type: searchParams.get('type'), schema: searchParams.get('schema'), - table: searchParams.get('table') + table: searchParams.get('table'), + fileset: searchParams.get('fileset') } - const { metalake, catalog, schema, table } = routeParams + const { metalake, catalog, type, schema, table, fileset } = routeParams const metalakeUrl = `?metalake=${metalake}` - const catalogUrl = `?metalake=${metalake}&catalog=${catalog}` - const schemaUrl = `?metalake=${metalake}&catalog=${catalog}&schema=${schema}` - const tableUrl = `?metalake=${metalake}&catalog=${catalog}&schema=${schema}&table=${table}` + const catalogUrl = `?metalake=${metalake}&catalog=${catalog}&type=${type}` + const schemaUrl = `?metalake=${metalake}&catalog=${catalog}&type=${type}&schema=${schema}` + const tableUrl = `?metalake=${metalake}&catalog=${catalog}&type=${type}&schema=${schema}&table=${table}` + const filesetUrl = `?metalake=${metalake}&catalog=${catalog}&type=${type}&schema=${schema}&fileset=${fileset}` const handleClick = (event, path) => { path === `?${searchParams.toString()}` && event.preventDefault() @@ -91,6 +94,19 @@ const MetalakePath = props => { )} + {fileset && ( + + handleClick(event, filesetUrl)} + underline='hover' + > + + {fileset} + + + )} ) } diff --git a/web/src/app/metalakes/metalake/rightContent/tabsContent/TabsContent.js b/web/src/app/metalakes/metalake/rightContent/tabsContent/TabsContent.js index 00ae259f367..7ae1a86543b 100644 --- a/web/src/app/metalakes/metalake/rightContent/tabsContent/TabsContent.js +++ b/web/src/app/metalakes/metalake/rightContent/tabsContent/TabsContent.js @@ -5,7 +5,7 @@ 'use client' -import { useState } from 'react' +import { useState, useEffect } from 'react' import Box from '@mui/material/Box' import Tab from '@mui/material/Tab' @@ -49,44 +49,60 @@ const CustomTabPanel = props => { } const TabsContent = () => { + let tableTitle = '' + const searchParams = useSearchParams() + const paramsSize = [...searchParams.keys()].length + const type = searchParams.get('type') const [tab, setTab] = useState('table') + const isNotNeedTableTab = type && type === 'fileset' && paramsSize === 5 const handleChangeTab = (event, newValue) => { setTab(newValue) } - let tableTitle = '' - const searchParams = useSearchParams() - const paramsSize = [...searchParams.keys()].length - switch (paramsSize) { case 1: tableTitle = 'Catalogs' break - case 2: - tableTitle = 'Schemas' - break case 3: - tableTitle = 'Tables' + tableTitle = 'Schemas' break case 4: + tableTitle = type === 'fileset' ? 'Filesets' : 'Tables' + break + case 5: tableTitle = 'Columns' break default: break } + useEffect(() => { + if (isNotNeedTableTab) { + setTab('details') + } else { + setTab('table') + } + + // eslint-disable-next-line react-hooks/exhaustive-deps + }, [searchParams]) + return ( - + {!isNotNeedTableTab ? ( + + ) : null} - - - + {!isNotNeedTableTab ? ( + + + + ) : null} + diff --git a/web/src/app/metalakes/metalake/rightContent/tabsContent/tableView/TableView.js b/web/src/app/metalakes/metalake/rightContent/tabsContent/tableView/TableView.js index 6ec15defc54..4c0ee996f5b 100644 --- a/web/src/app/metalakes/metalake/rightContent/tabsContent/tableView/TableView.js +++ b/web/src/app/metalakes/metalake/rightContent/tabsContent/tableView/TableView.js @@ -170,7 +170,7 @@ const TableView = () => { title='Delete' size='small' sx={{ color: theme => theme.palette.error.light }} - onClick={() => handleDelete({ name: row.name, type: 'catalog' })} + onClick={() => handleDelete({ name: row.name, type: 'catalog', catalogType: row.type })} data-refer={`delete-catalog-${row.name}`} > @@ -345,9 +345,9 @@ const TableView = () => { } } - const handleDelete = ({ name, type }) => { + const handleDelete = ({ name, type, catalogType }) => { setOpenConfirmDelete(true) - setConfirmCacheData({ name, type }) + setConfirmCacheData({ name, type, catalogType }) } const handleCloseConfirm = () => { @@ -358,7 +358,7 @@ const TableView = () => { const handleConfirmDeleteSubmit = () => { if (confirmCacheData) { if (confirmCacheData.type === 'catalog') { - dispatch(deleteCatalog({ metalake, catalog: confirmCacheData.name })) + dispatch(deleteCatalog({ metalake, catalog: confirmCacheData.name, type: confirmCacheData.catalogType })) } setOpenConfirmDelete(false) @@ -368,7 +368,7 @@ const TableView = () => { const checkColumns = () => { if (paramsSize == 1 && searchParams.has('metalake')) { return catalogsColumns - } else if (paramsSize == 4 && searchParams.has('table')) { + } else if (paramsSize == 5 && searchParams.has('table')) { return tableColumns } else { return columns diff --git a/web/src/lib/api/catalogs/index.js b/web/src/lib/api/catalogs/index.js index 45c1c6eb3fe..b8acee1fdd0 100644 --- a/web/src/lib/api/catalogs/index.js +++ b/web/src/lib/api/catalogs/index.js @@ -6,7 +6,7 @@ import { defHttp } from '@/lib/utils/axios' const Apis = { - GET: ({ metalake }) => `/api/metalakes/${metalake}/catalogs`, + GET: ({ metalake }) => `/api/metalakes/${metalake}/catalogs?details=true`, GET_DETAIL: ({ metalake, catalog }) => `/api/metalakes/${metalake}/catalogs/${catalog}`, CREATE: ({ metalake }) => `/api/metalakes/${metalake}/catalogs`, UPDATE: ({ metalake, catalog }) => `/api/metalakes/${metalake}/catalogs/${catalog}`, diff --git a/web/src/lib/api/filesets/index.js b/web/src/lib/api/filesets/index.js new file mode 100644 index 00000000000..a2e03b970d8 --- /dev/null +++ b/web/src/lib/api/filesets/index.js @@ -0,0 +1,24 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ + +import { defHttp } from '@/lib/utils/axios' + +const Apis = { + GET: ({ metalake, catalog, schema }) => `/api/metalakes/${metalake}/catalogs/${catalog}/schemas/${schema}/filesets`, + GET_DETAIL: ({ metalake, catalog, schema, fileset }) => + `/api/metalakes/${metalake}/catalogs/${catalog}/schemas/${schema}/filesets/${fileset}` +} + +export const getFilesetsApi = params => { + return defHttp.get({ + url: `${Apis.GET(params)}` + }) +} + +export const getFilesetDetailsApi = ({ metalake, catalog, schema, fileset }) => { + return defHttp.get({ + url: `${Apis.GET_DETAIL({ metalake, catalog, schema, fileset })}` + }) +} diff --git a/web/src/lib/store/metalakes/index.js b/web/src/lib/store/metalakes/index.js index 3e03140e38b..37a99f697ec 100644 --- a/web/src/lib/store/metalakes/index.js +++ b/web/src/lib/store/metalakes/index.js @@ -6,6 +6,7 @@ import { createSlice, createAsyncThunk } from '@reduxjs/toolkit' import { to, extractPlaceholder, updateTreeData, findInTree } from '@/lib/utils' +import toast from 'react-hot-toast' import _ from 'lodash-es' @@ -26,6 +27,7 @@ import { } from '@/lib/api/catalogs' import { getSchemasApi, getSchemaDetailsApi } from '@/lib/api/schemas' import { getTablesApi, getTableDetailsApi } from '@/lib/api/tables' +import { getFilesetsApi, getFilesetDetailsApi } from '@/lib/api/filesets' export const fetchMetalakes = createAsyncThunk('appMetalakes/fetchMetalakes', async (params, { getState }) => { const [err, res] = await to(getMetalakesApi()) @@ -85,7 +87,7 @@ export const setIntoTreeNodeWithFetch = createAsyncThunk( } const pathArr = extractPlaceholder(key) - const [metalake, catalog, schema, table] = pathArr + const [metalake, catalog, type, schema] = pathArr if (pathArr.length === 1) { const [err, res] = await to(getCatalogsApi({ metalake })) @@ -94,22 +96,23 @@ export const setIntoTreeNodeWithFetch = createAsyncThunk( throw new Error(err) } - const { identifiers = [] } = res + const { catalogs = [] } = res - result.data = identifiers.map(catalogItem => { + result.data = catalogs.map(catalogItem => { return { ...catalogItem, node: 'catalog', - id: `{{${metalake}}}{{${catalogItem.name}}}`, - key: `{{${metalake}}}{{${catalogItem.name}}}`, - path: `?${new URLSearchParams({ metalake, catalog: catalogItem.name }).toString()}`, + id: `{{${metalake}}}{{${catalogItem.name}}}{{${catalogItem.type}}}`, + key: `{{${metalake}}}{{${catalogItem.name}}}{{${catalogItem.type}}}`, + path: `?${new URLSearchParams({ metalake, catalog: catalogItem.name, type: catalogItem.type }).toString()}`, name: catalogItem.name, title: catalogItem.name, + namespace: [metalake], schemas: [], children: [] } }) - } else if (pathArr.length === 2) { + } else if (pathArr.length === 3) { const [err, res] = await to(getSchemasApi({ metalake, catalog })) if (err || !res) { @@ -122,16 +125,16 @@ export const setIntoTreeNodeWithFetch = createAsyncThunk( return { ...schemaItem, node: 'schema', - id: `{{${metalake}}}{{${catalog}}}{{${schemaItem.name}}}`, - key: `{{${metalake}}}{{${catalog}}}{{${schemaItem.name}}}`, - path: `?${new URLSearchParams({ metalake, catalog, schema: schemaItem.name }).toString()}`, + id: `{{${metalake}}}{{${catalog}}}{{${type}}}{{${schemaItem.name}}}`, + key: `{{${metalake}}}{{${catalog}}}{{${type}}}{{${schemaItem.name}}}`, + path: `?${new URLSearchParams({ metalake, catalog, type, schema: schemaItem.name }).toString()}`, name: schemaItem.name, title: schemaItem.name, tables: [], children: [] } }) - } else if (pathArr.length === 3) { + } else if (pathArr.length === 4 && type !== 'fileset') { const [err, res] = await to(getTablesApi({ metalake, catalog, schema })) const { identifiers = [] } = res @@ -144,9 +147,9 @@ export const setIntoTreeNodeWithFetch = createAsyncThunk( return { ...tableItem, node: 'table', - id: `{{${metalake}}}{{${catalog}}}{{${schema}}}{{${tableItem.name}}}`, - key: `{{${metalake}}}{{${catalog}}}{{${schema}}}{{${tableItem.name}}}`, - path: `?${new URLSearchParams({ metalake, catalog, schema, table: tableItem.name }).toString()}`, + id: `{{${metalake}}}{{${catalog}}}{{${type}}}{{${schema}}}{{${tableItem.name}}}`, + key: `{{${metalake}}}{{${catalog}}}{{${type}}}{{${schema}}}{{${tableItem.name}}}`, + path: `?${new URLSearchParams({ metalake, catalog, type, schema, table: tableItem.name }).toString()}`, name: tableItem.name, title: tableItem.name, isLeaf: true, @@ -154,6 +157,27 @@ export const setIntoTreeNodeWithFetch = createAsyncThunk( children: [] } }) + } else if (pathArr.length === 4 && type === 'fileset') { + const [err, res] = await to(getFilesetsApi({ metalake, catalog, schema })) + + const { identifiers = [] } = res + + if (err || !res) { + throw new Error(err) + } + + result.data = identifiers.map(filesetItem => { + return { + ...filesetItem, + node: 'fileset', + id: `{{${metalake}}}{{${catalog}}}{{${type}}}{{${schema}}}{{${filesetItem.name}}}`, + key: `{{${metalake}}}{{${catalog}}}{{${type}}}{{${schema}}}{{${filesetItem.name}}}`, + path: `?${new URLSearchParams({ metalake, catalog, type, schema, fileset: filesetItem.name }).toString()}`, + name: filesetItem.name, + title: filesetItem.name, + isLeaf: true + } + }) } return result @@ -188,17 +212,18 @@ export const fetchCatalogs = createAsyncThunk( throw new Error(err) } - const { identifiers = [] } = res + const { catalogs = [] } = res - const catalogs = identifiers.map(catalog => { + const catalogsData = catalogs.map(catalog => { return { ...catalog, node: 'catalog', - id: `{{${metalake}}}{{${catalog.name}}}`, - key: `{{${metalake}}}{{${catalog.name}}}`, - path: `?${new URLSearchParams({ metalake, catalog: catalog.name }).toString()}`, + id: `{{${metalake}}}{{${catalog.name}}}{{${catalog.type}}}`, + key: `{{${metalake}}}{{${catalog.name}}}{{${catalog.type}}}`, + path: `?${new URLSearchParams({ metalake, catalog: catalog.name, type: catalog.type }).toString()}`, name: catalog.name, title: catalog.name, + namespace: [metalake], schemas: [], children: [] } @@ -215,11 +240,12 @@ export const fetchCatalogs = createAsyncThunk( ? schema.children.map(table => { return { ...table, - id: `{{${metalake}}}{{${update.newCatalog.name}}}{{${schema.name}}}{{${table.name}}}`, - key: `{{${metalake}}}{{${update.newCatalog.name}}}{{${schema.name}}}{{${table.name}}}`, + id: `{{${metalake}}}{{${update.newCatalog.name}}}{{${update.newCatalog.type}}}{{${schema.name}}}{{${table.name}}}`, + key: `{{${metalake}}}{{${update.newCatalog.name}}}{{${update.newCatalog.type}}}{{${schema.name}}}{{${table.name}}}`, path: `?${new URLSearchParams({ metalake, catalog: update.newCatalog.name, + type: update.newCatalog.type, schema: schema.name, table: table.name }).toString()}` @@ -229,11 +255,12 @@ export const fetchCatalogs = createAsyncThunk( return { ...schema, - id: `{{${metalake}}}{{${update.newCatalog.name}}}{{${schema.name}}}`, - key: `{{${metalake}}}{{${update.newCatalog.name}}}{{${schema.name}}}`, + id: `{{${metalake}}}{{${update.newCatalog.name}}}{{${update.newCatalog.type}}}{{${schema.name}}}`, + key: `{{${metalake}}}{{${update.newCatalog.name}}}{{${update.newCatalog.type}}}{{${schema.name}}}`, path: `?${new URLSearchParams({ metalake, catalog: update.newCatalog.name, + type: update.newCatalog.type, schema: schema.name }).toString()}`, tables: tables, @@ -244,9 +271,13 @@ export const fetchCatalogs = createAsyncThunk( return { ...catalog, - id: `{{${metalake}}}{{${update.newCatalog.name}}}`, - key: `{{${metalake}}}{{${update.newCatalog.name}}}`, - path: `?${new URLSearchParams({ metalake, catalog: update.newCatalog.name }).toString()}`, + id: `{{${metalake}}}{{${update.newCatalog.name}}}{{${update.newCatalog.type}}}`, + key: `{{${metalake}}}{{${update.newCatalog.name}}}{{${update.newCatalog.type}}}`, + path: `?${new URLSearchParams({ + metalake, + catalog: update.newCatalog.name, + type: update.newCatalog.type + }).toString()}`, name: update.newCatalog.name, title: update.newCatalog.name, schemas: schemas, @@ -262,9 +293,9 @@ export const fetchCatalogs = createAsyncThunk( const expandedNodes = getState().metalakes.expandedNodes.map(node => { const [metalake, catalog, schema, table] = extractPlaceholder(node) if (catalog === update.catalog) { - const updatedNode = `{{${metalake}}}{{${update.newCatalog.name}}}${schema ? `{{${schema}}}` : ''}${ - table ? `{{${table}}}` : '' - }` + const updatedNode = `{{${metalake}}}{{${update.newCatalog.name}}}{{${update.newCatalog.type}}}${ + schema ? `{{${schema}}}` : '' + }${table ? `{{${table}}}` : ''}` return updatedNode } @@ -278,9 +309,9 @@ export const fetchCatalogs = createAsyncThunk( const loadedNodes = getState().metalakes.loadedNodes.map(node => { const [metalake, catalog, schema, table] = extractPlaceholder(node) if (catalog === update.catalog) { - const updatedNode = `{{${metalake}}}{{${update.newCatalog.name}}}${schema ? `{{${schema}}}` : ''}${ - table ? `{{${table}}}` : '' - }` + const updatedNode = `{{${metalake}}}{{${update.newCatalog.name}}}{{${update.newCatalog.type}}}${ + schema ? `{{${schema}}}` : '' + }${table ? `{{${table}}}` : ''}` return updatedNode } @@ -293,14 +324,14 @@ export const fetchCatalogs = createAsyncThunk( } } else { const mergedTree = _.values( - _.merge(_.keyBy(getState().metalakes.metalakeTree, 'key'), _.keyBy(catalogs, 'key')) + _.merge(_.keyBy(catalogsData, 'key'), _.keyBy(getState().metalakes.metalakeTree, 'key')) ) dispatch(setMetalakeTree(mergedTree)) } } return { - catalogs, + catalogs: catalogsData, page, init } @@ -335,16 +366,17 @@ export const createCatalog = createAsyncThunk( const catalogData = { ...catalogItem, node: 'catalog', - id: `{{${metalake}}}{{${catalogItem.name}}}`, - key: `{{${metalake}}}{{${catalogItem.name}}}`, - path: `?${new URLSearchParams({ metalake, catalog: catalogItem.name }).toString()}`, + id: `{{${metalake}}}{{${catalogItem.name}}}{{${catalogItem.type}}}`, + key: `{{${metalake}}}{{${catalogItem.name}}}{{${catalogItem.type}}}`, + path: `?${new URLSearchParams({ metalake, catalog: catalogItem.name, type: catalogItem.type }).toString()}`, name: catalogItem.name, title: catalogItem.name, + namespace: [metalake], schemas: [], children: [] } - dispatch(dispatch(fetchCatalogs({ metalake, init: true }))) + dispatch(fetchCatalogs({ metalake, init: true })) dispatch(addCatalogToTree(catalogData)) @@ -367,7 +399,7 @@ export const updateCatalog = createAsyncThunk( export const deleteCatalog = createAsyncThunk( 'appMetalakes/deleteCatalog', - async ({ metalake, catalog }, { dispatch }) => { + async ({ metalake, catalog, type }, { dispatch }) => { dispatch(setTableLoading(true)) const [err, res] = await to(deleteCatalogApi({ metalake, catalog })) dispatch(setTableLoading(false)) @@ -378,7 +410,7 @@ export const deleteCatalog = createAsyncThunk( dispatch(fetchCatalogs({ metalake, catalog, page: 'metalakes', init: true })) - dispatch(removeCatalogFromTree(`{{${metalake}}}{{${catalog}}}`)) + dispatch(removeCatalogFromTree(`{{${metalake}}}{{${catalog}}}{{${type}}}`)) return res } @@ -386,7 +418,7 @@ export const deleteCatalog = createAsyncThunk( export const fetchSchemas = createAsyncThunk( 'appMetalakes/fetchSchemas', - async ({ init, page, metalake, catalog }, { getState, dispatch }) => { + async ({ init, page, metalake, catalog, type }, { getState, dispatch }) => { if (init) { dispatch(setTableLoading(true)) } @@ -404,15 +436,15 @@ export const fetchSchemas = createAsyncThunk( const schemaItem = findInTree( getState().metalakes.metalakeTree, 'key', - `{{${metalake}}}{{${catalog}}}{{${schema.name}}}` + `{{${metalake}}}{{${catalog}}}{{${type}}}{{${schema.name}}}` ) return { ...schema, node: 'schema', - id: `{{${metalake}}}{{${catalog}}}{{${schema.name}}}`, - key: `{{${metalake}}}{{${catalog}}}{{${schema.name}}}`, - path: `?${new URLSearchParams({ metalake, catalog, schema: schema.name }).toString()}`, + id: `{{${metalake}}}{{${catalog}}}{{${type}}}{{${schema.name}}}`, + key: `{{${metalake}}}{{${catalog}}}{{${type}}}{{${schema.name}}}`, + path: `?${new URLSearchParams({ metalake, catalog, type, schema: schema.name }).toString()}`, name: schema.name, title: schema.name, tables: schemaItem ? schemaItem.children : [], @@ -420,10 +452,10 @@ export const fetchSchemas = createAsyncThunk( } }) - if (init && getState().metalakes.loadedNodes.includes(`{{${metalake}}}{{${catalog}}}`)) { + if (init && getState().metalakes.loadedNodes.includes(`{{${metalake}}}{{${catalog}}}{{${type}}}`)) { dispatch( setIntoTreeNodes({ - key: `{{${metalake}}}{{${catalog}}}`, + key: `{{${metalake}}}{{${catalog}}}{{${type}}}`, data: schemas, tree: getState().metalakes.metalakeTree }) @@ -434,7 +466,7 @@ export const fetchSchemas = createAsyncThunk( dispatch(fetchCatalogs({ metalake })) } - dispatch(setExpandedNodes([`{{${metalake}}}`, `{{${metalake}}}{{${catalog}}}`])) + dispatch(setExpandedNodes([`{{${metalake}}}`, `{{${metalake}}}{{${catalog}}}{{${type}}}`])) return { schemas, page, init } } @@ -475,9 +507,15 @@ export const fetchTables = createAsyncThunk( return { ...table, node: 'table', - id: `{{${metalake}}}{{${catalog}}}{{${schema}}}{{${table.name}}}`, - key: `{{${metalake}}}{{${catalog}}}{{${schema}}}{{${table.name}}}`, - path: `?${new URLSearchParams({ metalake, catalog, schema, table: table.name }).toString()}`, + id: `{{${metalake}}}{{${catalog}}}{{${'relational'}}}{{${schema}}}{{${table.name}}}`, + key: `{{${metalake}}}{{${catalog}}}{{${'relational'}}}{{${schema}}}{{${table.name}}}`, + path: `?${new URLSearchParams({ + metalake, + catalog, + type: 'relational', + schema, + table: table.name + }).toString()}`, name: table.name, title: table.name, isLeaf: true, @@ -486,10 +524,13 @@ export const fetchTables = createAsyncThunk( } }) - if (init && getState().metalakes.loadedNodes.includes(`{{${metalake}}}{{${catalog}}}{{${schema}}}`)) { + if ( + init && + getState().metalakes.loadedNodes.includes(`{{${metalake}}}{{${catalog}}}{{${'relational'}}}{{${schema}}}`) + ) { dispatch( setIntoTreeNodes({ - key: `{{${metalake}}}{{${catalog}}}{{${schema}}}`, + key: `{{${metalake}}}{{${catalog}}}{{${'relational'}}}{{${schema}}}`, data: tables, tree: getState().metalakes.metalakeTree }) @@ -503,8 +544,8 @@ export const fetchTables = createAsyncThunk( dispatch( setExpandedNodes([ `{{${metalake}}}`, - `{{${metalake}}}{{${catalog}}}`, - `{{${metalake}}}{{${catalog}}}{{${schema}}}` + `{{${metalake}}}{{${catalog}}}{{${'relational'}}}`, + `{{${metalake}}}{{${catalog}}}{{${'relational'}}}{{${schema}}}` ]) ) @@ -536,8 +577,8 @@ export const getTableDetails = createAsyncThunk( dispatch( setExpandedNodes([ `{{${metalake}}}`, - `{{${metalake}}}{{${catalog}}}`, - `{{${metalake}}}{{${catalog}}}{{${schema}}}` + `{{${metalake}}}{{${catalog}}}{{${'relational'}}}`, + `{{${metalake}}}{{${catalog}}}{{${'relational'}}}{{${schema}}}` ]) ) @@ -545,6 +586,104 @@ export const getTableDetails = createAsyncThunk( } ) +export const fetchFilesets = createAsyncThunk( + 'appMetalakes/fetchFilesets', + async ({ init, page, metalake, catalog, schema }, { getState, dispatch }) => { + if (init) { + dispatch(setTableLoading(true)) + } + + const [err, res] = await to(getFilesetsApi({ metalake, catalog, schema })) + dispatch(setTableLoading(false)) + + if (init && (err || !res)) { + dispatch(resetTableData()) + throw new Error(err) + } + + const { identifiers = [] } = res + + const filesets = identifiers.map(fileset => { + return { + ...fileset, + node: 'fileset', + id: `{{${metalake}}}{{${catalog}}}{{${'fileset'}}}{{${schema}}}{{${fileset.name}}}`, + key: `{{${metalake}}}{{${catalog}}}{{${'fileset'}}}{{${schema}}}{{${fileset.name}}}`, + path: `?${new URLSearchParams({ + metalake, + catalog, + type: 'fileset', + schema, + fileset: fileset.name + }).toString()}`, + name: fileset.name, + title: fileset.name, + isLeaf: true + } + }) + + if ( + init && + getState().metalakes.loadedNodes.includes(`{{${metalake}}}{{${catalog}}}{{${'fileset'}}}{{${schema}}}`) + ) { + dispatch( + setIntoTreeNodes({ + key: `{{${metalake}}}{{${catalog}}}{{${'fileset'}}}{{${schema}}}`, + data: filesets, + tree: getState().metalakes.metalakeTree + }) + ) + } + + if (getState().metalakes.metalakeTree.length === 0) { + dispatch(fetchCatalogs({ metalake })) + } + + dispatch( + setExpandedNodes([ + `{{${metalake}}}`, + `{{${metalake}}}{{${catalog}}}{{${'fileset'}}}`, + `{{${metalake}}}{{${catalog}}}{{${'fileset'}}}{{${schema}}}` + ]) + ) + + return { filesets, page, init } + } +) + +export const getFilesetDetails = createAsyncThunk( + 'appMetalakes/getFilesetDetails', + async ({ init, metalake, catalog, schema, fileset }, { getState, dispatch }) => { + dispatch(resetTableData()) + if (init) { + dispatch(setTableLoading(true)) + } + const [err, res] = await to(getFilesetDetailsApi({ metalake, catalog, schema, fileset })) + dispatch(setTableLoading(false)) + + if (err || !res) { + dispatch(resetTableData()) + throw new Error(err) + } + + const { fileset: resFileset } = res + + if (getState().metalakes.metalakeTree.length === 0) { + dispatch(fetchCatalogs({ metalake })) + } + + dispatch( + setExpandedNodes([ + `{{${metalake}}}`, + `{{${metalake}}}{{${catalog}}}{{${'fileset'}}}`, + `{{${metalake}}}{{${catalog}}}{{${'fileset'}}}{{${schema}}}` + ]) + ) + + return resFileset + } +) + export const appMetalakesSlice = createSlice({ name: 'appMetalakes', initialState: { @@ -555,6 +694,7 @@ export const appMetalakesSlice = createSlice({ schemas: [], tables: [], columns: [], + filesets: [], metalakeTree: [], loadedNodes: [], selectedNodes: [], @@ -605,6 +745,7 @@ export const appMetalakesSlice = createSlice({ state.schemas = [] state.tables = [] state.columns = [] + state.filesets = [] }, setTableLoading(state, action) { state.tableLoading = action.payload @@ -617,7 +758,12 @@ export const appMetalakesSlice = createSlice({ state.metalakeTree = updateTreeData(tree, key, data) }, addCatalogToTree(state, action) { - state.metalakeTree.push(action.payload) + const catalogIndex = state.metalakeTree.findIndex(c => c.key === action.payload.key) + if (catalogIndex === -1) { + state.metalakeTree.push(action.payload) + } else { + state.metalakeTree.splice(catalogIndex, 1, action.payload) + } }, removeCatalogFromTree(state, action) { state.metalakeTree = state.metalakeTree.filter(i => i.key !== action.payload) @@ -628,13 +774,28 @@ export const appMetalakesSlice = createSlice({ builder.addCase(fetchMetalakes.fulfilled, (state, action) => { state.metalakes = action.payload.metalakes }) + builder.addCase(fetchMetalakes.rejected, (state, action) => { + toast.error(action.error.message) + }) builder.addCase(setIntoTreeNodeWithFetch.fulfilled, (state, action) => { const { key, data, tree } = action.payload state.metalakeTree = updateTreeData(tree, key, data) }) + builder.addCase(setIntoTreeNodeWithFetch.rejected, (state, action) => { + toast.error(action.error.message) + }) builder.addCase(getMetalakeDetails.fulfilled, (state, action) => { state.activatedDetails = action.payload }) + builder.addCase(getMetalakeDetails.rejected, (state, action) => { + toast.error(action.error.message) + }) + builder.addCase(createCatalog.rejected, (state, action) => { + toast.error(action.error.message) + }) + builder.addCase(updateCatalog.rejected, (state, action) => { + toast.error(action.error.message) + }) builder.addCase(fetchCatalogs.fulfilled, (state, action) => { state.catalogs = action.payload.catalogs @@ -646,28 +807,65 @@ export const appMetalakesSlice = createSlice({ state.metalakeTree = action.payload.catalogs } }) + builder.addCase(fetchCatalogs.rejected, (state, action) => { + toast.error(action.error.message) + }) builder.addCase(getCatalogDetails.fulfilled, (state, action) => { state.activatedDetails = action.payload }) + builder.addCase(getCatalogDetails.rejected, (state, action) => { + toast.error(action.error.message) + }) + builder.addCase(deleteCatalog.rejected, (state, action) => { + toast.error(action.error.message) + }) builder.addCase(fetchSchemas.fulfilled, (state, action) => { state.schemas = action.payload.schemas if (action.payload.init) { state.tableData = action.payload.schemas } }) + builder.addCase(fetchSchemas.rejected, (state, action) => { + toast.error(action.error.message) + }) builder.addCase(getSchemaDetails.fulfilled, (state, action) => { state.activatedDetails = action.payload }) + builder.addCase(getSchemaDetails.rejected, (state, action) => { + toast.error(action.error.message) + }) builder.addCase(fetchTables.fulfilled, (state, action) => { state.tables = action.payload.tables if (action.payload.init) { state.tableData = action.payload.tables } }) + builder.addCase(fetchTables.rejected, (state, action) => { + toast.error(action.error.message) + }) builder.addCase(getTableDetails.fulfilled, (state, action) => { state.activatedDetails = action.payload state.tableData = action.payload.columns || [] }) + builder.addCase(getTableDetails.rejected, (state, action) => { + toast.error(action.error.message) + }) + builder.addCase(fetchFilesets.fulfilled, (state, action) => { + state.filesets = action.payload.filesets + if (action.payload.init) { + state.tableData = action.payload.filesets + } + }) + builder.addCase(fetchFilesets.rejected, (state, action) => { + toast.error(action.error.message) + }) + builder.addCase(getFilesetDetails.fulfilled, (state, action) => { + state.activatedDetails = action.payload + state.tableData = [] + }) + builder.addCase(getFilesetDetails.rejected, (state, action) => { + toast.error(action.error.message) + }) } }) diff --git a/web/src/lib/utils/initial.js b/web/src/lib/utils/initial.js index 2eaa4d9ac76..476879b1e4b 100644 --- a/web/src/lib/utils/initial.js +++ b/web/src/lib/utils/initial.js @@ -4,6 +4,11 @@ */ export const providers = [ + { + label: 'hadoop', + value: 'hadoop', + defaultProps: [] + }, { label: 'hive', value: 'hive', From fd276b4dc0dbd4516632f62bff147befae21db6f Mon Sep 17 00:00:00 2001 From: Shaofeng Shi Date: Mon, 1 Apr 2024 14:10:54 +0800 Subject: [PATCH 09/20] [#2737] fix(catalog-jdbc-mysql): MySqlColumnDefaultValueConverter DATE conversion failed. (#2743) ### What changes were proposed in this pull request? Fix a bug for a Date string as column default value in MySqlColumnDefaultValueConverter ### Why are the changes needed? Use date pattern for such a date type value. Fix: #2737 ### Does this PR introduce any user-facing change? No ### How was this patch tested? UT covered, which is added by charliecheng630 --- .../jdbc/converter/JdbcColumnDefaultValueConverter.java | 2 ++ .../mysql/converter/MysqlColumnDefaultValueConverter.java | 2 +- .../catalog/mysql/integration/test/CatalogMysqlIT.java | 5 +++++ 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/catalogs/catalog-jdbc-common/src/main/java/com/datastrato/gravitino/catalog/jdbc/converter/JdbcColumnDefaultValueConverter.java b/catalogs/catalog-jdbc-common/src/main/java/com/datastrato/gravitino/catalog/jdbc/converter/JdbcColumnDefaultValueConverter.java index 2c693949f65..4c879535629 100644 --- a/catalogs/catalog-jdbc-common/src/main/java/com/datastrato/gravitino/catalog/jdbc/converter/JdbcColumnDefaultValueConverter.java +++ b/catalogs/catalog-jdbc-common/src/main/java/com/datastrato/gravitino/catalog/jdbc/converter/JdbcColumnDefaultValueConverter.java @@ -19,6 +19,8 @@ public class JdbcColumnDefaultValueConverter { protected static final String NULL = "NULL"; protected static final DateTimeFormatter DATE_TIME_FORMATTER = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); + protected static final DateTimeFormatter DATE_FORMATTER = + DateTimeFormatter.ofPattern("yyyy-MM-dd"); public String fromGravitino(Expression defaultValue) { if (DEFAULT_VALUE_NOT_SET.equals(defaultValue)) { diff --git a/catalogs/catalog-jdbc-mysql/src/main/java/com/datastrato/gravitino/catalog/mysql/converter/MysqlColumnDefaultValueConverter.java b/catalogs/catalog-jdbc-mysql/src/main/java/com/datastrato/gravitino/catalog/mysql/converter/MysqlColumnDefaultValueConverter.java index 21adafc0821..b34687d095c 100644 --- a/catalogs/catalog-jdbc-mysql/src/main/java/com/datastrato/gravitino/catalog/mysql/converter/MysqlColumnDefaultValueConverter.java +++ b/catalogs/catalog-jdbc-mysql/src/main/java/com/datastrato/gravitino/catalog/mysql/converter/MysqlColumnDefaultValueConverter.java @@ -77,7 +77,7 @@ public Expression toGravitino( Integer.parseInt(type.getColumnSize()), Integer.parseInt(type.getScale()))); case DATE: - return Literals.dateLiteral(LocalDate.parse(columnDefaultValue, DATE_TIME_FORMATTER)); + return Literals.dateLiteral(LocalDate.parse(columnDefaultValue, DATE_FORMATTER)); case TIME: return Literals.timeLiteral(LocalTime.parse(columnDefaultValue, DATE_TIME_FORMATTER)); case TIMESTAMP: diff --git a/catalogs/catalog-jdbc-mysql/src/test/java/com/datastrato/gravitino/catalog/mysql/integration/test/CatalogMysqlIT.java b/catalogs/catalog-jdbc-mysql/src/test/java/com/datastrato/gravitino/catalog/mysql/integration/test/CatalogMysqlIT.java index d4184fbb64e..b07600f6ffb 100644 --- a/catalogs/catalog-jdbc-mysql/src/test/java/com/datastrato/gravitino/catalog/mysql/integration/test/CatalogMysqlIT.java +++ b/catalogs/catalog-jdbc-mysql/src/test/java/com/datastrato/gravitino/catalog/mysql/integration/test/CatalogMysqlIT.java @@ -464,6 +464,7 @@ void testColumnDefaultValueConverter() { + " date_col_2 date,\n" + " date_col_3 date DEFAULT (CURRENT_DATE + INTERVAL 1 YEAR),\n" + " date_col_4 date DEFAULT (CURRENT_DATE),\n" + + " date_col_5 date DEFAULT '2024-04-01',\n" + " timestamp_col_1 timestamp default '2012-12-31 11:30:45',\n" + " timestamp_col_2 timestamp default 19830905,\n" + " decimal_6_2_col_1 decimal(6, 2) default 1.2\n" @@ -518,6 +519,10 @@ void testColumnDefaultValueConverter() { case "date_col_4": Assertions.assertEquals(UnparsedExpression.of("curdate()"), column.defaultValue()); break; + case "date_col_5": + Assertions.assertEquals( + Literals.of("2024-04-01", Types.DateType.get()), column.defaultValue()); + break; case "timestamp_col_1": Assertions.assertEquals( Literals.timestampLiteral("2012-12-31T11:30:45"), column.defaultValue()); From 0dcd308dce7d2d83ebfa147da023e98c61577b09 Mon Sep 17 00:00:00 2001 From: qqqttt123 <148952220+qqqttt123@users.noreply.github.com> Date: Mon, 1 Apr 2024 14:33:17 +0800 Subject: [PATCH 10/20] [#2238] feat(server): Add the operations for the user (#2733) ### What changes were proposed in this pull request? Add the operations for the user. ### Why are the changes needed? Fix: #2238 ### Does this PR introduce _any_ user-facing change? Yes, I will add the open api and the document in the later pr. ### How was this patch tested? Add a new UT. --------- Co-authored-by: Heng Qin --- .../gravitino/dto/authorization/UserDTO.java | 139 +++++++++ .../dto/requests/UserAddRequest.java | 53 ++++ .../dto/responses/RemoveResponse.java | 43 +++ .../gravitino/dto/responses/UserResponse.java | 54 ++++ .../gravitino/dto/util/DTOConverters.java | 20 ++ .../dto/responses/TestResponses.java | 16 + .../gravitino/server/GravitinoServer.java | 2 + .../server/web/rest/ExceptionHandlers.java | 38 +++ .../server/web/rest/OperationType.java | 4 +- .../server/web/rest/UserOperations.java | 123 ++++++++ .../server/web/rest/TestUserOperations.java | 282 ++++++++++++++++++ 11 files changed, 773 insertions(+), 1 deletion(-) create mode 100644 common/src/main/java/com/datastrato/gravitino/dto/authorization/UserDTO.java create mode 100644 common/src/main/java/com/datastrato/gravitino/dto/requests/UserAddRequest.java create mode 100644 common/src/main/java/com/datastrato/gravitino/dto/responses/RemoveResponse.java create mode 100644 common/src/main/java/com/datastrato/gravitino/dto/responses/UserResponse.java create mode 100644 server/src/main/java/com/datastrato/gravitino/server/web/rest/UserOperations.java create mode 100644 server/src/test/java/com/datastrato/gravitino/server/web/rest/TestUserOperations.java diff --git a/common/src/main/java/com/datastrato/gravitino/dto/authorization/UserDTO.java b/common/src/main/java/com/datastrato/gravitino/dto/authorization/UserDTO.java new file mode 100644 index 00000000000..f76a912c900 --- /dev/null +++ b/common/src/main/java/com/datastrato/gravitino/dto/authorization/UserDTO.java @@ -0,0 +1,139 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.dto.authorization; + +import com.datastrato.gravitino.Audit; +import com.datastrato.gravitino.authorization.User; +import com.datastrato.gravitino.dto.AuditDTO; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import java.util.Collections; +import java.util.List; +import org.apache.commons.lang3.StringUtils; + +/** Represents a User Data Transfer Object (DTO). */ +public class UserDTO implements User { + + @JsonProperty("name") + private String name; + + @JsonProperty("audit") + private AuditDTO audit; + + @JsonProperty("roles") + private List roles; + + /** Default constructor for Jackson deserialization. */ + protected UserDTO() {} + + /** + * Creates a new instance of UserDTO. + * + * @param name The name of the User DTO. + * @param roles The roles of the User DTO. + * @param audit The audit information of the User DTO. + */ + protected UserDTO(String name, List roles, AuditDTO audit) { + this.name = name; + this.audit = audit; + this.roles = roles; + } + + /** @return The name of the User DTO. */ + @Override + public String name() { + return name; + } + + /** + * The roles of the user. A user can have multiple roles. Every role binds several privileges. + * + * @return The roles of the user. + */ + @Override + public List roles() { + return roles; + } + + /** @return The audit information of the User DTO. */ + @Override + public Audit auditInfo() { + return audit; + } + + /** + * Creates a new Builder for constructing an User DTO. + * + * @return A new Builder instance. + */ + public static Builder builder() { + return new Builder(); + } + + /** + * Builder class for constructing a UserDTO instance. + * + * @param The type of the builder instance. + */ + public static class Builder { + + /** The name of the user. */ + protected String name; + + /** The roles of the user. */ + protected List roles = Collections.emptyList(); + + /** The audit information of the user. */ + protected AuditDTO audit; + + /** + * Sets the name of the user. + * + * @param name The name of the user. + * @return The builder instance. + */ + public S withName(String name) { + this.name = name; + return (S) this; + } + + /** + * Sets the roles of the user. + * + * @param roles The roles of the user. + * @return The builder instance. + */ + public S withRoles(List roles) { + if (roles != null) { + this.roles = roles; + } + + return (S) this; + } + + /** + * Sets the audit information of the user. + * + * @param audit The audit information of the user. + * @return The builder instance. + */ + public S withAudit(AuditDTO audit) { + this.audit = audit; + return (S) this; + } + + /** + * Builds an instance of UserDTO using the builder's properties. + * + * @return An instance of UserDTO. + * @throws IllegalArgumentException If the name or audit are not set. + */ + public UserDTO build() { + Preconditions.checkArgument(StringUtils.isNotBlank(name), "name cannot be null or empty"); + Preconditions.checkArgument(audit != null, "audit cannot be null"); + return new UserDTO(name, roles, audit); + } + } +} diff --git a/common/src/main/java/com/datastrato/gravitino/dto/requests/UserAddRequest.java b/common/src/main/java/com/datastrato/gravitino/dto/requests/UserAddRequest.java new file mode 100644 index 00000000000..886f4d2f309 --- /dev/null +++ b/common/src/main/java/com/datastrato/gravitino/dto/requests/UserAddRequest.java @@ -0,0 +1,53 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.dto.requests; + +import com.datastrato.gravitino.rest.RESTRequest; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import lombok.Builder; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.ToString; +import lombok.extern.jackson.Jacksonized; +import org.apache.commons.lang3.StringUtils; + +/** Represents a request to add a user. */ +@Getter +@EqualsAndHashCode +@ToString +@Builder +@Jacksonized +public class UserAddRequest implements RESTRequest { + + @JsonProperty("name") + private final String name; + + /** Default constructor for UserAddRequest. (Used for Jackson deserialization.) */ + public UserAddRequest() { + this(null); + } + + /** + * Creates a new UserAddRequest. + * + * @param name The name of the user. + */ + public UserAddRequest(String name) { + super(); + this.name = name; + } + + /** + * Validates the {@link UserAddRequest} request. + * + * @throws IllegalArgumentException If the request is invalid, this exception is thrown. + */ + @Override + public void validate() throws IllegalArgumentException { + Preconditions.checkArgument( + StringUtils.isNotBlank(name), "\"name\" field is required and cannot be empty"); + } +} diff --git a/common/src/main/java/com/datastrato/gravitino/dto/responses/RemoveResponse.java b/common/src/main/java/com/datastrato/gravitino/dto/responses/RemoveResponse.java new file mode 100644 index 00000000000..84bba97fb2f --- /dev/null +++ b/common/src/main/java/com/datastrato/gravitino/dto/responses/RemoveResponse.java @@ -0,0 +1,43 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.dto.responses; + +import com.fasterxml.jackson.annotation.JsonProperty; +import lombok.EqualsAndHashCode; +import lombok.ToString; + +/** Represents a response for a remove operation. */ +@ToString +@EqualsAndHashCode(callSuper = true) +public class RemoveResponse extends BaseResponse { + + @JsonProperty("removed") + private final boolean removed; + + /** + * Constructor for RemoveResponse. + * + * @param removed Whether the remove operation was successful. + */ + public RemoveResponse(boolean removed) { + super(0); + this.removed = removed; + } + + /** Default constructor for RemoveResponse (used by Jackson deserializer). */ + public RemoveResponse() { + super(); + this.removed = false; + } + + /** + * Returns whether the remove operation was successful. + * + * @return True if the remove operation was successful, otherwise false. + */ + public boolean removed() { + return removed; + } +} diff --git a/common/src/main/java/com/datastrato/gravitino/dto/responses/UserResponse.java b/common/src/main/java/com/datastrato/gravitino/dto/responses/UserResponse.java new file mode 100644 index 00000000000..277be31d6a9 --- /dev/null +++ b/common/src/main/java/com/datastrato/gravitino/dto/responses/UserResponse.java @@ -0,0 +1,54 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.dto.responses; + +import com.datastrato.gravitino.dto.authorization.UserDTO; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.ToString; +import org.apache.commons.lang3.StringUtils; + +/** Represents a response for a user. */ +@Getter +@ToString +@EqualsAndHashCode(callSuper = true) +public class UserResponse extends BaseResponse { + + @JsonProperty("user") + private final UserDTO user; + + /** + * Constructor for UserResponse. + * + * @param user The user data transfer object. + */ + public UserResponse(UserDTO user) { + super(0); + this.user = user; + } + + /** Default constructor for UserResponse. (Used for Jackson deserialization.) */ + public UserResponse() { + super(); + this.user = null; + } + + /** + * Validates the response data. + * + * @throws IllegalArgumentException if the name or audit is not set. + */ + @Override + public void validate() throws IllegalArgumentException { + super.validate(); + + Preconditions.checkArgument(user != null, "user must not be null"); + Preconditions.checkArgument( + StringUtils.isNotBlank(user.name()), "user 'name' must not be null and empty"); + Preconditions.checkArgument(user.auditInfo() != null, "user 'auditInfo' must not be null"); + } +} diff --git a/common/src/main/java/com/datastrato/gravitino/dto/util/DTOConverters.java b/common/src/main/java/com/datastrato/gravitino/dto/util/DTOConverters.java index e06c3c17bc2..049c86bf0b1 100644 --- a/common/src/main/java/com/datastrato/gravitino/dto/util/DTOConverters.java +++ b/common/src/main/java/com/datastrato/gravitino/dto/util/DTOConverters.java @@ -9,9 +9,11 @@ import com.datastrato.gravitino.Audit; import com.datastrato.gravitino.Catalog; import com.datastrato.gravitino.Metalake; +import com.datastrato.gravitino.authorization.User; import com.datastrato.gravitino.dto.AuditDTO; import com.datastrato.gravitino.dto.CatalogDTO; import com.datastrato.gravitino.dto.MetalakeDTO; +import com.datastrato.gravitino.dto.authorization.UserDTO; import com.datastrato.gravitino.dto.file.FilesetDTO; import com.datastrato.gravitino.dto.rel.ColumnDTO; import com.datastrato.gravitino.dto.rel.DistributionDTO; @@ -327,6 +329,24 @@ public static IndexDTO toDTO(Index index) { .build(); } + /** + * Converts a user implementation to a UserDTO. + * + * @param user The user implementation. + * @return The user DTO. + */ + public static UserDTO toDTO(User user) { + if (user instanceof UserDTO) { + return (UserDTO) user; + } + + return UserDTO.builder() + .withName(user.name()) + .withRoles(user.roles()) + .withAudit(toDTO(user.auditInfo())) + .build(); + } + /** * Converts a Expression to an FunctionArg DTO. * diff --git a/common/src/test/java/com/datastrato/gravitino/dto/responses/TestResponses.java b/common/src/test/java/com/datastrato/gravitino/dto/responses/TestResponses.java index efeb57c11c7..6176b6bf7ad 100644 --- a/common/src/test/java/com/datastrato/gravitino/dto/responses/TestResponses.java +++ b/common/src/test/java/com/datastrato/gravitino/dto/responses/TestResponses.java @@ -14,6 +14,7 @@ import com.datastrato.gravitino.dto.AuditDTO; import com.datastrato.gravitino.dto.CatalogDTO; import com.datastrato.gravitino.dto.MetalakeDTO; +import com.datastrato.gravitino.dto.authorization.UserDTO; import com.datastrato.gravitino.dto.rel.ColumnDTO; import com.datastrato.gravitino.dto.rel.SchemaDTO; import com.datastrato.gravitino.dto.rel.TableDTO; @@ -223,4 +224,19 @@ void testOAuthErrorException() throws IllegalArgumentException { OAuth2ErrorResponse response = new OAuth2ErrorResponse(); assertThrows(IllegalArgumentException.class, () -> response.validate()); } + + @Test + void testUserResponse() throws IllegalArgumentException { + AuditDTO audit = + AuditDTO.builder().withCreator("creator").withCreateTime(Instant.now()).build(); + UserDTO user = UserDTO.builder().withName("user1").withAudit(audit).build(); + UserResponse response = new UserResponse(user); + response.validate(); // No exception thrown + } + + @Test + void testUserResponseException() throws IllegalArgumentException { + UserResponse user = new UserResponse(); + assertThrows(IllegalArgumentException.class, () -> user.validate()); + } } 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 83533364a07..aae0fe8e860 100644 --- a/server/src/main/java/com/datastrato/gravitino/server/GravitinoServer.java +++ b/server/src/main/java/com/datastrato/gravitino/server/GravitinoServer.java @@ -5,6 +5,7 @@ package com.datastrato.gravitino.server; import com.datastrato.gravitino.GravitinoEnv; +import com.datastrato.gravitino.authorization.AccessControlManager; import com.datastrato.gravitino.catalog.CatalogManager; import com.datastrato.gravitino.catalog.FilesetOperationDispatcher; import com.datastrato.gravitino.catalog.SchemaOperationDispatcher; @@ -75,6 +76,7 @@ private void initializeRestApi() { protected void configure() { bind(gravitinoEnv.metalakesManager()).to(MetalakeManager.class).ranked(1); bind(gravitinoEnv.catalogManager()).to(CatalogManager.class).ranked(1); + bind(gravitinoEnv.accessControlManager()).to(AccessControlManager.class).ranked(1); bind(gravitinoEnv.schemaOperationDispatcher()) .to(SchemaOperationDispatcher.class) .ranked(1); diff --git a/server/src/main/java/com/datastrato/gravitino/server/web/rest/ExceptionHandlers.java b/server/src/main/java/com/datastrato/gravitino/server/web/rest/ExceptionHandlers.java index 7f4cb41c6e9..8af5693608e 100644 --- a/server/src/main/java/com/datastrato/gravitino/server/web/rest/ExceptionHandlers.java +++ b/server/src/main/java/com/datastrato/gravitino/server/web/rest/ExceptionHandlers.java @@ -13,6 +13,7 @@ import com.datastrato.gravitino.exceptions.PartitionAlreadyExistsException; import com.datastrato.gravitino.exceptions.SchemaAlreadyExistsException; import com.datastrato.gravitino.exceptions.TableAlreadyExistsException; +import com.datastrato.gravitino.exceptions.UserAlreadyExistsException; import com.datastrato.gravitino.server.web.Utils; import com.google.common.annotations.VisibleForTesting; import javax.ws.rs.core.Response; @@ -55,6 +56,11 @@ public static Response handleFilesetException( return FilesetExceptionHandler.INSTANCE.handle(op, fileset, schema, e); } + public static Response handleUserException( + OperationType op, String user, String metalake, Exception e) { + return UserExceptionHandler.INSTANCE.handle(op, user, metalake, e); + } + private static class PartitionExceptionHandler extends BaseExceptionHandler { private static final ExceptionHandler INSTANCE = new PartitionExceptionHandler(); @@ -254,6 +260,38 @@ public Response handle(OperationType op, String fileset, String schema, Exceptio } } + private static class UserExceptionHandler extends BaseExceptionHandler { + + private static final ExceptionHandler INSTANCE = new UserExceptionHandler(); + + private static String getUserErrorMsg( + String fileset, String operation, String metalake, String reason) { + return String.format( + "Failed to operate user %s operation [%s] under metalake [%s], reason [%s]", + fileset, operation, metalake, reason); + } + + @Override + public Response handle(OperationType op, String user, String metalake, Exception e) { + String formatted = StringUtil.isBlank(user) ? "" : " [" + user + "]"; + String errorMsg = getUserErrorMsg(formatted, op.name(), metalake, getErrorMsg(e)); + LOG.warn(errorMsg, e); + + if (e instanceof IllegalArgumentException) { + return Utils.illegalArguments(errorMsg, e); + + } else if (e instanceof NotFoundException) { + return Utils.notFound(errorMsg, e); + + } else if (e instanceof UserAlreadyExistsException) { + return Utils.alreadyExists(errorMsg, e); + + } else { + return super.handle(op, user, metalake, e); + } + } + } + @VisibleForTesting static class BaseExceptionHandler extends ExceptionHandler { diff --git a/server/src/main/java/com/datastrato/gravitino/server/web/rest/OperationType.java b/server/src/main/java/com/datastrato/gravitino/server/web/rest/OperationType.java index c914b9f90cb..78a4f56aa0a 100644 --- a/server/src/main/java/com/datastrato/gravitino/server/web/rest/OperationType.java +++ b/server/src/main/java/com/datastrato/gravitino/server/web/rest/OperationType.java @@ -11,5 +11,7 @@ public enum OperationType { ALTER, DROP, /** This is a special operation type that is used to get a partition from a table. */ - GET + GET, + ADD, + REMOVE } diff --git a/server/src/main/java/com/datastrato/gravitino/server/web/rest/UserOperations.java b/server/src/main/java/com/datastrato/gravitino/server/web/rest/UserOperations.java new file mode 100644 index 00000000000..759b71b3f12 --- /dev/null +++ b/server/src/main/java/com/datastrato/gravitino/server/web/rest/UserOperations.java @@ -0,0 +1,123 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.server.web.rest; + +import com.codahale.metrics.annotation.ResponseMetered; +import com.codahale.metrics.annotation.Timed; +import com.datastrato.gravitino.NameIdentifier; +import com.datastrato.gravitino.authorization.AccessControlManager; +import com.datastrato.gravitino.dto.requests.UserAddRequest; +import com.datastrato.gravitino.dto.responses.RemoveResponse; +import com.datastrato.gravitino.dto.responses.UserResponse; +import com.datastrato.gravitino.dto.util.DTOConverters; +import com.datastrato.gravitino.lock.LockType; +import com.datastrato.gravitino.lock.TreeLockUtils; +import com.datastrato.gravitino.meta.CatalogEntity; +import com.datastrato.gravitino.meta.UserEntity; +import com.datastrato.gravitino.metrics.MetricNames; +import com.datastrato.gravitino.server.web.Utils; +import javax.inject.Inject; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.DELETE; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.Response; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Path("/metalakes/{metalake}/users") +public class UserOperations { + + private static final Logger LOG = LoggerFactory.getLogger(UserOperations.class); + + private final AccessControlManager accessControlManager; + + @Context private HttpServletRequest httpRequest; + + @Inject + public UserOperations(AccessControlManager accessControlManager) { + this.accessControlManager = accessControlManager; + } + + @GET + @Path("{user}") + @Produces("application/vnd.gravitino.v1+json") + @Timed(name = "get-user." + MetricNames.HTTP_PROCESS_DURATION, absolute = true) + @ResponseMetered(name = "get-user", absolute = true) + public Response getUser(@PathParam("metalake") String metalake, @PathParam("user") String user) { + try { + NameIdentifier ident = ofUser(metalake, user); + return Utils.doAs( + httpRequest, + () -> + Utils.ok( + new UserResponse( + DTOConverters.toDTO( + TreeLockUtils.doWithTreeLock( + ident, + LockType.READ, + () -> accessControlManager.getUser(metalake, user)))))); + } catch (Exception e) { + return ExceptionHandlers.handleUserException(OperationType.GET, user, metalake, e); + } + } + + @POST + @Produces("application/vnd.gravitino.v1+json") + @Timed(name = "add-user." + MetricNames.HTTP_PROCESS_DURATION, absolute = true) + @ResponseMetered(name = "add-user", absolute = true) + public Response addUser(@PathParam("metalake") String metalake, UserAddRequest request) { + try { + NameIdentifier ident = ofUser(metalake, request.getName()); + return Utils.doAs( + httpRequest, + () -> + Utils.ok( + new UserResponse( + DTOConverters.toDTO( + TreeLockUtils.doWithTreeLock( + ident, + LockType.WRITE, + () -> accessControlManager.addUser(metalake, request.getName())))))); + } catch (Exception e) { + return ExceptionHandlers.handleUserException( + OperationType.ADD, request.getName(), metalake, e); + } + } + + @DELETE + @Path("{user}") + @Produces("application/vnd.gravitino.v1+json") + @Timed(name = "remove-user." + MetricNames.HTTP_PROCESS_DURATION, absolute = true) + @ResponseMetered(name = "remove-user", absolute = true) + public Response removeUser( + @PathParam("metalake") String metalake, @PathParam("user") String user) { + try { + return Utils.doAs( + httpRequest, + () -> { + NameIdentifier ident = ofUser(metalake, user); + boolean removed = + TreeLockUtils.doWithTreeLock( + ident, LockType.WRITE, () -> accessControlManager.removeUser(metalake, user)); + if (!removed) { + LOG.warn("Failed to remove user {} under metalake {}", user, metalake); + } + return Utils.ok(new RemoveResponse(removed)); + }); + } catch (Exception e) { + return ExceptionHandlers.handleUserException(OperationType.REMOVE, user, metalake, e); + } + } + + private NameIdentifier ofUser(String metalake, String user) { + return NameIdentifier.of( + metalake, CatalogEntity.SYSTEM_CATALOG_RESERVED_NAME, UserEntity.USER_SCHEMA_NAME, user); + } +} diff --git a/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestUserOperations.java b/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestUserOperations.java new file mode 100644 index 00000000000..a49b7e9a46f --- /dev/null +++ b/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestUserOperations.java @@ -0,0 +1,282 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.server.web.rest; + +import static com.datastrato.gravitino.Configs.TREE_LOCK_CLEAN_INTERVAL; +import static com.datastrato.gravitino.Configs.TREE_LOCK_MAX_NODE_IN_MEMORY; +import static com.datastrato.gravitino.Configs.TREE_LOCK_MIN_NODE_IN_MEMORY; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.datastrato.gravitino.Config; +import com.datastrato.gravitino.GravitinoEnv; +import com.datastrato.gravitino.authorization.AccessControlManager; +import com.datastrato.gravitino.authorization.User; +import com.datastrato.gravitino.dto.authorization.UserDTO; +import com.datastrato.gravitino.dto.requests.UserAddRequest; +import com.datastrato.gravitino.dto.responses.ErrorConstants; +import com.datastrato.gravitino.dto.responses.ErrorResponse; +import com.datastrato.gravitino.dto.responses.RemoveResponse; +import com.datastrato.gravitino.dto.responses.UserResponse; +import com.datastrato.gravitino.exceptions.NoSuchMetalakeException; +import com.datastrato.gravitino.exceptions.NoSuchUserException; +import com.datastrato.gravitino.exceptions.UserAlreadyExistsException; +import com.datastrato.gravitino.lock.LockManager; +import com.datastrato.gravitino.meta.AuditInfo; +import com.datastrato.gravitino.meta.UserEntity; +import com.datastrato.gravitino.rest.RESTUtils; +import java.io.IOException; +import java.time.Instant; +import java.util.Collections; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.client.Entity; +import javax.ws.rs.core.Application; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import org.glassfish.hk2.utilities.binding.AbstractBinder; +import org.glassfish.jersey.server.ResourceConfig; +import org.glassfish.jersey.test.JerseyTest; +import org.glassfish.jersey.test.TestProperties; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +public class TestUserOperations extends JerseyTest { + + private final AccessControlManager manager = mock(AccessControlManager.class); + + private static class MockServletRequestFactory extends ServletRequestFactoryBase { + @Override + public HttpServletRequest get() { + HttpServletRequest request = mock(HttpServletRequest.class); + when(request.getRemoteUser()).thenReturn(null); + return request; + } + } + + @BeforeAll + public static void setup() { + Config config = mock(Config.class); + Mockito.doReturn(100000L).when(config).get(TREE_LOCK_MAX_NODE_IN_MEMORY); + Mockito.doReturn(1000L).when(config).get(TREE_LOCK_MIN_NODE_IN_MEMORY); + Mockito.doReturn(36000L).when(config).get(TREE_LOCK_CLEAN_INTERVAL); + GravitinoEnv.getInstance().setLockManager(new LockManager(config)); + } + + @Override + protected Application configure() { + try { + forceSet( + TestProperties.CONTAINER_PORT, String.valueOf(RESTUtils.findAvailablePort(2000, 3000))); + } catch (IOException e) { + throw new RuntimeException(e); + } + + ResourceConfig resourceConfig = new ResourceConfig(); + resourceConfig.register(UserOperations.class); + resourceConfig.register( + new AbstractBinder() { + @Override + protected void configure() { + bind(manager).to(AccessControlManager.class).ranked(2); + bindFactory(MockServletRequestFactory.class).to(HttpServletRequest.class); + } + }); + + return resourceConfig; + } + + @Test + public void testAddUser() { + UserAddRequest req = new UserAddRequest("user1"); + User user = buildUser("user1"); + + when(manager.addUser(any(), any())).thenReturn(user); + + Response resp = + target("/metalakes/metalake1/users") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .post(Entity.entity(req, MediaType.APPLICATION_JSON_TYPE)); + + Assertions.assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + Assertions.assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getMediaType()); + + UserResponse userResponse = resp.readEntity(UserResponse.class); + Assertions.assertEquals(0, userResponse.getCode()); + + UserDTO userDTO = userResponse.getUser(); + Assertions.assertEquals("user1", userDTO.name()); + Assertions.assertNotNull(userDTO.roles()); + Assertions.assertTrue(userDTO.roles().isEmpty()); + + // Test to throw NoSuchMetalakeException + doThrow(new NoSuchMetalakeException("mock error")).when(manager).addUser(any(), any()); + Response resp1 = + target("/metalakes/metalake1/users") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .post(Entity.entity(req, MediaType.APPLICATION_JSON_TYPE)); + + Assertions.assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp1.getStatus()); + Assertions.assertEquals(MediaType.APPLICATION_JSON_TYPE, resp1.getMediaType()); + + ErrorResponse errorResponse = resp1.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.NOT_FOUND_CODE, errorResponse.getCode()); + Assertions.assertEquals(NoSuchMetalakeException.class.getSimpleName(), errorResponse.getType()); + + // Test to throw UserAlreadyExistsException + doThrow(new UserAlreadyExistsException("mock error")).when(manager).addUser(any(), any()); + Response resp2 = + target("/metalakes/metalake1/users") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .post(Entity.entity(req, MediaType.APPLICATION_JSON_TYPE)); + + Assertions.assertEquals(Response.Status.CONFLICT.getStatusCode(), resp2.getStatus()); + + ErrorResponse errorResponse1 = resp2.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.ALREADY_EXISTS_CODE, errorResponse1.getCode()); + Assertions.assertEquals( + UserAlreadyExistsException.class.getSimpleName(), errorResponse1.getType()); + + // Test to throw internal RuntimeException + doThrow(new RuntimeException("mock error")).when(manager).addUser(any(), any()); + Response resp3 = + target("/metalakes/metalake1/users") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .post(Entity.entity(req, MediaType.APPLICATION_JSON_TYPE)); + + Assertions.assertEquals( + Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), resp3.getStatus()); + + ErrorResponse errorResponse2 = resp3.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.INTERNAL_ERROR_CODE, errorResponse2.getCode()); + Assertions.assertEquals(RuntimeException.class.getSimpleName(), errorResponse2.getType()); + } + + @Test + public void testGetUser() { + + User user = buildUser("user1"); + + when(manager.getUser(any(), any())).thenReturn(user); + + Response resp = + target("/metalakes/metalake1/users/user1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .get(); + + Assertions.assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + UserResponse userResponse = resp.readEntity(UserResponse.class); + Assertions.assertEquals(0, userResponse.getCode()); + UserDTO userDTO = userResponse.getUser(); + Assertions.assertEquals("user1", userDTO.name()); + Assertions.assertNotNull(userDTO.roles()); + Assertions.assertTrue(userDTO.roles().isEmpty()); + + // Test to throw NoSuchMetalakeException + doThrow(new NoSuchMetalakeException("mock error")).when(manager).getUser(any(), any()); + Response resp1 = + target("/metalakes/metalake1/users/user1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .get(); + + Assertions.assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp1.getStatus()); + + ErrorResponse errorResponse = resp1.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.NOT_FOUND_CODE, errorResponse.getCode()); + Assertions.assertEquals(NoSuchMetalakeException.class.getSimpleName(), errorResponse.getType()); + + // Test to throw NoSuchUserException + doThrow(new NoSuchUserException("mock error")).when(manager).getUser(any(), any()); + Response resp2 = + target("/metalakes/metalake1/users/user1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .get(); + + Assertions.assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp2.getStatus()); + + ErrorResponse errorResponse1 = resp2.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.NOT_FOUND_CODE, errorResponse1.getCode()); + Assertions.assertEquals(NoSuchUserException.class.getSimpleName(), errorResponse1.getType()); + + // Test to throw internal RuntimeException + doThrow(new RuntimeException("mock error")).when(manager).getUser(any(), any()); + Response resp3 = + target("/metalakes/metalake1/users/user1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .get(); + + Assertions.assertEquals( + Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), resp3.getStatus()); + + ErrorResponse errorResponse2 = resp3.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.INTERNAL_ERROR_CODE, errorResponse2.getCode()); + Assertions.assertEquals(RuntimeException.class.getSimpleName(), errorResponse2.getType()); + } + + private User buildUser(String user) { + return UserEntity.builder() + .withId(1L) + .withName(user) + .withRoles(Collections.emptyList()) + .withAuditInfo( + AuditInfo.builder().withCreator("creator").withCreateTime(Instant.now()).build()) + .build(); + } + + @Test + public void testRemoveUser() { + when(manager.removeUser(any(), any())).thenReturn(true); + + Response resp = + target("/metalakes/metalake1/users/user1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .delete(); + + Assertions.assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + RemoveResponse removeResponse = resp.readEntity(RemoveResponse.class); + Assertions.assertEquals(0, removeResponse.getCode()); + Assertions.assertTrue(removeResponse.removed()); + + // Test when failed to remove user + when(manager.removeUser(any(), any())).thenReturn(false); + Response resp2 = + target("/metalakes/metalake1/users/user1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .delete(); + + Assertions.assertEquals(Response.Status.OK.getStatusCode(), resp2.getStatus()); + RemoveResponse removeResponse2 = resp2.readEntity(RemoveResponse.class); + Assertions.assertEquals(0, removeResponse2.getCode()); + Assertions.assertFalse(removeResponse2.removed()); + + doThrow(new RuntimeException("mock error")).when(manager).removeUser(any(), any()); + Response resp3 = + target("/metalakes/metalake1/users/user1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .delete(); + + Assertions.assertEquals( + Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), resp3.getStatus()); + + ErrorResponse errorResponse = resp3.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.INTERNAL_ERROR_CODE, errorResponse.getCode()); + Assertions.assertEquals(RuntimeException.class.getSimpleName(), errorResponse.getType()); + } +} From f72a48d661bbeeeafc6efc34c3418352fbdbd9e5 Mon Sep 17 00:00:00 2001 From: danhuawang <154112360+danhuawang@users.noreply.github.com> Date: Mon, 1 Apr 2024 14:53:25 +0800 Subject: [PATCH 11/20] [#2731]improvement(IT): crob workflow for ITs (#2748) ### What changes were proposed in this pull request? Setup crob job to run all ITs nightly. ### Why are the changes needed? Regularly run all ITs. Fix: #2731 ### Does this PR introduce _any_ user-facing change? N/A ### How was this patch tested? CI --- .github/workflows/cron-integration-test.yml | 96 +++++++++++++++++++++ 1 file changed, 96 insertions(+) create mode 100644 .github/workflows/cron-integration-test.yml diff --git a/.github/workflows/cron-integration-test.yml b/.github/workflows/cron-integration-test.yml new file mode 100644 index 00000000000..a69ee727602 --- /dev/null +++ b/.github/workflows/cron-integration-test.yml @@ -0,0 +1,96 @@ +name: Cron Integration Test + +# Controls when the workflow will run +on: + schedule: # Runs by default on main branch + - cron: '0 19 * * *' # Runs every day at 19:00 PM UTC, equal to 03:00 AM the next day in GMT+8 time zone + +concurrency: + group: ${{ github.workflow }}-${{ github.ref }} + cancel-in-progress: ${{ contains(github.ref, 'main') }} + +jobs: + changes: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v3 + - uses: dorny/paths-filter@v2 + id: filter + with: + filters: | + source_changes: + - .github/** + - api/** + - bin/** + - catalogs/** + - clients/** + - common/** + - conf/** + - core/** + - dev/** + - gradle/** + - integration-test/** + - meta/** + - server/** + - server-common/** + - spark-connector/** + - trino-connector/** + - web/** + - docs/open-api/** + - build.gradle.kts + - gradle.properties + - gradlew + - setting.gradle.kts + outputs: + source_changes: ${{ steps.filter.outputs.source_changes }} + + # Integration test for AMD64 architecture + test-amd64-arch: + needs: changes + if: needs.changes.outputs.source_changes == 'true' + runs-on: ubuntu-latest + timeout-minutes: 30 + strategy: + matrix: + architecture: [linux/amd64] + java-version: [ 8, 11, 17 ] + test-mode: [ embedded, deploy ] + env: + DOCKER_RUN_NAME: hive-amd64 + PLATFORM: ${{ matrix.architecture }} + steps: + - uses: actions/checkout@v3 + + - uses: actions/setup-java@v3 + with: + java-version: ${{ matrix.java-version }} + distribution: 'temurin' + + - name: Set up QEMU + uses: docker/setup-qemu-action@v2 + + - name: Package Gravitino + run: | + ./gradlew build -x test -PjdkVersion=${{ matrix.java-version }} + ./gradlew compileDistribution -x test -PjdkVersion=${{ matrix.java-version }} + + - name: Setup debug Github Action + if: ${{ contains(github.event.pull_request.labels.*.name, 'debug action') }} + uses: csexton/debugger-action@master + + - name: Integration Test + id: integrationTest + run: | + ./gradlew test --rerun-tasks -PskipTests -PtestMode=${{ matrix.test-mode }} -PjdkVersion=${{ matrix.java-version }} + + - name: Upload integrate tests reports + uses: actions/upload-artifact@v3 + if: ${{ failure() && steps.integrationTest.outcome == 'failure' }} + with: + name: integrate test reports + path: | + build/reports + integration-test/build/integration-test.log + distribution/package/logs/gravitino-server.out + distribution/package/logs/gravitino-server.log + catalogs/**/*.log \ No newline at end of file From c433f8418bf400e8eb348f9a14f10bcad51dea4e Mon Sep 17 00:00:00 2001 From: xloya <982052490@qq.com> Date: Mon, 1 Apr 2024 17:33:42 +0800 Subject: [PATCH 12/20] [#2745] fix(test): Extend sleep time in the `testInternalCache` unit test to fix flaky test (#2744) ### What changes were proposed in this pull request? `Caffeine Cache`'s use of `RemoveListener` is not stable and may cause a single test to fail in some cases. Therefore, the waiting time in the single test is extended to ensure that the `Cache` is cleaned normally. ![image](https://github.com/datastrato/gravitino/assets/26177232/7ecb8a6b-28a0-4146-91bd-0e633f9fe53b) ### Why are the changes needed? Fix: #2745 --------- Co-authored-by: xiaojiebao --- clients/filesystem-hadoop3/build.gradle.kts | 1 + .../TestGravitinoVirtualFileSystem.java | 30 ++++++++++++++----- gradle/libs.versions.toml | 2 ++ 3 files changed, 25 insertions(+), 8 deletions(-) diff --git a/clients/filesystem-hadoop3/build.gradle.kts b/clients/filesystem-hadoop3/build.gradle.kts index a6ad7ad365d..d90bc5d151b 100644 --- a/clients/filesystem-hadoop3/build.gradle.kts +++ b/clients/filesystem-hadoop3/build.gradle.kts @@ -14,6 +14,7 @@ dependencies { implementation(project(":clients:client-java-runtime", configuration = "shadow")) implementation(libs.caffeine) + testImplementation(libs.awaitility) testImplementation(libs.hadoop3.common) testImplementation(libs.junit.jupiter.api) testImplementation(libs.junit.jupiter.params) diff --git a/clients/filesystem-hadoop3/src/test/java/com/datastrato/gravitino/filesystem/hadoop3/TestGravitinoVirtualFileSystem.java b/clients/filesystem-hadoop3/src/test/java/com/datastrato/gravitino/filesystem/hadoop3/TestGravitinoVirtualFileSystem.java index 6c016f42bf2..85a6b8363d8 100644 --- a/clients/filesystem-hadoop3/src/test/java/com/datastrato/gravitino/filesystem/hadoop3/TestGravitinoVirtualFileSystem.java +++ b/clients/filesystem-hadoop3/src/test/java/com/datastrato/gravitino/filesystem/hadoop3/TestGravitinoVirtualFileSystem.java @@ -20,10 +20,12 @@ import java.util.Comparator; import java.util.List; import java.util.Objects; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.awaitility.Awaitility; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; @@ -151,7 +153,7 @@ public void testFSCache() throws IOException { } @Test - public void testInternalCache() throws IOException, InterruptedException { + public void testInternalCache() throws IOException { Configuration configuration = new Configuration(conf); configuration.set( GravitinoVirtualFileSystemConfiguration.FS_GRAVITINO_FILESET_CACHE_MAX_CAPACITY_KEY, "1"); @@ -187,15 +189,27 @@ public void testInternalCache() throws IOException, InterruptedException { localPath2.toString()); FileSystemTestUtils.mkdirs(filesetPath2, fs); - Thread.sleep(1000); - assertNull( - ((GravitinoVirtualFileSystem) fs) - .getFilesetCache() - .getIfPresent(NameIdentifier.of(metalakeName, catalogName, schemaName, "fileset1"))); + Awaitility.await() + .atMost(5, TimeUnit.SECONDS) + .pollInterval(1, TimeUnit.SECONDS) + .untilAsserted( + () -> + assertNull( + ((GravitinoVirtualFileSystem) fs) + .getFilesetCache() + .getIfPresent( + NameIdentifier.of( + metalakeName, catalogName, schemaName, "fileset1")))); // expired by time - Thread.sleep(1000); - assertEquals(0, ((GravitinoVirtualFileSystem) fs).getFilesetCache().asMap().size()); + Awaitility.await() + .atMost(5, TimeUnit.SECONDS) + .pollInterval(1, TimeUnit.SECONDS) + .untilAsserted( + () -> + assertEquals( + 0, ((GravitinoVirtualFileSystem) fs).getFilesetCache().asMap().size())); + assertNull( ((GravitinoVirtualFileSystem) fs) .getFilesetCache() diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 39a9459ba57..e5dbf1a46fe 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -52,6 +52,7 @@ h2db = "1.4.200" kyuubi = "1.8.0" kafka = "3.4.0" curator = "2.12.0" +awaitility = "4.2.1" protobuf-plugin = "0.9.2" spotless-plugin = '6.11.0' @@ -162,6 +163,7 @@ selenium = { group = "org.seleniumhq.selenium", name = "selenium-java", version. rauschig = { group = "org.rauschig", name = "jarchivelib", version.ref = "rauschig" } mybatis = { group = "org.mybatis", name = "mybatis", version.ref = "mybatis"} h2db = { group = "com.h2database", name = "h2", version.ref = "h2db"} +awaitility = { group = "org.awaitility", name = "awaitility", version.ref = "awaitility" } [bundles] log4j = ["slf4j-api", "log4j-slf4j2-impl", "log4j-api", "log4j-core", "log4j-12-api"] From fc90986d7a75fb5f33145705cef609246d7754fe Mon Sep 17 00:00:00 2001 From: mchades Date: Mon, 1 Apr 2024 19:54:32 +0800 Subject: [PATCH 13/20] [#2611] feat(server): Add server-side REST API support for topic (#2703) ### What changes were proposed in this pull request? This PR adds RESTful API implementation of messaging catalog topic operations ### Why are the changes needed? Part of Kafka catalog support work Fix: #2611 ### Does this PR introduce _any_ user-facing change? yes, add several topic operations RESTful API ### How was this patch tested? UTs --- .../datastrato/gravitino/NameIdentifier.java | 15 + .../com/datastrato/gravitino/Namespace.java | 12 + .../gravitino/dto/messaging/TopicDTO.java | 129 ++++++ .../dto/requests/TopicCreateRequest.java | 61 +++ .../dto/requests/TopicUpdateRequest.java | 175 ++++++++ .../dto/requests/TopicUpdatesRequest.java | 46 ++ .../dto/responses/TopicResponse.java | 48 +++ .../gravitino/dto/util/DTOConverters.java | 17 + .../gravitino/server/GravitinoServer.java | 3 + .../server/web/rest/ExceptionHandlers.java | 37 ++ .../server/web/rest/TopicOperations.java | 207 +++++++++ .../server/web/rest/TestTopicOperations.java | 398 ++++++++++++++++++ 12 files changed, 1148 insertions(+) create mode 100644 common/src/main/java/com/datastrato/gravitino/dto/messaging/TopicDTO.java create mode 100644 common/src/main/java/com/datastrato/gravitino/dto/requests/TopicCreateRequest.java create mode 100644 common/src/main/java/com/datastrato/gravitino/dto/requests/TopicUpdateRequest.java create mode 100644 common/src/main/java/com/datastrato/gravitino/dto/requests/TopicUpdatesRequest.java create mode 100644 common/src/main/java/com/datastrato/gravitino/dto/responses/TopicResponse.java create mode 100644 server/src/main/java/com/datastrato/gravitino/server/web/rest/TopicOperations.java create mode 100644 server/src/test/java/com/datastrato/gravitino/server/web/rest/TestTopicOperations.java diff --git a/api/src/main/java/com/datastrato/gravitino/NameIdentifier.java b/api/src/main/java/com/datastrato/gravitino/NameIdentifier.java index 6a79a4f188f..3d9389bb3ed 100644 --- a/api/src/main/java/com/datastrato/gravitino/NameIdentifier.java +++ b/api/src/main/java/com/datastrato/gravitino/NameIdentifier.java @@ -113,6 +113,21 @@ public static NameIdentifier ofFileset( return NameIdentifier.of(metalake, catalog, schema, fileset); } + /** + * Create the topic {@link NameIdentifier} with the given metalake, catalog, schema and topic + * name. + * + * @param metalake The metalake name + * @param catalog The catalog name + * @param schema The schema name + * @param topic The topic name + * @return The created topic {@link NameIdentifier} + */ + public static NameIdentifier ofTopic( + String metalake, String catalog, String schema, String topic) { + return NameIdentifier.of(metalake, catalog, schema, topic); + } + /** * Check the given {@link NameIdentifier} is a metalake identifier. Throw an {@link * IllegalNameIdentifierException} if it's not. diff --git a/api/src/main/java/com/datastrato/gravitino/Namespace.java b/api/src/main/java/com/datastrato/gravitino/Namespace.java index d61e7ec46ad..3a38025cf7c 100644 --- a/api/src/main/java/com/datastrato/gravitino/Namespace.java +++ b/api/src/main/java/com/datastrato/gravitino/Namespace.java @@ -105,6 +105,18 @@ public static Namespace ofFileset(String metalake, String catalog, String schema return of(metalake, catalog, schema); } + /** + * Create a namespace for topic. + * + * @param metalake The metalake name + * @param catalog The catalog name + * @param schema The schema name + * @return A namespace for topic + */ + public static Namespace ofTopic(String metalake, String catalog, String schema) { + return of(metalake, catalog, schema); + } + /** * Check if the given metalake namespace is legal, throw an {@link IllegalNamespaceException} if * it's illegal. diff --git a/common/src/main/java/com/datastrato/gravitino/dto/messaging/TopicDTO.java b/common/src/main/java/com/datastrato/gravitino/dto/messaging/TopicDTO.java new file mode 100644 index 00000000000..6c0116dd786 --- /dev/null +++ b/common/src/main/java/com/datastrato/gravitino/dto/messaging/TopicDTO.java @@ -0,0 +1,129 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.dto.messaging; + +import com.datastrato.gravitino.Audit; +import com.datastrato.gravitino.dto.AuditDTO; +import com.datastrato.gravitino.messaging.Topic; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.Map; +import javax.annotation.Nullable; + +/** Represents a topic DTO (Data Transfer Object). */ +public class TopicDTO implements Topic { + + /** @return a new builder for constructing a Topic DTO. */ + public static Builder builder() { + return new Builder(); + } + + @JsonProperty("name") + private String name; + + @JsonProperty("comment") + private String comment; + + @JsonProperty("properties") + private Map properties; + + @JsonProperty("audit") + private AuditDTO audit; + + private TopicDTO() {} + + /** + * Constructs a Topic DTO. + * + * @param name The name of the topic. + * @param comment The comment associated with the topic. + * @param properties The properties associated with the topic. + * @param audit The audit information for the topic. + */ + private TopicDTO(String name, String comment, Map properties, AuditDTO audit) { + this.name = name; + this.comment = comment; + this.properties = properties; + this.audit = audit; + } + + @Override + public String name() { + return name; + } + + @Nullable + @Override + public String comment() { + return comment; + } + + @Override + public Map properties() { + return properties; + } + + @Override + public Audit auditInfo() { + return audit; + } + + /** A builder for constructing a Topic DTO. */ + public static class Builder { + private final TopicDTO topic; + + private Builder() { + topic = new TopicDTO(); + } + + /** + * Sets the name of the topic. + * + * @param name The name of the topic. + * @return The builder instance. + */ + public Builder withName(String name) { + topic.name = name; + return this; + } + + /** + * Sets the comment associated with the topic. + * + * @param comment The comment associated with the topic. + * @return The builder instance. + */ + public Builder withComment(String comment) { + topic.comment = comment; + return this; + } + + /** + * Sets the properties associated with the topic. + * + * @param properties The properties associated with the topic. + * @return The builder instance. + */ + public Builder withProperties(Map properties) { + topic.properties = properties; + return this; + } + + /** + * Sets the audit information for the topic. + * + * @param audit The audit information for the topic. + * @return The builder instance. + */ + public Builder withAudit(AuditDTO audit) { + topic.audit = audit; + return this; + } + + /** @return The constructed Topic DTO. */ + public TopicDTO build() { + return topic; + } + } +} diff --git a/common/src/main/java/com/datastrato/gravitino/dto/requests/TopicCreateRequest.java b/common/src/main/java/com/datastrato/gravitino/dto/requests/TopicCreateRequest.java new file mode 100644 index 00000000000..df4ca507125 --- /dev/null +++ b/common/src/main/java/com/datastrato/gravitino/dto/requests/TopicCreateRequest.java @@ -0,0 +1,61 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.dto.requests; + +import com.datastrato.gravitino.rest.RESTRequest; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import java.util.Map; +import javax.annotation.Nullable; +import lombok.Builder; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.ToString; +import lombok.extern.jackson.Jacksonized; +import org.apache.commons.lang3.StringUtils; + +/** Represents a request to create a topic. */ +@Getter +@EqualsAndHashCode +@ToString +@Builder +@Jacksonized +public class TopicCreateRequest implements RESTRequest { + @JsonProperty("name") + private final String name; + + @Nullable + @JsonProperty("comment") + private final String comment; + + @Nullable + @JsonProperty("properties") + private final Map properties; + + /** Default constructor for Jackson deserialization. */ + public TopicCreateRequest() { + this(null, null, null); + } + + /** + * Creates a topic create request. + * + * @param name The name of the topic. + * @param comment The comment of the topic. + * @param properties The properties of the topic. + */ + public TopicCreateRequest( + String name, @Nullable String comment, @Nullable Map properties) { + this.name = name; + this.comment = comment; + this.properties = properties; + } + + @Override + public void validate() throws IllegalArgumentException { + Preconditions.checkArgument( + StringUtils.isNotBlank(name), "\"name\" field is required and cannot be empty"); + } +} diff --git a/common/src/main/java/com/datastrato/gravitino/dto/requests/TopicUpdateRequest.java b/common/src/main/java/com/datastrato/gravitino/dto/requests/TopicUpdateRequest.java new file mode 100644 index 00000000000..1d1f9918b1b --- /dev/null +++ b/common/src/main/java/com/datastrato/gravitino/dto/requests/TopicUpdateRequest.java @@ -0,0 +1,175 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.dto.requests; + +import com.datastrato.gravitino.messaging.TopicChange; +import com.datastrato.gravitino.rest.RESTRequest; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.base.Preconditions; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.ToString; +import org.apache.commons.lang3.StringUtils; + +/** Represents a request to update a topic. */ +@JsonIgnoreProperties(ignoreUnknown = true) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY) +@JsonSubTypes({ + @JsonSubTypes.Type( + value = TopicUpdateRequest.UpdateTopicCommentRequest.class, + name = "updateComment"), + @JsonSubTypes.Type( + value = TopicUpdateRequest.SetTopicPropertyRequest.class, + name = "setProperty"), + @JsonSubTypes.Type( + value = TopicUpdateRequest.RemoveTopicPropertyRequest.class, + name = "removeProperty") +}) +public interface TopicUpdateRequest extends RESTRequest { + + /** @return The topic change. */ + TopicChange topicChange(); + + /** Represents a request to update the comment of a topic. */ + @EqualsAndHashCode + @ToString + @Getter + class UpdateTopicCommentRequest implements TopicUpdateRequest { + + @JsonProperty("newComment") + private final String newComment; + + /** + * Constructor for UpdateTopicCommentRequest. + * + * @param newComment the new comment of the topic + */ + public UpdateTopicCommentRequest(String newComment) { + this.newComment = newComment; + } + + /** Default constructor for Jackson deserialization. */ + public UpdateTopicCommentRequest() { + this(null); + } + + /** + * Validates the request. + * + * @throws IllegalArgumentException If the request is invalid, this exception is thrown. + */ + @Override + public void validate() throws IllegalArgumentException { + Preconditions.checkArgument( + StringUtils.isNotBlank(newComment), + "\"newComment\" field is required and cannot be empty"); + } + + /** + * Returns the topic change. + * + * @return An instance of TopicChange. + */ + @Override + public TopicChange topicChange() { + return TopicChange.updateComment(newComment); + } + } + + /** Represents a request to set a property of a Topic. */ + @EqualsAndHashCode + @ToString + @Getter + class SetTopicPropertyRequest implements TopicUpdateRequest { + + @JsonProperty("property") + private final String property; + + @JsonProperty("value") + private final String value; + + /** + * Constructor for SetTopicPropertyRequest. + * + * @param property the property to set + * @param value the value to set + */ + public SetTopicPropertyRequest(String property, String value) { + this.property = property; + this.value = value; + } + + /** Default constructor for Jackson deserialization. */ + public SetTopicPropertyRequest() { + this(null, null); + } + + /** + * Validates the request. + * + * @throws IllegalArgumentException If the request is invalid, this exception is thrown. + */ + @Override + public void validate() throws IllegalArgumentException { + Preconditions.checkArgument( + StringUtils.isNotBlank(property), "\"property\" field is required and cannot be empty"); + Preconditions.checkArgument(value != null, "\"value\" field is required and cannot be null"); + } + + /** + * Returns the topic change. + * + * @return An instance of TopicChange. + */ + @Override + public TopicChange topicChange() { + return TopicChange.setProperty(property, value); + } + } + + /** Represents a request to remove a property of a topic. */ + @EqualsAndHashCode + @ToString + @Getter + class RemoveTopicPropertyRequest implements TopicUpdateRequest { + + @JsonProperty("property") + private final String property; + + /** + * Constructor for RemoveTopicPropertyRequest. + * + * @param property the property to remove + */ + public RemoveTopicPropertyRequest(String property) { + this.property = property; + } + + /** Default constructor for Jackson deserialization. */ + public RemoveTopicPropertyRequest() { + this(null); + } + + /** + * Validates the request. + * + * @throws IllegalArgumentException If the request is invalid, this exception is thrown. + */ + @Override + public void validate() throws IllegalArgumentException { + Preconditions.checkArgument( + StringUtils.isNotBlank(property), "\"property\" field is required and cannot be empty"); + } + + /** @return An instance of TopicChange. */ + @Override + public TopicChange topicChange() { + return TopicChange.removeProperty(property); + } + } +} diff --git a/common/src/main/java/com/datastrato/gravitino/dto/requests/TopicUpdatesRequest.java b/common/src/main/java/com/datastrato/gravitino/dto/requests/TopicUpdatesRequest.java new file mode 100644 index 00000000000..e7ea29edf30 --- /dev/null +++ b/common/src/main/java/com/datastrato/gravitino/dto/requests/TopicUpdatesRequest.java @@ -0,0 +1,46 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.dto.requests; + +import com.datastrato.gravitino.rest.RESTRequest; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.ToString; + +/** Represents a request to update a topic. */ +@Getter +@EqualsAndHashCode +@ToString +public class TopicUpdatesRequest implements RESTRequest { + + @JsonProperty("updates") + private final List updates; + + /** + * Creates a new TopicUpdatesRequest. + * + * @param updates The updates to apply to the topic. + */ + public TopicUpdatesRequest(List updates) { + this.updates = updates; + } + + /** This is the constructor that is used by Jackson deserializer */ + public TopicUpdatesRequest() { + this(null); + } + + /** + * Validates the request. + * + * @throws IllegalArgumentException If the request is invalid, this exception is thrown. + */ + @Override + public void validate() throws IllegalArgumentException { + updates.forEach(RESTRequest::validate); + } +} diff --git a/common/src/main/java/com/datastrato/gravitino/dto/responses/TopicResponse.java b/common/src/main/java/com/datastrato/gravitino/dto/responses/TopicResponse.java new file mode 100644 index 00000000000..d51561abb7b --- /dev/null +++ b/common/src/main/java/com/datastrato/gravitino/dto/responses/TopicResponse.java @@ -0,0 +1,48 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.dto.responses; + +import com.datastrato.gravitino.dto.messaging.TopicDTO; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.ToString; +import org.apache.commons.lang3.StringUtils; + +/** Represents a response to a topic. */ +@Getter +@ToString +@EqualsAndHashCode(callSuper = true) +public class TopicResponse extends BaseResponse { + + @JsonProperty("topic") + private final TopicDTO topic; + + /** + * Creates a new TopicResponse. + * + * @param topic The topic DTO object. + */ + public TopicResponse(TopicDTO topic) { + super(0); + this.topic = topic; + } + + /** This is the constructor that is used by Jackson deserializer */ + public TopicResponse() { + super(); + this.topic = null; + } + + @Override + public void validate() throws IllegalArgumentException { + super.validate(); + + Preconditions.checkArgument(topic != null, "topic must not be null"); + Preconditions.checkArgument( + StringUtils.isNotBlank(topic.name()), "topic 'name' must not be null and empty"); + } +} diff --git a/common/src/main/java/com/datastrato/gravitino/dto/util/DTOConverters.java b/common/src/main/java/com/datastrato/gravitino/dto/util/DTOConverters.java index 049c86bf0b1..e9868b9b376 100644 --- a/common/src/main/java/com/datastrato/gravitino/dto/util/DTOConverters.java +++ b/common/src/main/java/com/datastrato/gravitino/dto/util/DTOConverters.java @@ -15,6 +15,7 @@ import com.datastrato.gravitino.dto.MetalakeDTO; import com.datastrato.gravitino.dto.authorization.UserDTO; import com.datastrato.gravitino.dto.file.FilesetDTO; +import com.datastrato.gravitino.dto.messaging.TopicDTO; import com.datastrato.gravitino.dto.rel.ColumnDTO; import com.datastrato.gravitino.dto.rel.DistributionDTO; import com.datastrato.gravitino.dto.rel.SchemaDTO; @@ -42,6 +43,7 @@ import com.datastrato.gravitino.dto.rel.partitions.PartitionDTO; import com.datastrato.gravitino.dto.rel.partitions.RangePartitionDTO; import com.datastrato.gravitino.file.Fileset; +import com.datastrato.gravitino.messaging.Topic; import com.datastrato.gravitino.rel.Column; import com.datastrato.gravitino.rel.Schema; import com.datastrato.gravitino.rel.Table; @@ -417,6 +419,21 @@ public static FilesetDTO toDTO(Fileset fileset) { .build(); } + /** + * Converts a Topic to a TopicDTO. + * + * @param topic The topic to be converted. + * @return The topic DTO. + */ + public static TopicDTO toDTO(Topic topic) { + return TopicDTO.builder() + .withName(topic.name()) + .withComment(topic.comment()) + .withProperties(topic.properties()) + .withAudit(toDTO(topic.auditInfo())) + .build(); + } + /** * Converts an array of Columns to an array of ColumnDTOs. * 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 aae0fe8e860..55702776567 100644 --- a/server/src/main/java/com/datastrato/gravitino/server/GravitinoServer.java +++ b/server/src/main/java/com/datastrato/gravitino/server/GravitinoServer.java @@ -86,6 +86,9 @@ protected void configure() { bind(gravitinoEnv.filesetOperationDispatcher()) .to(FilesetOperationDispatcher.class) .ranked(1); + bind(gravitinoEnv.topicOperationDispatcher()) + .to(com.datastrato.gravitino.catalog.TopicOperationDispatcher.class) + .ranked(1); } }); register(ObjectMapperProvider.class).register(JacksonFeature.class); diff --git a/server/src/main/java/com/datastrato/gravitino/server/web/rest/ExceptionHandlers.java b/server/src/main/java/com/datastrato/gravitino/server/web/rest/ExceptionHandlers.java index 8af5693608e..2c4c9e5f5ca 100644 --- a/server/src/main/java/com/datastrato/gravitino/server/web/rest/ExceptionHandlers.java +++ b/server/src/main/java/com/datastrato/gravitino/server/web/rest/ExceptionHandlers.java @@ -13,6 +13,7 @@ import com.datastrato.gravitino.exceptions.PartitionAlreadyExistsException; import com.datastrato.gravitino.exceptions.SchemaAlreadyExistsException; import com.datastrato.gravitino.exceptions.TableAlreadyExistsException; +import com.datastrato.gravitino.exceptions.TopicAlreadyExistsException; import com.datastrato.gravitino.exceptions.UserAlreadyExistsException; import com.datastrato.gravitino.server.web.Utils; import com.google.common.annotations.VisibleForTesting; @@ -61,6 +62,11 @@ public static Response handleUserException( return UserExceptionHandler.INSTANCE.handle(op, user, metalake, e); } + public static Response handleTopicException( + OperationType op, String topic, String schema, Exception e) { + return TopicExceptionHandler.INSTANCE.handle(op, topic, schema, e); + } + private static class PartitionExceptionHandler extends BaseExceptionHandler { private static final ExceptionHandler INSTANCE = new PartitionExceptionHandler(); @@ -292,6 +298,37 @@ public Response handle(OperationType op, String user, String metalake, Exception } } + private static class TopicExceptionHandler extends BaseExceptionHandler { + private static final ExceptionHandler INSTANCE = new TopicExceptionHandler(); + + private static String getTopicErrorMsg( + String topic, String operation, String schema, String reason) { + return String.format( + "Failed to operate topic(s)%s operation [%s] under schema [%s], reason [%s]", + topic, operation, schema, reason); + } + + @Override + public Response handle(OperationType op, String topic, String schema, Exception e) { + String formatted = StringUtil.isBlank(topic) ? "" : " [" + topic + "]"; + String errorMsg = getTopicErrorMsg(formatted, op.name(), schema, getErrorMsg(e)); + LOG.warn(errorMsg, e); + + if (e instanceof IllegalArgumentException) { + return Utils.illegalArguments(errorMsg, e); + + } else if (e instanceof NotFoundException) { + return Utils.notFound(errorMsg, e); + + } else if (e instanceof TopicAlreadyExistsException) { + return Utils.alreadyExists(errorMsg, e); + + } else { + return super.handle(op, topic, schema, e); + } + } + } + @VisibleForTesting static class BaseExceptionHandler extends ExceptionHandler { diff --git a/server/src/main/java/com/datastrato/gravitino/server/web/rest/TopicOperations.java b/server/src/main/java/com/datastrato/gravitino/server/web/rest/TopicOperations.java new file mode 100644 index 00000000000..e09440432b9 --- /dev/null +++ b/server/src/main/java/com/datastrato/gravitino/server/web/rest/TopicOperations.java @@ -0,0 +1,207 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.server.web.rest; + +import com.codahale.metrics.annotation.ResponseMetered; +import com.codahale.metrics.annotation.Timed; +import com.datastrato.gravitino.NameIdentifier; +import com.datastrato.gravitino.Namespace; +import com.datastrato.gravitino.catalog.TopicOperationDispatcher; +import com.datastrato.gravitino.dto.requests.TopicCreateRequest; +import com.datastrato.gravitino.dto.requests.TopicUpdateRequest; +import com.datastrato.gravitino.dto.requests.TopicUpdatesRequest; +import com.datastrato.gravitino.dto.responses.DropResponse; +import com.datastrato.gravitino.dto.responses.EntityListResponse; +import com.datastrato.gravitino.dto.responses.TopicResponse; +import com.datastrato.gravitino.dto.util.DTOConverters; +import com.datastrato.gravitino.lock.LockType; +import com.datastrato.gravitino.lock.TreeLockUtils; +import com.datastrato.gravitino.messaging.Topic; +import com.datastrato.gravitino.messaging.TopicChange; +import com.datastrato.gravitino.metrics.MetricNames; +import com.datastrato.gravitino.server.web.Utils; +import javax.inject.Inject; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.DELETE; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.PUT; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.Response; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Path("/metalakes/{metalake}/catalogs/{catalog}/schemas/{schema}/topics") +public class TopicOperations { + private static final Logger LOG = LoggerFactory.getLogger(TopicOperations.class); + + private final TopicOperationDispatcher dispatcher; + + @Context private HttpServletRequest httpRequest; + + @Inject + public TopicOperations(TopicOperationDispatcher dispatcher) { + this.dispatcher = dispatcher; + } + + @GET + @Produces("application/vnd.gravitino.v1+json") + @Timed(name = "list-topic." + MetricNames.HTTP_PROCESS_DURATION, absolute = true) + public Response listTopics( + @PathParam("metalake") String metalake, + @PathParam("catalog") String catalog, + @PathParam("schema") String schema) { + try { + return Utils.doAs( + httpRequest, + () -> { + LOG.info("Listing topics under schema: {}.{}.{}", metalake, catalog, schema); + Namespace topicNS = Namespace.ofTopic(metalake, catalog, schema); + NameIdentifier[] topics = + TreeLockUtils.doWithTreeLock( + NameIdentifier.of(metalake, catalog, schema), + LockType.WRITE, + () -> dispatcher.listTopics(topicNS)); + return Utils.ok(new EntityListResponse(topics)); + }); + } catch (Exception e) { + return ExceptionHandlers.handleFilesetException(OperationType.LIST, "", schema, e); + } + } + + @POST + @Produces("application/vnd.gravitino.v1+json") + @Timed(name = "create-topic." + MetricNames.HTTP_PROCESS_DURATION, absolute = true) + @ResponseMetered(name = "create-topic", absolute = true) + public Response createTopic( + @PathParam("metalake") String metalake, + @PathParam("catalog") String catalog, + @PathParam("schema") String schema, + TopicCreateRequest request) { + try { + return Utils.doAs( + httpRequest, + () -> { + LOG.info( + "Creating topic under schema: {}.{}.{}.{}", + metalake, + catalog, + schema, + request.getName()); + request.validate(); + NameIdentifier ident = + NameIdentifier.ofTopic(metalake, catalog, schema, request.getName()); + + Topic topic = + TreeLockUtils.doWithTreeLock( + ident, + LockType.WRITE, + () -> + dispatcher.createTopic( + ident, + request.getComment(), + null /* dataLayout, always null because it's not supported yet.*/, + request.getProperties())); + return Utils.ok(new TopicResponse(DTOConverters.toDTO(topic))); + }); + } catch (Exception e) { + return ExceptionHandlers.handleTopicException( + OperationType.CREATE, request.getName(), schema, e); + } + } + + @GET + @Path("/{topic}") + @Produces("application/vnd.gravitino.v1+json") + @Timed(name = "load-topic." + MetricNames.HTTP_PROCESS_DURATION, absolute = true) + @ResponseMetered(name = "load-topic", absolute = true) + public Response loadTopic( + @PathParam("metalake") String metalake, + @PathParam("catalog") String catalog, + @PathParam("schema") String schema, + @PathParam("topic") String topic) { + try { + return Utils.doAs( + httpRequest, + () -> { + LOG.info("Loading topic: {}.{}.{}.{}", metalake, catalog, schema, topic); + NameIdentifier ident = NameIdentifier.ofTopic(metalake, catalog, schema, topic); + Topic t = + TreeLockUtils.doWithTreeLock( + ident, LockType.READ, () -> dispatcher.loadTopic(ident)); + return Utils.ok(new TopicResponse(DTOConverters.toDTO(t))); + }); + } catch (Exception e) { + return ExceptionHandlers.handleTopicException(OperationType.LOAD, topic, schema, e); + } + } + + @PUT + @Path("/{topic}") + @Produces("application/vnd.gravitino.v1+json") + @Timed(name = "alter-topic." + MetricNames.HTTP_PROCESS_DURATION, absolute = true) + @ResponseMetered(name = "alter-topic", absolute = true) + public Response alterTopic( + @PathParam("metalake") String metalake, + @PathParam("catalog") String catalog, + @PathParam("schema") String schema, + @PathParam("topic") String topic, + TopicUpdatesRequest request) { + try { + return Utils.doAs( + httpRequest, + () -> { + LOG.info("Altering topic: {}.{}.{}.{}", metalake, catalog, schema, topic); + request.validate(); + NameIdentifier ident = NameIdentifier.ofTopic(metalake, catalog, schema, topic); + TopicChange[] changes = + request.getUpdates().stream() + .map(TopicUpdateRequest::topicChange) + .toArray(TopicChange[]::new); + + Topic t = + TreeLockUtils.doWithTreeLock( + ident, LockType.WRITE, () -> dispatcher.alterTopic(ident, changes)); + return Utils.ok(new TopicResponse(DTOConverters.toDTO(t))); + }); + } catch (Exception e) { + return ExceptionHandlers.handleTopicException(OperationType.ALTER, topic, schema, e); + } + } + + @DELETE + @Path("/{topic}") + @Produces("application/vnd.gravitino.v1+json") + @Timed(name = "drop-topic." + MetricNames.HTTP_PROCESS_DURATION, absolute = true) + @ResponseMetered(name = "drop-topic", absolute = true) + public Response dropTopic( + @PathParam("metalake") String metalake, + @PathParam("catalog") String catalog, + @PathParam("schema") String schema, + @PathParam("topic") String topic) { + try { + return Utils.doAs( + httpRequest, + () -> { + LOG.info("Dropping topic under schema: {}.{}.{}", metalake, catalog, schema); + NameIdentifier ident = NameIdentifier.ofTopic(metalake, catalog, schema, topic); + boolean dropped = + TreeLockUtils.doWithTreeLock( + ident, LockType.WRITE, () -> dispatcher.dropTopic(ident)); + + if (!dropped) { + LOG.warn("Failed to drop topic {} under schema {}", topic, schema); + } + + return Utils.ok(new DropResponse(dropped)); + }); + } catch (Exception e) { + return ExceptionHandlers.handleTopicException(OperationType.DROP, topic, schema, e); + } + } +} diff --git a/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestTopicOperations.java b/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestTopicOperations.java new file mode 100644 index 00000000000..35acfedf277 --- /dev/null +++ b/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestTopicOperations.java @@ -0,0 +1,398 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.server.web.rest; + +import static com.datastrato.gravitino.Configs.TREE_LOCK_CLEAN_INTERVAL; +import static com.datastrato.gravitino.Configs.TREE_LOCK_MAX_NODE_IN_MEMORY; +import static com.datastrato.gravitino.Configs.TREE_LOCK_MIN_NODE_IN_MEMORY; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.datastrato.gravitino.Audit; +import com.datastrato.gravitino.Config; +import com.datastrato.gravitino.GravitinoEnv; +import com.datastrato.gravitino.NameIdentifier; +import com.datastrato.gravitino.catalog.TopicOperationDispatcher; +import com.datastrato.gravitino.dto.messaging.TopicDTO; +import com.datastrato.gravitino.dto.requests.TopicCreateRequest; +import com.datastrato.gravitino.dto.requests.TopicUpdateRequest; +import com.datastrato.gravitino.dto.requests.TopicUpdatesRequest; +import com.datastrato.gravitino.dto.responses.DropResponse; +import com.datastrato.gravitino.dto.responses.EntityListResponse; +import com.datastrato.gravitino.dto.responses.ErrorConstants; +import com.datastrato.gravitino.dto.responses.ErrorResponse; +import com.datastrato.gravitino.dto.responses.TopicResponse; +import com.datastrato.gravitino.exceptions.NoSuchSchemaException; +import com.datastrato.gravitino.exceptions.TopicAlreadyExistsException; +import com.datastrato.gravitino.lock.LockManager; +import com.datastrato.gravitino.messaging.Topic; +import com.datastrato.gravitino.messaging.TopicChange; +import com.datastrato.gravitino.rest.RESTUtils; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.time.Instant; +import java.util.Map; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.client.Entity; +import javax.ws.rs.core.Application; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import org.glassfish.jersey.internal.inject.AbstractBinder; +import org.glassfish.jersey.server.ResourceConfig; +import org.glassfish.jersey.test.JerseyTest; +import org.glassfish.jersey.test.TestProperties; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +public class TestTopicOperations extends JerseyTest { + + private static class MockServletRequestFactory extends ServletRequestFactoryBase { + @Override + public HttpServletRequest get() { + HttpServletRequest request = mock(HttpServletRequest.class); + when(request.getRemoteUser()).thenReturn(null); + return request; + } + } + + private TopicOperationDispatcher dispatcher = mock(TopicOperationDispatcher.class); + private final String metalake = "metalake"; + private final String catalog = "catalog1"; + private final String schema = "default"; + + @BeforeAll + public static void setup() { + Config config = mock(Config.class); + Mockito.doReturn(100000L).when(config).get(TREE_LOCK_MAX_NODE_IN_MEMORY); + Mockito.doReturn(1000L).when(config).get(TREE_LOCK_MIN_NODE_IN_MEMORY); + Mockito.doReturn(36000L).when(config).get(TREE_LOCK_CLEAN_INTERVAL); + GravitinoEnv.getInstance().setLockManager(new LockManager(config)); + } + + @Override + protected Application configure() { + try { + forceSet( + TestProperties.CONTAINER_PORT, String.valueOf(RESTUtils.findAvailablePort(2000, 3000))); + } catch (IOException e) { + throw new RuntimeException(e); + } + + ResourceConfig resourceConfig = new ResourceConfig(); + resourceConfig.register(TopicOperations.class); + resourceConfig.register( + new AbstractBinder() { + @Override + protected void configure() { + bind(dispatcher).to(TopicOperationDispatcher.class).ranked(2); + bindFactory(TestTopicOperations.MockServletRequestFactory.class) + .to(HttpServletRequest.class); + } + }); + + return resourceConfig; + } + + @Test + public void testListTopics() { + NameIdentifier topic1 = NameIdentifier.of(metalake, catalog, schema, "topic1"); + NameIdentifier topic2 = NameIdentifier.of(metalake, catalog, schema, "topic2"); + + when(dispatcher.listTopics(any())).thenReturn(new NameIdentifier[] {topic1, topic2}); + + Response resp = + target(topicPath(metalake, catalog, schema)) + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .get(); + + Assertions.assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + Assertions.assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getMediaType()); + + EntityListResponse listResp = resp.readEntity(EntityListResponse.class); + Assertions.assertEquals(0, listResp.getCode()); + + NameIdentifier[] topics = listResp.identifiers(); + Assertions.assertEquals(2, topics.length); + Assertions.assertEquals(topic1, topics[0]); + Assertions.assertEquals(topic2, topics[1]); + + // Test throw NoSuchSchemaException + doThrow(new NoSuchSchemaException("mock error")).when(dispatcher).listTopics(any()); + Response resp1 = + target(topicPath(metalake, catalog, schema)) + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .get(); + + Assertions.assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp1.getStatus()); + + ErrorResponse errorResp = resp1.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.NOT_FOUND_CODE, errorResp.getCode()); + Assertions.assertEquals(NoSuchSchemaException.class.getSimpleName(), errorResp.getType()); + + // Test throw RuntimeException + doThrow(new RuntimeException("mock error")).when(dispatcher).listTopics(any()); + Response resp2 = + target(topicPath(metalake, catalog, schema)) + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .get(); + + Assertions.assertEquals( + Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), resp2.getStatus()); + + ErrorResponse errorResp2 = resp2.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.INTERNAL_ERROR_CODE, errorResp2.getCode()); + Assertions.assertEquals(RuntimeException.class.getSimpleName(), errorResp2.getType()); + } + + @Test + public void testLoadTopic() { + Topic topic = mockTopic("topic1", "comment", ImmutableMap.of("key1", "value1")); + when(dispatcher.loadTopic(any())).thenReturn(topic); + + Response resp = + target(topicPath(metalake, catalog, schema) + "/topic1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .get(); + Assertions.assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + TopicResponse topicResp = resp.readEntity(TopicResponse.class); + Assertions.assertEquals(0, topicResp.getCode()); + + TopicDTO topicDTO = topicResp.getTopic(); + Assertions.assertEquals("topic1", topicDTO.name()); + Assertions.assertEquals("comment", topicDTO.comment()); + Assertions.assertEquals(ImmutableMap.of("key1", "value1"), topicDTO.properties()); + + // Test throw NoSuchSchemaException + doThrow(new NoSuchSchemaException("mock error")).when(dispatcher).loadTopic(any()); + Response resp1 = + target(topicPath(metalake, catalog, schema) + "/topic1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .get(); + Assertions.assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp1.getStatus()); + + ErrorResponse errorResp = resp1.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.NOT_FOUND_CODE, errorResp.getCode()); + Assertions.assertEquals(NoSuchSchemaException.class.getSimpleName(), errorResp.getType()); + + // Test throw RuntimeException + doThrow(new RuntimeException("mock error")).when(dispatcher).loadTopic(any()); + + Response resp2 = + target(topicPath(metalake, catalog, schema) + "/topic1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .get(); + Assertions.assertEquals( + Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), resp2.getStatus()); + + ErrorResponse errorResp2 = resp2.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.INTERNAL_ERROR_CODE, errorResp2.getCode()); + Assertions.assertEquals(RuntimeException.class.getSimpleName(), errorResp2.getType()); + } + + @Test + public void testCreateTopic() { + Topic topic = mockTopic("topic1", "comment", ImmutableMap.of("key1", "value1")); + when(dispatcher.createTopic(any(), any(), any(), any())).thenReturn(topic); + + TopicCreateRequest req = + TopicCreateRequest.builder() + .name("topic1") + .comment("comment") + .properties(ImmutableMap.of("key1", "value1")) + .build(); + Response resp = + target(topicPath(metalake, catalog, schema)) + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .post(Entity.entity(req, MediaType.APPLICATION_JSON_TYPE)); + Assertions.assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + TopicResponse topicResp = resp.readEntity(TopicResponse.class); + Assertions.assertEquals(0, topicResp.getCode()); + + TopicDTO topicDTO = topicResp.getTopic(); + Assertions.assertEquals("topic1", topicDTO.name()); + Assertions.assertEquals("comment", topicDTO.comment()); + Assertions.assertEquals(ImmutableMap.of("key1", "value1"), topicDTO.properties()); + + // Test throw NoSuchSchemaException + doThrow(new NoSuchSchemaException("mock error")) + .when(dispatcher) + .createTopic(any(), any(), any(), any()); + + Response resp1 = + target(topicPath(metalake, catalog, schema)) + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .post(Entity.entity(req, MediaType.APPLICATION_JSON_TYPE)); + Assertions.assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp1.getStatus()); + + ErrorResponse errorResp = resp1.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.NOT_FOUND_CODE, errorResp.getCode()); + Assertions.assertEquals(NoSuchSchemaException.class.getSimpleName(), errorResp.getType()); + + // Test throw TopicAlreadyExistsException + doThrow(new TopicAlreadyExistsException("mock error")) + .when(dispatcher) + .createTopic(any(), any(), any(), any()); + + Response resp2 = + target(topicPath(metalake, catalog, schema)) + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .post(Entity.entity(req, MediaType.APPLICATION_JSON_TYPE)); + Assertions.assertEquals(Response.Status.CONFLICT.getStatusCode(), resp2.getStatus()); + + ErrorResponse errorResp2 = resp2.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.ALREADY_EXISTS_CODE, errorResp2.getCode()); + Assertions.assertEquals( + TopicAlreadyExistsException.class.getSimpleName(), errorResp2.getType()); + + // Test throw RuntimeException + doThrow(new RuntimeException("mock error")) + .when(dispatcher) + .createTopic(any(), any(), any(), any()); + + Response resp3 = + target(topicPath(metalake, catalog, schema)) + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .post(Entity.entity(req, MediaType.APPLICATION_JSON_TYPE)); + Assertions.assertEquals( + Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), resp3.getStatus()); + + ErrorResponse errorResp3 = resp3.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.INTERNAL_ERROR_CODE, errorResp3.getCode()); + Assertions.assertEquals(RuntimeException.class.getSimpleName(), errorResp3.getType()); + } + + @Test + public void testSetTopicProperties() { + TopicUpdateRequest req = new TopicUpdateRequest.SetTopicPropertyRequest("key1", "value1"); + Topic topic = mockTopic("topic1", "comment", ImmutableMap.of("key1", "value1")); + assertUpdateTopic(new TopicUpdatesRequest(ImmutableList.of(req)), topic); + } + + @Test + public void testRemoveTopicProperties() { + TopicUpdateRequest req = new TopicUpdateRequest.RemoveTopicPropertyRequest("key1"); + Topic topic = mockTopic("topic1", "comment", ImmutableMap.of()); + assertUpdateTopic(new TopicUpdatesRequest(ImmutableList.of(req)), topic); + } + + @Test + public void testUpdateTopicComment() { + TopicUpdateRequest req = new TopicUpdateRequest.UpdateTopicCommentRequest("new comment"); + Topic topic = mockTopic("topic1", "new comment", ImmutableMap.of()); + assertUpdateTopic(new TopicUpdatesRequest(ImmutableList.of(req)), topic); + } + + @Test + public void testMultiUpdateRequest() { + TopicUpdateRequest req1 = new TopicUpdateRequest.UpdateTopicCommentRequest("new comment"); + TopicUpdateRequest req2 = new TopicUpdateRequest.SetTopicPropertyRequest("key1", "value1"); + // update key1=value2 + TopicUpdateRequest req3 = new TopicUpdateRequest.SetTopicPropertyRequest("key1", "value2"); + TopicUpdateRequest req4 = new TopicUpdateRequest.SetTopicPropertyRequest("key2", "value2"); + // remove key2 + TopicUpdateRequest req5 = new TopicUpdateRequest.RemoveTopicPropertyRequest("key2"); + + Topic topic = mockTopic("topic1", "new comment", ImmutableMap.of("key1", "value1")); + assertUpdateTopic( + new TopicUpdatesRequest(ImmutableList.of(req1, req2, req3, req4, req5)), topic); + } + + @Test + public void testDropTopic() { + when(dispatcher.dropTopic(any())).thenReturn(true); + Response resp = + target(topicPath(metalake, catalog, schema) + "/topic1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .delete(); + Assertions.assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + DropResponse dropResp = resp.readEntity(DropResponse.class); + Assertions.assertEquals(0, dropResp.getCode()); + Assertions.assertTrue(dropResp.dropped()); + + // test dropTopic return false + when(dispatcher.dropTopic(any())).thenReturn(false); + Response resp1 = + target(topicPath(metalake, catalog, schema) + "/topic1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .delete(); + Assertions.assertEquals(Response.Status.OK.getStatusCode(), resp1.getStatus()); + + DropResponse dropResp1 = resp1.readEntity(DropResponse.class); + Assertions.assertEquals(0, dropResp1.getCode()); + Assertions.assertFalse(dropResp1.dropped()); + + // test throw RuntimeException + doThrow(new RuntimeException("mock error")).when(dispatcher).dropTopic(any()); + Response resp2 = + target(topicPath(metalake, catalog, schema) + "/topic1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .delete(); + + Assertions.assertEquals( + Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), resp2.getStatus()); + + ErrorResponse errorResp2 = resp2.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.INTERNAL_ERROR_CODE, errorResp2.getCode()); + Assertions.assertEquals(RuntimeException.class.getSimpleName(), errorResp2.getType()); + } + + private void assertUpdateTopic(TopicUpdatesRequest req, Topic updatedTopic) { + when(dispatcher.alterTopic(any(), any(TopicChange.class))).thenReturn(updatedTopic); + + Response resp1 = + target(topicPath(metalake, catalog, schema) + "/topic1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .put(Entity.entity(req, MediaType.APPLICATION_JSON_TYPE)); + Assertions.assertEquals(Response.Status.OK.getStatusCode(), resp1.getStatus()); + + TopicResponse topicResp = resp1.readEntity(TopicResponse.class); + Assertions.assertEquals(0, topicResp.getCode()); + + TopicDTO topicDTO = topicResp.getTopic(); + Assertions.assertEquals(updatedTopic.name(), topicDTO.name()); + Assertions.assertEquals(updatedTopic.comment(), topicDTO.comment()); + Assertions.assertEquals(updatedTopic.properties(), topicDTO.properties()); + } + + private Topic mockTopic(String name, String comment, Map properties) { + Topic mockedTopic = mock(Topic.class); + when(mockedTopic.name()).thenReturn(name); + when(mockedTopic.comment()).thenReturn(comment); + when(mockedTopic.properties()).thenReturn(properties); + + Audit mockAudit = mock(Audit.class); + when(mockAudit.creator()).thenReturn("gravitino"); + when(mockAudit.createTime()).thenReturn(Instant.now()); + when(mockedTopic.auditInfo()).thenReturn(mockAudit); + + return mockedTopic; + } + + private String topicPath(String metalake, String catalog, String schema) { + return "/metalakes/" + metalake + "/catalogs/" + catalog + "/schemas/" + schema + "/topics"; + } +} From cf0fb50cc77b3c1bea0bf9cfa85c51228e3783d8 Mon Sep 17 00:00:00 2001 From: danhuawang <154112360+danhuawang@users.noreply.github.com> Date: Mon, 1 Apr 2024 19:55:47 +0800 Subject: [PATCH 14/20] =?UTF-8?q?[#2748]=20fix(CI):=20free=20up=20disk=20s?= =?UTF-8?q?pace,=20because=20doris=20container=20need=20more=20than=2016GB?= =?UTF-8?q?=20disk=20=E2=80=A6=20(#2751)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Free up disk space ### Why are the changes needed? Because doris container need more than 16GB disk size Fix: #2731 ### Does this PR introduce _any_ user-facing change? N/A ### How was this patch tested? CI --- .github/workflows/cron-integration-test.yml | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/.github/workflows/cron-integration-test.yml b/.github/workflows/cron-integration-test.yml index a69ee727602..7f3ed5aa152 100644 --- a/.github/workflows/cron-integration-test.yml +++ b/.github/workflows/cron-integration-test.yml @@ -49,7 +49,7 @@ jobs: needs: changes if: needs.changes.outputs.source_changes == 'true' runs-on: ubuntu-latest - timeout-minutes: 30 + timeout-minutes: 60 strategy: matrix: architecture: [linux/amd64] @@ -78,6 +78,10 @@ jobs: if: ${{ contains(github.event.pull_request.labels.*.name, 'debug action') }} uses: csexton/debugger-action@master + - name: Free up disk space + run: | + dev/ci/util_free_space.sh + - name: Integration Test id: integrationTest run: | From a1c1d53777d89bdc2c467f92fe115df98e40fb29 Mon Sep 17 00:00:00 2001 From: qqqttt123 <148952220+qqqttt123@users.noreply.github.com> Date: Mon, 1 Apr 2024 20:50:02 +0800 Subject: [PATCH 15/20] [#2240] feat(server): Add the operations for the group (#2752) ### What changes were proposed in this pull request? Add the operations for the group ### Why are the changes needed? Fix: #2240 ### Does this PR introduce _any_ user-facing change? Yes, I will add the document in the later pr. ### How was this patch tested? Add a new UT. --------- Co-authored-by: Heng Qin --- .../gravitino/dto/authorization/GroupDTO.java | 139 +++++++++ .../dto/requests/GroupAddRequest.java | 53 ++++ .../dto/responses/GroupResponse.java | 54 ++++ .../gravitino/dto/util/DTOConverters.java | 20 ++ .../dto/responses/TestResponses.java | 15 + .../TestAccessControlManager.java | 6 - .../server/web/rest/ExceptionHandlers.java | 42 ++- .../server/web/rest/GroupOperations.java | 124 ++++++++ .../server/web/rest/TestGroupOperations.java | 281 ++++++++++++++++++ 9 files changed, 726 insertions(+), 8 deletions(-) create mode 100644 common/src/main/java/com/datastrato/gravitino/dto/authorization/GroupDTO.java create mode 100644 common/src/main/java/com/datastrato/gravitino/dto/requests/GroupAddRequest.java create mode 100644 common/src/main/java/com/datastrato/gravitino/dto/responses/GroupResponse.java create mode 100644 server/src/main/java/com/datastrato/gravitino/server/web/rest/GroupOperations.java create mode 100644 server/src/test/java/com/datastrato/gravitino/server/web/rest/TestGroupOperations.java diff --git a/common/src/main/java/com/datastrato/gravitino/dto/authorization/GroupDTO.java b/common/src/main/java/com/datastrato/gravitino/dto/authorization/GroupDTO.java new file mode 100644 index 00000000000..2c4720bc9d4 --- /dev/null +++ b/common/src/main/java/com/datastrato/gravitino/dto/authorization/GroupDTO.java @@ -0,0 +1,139 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.dto.authorization; + +import com.datastrato.gravitino.Audit; +import com.datastrato.gravitino.authorization.Group; +import com.datastrato.gravitino.dto.AuditDTO; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import java.util.Collections; +import java.util.List; +import org.apache.commons.lang3.StringUtils; + +/** Represents a Group Data Transfer Object (DTO). */ +public class GroupDTO implements Group { + + @JsonProperty("name") + private String name; + + @JsonProperty("audit") + private AuditDTO audit; + + @JsonProperty("roles") + private List roles; + + /** Default constructor for Jackson deserialization. */ + protected GroupDTO() {} + + /** + * Creates a new instance of GroupDTO. + * + * @param name The name of the Group DTO. + * @param roles The roles of the Group DTO. + * @param audit The audit information of the Group DTO. + */ + protected GroupDTO(String name, List roles, AuditDTO audit) { + this.name = name; + this.audit = audit; + this.roles = roles; + } + + /** @return The name of the Group DTO. */ + @Override + public String name() { + return name; + } + + /** + * The roles of the group. A group can have multiple roles. Every role binds several privileges. + * + * @return The roles of the group. + */ + @Override + public List roles() { + return roles; + } + + /** @return The audit information of the Group DTO. */ + @Override + public Audit auditInfo() { + return audit; + } + + /** + * Creates a new Builder for constructing a Group DTO. + * + * @return A new Builder instance. + */ + public static Builder builder() { + return new Builder(); + } + + /** + * Builder class for constructing a GroupDTO instance. + * + * @param The type of the builder instance. + */ + public static class Builder { + + /** The name of the group. */ + protected String name; + + /** The roles of the group. */ + protected List roles = Collections.emptyList(); + + /** The audit information of the group. */ + protected AuditDTO audit; + + /** + * Sets the name of the group. + * + * @param name The name of the group. + * @return The builder instance. + */ + public S withName(String name) { + this.name = name; + return (S) this; + } + + /** + * Sets the roles of the group. + * + * @param roles The roles of the group. + * @return The builder instance. + */ + public S withRoles(List roles) { + if (roles != null) { + this.roles = roles; + } + + return (S) this; + } + + /** + * Sets the audit information of the group. + * + * @param audit The audit information of the group. + * @return The builder instance. + */ + public S withAudit(AuditDTO audit) { + this.audit = audit; + return (S) this; + } + + /** + * Builds an instance of GroupDTO using the builder's properties. + * + * @return An instance of GroupDTO. + * @throws IllegalArgumentException If the name or audit are not set. + */ + public GroupDTO build() { + Preconditions.checkArgument(StringUtils.isNotBlank(name), "name cannot be null or empty"); + Preconditions.checkArgument(audit != null, "audit cannot be null"); + return new GroupDTO(name, roles, audit); + } + } +} diff --git a/common/src/main/java/com/datastrato/gravitino/dto/requests/GroupAddRequest.java b/common/src/main/java/com/datastrato/gravitino/dto/requests/GroupAddRequest.java new file mode 100644 index 00000000000..5c41fdaefe6 --- /dev/null +++ b/common/src/main/java/com/datastrato/gravitino/dto/requests/GroupAddRequest.java @@ -0,0 +1,53 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.dto.requests; + +import com.datastrato.gravitino.rest.RESTRequest; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import lombok.Builder; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.ToString; +import lombok.extern.jackson.Jacksonized; +import org.apache.commons.lang3.StringUtils; + +/** Represents a request to add a group. */ +@Getter +@EqualsAndHashCode +@ToString +@Builder +@Jacksonized +public class GroupAddRequest implements RESTRequest { + + @JsonProperty("name") + private final String name; + + /** Default constructor for GroupAddRequest. (Used for Jackson deserialization.) */ + public GroupAddRequest() { + this(null); + } + + /** + * Creates a new GroupAddRequest. + * + * @param name The name of the group. + */ + public GroupAddRequest(String name) { + super(); + this.name = name; + } + + /** + * Validates the {@link GroupAddRequest} request. + * + * @throws IllegalArgumentException If the request is invalid, this exception is thrown. + */ + @Override + public void validate() throws IllegalArgumentException { + Preconditions.checkArgument( + StringUtils.isNotBlank(name), "\"name\" field is required and cannot be empty"); + } +} diff --git a/common/src/main/java/com/datastrato/gravitino/dto/responses/GroupResponse.java b/common/src/main/java/com/datastrato/gravitino/dto/responses/GroupResponse.java new file mode 100644 index 00000000000..100f2fa13bb --- /dev/null +++ b/common/src/main/java/com/datastrato/gravitino/dto/responses/GroupResponse.java @@ -0,0 +1,54 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.dto.responses; + +import com.datastrato.gravitino.dto.authorization.GroupDTO; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.ToString; +import org.apache.commons.lang3.StringUtils; + +/** Represents a response for a group. */ +@Getter +@ToString +@EqualsAndHashCode(callSuper = true) +public class GroupResponse extends BaseResponse { + + @JsonProperty("group") + private final GroupDTO group; + + /** + * Constructor for GroupResponse. + * + * @param group The group data transfer object. + */ + public GroupResponse(GroupDTO group) { + super(0); + this.group = group; + } + + /** Default constructor for GroupResponse. (Used for Jackson deserialization.) */ + public GroupResponse() { + super(); + this.group = null; + } + + /** + * Validates the response data. + * + * @throws IllegalArgumentException if the name or audit is not set. + */ + @Override + public void validate() throws IllegalArgumentException { + super.validate(); + + Preconditions.checkArgument(group != null, "group must not be null"); + Preconditions.checkArgument( + StringUtils.isNotBlank(group.name()), "group 'name' must not be null and empty"); + Preconditions.checkArgument(group.auditInfo() != null, "group 'auditInfo' must not be null"); + } +} diff --git a/common/src/main/java/com/datastrato/gravitino/dto/util/DTOConverters.java b/common/src/main/java/com/datastrato/gravitino/dto/util/DTOConverters.java index e9868b9b376..a2050e53548 100644 --- a/common/src/main/java/com/datastrato/gravitino/dto/util/DTOConverters.java +++ b/common/src/main/java/com/datastrato/gravitino/dto/util/DTOConverters.java @@ -9,10 +9,12 @@ import com.datastrato.gravitino.Audit; import com.datastrato.gravitino.Catalog; import com.datastrato.gravitino.Metalake; +import com.datastrato.gravitino.authorization.Group; import com.datastrato.gravitino.authorization.User; import com.datastrato.gravitino.dto.AuditDTO; import com.datastrato.gravitino.dto.CatalogDTO; import com.datastrato.gravitino.dto.MetalakeDTO; +import com.datastrato.gravitino.dto.authorization.GroupDTO; import com.datastrato.gravitino.dto.authorization.UserDTO; import com.datastrato.gravitino.dto.file.FilesetDTO; import com.datastrato.gravitino.dto.messaging.TopicDTO; @@ -349,6 +351,24 @@ public static UserDTO toDTO(User user) { .build(); } + /** + * Converts a group implementation to a GroupDTO. + * + * @param group The group implementation. + * @return The group DTO. + */ + public static GroupDTO toDTO(Group group) { + if (group instanceof GroupDTO) { + return (GroupDTO) group; + } + + return GroupDTO.builder() + .withName(group.name()) + .withRoles(group.roles()) + .withAudit(toDTO(group.auditInfo())) + .build(); + } + /** * Converts a Expression to an FunctionArg DTO. * diff --git a/common/src/test/java/com/datastrato/gravitino/dto/responses/TestResponses.java b/common/src/test/java/com/datastrato/gravitino/dto/responses/TestResponses.java index 6176b6bf7ad..4465cc34f11 100644 --- a/common/src/test/java/com/datastrato/gravitino/dto/responses/TestResponses.java +++ b/common/src/test/java/com/datastrato/gravitino/dto/responses/TestResponses.java @@ -14,6 +14,7 @@ import com.datastrato.gravitino.dto.AuditDTO; import com.datastrato.gravitino.dto.CatalogDTO; import com.datastrato.gravitino.dto.MetalakeDTO; +import com.datastrato.gravitino.dto.authorization.GroupDTO; import com.datastrato.gravitino.dto.authorization.UserDTO; import com.datastrato.gravitino.dto.rel.ColumnDTO; import com.datastrato.gravitino.dto.rel.SchemaDTO; @@ -239,4 +240,18 @@ void testUserResponseException() throws IllegalArgumentException { UserResponse user = new UserResponse(); assertThrows(IllegalArgumentException.class, () -> user.validate()); } + + void testGroupResponse() throws IllegalArgumentException { + AuditDTO audit = + AuditDTO.builder().withCreator("creator").withCreateTime(Instant.now()).build(); + GroupDTO group = GroupDTO.builder().withName("group1").withAudit(audit).build(); + GroupResponse response = new GroupResponse(group); + response.validate(); // No exception thrown + } + + @Test + void testGroupResponseException() throws IllegalArgumentException { + GroupResponse group = new GroupResponse(); + assertThrows(IllegalArgumentException.class, () -> group.validate()); + } } diff --git a/core/src/test/java/com/datastrato/gravitino/authorization/TestAccessControlManager.java b/core/src/test/java/com/datastrato/gravitino/authorization/TestAccessControlManager.java index 2a1e023d6cd..03c51ad03e9 100644 --- a/core/src/test/java/com/datastrato/gravitino/authorization/TestAccessControlManager.java +++ b/core/src/test/java/com/datastrato/gravitino/authorization/TestAccessControlManager.java @@ -64,7 +64,6 @@ public static void tearDown() throws IOException { @Test public void testAddUser() { - User user = accessControlManager.addUser("metalake", "testAdd"); Assertions.assertEquals("testAdd", user.name()); Assertions.assertTrue(user.roles().isEmpty()); @@ -82,7 +81,6 @@ public void testAddUser() { @Test public void testGetUser() { - accessControlManager.addUser("metalake", "testGet"); User user = accessControlManager.getUser("metalake", "testGet"); @@ -97,7 +95,6 @@ public void testGetUser() { @Test public void testRemoveUser() { - accessControlManager.addUser("metalake", "testRemove"); // Test to remove user @@ -111,7 +108,6 @@ public void testRemoveUser() { @Test public void testAddGroup() { - Group group = accessControlManager.addGroup("metalake", "testAdd"); Assertions.assertEquals("testAdd", group.name()); Assertions.assertTrue(group.roles().isEmpty()); @@ -129,7 +125,6 @@ public void testAddGroup() { @Test public void testGetGroup() { - accessControlManager.addGroup("metalake", "testGet"); Group group = accessControlManager.getGroup("metalake", "testGet"); @@ -145,7 +140,6 @@ public void testGetGroup() { @Test public void testRemoveGroup() { - accessControlManager.addGroup("metalake", "testRemove"); // Test to remove group diff --git a/server/src/main/java/com/datastrato/gravitino/server/web/rest/ExceptionHandlers.java b/server/src/main/java/com/datastrato/gravitino/server/web/rest/ExceptionHandlers.java index 2c4c9e5f5ca..9b32d3c82e8 100644 --- a/server/src/main/java/com/datastrato/gravitino/server/web/rest/ExceptionHandlers.java +++ b/server/src/main/java/com/datastrato/gravitino/server/web/rest/ExceptionHandlers.java @@ -6,6 +6,7 @@ import com.datastrato.gravitino.exceptions.CatalogAlreadyExistsException; import com.datastrato.gravitino.exceptions.FilesetAlreadyExistsException; +import com.datastrato.gravitino.exceptions.GroupAlreadyExistsException; import com.datastrato.gravitino.exceptions.MetalakeAlreadyExistsException; import com.datastrato.gravitino.exceptions.NoSuchMetalakeException; import com.datastrato.gravitino.exceptions.NonEmptySchemaException; @@ -62,6 +63,11 @@ public static Response handleUserException( return UserExceptionHandler.INSTANCE.handle(op, user, metalake, e); } + public static Response handleGroupException( + OperationType op, String group, String metalake, Exception e) { + return GroupExceptionHandler.INSTANCE.handle(op, group, metalake, e); + } + public static Response handleTopicException( OperationType op, String topic, String schema, Exception e) { return TopicExceptionHandler.INSTANCE.handle(op, topic, schema, e); @@ -271,10 +277,10 @@ private static class UserExceptionHandler extends BaseExceptionHandler { private static final ExceptionHandler INSTANCE = new UserExceptionHandler(); private static String getUserErrorMsg( - String fileset, String operation, String metalake, String reason) { + String user, String operation, String metalake, String reason) { return String.format( "Failed to operate user %s operation [%s] under metalake [%s], reason [%s]", - fileset, operation, metalake, reason); + user, operation, metalake, reason); } @Override @@ -298,6 +304,38 @@ public Response handle(OperationType op, String user, String metalake, Exception } } + private static class GroupExceptionHandler extends BaseExceptionHandler { + + private static final ExceptionHandler INSTANCE = new GroupExceptionHandler(); + + private static String getGroupErrorMsg( + String group, String operation, String metalake, String reason) { + return String.format( + "Failed to operate group %s operation [%s] under metalake [%s], reason [%s]", + group, operation, metalake, reason); + } + + @Override + public Response handle(OperationType op, String group, String metalake, Exception e) { + String formatted = StringUtil.isBlank(group) ? "" : " [" + group + "]"; + String errorMsg = getGroupErrorMsg(formatted, op.name(), metalake, getErrorMsg(e)); + LOG.warn(errorMsg, e); + + if (e instanceof IllegalArgumentException) { + return Utils.illegalArguments(errorMsg, e); + + } else if (e instanceof NotFoundException) { + return Utils.notFound(errorMsg, e); + + } else if (e instanceof GroupAlreadyExistsException) { + return Utils.alreadyExists(errorMsg, e); + + } else { + return super.handle(op, group, metalake, e); + } + } + } + private static class TopicExceptionHandler extends BaseExceptionHandler { private static final ExceptionHandler INSTANCE = new TopicExceptionHandler(); diff --git a/server/src/main/java/com/datastrato/gravitino/server/web/rest/GroupOperations.java b/server/src/main/java/com/datastrato/gravitino/server/web/rest/GroupOperations.java new file mode 100644 index 00000000000..692d7bf31c9 --- /dev/null +++ b/server/src/main/java/com/datastrato/gravitino/server/web/rest/GroupOperations.java @@ -0,0 +1,124 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.server.web.rest; + +import com.codahale.metrics.annotation.ResponseMetered; +import com.codahale.metrics.annotation.Timed; +import com.datastrato.gravitino.NameIdentifier; +import com.datastrato.gravitino.authorization.AccessControlManager; +import com.datastrato.gravitino.dto.requests.GroupAddRequest; +import com.datastrato.gravitino.dto.responses.GroupResponse; +import com.datastrato.gravitino.dto.responses.RemoveResponse; +import com.datastrato.gravitino.dto.util.DTOConverters; +import com.datastrato.gravitino.lock.LockType; +import com.datastrato.gravitino.lock.TreeLockUtils; +import com.datastrato.gravitino.meta.CatalogEntity; +import com.datastrato.gravitino.meta.GroupEntity; +import com.datastrato.gravitino.metrics.MetricNames; +import com.datastrato.gravitino.server.web.Utils; +import javax.inject.Inject; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.DELETE; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.Response; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Path("/metalakes/{metalake}/groups") +public class GroupOperations { + + private static final Logger LOG = LoggerFactory.getLogger(GroupOperations.class); + + private final AccessControlManager accessControlManager; + + @Context private HttpServletRequest httpRequest; + + @Inject + public GroupOperations(AccessControlManager accessControlManager) { + this.accessControlManager = accessControlManager; + } + + @GET + @Path("{group}") + @Produces("application/vnd.gravitino.v1+json") + @Timed(name = "get-group." + MetricNames.HTTP_PROCESS_DURATION, absolute = true) + @ResponseMetered(name = "get-group", absolute = true) + public Response getGroup( + @PathParam("metalake") String metalake, @PathParam("group") String group) { + try { + NameIdentifier ident = ofGroup(metalake, group); + return Utils.doAs( + httpRequest, + () -> + Utils.ok( + new GroupResponse( + DTOConverters.toDTO( + TreeLockUtils.doWithTreeLock( + ident, + LockType.READ, + () -> accessControlManager.getGroup(metalake, group)))))); + } catch (Exception e) { + return ExceptionHandlers.handleGroupException(OperationType.GET, group, metalake, e); + } + } + + @POST + @Produces("application/vnd.gravitino.v1+json") + @Timed(name = "add-group." + MetricNames.HTTP_PROCESS_DURATION, absolute = true) + @ResponseMetered(name = "add-group", absolute = true) + public Response addGroup(@PathParam("metalake") String metalake, GroupAddRequest request) { + try { + NameIdentifier ident = ofGroup(metalake, request.getName()); + return Utils.doAs( + httpRequest, + () -> + Utils.ok( + new GroupResponse( + DTOConverters.toDTO( + TreeLockUtils.doWithTreeLock( + ident, + LockType.WRITE, + () -> accessControlManager.addGroup(metalake, request.getName())))))); + } catch (Exception e) { + return ExceptionHandlers.handleGroupException( + OperationType.ADD, request.getName(), metalake, e); + } + } + + @DELETE + @Path("{group}") + @Produces("application/vnd.gravitino.v1+json") + @Timed(name = "remove-group." + MetricNames.HTTP_PROCESS_DURATION, absolute = true) + @ResponseMetered(name = "remove-group", absolute = true) + public Response removeGroup( + @PathParam("metalake") String metalake, @PathParam("group") String group) { + try { + return Utils.doAs( + httpRequest, + () -> { + NameIdentifier ident = ofGroup(metalake, group); + boolean removed = + TreeLockUtils.doWithTreeLock( + ident, LockType.WRITE, () -> accessControlManager.removeGroup(metalake, group)); + if (!removed) { + LOG.warn("Failed to remove group {} under metalake {}", group, metalake); + } + return Utils.ok(new RemoveResponse(removed)); + }); + } catch (Exception e) { + return ExceptionHandlers.handleGroupException(OperationType.REMOVE, group, metalake, e); + } + } + + private NameIdentifier ofGroup(String metalake, String group) { + return NameIdentifier.of( + metalake, CatalogEntity.SYSTEM_CATALOG_RESERVED_NAME, GroupEntity.GROUP_SCHEMA_NAME, group); + } +} diff --git a/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestGroupOperations.java b/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestGroupOperations.java new file mode 100644 index 00000000000..d7d175236b4 --- /dev/null +++ b/server/src/test/java/com/datastrato/gravitino/server/web/rest/TestGroupOperations.java @@ -0,0 +1,281 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.server.web.rest; + +import static com.datastrato.gravitino.Configs.TREE_LOCK_CLEAN_INTERVAL; +import static com.datastrato.gravitino.Configs.TREE_LOCK_MAX_NODE_IN_MEMORY; +import static com.datastrato.gravitino.Configs.TREE_LOCK_MIN_NODE_IN_MEMORY; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.datastrato.gravitino.Config; +import com.datastrato.gravitino.GravitinoEnv; +import com.datastrato.gravitino.authorization.AccessControlManager; +import com.datastrato.gravitino.authorization.Group; +import com.datastrato.gravitino.dto.authorization.GroupDTO; +import com.datastrato.gravitino.dto.requests.GroupAddRequest; +import com.datastrato.gravitino.dto.responses.ErrorConstants; +import com.datastrato.gravitino.dto.responses.ErrorResponse; +import com.datastrato.gravitino.dto.responses.GroupResponse; +import com.datastrato.gravitino.dto.responses.RemoveResponse; +import com.datastrato.gravitino.exceptions.GroupAlreadyExistsException; +import com.datastrato.gravitino.exceptions.NoSuchGroupException; +import com.datastrato.gravitino.exceptions.NoSuchMetalakeException; +import com.datastrato.gravitino.lock.LockManager; +import com.datastrato.gravitino.meta.AuditInfo; +import com.datastrato.gravitino.meta.GroupEntity; +import com.datastrato.gravitino.rest.RESTUtils; +import java.io.IOException; +import java.time.Instant; +import java.util.Collections; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.client.Entity; +import javax.ws.rs.core.Application; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import org.glassfish.hk2.utilities.binding.AbstractBinder; +import org.glassfish.jersey.server.ResourceConfig; +import org.glassfish.jersey.test.JerseyTest; +import org.glassfish.jersey.test.TestProperties; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +public class TestGroupOperations extends JerseyTest { + + private final AccessControlManager manager = mock(AccessControlManager.class); + + private static class MockServletRequestFactory extends ServletRequestFactoryBase { + @Override + public HttpServletRequest get() { + HttpServletRequest request = mock(HttpServletRequest.class); + when(request.getRemoteUser()).thenReturn(null); + return request; + } + } + + @BeforeAll + public static void setup() { + Config config = mock(Config.class); + Mockito.doReturn(100000L).when(config).get(TREE_LOCK_MAX_NODE_IN_MEMORY); + Mockito.doReturn(1000L).when(config).get(TREE_LOCK_MIN_NODE_IN_MEMORY); + Mockito.doReturn(36000L).when(config).get(TREE_LOCK_CLEAN_INTERVAL); + GravitinoEnv.getInstance().setLockManager(new LockManager(config)); + } + + @Override + protected Application configure() { + try { + forceSet( + TestProperties.CONTAINER_PORT, String.valueOf(RESTUtils.findAvailablePort(2000, 3000))); + } catch (IOException e) { + throw new RuntimeException(e); + } + + ResourceConfig resourceConfig = new ResourceConfig(); + resourceConfig.register(GroupOperations.class); + resourceConfig.register( + new AbstractBinder() { + @Override + protected void configure() { + bind(manager).to(AccessControlManager.class).ranked(2); + bindFactory(MockServletRequestFactory.class).to(HttpServletRequest.class); + } + }); + + return resourceConfig; + } + + @Test + public void testAddGroup() { + GroupAddRequest req = new GroupAddRequest("group1"); + Group group = buildGroup("group1"); + + when(manager.addGroup(any(), any())).thenReturn(group); + + Response resp = + target("/metalakes/metalake1/groups") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .post(Entity.entity(req, MediaType.APPLICATION_JSON_TYPE)); + + Assertions.assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + Assertions.assertEquals(MediaType.APPLICATION_JSON_TYPE, resp.getMediaType()); + + GroupResponse groupResponse = resp.readEntity(GroupResponse.class); + Assertions.assertEquals(0, groupResponse.getCode()); + + GroupDTO groupDTO = groupResponse.getGroup(); + Assertions.assertEquals("group1", groupDTO.name()); + Assertions.assertNotNull(groupDTO.roles()); + Assertions.assertTrue(groupDTO.roles().isEmpty()); + + // Test to throw NoSuchMetalakeException + doThrow(new NoSuchMetalakeException("mock error")).when(manager).addGroup(any(), any()); + Response resp1 = + target("/metalakes/metalake1/groups") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .post(Entity.entity(req, MediaType.APPLICATION_JSON_TYPE)); + + Assertions.assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp1.getStatus()); + Assertions.assertEquals(MediaType.APPLICATION_JSON_TYPE, resp1.getMediaType()); + + ErrorResponse errorResponse = resp1.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.NOT_FOUND_CODE, errorResponse.getCode()); + Assertions.assertEquals(NoSuchMetalakeException.class.getSimpleName(), errorResponse.getType()); + + // Test to throw GroupAlreadyExistsException + doThrow(new GroupAlreadyExistsException("mock error")).when(manager).addGroup(any(), any()); + Response resp2 = + target("/metalakes/metalake1/groups") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .post(Entity.entity(req, MediaType.APPLICATION_JSON_TYPE)); + + Assertions.assertEquals(Response.Status.CONFLICT.getStatusCode(), resp2.getStatus()); + + ErrorResponse errorResponse1 = resp2.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.ALREADY_EXISTS_CODE, errorResponse1.getCode()); + Assertions.assertEquals( + GroupAlreadyExistsException.class.getSimpleName(), errorResponse1.getType()); + + // Test to throw internal RuntimeException + doThrow(new RuntimeException("mock error")).when(manager).addGroup(any(), any()); + Response resp3 = + target("/metalakes/metalake1/groups") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .post(Entity.entity(req, MediaType.APPLICATION_JSON_TYPE)); + + Assertions.assertEquals( + Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), resp3.getStatus()); + + ErrorResponse errorResponse2 = resp3.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.INTERNAL_ERROR_CODE, errorResponse2.getCode()); + Assertions.assertEquals(RuntimeException.class.getSimpleName(), errorResponse2.getType()); + } + + @Test + public void testGetGroup() { + Group group = buildGroup("group1"); + + when(manager.getGroup(any(), any())).thenReturn(group); + + Response resp = + target("/metalakes/metalake1/groups/group1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .get(); + + Assertions.assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + GroupResponse groupResponse = resp.readEntity(GroupResponse.class); + Assertions.assertEquals(0, groupResponse.getCode()); + GroupDTO groupDTO = groupResponse.getGroup(); + Assertions.assertEquals("group1", groupDTO.name()); + Assertions.assertNotNull(groupDTO.roles()); + Assertions.assertTrue(groupDTO.roles().isEmpty()); + + // Test to throw NoSuchMetalakeException + doThrow(new NoSuchMetalakeException("mock error")).when(manager).getGroup(any(), any()); + Response resp1 = + target("/metalakes/metalake1/groups/group1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .get(); + + Assertions.assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp1.getStatus()); + + ErrorResponse errorResponse = resp1.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.NOT_FOUND_CODE, errorResponse.getCode()); + Assertions.assertEquals(NoSuchMetalakeException.class.getSimpleName(), errorResponse.getType()); + + // Test to throw NoSuchGroupException + doThrow(new NoSuchGroupException("mock error")).when(manager).getGroup(any(), any()); + Response resp2 = + target("/metalakes/metalake1/groups/group1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .get(); + + Assertions.assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp2.getStatus()); + + ErrorResponse errorResponse1 = resp2.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.NOT_FOUND_CODE, errorResponse1.getCode()); + Assertions.assertEquals(NoSuchGroupException.class.getSimpleName(), errorResponse1.getType()); + + // Test to throw internal RuntimeException + doThrow(new RuntimeException("mock error")).when(manager).getGroup(any(), any()); + Response resp3 = + target("/metalakes/metalake1/groups/group1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .get(); + + Assertions.assertEquals( + Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), resp3.getStatus()); + + ErrorResponse errorResponse2 = resp3.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.INTERNAL_ERROR_CODE, errorResponse2.getCode()); + Assertions.assertEquals(RuntimeException.class.getSimpleName(), errorResponse2.getType()); + } + + private Group buildGroup(String group) { + return GroupEntity.builder() + .withId(1L) + .withName(group) + .withRoles(Collections.emptyList()) + .withAuditInfo( + AuditInfo.builder().withCreator("creator").withCreateTime(Instant.now()).build()) + .build(); + } + + @Test + public void testRemoveGroup() { + when(manager.removeGroup(any(), any())).thenReturn(true); + + Response resp = + target("/metalakes/metalake1/groups/group1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .delete(); + + Assertions.assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + RemoveResponse removeResponse = resp.readEntity(RemoveResponse.class); + Assertions.assertEquals(0, removeResponse.getCode()); + Assertions.assertTrue(removeResponse.removed()); + + // Test when failed to remove group + when(manager.removeGroup(any(), any())).thenReturn(false); + Response resp2 = + target("/metalakes/metalake1/groups/group1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .delete(); + + Assertions.assertEquals(Response.Status.OK.getStatusCode(), resp2.getStatus()); + RemoveResponse removeResponse2 = resp2.readEntity(RemoveResponse.class); + Assertions.assertEquals(0, removeResponse2.getCode()); + Assertions.assertFalse(removeResponse2.removed()); + + doThrow(new RuntimeException("mock error")).when(manager).removeGroup(any(), any()); + Response resp3 = + target("/metalakes/metalake1/groups/group1") + .request(MediaType.APPLICATION_JSON_TYPE) + .accept("application/vnd.gravitino.v1+json") + .delete(); + + Assertions.assertEquals( + Response.Status.INTERNAL_SERVER_ERROR.getStatusCode(), resp3.getStatus()); + + ErrorResponse errorResponse = resp3.readEntity(ErrorResponse.class); + Assertions.assertEquals(ErrorConstants.INTERNAL_ERROR_CODE, errorResponse.getCode()); + Assertions.assertEquals(RuntimeException.class.getSimpleName(), errorResponse.getType()); + } +} From 4d423a7e41f027006d0afac8dc993114bed9dc9a Mon Sep 17 00:00:00 2001 From: mchades Date: Mon, 1 Apr 2024 22:34:43 +0800 Subject: [PATCH 16/20] [#2612] feat(client): Add Java client support for messaging catalog (#2722) ### What changes were proposed in this pull request? Add Java client support for the messaging catalog ### Why are the changes needed? Fix: #2612 ### Does this PR introduce _any_ user-facing change? Add Java client support for the messaging catalog ### How was this patch tested? UTs --- .../com/datastrato/gravitino/Catalog.java | 5 +- .../datastrato/gravitino/NameIdentifier.java | 11 + .../com/datastrato/gravitino/Namespace.java | 13 + .../gravitino/client/DTOConverters.java | 28 ++ .../gravitino/client/ErrorHandlers.java | 46 +++ .../gravitino/client/FilesetCatalog.java | 1 + .../gravitino/client/MessagingCatalog.java | 226 +++++++++++++ .../client/TestGravitinoMetalake.java | 4 +- .../client/TestMessagingCatalog.java | 310 ++++++++++++++++++ 9 files changed, 641 insertions(+), 3 deletions(-) create mode 100644 clients/client-java/src/main/java/com/datastrato/gravitino/client/MessagingCatalog.java create mode 100644 clients/client-java/src/test/java/com/datastrato/gravitino/client/TestMessagingCatalog.java diff --git a/api/src/main/java/com/datastrato/gravitino/Catalog.java b/api/src/main/java/com/datastrato/gravitino/Catalog.java index e9339c6a6c3..8ebeaf703e0 100644 --- a/api/src/main/java/com/datastrato/gravitino/Catalog.java +++ b/api/src/main/java/com/datastrato/gravitino/Catalog.java @@ -27,7 +27,10 @@ enum Type { FILESET, /** Catalog Type for Message Queue, like kafka://topic */ - MESSAGING + MESSAGING, + + /** Catalog Type for test only. */ + UNSUPPORTED } /** diff --git a/api/src/main/java/com/datastrato/gravitino/NameIdentifier.java b/api/src/main/java/com/datastrato/gravitino/NameIdentifier.java index 3d9389bb3ed..b3c40bbdbcf 100644 --- a/api/src/main/java/com/datastrato/gravitino/NameIdentifier.java +++ b/api/src/main/java/com/datastrato/gravitino/NameIdentifier.java @@ -185,6 +185,17 @@ public static void checkFileset(NameIdentifier ident) { Namespace.checkFileset(ident.namespace); } + /** + * Check the given {@link NameIdentifier} is a topic identifier. Throw an {@link + * IllegalNameIdentifierException} if it's not. + * + * @param ident The topic {@link NameIdentifier} to check. + */ + public static void checkTopic(NameIdentifier ident) { + check(ident != null, "Topic identifier must not be null"); + Namespace.checkTopic(ident.namespace); + } + /** * Create a {@link NameIdentifier} from the given identifier string. * diff --git a/api/src/main/java/com/datastrato/gravitino/Namespace.java b/api/src/main/java/com/datastrato/gravitino/Namespace.java index 3a38025cf7c..ae469ff3d4c 100644 --- a/api/src/main/java/com/datastrato/gravitino/Namespace.java +++ b/api/src/main/java/com/datastrato/gravitino/Namespace.java @@ -182,6 +182,19 @@ public static void checkFileset(Namespace namespace) { namespace); } + /** + * Check if the given topic namespace is legal, throw an {@link IllegalNamespaceException} if it's + * illegal. + * + * @param namespace The topic namespace + */ + public static void checkTopic(Namespace namespace) { + check( + namespace != null && namespace.length() == 3, + "Topic namespace must be non-null and have 3 levels, the input namespace is %s", + namespace); + } + private Namespace(String[] levels) { this.levels = levels; } diff --git a/clients/client-java/src/main/java/com/datastrato/gravitino/client/DTOConverters.java b/clients/client-java/src/main/java/com/datastrato/gravitino/client/DTOConverters.java index b040357aa8a..5fdf05889de 100644 --- a/clients/client-java/src/main/java/com/datastrato/gravitino/client/DTOConverters.java +++ b/clients/client-java/src/main/java/com/datastrato/gravitino/client/DTOConverters.java @@ -15,7 +15,9 @@ import com.datastrato.gravitino.dto.requests.MetalakeUpdateRequest; import com.datastrato.gravitino.dto.requests.SchemaUpdateRequest; import com.datastrato.gravitino.dto.requests.TableUpdateRequest; +import com.datastrato.gravitino.dto.requests.TopicUpdateRequest; import com.datastrato.gravitino.file.FilesetChange; +import com.datastrato.gravitino.messaging.TopicChange; import com.datastrato.gravitino.rel.SchemaChange; import com.datastrato.gravitino.rel.TableChange; @@ -82,6 +84,15 @@ static Catalog toCatalog(CatalogDTO catalog, RESTClient client) { .build(); case MESSAGING: + return MessagingCatalog.builder() + .withName(catalog.name()) + .withType(catalog.type()) + .withProvider(catalog.provider()) + .withComment(catalog.comment()) + .withProperties(catalog.properties()) + .withAudit((AuditDTO) catalog.auditInfo()) + .withRestClient(client) + .build(); default: throw new UnsupportedOperationException("Unsupported catalog type: " + catalog.type()); } @@ -183,6 +194,23 @@ static FilesetUpdateRequest toFilesetUpdateRequest(FilesetChange change) { } } + static TopicUpdateRequest toTopicUpdateRequest(TopicChange change) { + if (change instanceof TopicChange.UpdateTopicComment) { + return new TopicUpdateRequest.UpdateTopicCommentRequest( + ((TopicChange.UpdateTopicComment) change).getNewComment()); + } else if (change instanceof TopicChange.SetProperty) { + return new TopicUpdateRequest.SetTopicPropertyRequest( + ((TopicChange.SetProperty) change).getProperty(), + ((TopicChange.SetProperty) change).getValue()); + } else if (change instanceof TopicChange.RemoveProperty) { + return new TopicUpdateRequest.RemoveTopicPropertyRequest( + ((TopicChange.RemoveProperty) change).getProperty()); + } else { + throw new IllegalArgumentException( + "Unknown change type: " + change.getClass().getSimpleName()); + } + } + private static TableUpdateRequest toColumnUpdateRequest(TableChange.ColumnChange change) { if (change instanceof TableChange.AddColumn) { TableChange.AddColumn addColumn = (TableChange.AddColumn) change; diff --git a/clients/client-java/src/main/java/com/datastrato/gravitino/client/ErrorHandlers.java b/clients/client-java/src/main/java/com/datastrato/gravitino/client/ErrorHandlers.java index 60b1ec49bc2..61bac3470d7 100644 --- a/clients/client-java/src/main/java/com/datastrato/gravitino/client/ErrorHandlers.java +++ b/clients/client-java/src/main/java/com/datastrato/gravitino/client/ErrorHandlers.java @@ -17,12 +17,14 @@ import com.datastrato.gravitino.exceptions.NoSuchPartitionException; import com.datastrato.gravitino.exceptions.NoSuchSchemaException; import com.datastrato.gravitino.exceptions.NoSuchTableException; +import com.datastrato.gravitino.exceptions.NoSuchTopicException; import com.datastrato.gravitino.exceptions.NonEmptySchemaException; import com.datastrato.gravitino.exceptions.NotFoundException; import com.datastrato.gravitino.exceptions.PartitionAlreadyExistsException; import com.datastrato.gravitino.exceptions.RESTException; import com.datastrato.gravitino.exceptions.SchemaAlreadyExistsException; import com.datastrato.gravitino.exceptions.TableAlreadyExistsException; +import com.datastrato.gravitino.exceptions.TopicAlreadyExistsException; import com.datastrato.gravitino.exceptions.UnauthorizedException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; @@ -112,6 +114,15 @@ public static Consumer filesetErrorHandler() { return FilesetErrorHandler.INSTANCE; } + /** + * Creates an error handler specific to Topic operations. + * + * @return A Consumer representing the Topic error handler. + */ + public static Consumer topicErrorHandler() { + return TopicErrorHandler.INSTANCE; + } + private ErrorHandlers() {} /** @@ -410,6 +421,41 @@ public void accept(ErrorResponse errorResponse) { } } + /** Error handler specific to Topic operations. */ + @SuppressWarnings("FormatStringAnnotation") + private static class TopicErrorHandler extends RestErrorHandler { + + private static final TopicErrorHandler INSTANCE = new TopicErrorHandler(); + + @Override + public void accept(ErrorResponse errorResponse) { + String errorMessage = formatErrorMessage(errorResponse); + + switch (errorResponse.getCode()) { + case ErrorConstants.ILLEGAL_ARGUMENTS_CODE: + throw new IllegalArgumentException(errorMessage); + + case ErrorConstants.NOT_FOUND_CODE: + if (errorResponse.getType().equals(NoSuchSchemaException.class.getSimpleName())) { + throw new NoSuchSchemaException(errorMessage); + } else if (errorResponse.getType().equals(NoSuchTopicException.class.getSimpleName())) { + throw new NoSuchTopicException(errorMessage); + } else { + throw new NotFoundException(errorMessage); + } + + case ErrorConstants.ALREADY_EXISTS_CODE: + throw new TopicAlreadyExistsException(errorMessage); + + case ErrorConstants.INTERNAL_ERROR_CODE: + throw new RuntimeException(errorMessage); + + default: + super.accept(errorResponse); + } + } + } + /** Generic error handler for REST requests. */ private static class RestErrorHandler extends ErrorHandler { private static final ErrorHandler INSTANCE = new RestErrorHandler(); diff --git a/clients/client-java/src/main/java/com/datastrato/gravitino/client/FilesetCatalog.java b/clients/client-java/src/main/java/com/datastrato/gravitino/client/FilesetCatalog.java index 9ae5b1fe04b..ff95b79e086 100644 --- a/clients/client-java/src/main/java/com/datastrato/gravitino/client/FilesetCatalog.java +++ b/clients/client-java/src/main/java/com/datastrato/gravitino/client/FilesetCatalog.java @@ -164,6 +164,7 @@ public Fileset alterFileset(NameIdentifier ident, FilesetChange... changes) .map(DTOConverters::toFilesetUpdateRequest) .collect(Collectors.toList()); FilesetUpdatesRequest req = new FilesetUpdatesRequest(updates); + req.validate(); FilesetResponse resp = restClient.put( diff --git a/clients/client-java/src/main/java/com/datastrato/gravitino/client/MessagingCatalog.java b/clients/client-java/src/main/java/com/datastrato/gravitino/client/MessagingCatalog.java new file mode 100644 index 00000000000..cd8f83781a0 --- /dev/null +++ b/clients/client-java/src/main/java/com/datastrato/gravitino/client/MessagingCatalog.java @@ -0,0 +1,226 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.client; + +import com.datastrato.gravitino.NameIdentifier; +import com.datastrato.gravitino.Namespace; +import com.datastrato.gravitino.dto.AuditDTO; +import com.datastrato.gravitino.dto.CatalogDTO; +import com.datastrato.gravitino.dto.requests.TopicCreateRequest; +import com.datastrato.gravitino.dto.requests.TopicUpdateRequest; +import com.datastrato.gravitino.dto.requests.TopicUpdatesRequest; +import com.datastrato.gravitino.dto.responses.DropResponse; +import com.datastrato.gravitino.dto.responses.EntityListResponse; +import com.datastrato.gravitino.dto.responses.TopicResponse; +import com.datastrato.gravitino.exceptions.NoSuchSchemaException; +import com.datastrato.gravitino.exceptions.NoSuchTopicException; +import com.datastrato.gravitino.exceptions.TopicAlreadyExistsException; +import com.datastrato.gravitino.messaging.DataLayout; +import com.datastrato.gravitino.messaging.Topic; +import com.datastrato.gravitino.messaging.TopicCatalog; +import com.datastrato.gravitino.messaging.TopicChange; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; + +/** + * Messaging catalog is a catalog implementation that supports messaging-like metadata operations, + * for example, topics list, creation, update and deletion. A Messaging catalog is under the + * metalake. + */ +public class MessagingCatalog extends BaseSchemaCatalog implements TopicCatalog { + + MessagingCatalog( + String name, + Type type, + String provider, + String comment, + Map properties, + AuditDTO auditDTO, + RESTClient restClient) { + super(name, type, provider, comment, properties, auditDTO, restClient); + } + + /** @return A new builder for {@link MessagingCatalog}. */ + public static Builder builder() { + return new Builder(); + } + + @Override + public TopicCatalog asTopicCatalog() throws UnsupportedOperationException { + return this; + } + + /** + * List all the topics under the given namespace. + * + * @param namespace The namespace to list the topics under it. + * @return An array of {@link NameIdentifier} of the topics under the specified namespace. + * @throws NoSuchSchemaException if the schema with specified namespace does not exist. + */ + @Override + public NameIdentifier[] listTopics(Namespace namespace) throws NoSuchSchemaException { + Namespace.checkTopic(namespace); + + EntityListResponse resp = + restClient.get( + formatTopicRequestPath(namespace), + EntityListResponse.class, + Collections.emptyMap(), + ErrorHandlers.topicErrorHandler()); + + resp.validate(); + + return resp.identifiers(); + } + + /** + * Load the topic with the given identifier. + * + * @param ident The identifier of the topic to load. + * @return The {@link Topic} with the specified identifier. + * @throws NoSuchTopicException if the topic with the specified identifier does not exist. + */ + @Override + public Topic loadTopic(NameIdentifier ident) throws NoSuchTopicException { + NameIdentifier.checkTopic(ident); + + TopicResponse resp = + restClient.get( + formatTopicRequestPath(ident.namespace()) + "/" + ident.name(), + TopicResponse.class, + Collections.emptyMap(), + ErrorHandlers.topicErrorHandler()); + resp.validate(); + + return resp.getTopic(); + } + + /** + * Create a new topic with the given identifier, comment, data layout and properties. + * + * @param ident A topic identifier. + * @param comment The comment of the topic object. Null is set if no comment is specified. + * @param dataLayout The message schema of the topic object. Always null because it's not + * supported yet. + * @param properties The properties of the topic object. Empty map is set if no properties are + * specified. + * @return The created topic object. + * @throws NoSuchSchemaException if the schema with specified namespace does not exist. + * @throws TopicAlreadyExistsException if the topic with specified identifier already exists. + */ + @Override + public Topic createTopic( + NameIdentifier ident, String comment, DataLayout dataLayout, Map properties) + throws NoSuchSchemaException, TopicAlreadyExistsException { + NameIdentifier.checkTopic(ident); + + TopicCreateRequest req = + TopicCreateRequest.builder() + .name(ident.name()) + .comment(comment) + .properties(properties) + .build(); + + TopicResponse resp = + restClient.post( + formatTopicRequestPath(ident.namespace()), + req, + TopicResponse.class, + Collections.emptyMap(), + ErrorHandlers.topicErrorHandler()); + resp.validate(); + + return resp.getTopic(); + } + + /** + * Alter the topic with the given identifier. + * + * @param ident A topic identifier. + * @param changes The changes to apply to the topic. + * @return The altered topic object. + * @throws NoSuchTopicException if the topic with the specified identifier does not exist. + * @throws IllegalArgumentException if the changes are invalid. + */ + @Override + public Topic alterTopic(NameIdentifier ident, TopicChange... changes) + throws NoSuchTopicException, IllegalArgumentException { + NameIdentifier.checkTopic(ident); + + List updates = + Arrays.stream(changes) + .map(DTOConverters::toTopicUpdateRequest) + .collect(Collectors.toList()); + TopicUpdatesRequest updatesRequest = new TopicUpdatesRequest(updates); + updatesRequest.validate(); + + TopicResponse resp = + restClient.put( + formatTopicRequestPath(ident.namespace()) + "/" + ident.name(), + updatesRequest, + TopicResponse.class, + Collections.emptyMap(), + ErrorHandlers.topicErrorHandler()); + resp.validate(); + + return resp.getTopic(); + } + + /** + * Drop the topic with the given identifier. + * + * @param ident A topic identifier. + * @return True if the topic is dropped successfully, false the topic does not exist. + */ + @Override + public boolean dropTopic(NameIdentifier ident) { + NameIdentifier.checkTopic(ident); + + DropResponse resp = + restClient.delete( + formatTopicRequestPath(ident.namespace()) + "/" + ident.name(), + DropResponse.class, + Collections.emptyMap(), + ErrorHandlers.topicErrorHandler()); + resp.validate(); + + return resp.dropped(); + } + + @VisibleForTesting + static String formatTopicRequestPath(Namespace ns) { + Namespace schemaNs = Namespace.of(ns.level(0), ns.level(1)); + return formatSchemaRequestPath(schemaNs) + "/" + ns.level(2) + "/topics"; + } + + static class Builder extends CatalogDTO.Builder { + /** The REST client to send the requests. */ + private RESTClient restClient; + + private Builder() {} + + Builder withRestClient(RESTClient restClient) { + this.restClient = restClient; + return this; + } + + @Override + public MessagingCatalog build() { + Preconditions.checkArgument(StringUtils.isNotBlank(name), "name must not be blank"); + Preconditions.checkArgument(type != null, "type must not be null"); + Preconditions.checkArgument(StringUtils.isNotBlank(provider), "provider must not be blank"); + Preconditions.checkArgument(audit != null, "audit must not be null"); + Preconditions.checkArgument(restClient != null, "restClient must be set"); + + return new MessagingCatalog(name, type, provider, comment, properties, audit, restClient); + } + } +} diff --git a/clients/client-java/src/test/java/com/datastrato/gravitino/client/TestGravitinoMetalake.java b/clients/client-java/src/test/java/com/datastrato/gravitino/client/TestGravitinoMetalake.java index 46d5f1c6128..b8f109736a8 100644 --- a/clients/client-java/src/test/java/com/datastrato/gravitino/client/TestGravitinoMetalake.java +++ b/clients/client-java/src/test/java/com/datastrato/gravitino/client/TestGravitinoMetalake.java @@ -190,7 +190,7 @@ public void testLoadCatalog() throws JsonProcessingException { CatalogDTO.builder() .withName("mock") .withComment("comment") - .withType(Catalog.Type.MESSAGING) + .withType(Catalog.Type.UNSUPPORTED) .withProvider("test") .withAudit( AuditDTO.builder().withCreator("creator").withCreateTime(Instant.now()).build()) @@ -250,7 +250,7 @@ public void testCreateCatalog() throws JsonProcessingException { CatalogDTO.builder() .withName("mock") .withComment("comment") - .withType(Catalog.Type.MESSAGING) + .withType(Catalog.Type.UNSUPPORTED) .withProvider("test") .withAudit( AuditDTO.builder().withCreator("creator").withCreateTime(Instant.now()).build()) diff --git a/clients/client-java/src/test/java/com/datastrato/gravitino/client/TestMessagingCatalog.java b/clients/client-java/src/test/java/com/datastrato/gravitino/client/TestMessagingCatalog.java new file mode 100644 index 00000000000..89b8b4050ce --- /dev/null +++ b/clients/client-java/src/test/java/com/datastrato/gravitino/client/TestMessagingCatalog.java @@ -0,0 +1,310 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.client; + +import static org.apache.hc.core5.http.HttpStatus.SC_CONFLICT; +import static org.apache.hc.core5.http.HttpStatus.SC_NOT_FOUND; +import static org.apache.hc.core5.http.HttpStatus.SC_OK; +import static org.apache.hc.core5.http.HttpStatus.SC_SERVER_ERROR; + +import com.datastrato.gravitino.Catalog; +import com.datastrato.gravitino.NameIdentifier; +import com.datastrato.gravitino.dto.AuditDTO; +import com.datastrato.gravitino.dto.CatalogDTO; +import com.datastrato.gravitino.dto.messaging.TopicDTO; +import com.datastrato.gravitino.dto.requests.CatalogCreateRequest; +import com.datastrato.gravitino.dto.requests.TopicCreateRequest; +import com.datastrato.gravitino.dto.requests.TopicUpdateRequest; +import com.datastrato.gravitino.dto.requests.TopicUpdatesRequest; +import com.datastrato.gravitino.dto.responses.CatalogResponse; +import com.datastrato.gravitino.dto.responses.DropResponse; +import com.datastrato.gravitino.dto.responses.EntityListResponse; +import com.datastrato.gravitino.dto.responses.ErrorResponse; +import com.datastrato.gravitino.dto.responses.TopicResponse; +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.fasterxml.jackson.core.JsonProcessingException; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import java.time.Instant; +import org.apache.hc.core5.http.Method; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +public class TestMessagingCatalog extends TestBase { + protected static Catalog catalog; + + private static GravitinoMetalake metalake; + + protected static final String metalakeName = "testMetalake"; + + protected static final String catalogName = "testMessagingCatalog"; + + private static final String provider = "test"; + + @BeforeAll + public static void setUp() throws Exception { + TestBase.setUp(); + + metalake = TestGravitinoMetalake.createMetalake(client, metalakeName); + CatalogDTO mockCatalog = + CatalogDTO.builder() + .withName(catalogName) + .withType(CatalogDTO.Type.MESSAGING) + .withProvider(provider) + .withComment("comment") + .withProperties(ImmutableMap.of("k1", "k2")) + .withAudit( + AuditDTO.builder().withCreator("creator").withCreateTime(Instant.now()).build()) + .build(); + + CatalogCreateRequest catalogCreateRequest = + new CatalogCreateRequest( + catalogName, + CatalogDTO.Type.MESSAGING, + provider, + "comment", + ImmutableMap.of("k1", "k2")); + CatalogResponse catalogResponse = new CatalogResponse(mockCatalog); + buildMockResource( + Method.POST, + "/api/metalakes/" + metalakeName + "/catalogs", + catalogCreateRequest, + catalogResponse, + SC_OK); + + catalog = + metalake.createCatalog( + NameIdentifier.of(metalakeName, catalogName), + CatalogDTO.Type.MESSAGING, + provider, + "comment", + ImmutableMap.of("k1", "k2")); + } + + @Test + public void testListTopics() throws Exception { + NameIdentifier topic1 = NameIdentifier.of(metalakeName, catalogName, "schema1", "topic1"); + NameIdentifier topic2 = NameIdentifier.of(metalakeName, catalogName, "schema1", "topic2"); + String topicPath = withSlash(MessagingCatalog.formatTopicRequestPath(topic1.namespace())); + + EntityListResponse entityListResponse = + new EntityListResponse(new NameIdentifier[] {topic1, topic2}); + buildMockResource(Method.GET, topicPath, null, entityListResponse, SC_OK); + NameIdentifier[] topics = ((MessagingCatalog) catalog).listTopics(topic1.namespace()); + + Assertions.assertEquals(2, topics.length); + Assertions.assertEquals(topic1, topics[0]); + Assertions.assertEquals(topic2, topics[1]); + + // Throw schema not found exception + ErrorResponse errResp = + ErrorResponse.notFound(NoSuchSchemaException.class.getSimpleName(), "schema not found"); + buildMockResource(Method.GET, topicPath, null, errResp, SC_NOT_FOUND); + Assertions.assertThrows( + NoSuchSchemaException.class, + () -> catalog.asTopicCatalog().listTopics(topic1.namespace()), + "schema not found"); + + // Throw Runtime exception + ErrorResponse errResp2 = ErrorResponse.internalError("internal error"); + buildMockResource(Method.GET, topicPath, null, errResp2, SC_SERVER_ERROR); + Assertions.assertThrows( + RuntimeException.class, + () -> catalog.asTopicCatalog().listTopics(topic1.namespace()), + "internal error"); + } + + @Test + public void testLoadTopic() throws JsonProcessingException { + NameIdentifier topic = NameIdentifier.of(metalakeName, catalogName, "schema1", "topic1"); + String topicPath = + withSlash(MessagingCatalog.formatTopicRequestPath(topic.namespace()) + "/" + topic.name()); + + TopicDTO mockTopic = mockTopicDTO(topic.name(), "comment", ImmutableMap.of("k1", "k2")); + TopicResponse topicResponse = new TopicResponse(mockTopic); + buildMockResource(Method.GET, topicPath, null, topicResponse, SC_OK); + + Topic loadedTopic = catalog.asTopicCatalog().loadTopic(topic); + Assertions.assertNotNull(loadedTopic); + assertTopic(mockTopic, loadedTopic); + + // test NoSuchSchemaException + ErrorResponse errResp = + ErrorResponse.notFound(NoSuchSchemaException.class.getSimpleName(), "schema not found"); + buildMockResource(Method.GET, topicPath, null, errResp, SC_NOT_FOUND); + Exception ex = + Assertions.assertThrows( + NoSuchSchemaException.class, () -> catalog.asTopicCatalog().loadTopic(topic)); + Assertions.assertEquals("schema not found", ex.getMessage()); + } + + @Test + public void testCreateTopic() throws JsonProcessingException { + NameIdentifier topic = NameIdentifier.of(metalakeName, catalogName, "schema1", "topic1"); + String topicPath = withSlash(MessagingCatalog.formatTopicRequestPath(topic.namespace())); + + TopicDTO mockTopic = mockTopicDTO(topic.name(), "comment", ImmutableMap.of("k1", "k2")); + + TopicCreateRequest topicCreateRequest = + new TopicCreateRequest(topic.name(), "comment", ImmutableMap.of("k1", "k2")); + TopicResponse topicResponse = new TopicResponse(mockTopic); + buildMockResource(Method.POST, topicPath, topicCreateRequest, topicResponse, SC_OK); + + Topic createdTopic = + catalog.asTopicCatalog().createTopic(topic, "comment", null, ImmutableMap.of("k1", "k2")); + Assertions.assertNotNull(createdTopic); + assertTopic(mockTopic, createdTopic); + + // test NoSuchSchemaException + ErrorResponse errResp = + ErrorResponse.notFound(NoSuchSchemaException.class.getSimpleName(), "schema not found"); + buildMockResource(Method.POST, topicPath, topicCreateRequest, errResp, SC_NOT_FOUND); + Exception ex = + Assertions.assertThrows( + NoSuchSchemaException.class, + () -> + catalog + .asTopicCatalog() + .createTopic(topic, "comment", null, ImmutableMap.of("k1", "k2"))); + Assertions.assertEquals("schema not found", ex.getMessage()); + + // test TopicAlreadyExistsException + ErrorResponse errResp2 = + ErrorResponse.alreadyExists( + TopicAlreadyExistsException.class.getSimpleName(), "topic already exists"); + buildMockResource(Method.POST, topicPath, topicCreateRequest, errResp2, SC_CONFLICT); + + Exception ex2 = + Assertions.assertThrows( + TopicAlreadyExistsException.class, + () -> + catalog + .asTopicCatalog() + .createTopic(topic, "comment", null, ImmutableMap.of("k1", "k2"))); + Assertions.assertEquals("topic already exists", ex2.getMessage()); + + // test RuntimeException + ErrorResponse errResp3 = ErrorResponse.internalError("internal error"); + buildMockResource(Method.POST, topicPath, topicCreateRequest, errResp3, SC_SERVER_ERROR); + Exception ex3 = + Assertions.assertThrows( + RuntimeException.class, + () -> + catalog + .asTopicCatalog() + .createTopic(topic, "comment", null, ImmutableMap.of("k1", "k2"))); + Assertions.assertEquals("internal error", ex3.getMessage()); + } + + @Test + public void testAlterTopic() throws JsonProcessingException { + NameIdentifier topic = NameIdentifier.of(metalakeName, catalogName, "schema1", "topic1"); + String topicPath = + withSlash(MessagingCatalog.formatTopicRequestPath(topic.namespace()) + "/" + topic.name()); + + // test alter topic comment + TopicUpdateRequest req1 = new TopicUpdateRequest.UpdateTopicCommentRequest("new comment"); + TopicDTO mockTopic1 = mockTopicDTO(topic.name(), "new comment", ImmutableMap.of("k1", "v1")); + TopicResponse resp1 = new TopicResponse(mockTopic1); + buildMockResource( + Method.PUT, topicPath, new TopicUpdatesRequest(ImmutableList.of(req1)), resp1, SC_OK); + + Topic alteredTopic = catalog.asTopicCatalog().alterTopic(topic, req1.topicChange()); + assertTopic(mockTopic1, alteredTopic); + + // test set topic properties + TopicUpdateRequest req2 = new TopicUpdateRequest.SetTopicPropertyRequest("k2", "v2"); + TopicDTO mockTopic2 = + mockTopicDTO(topic.name(), "new comment", ImmutableMap.of("k1", "v1", "k2", "v2")); + TopicResponse resp2 = new TopicResponse(mockTopic2); + buildMockResource( + Method.PUT, topicPath, new TopicUpdatesRequest(ImmutableList.of(req2)), resp2, SC_OK); + + alteredTopic = catalog.asTopicCatalog().alterTopic(topic, req2.topicChange()); + assertTopic(mockTopic2, alteredTopic); + + // test remove topic properties + TopicUpdateRequest req3 = new TopicUpdateRequest.RemoveTopicPropertyRequest("k2"); + TopicDTO mockTopic3 = mockTopicDTO(topic.name(), "new comment", ImmutableMap.of("k1", "v1")); + TopicResponse resp3 = new TopicResponse(mockTopic3); + buildMockResource( + Method.PUT, topicPath, new TopicUpdatesRequest(ImmutableList.of(req3)), resp3, SC_OK); + + alteredTopic = catalog.asTopicCatalog().alterTopic(topic, req3.topicChange()); + assertTopic(mockTopic3, alteredTopic); + + // test NoSuchTopicException + ErrorResponse errResp = + ErrorResponse.notFound(NoSuchTopicException.class.getSimpleName(), "topic not found"); + buildMockResource( + Method.PUT, + topicPath, + new TopicUpdatesRequest(ImmutableList.of(req1)), + errResp, + SC_NOT_FOUND); + Exception ex = + Assertions.assertThrows( + NoSuchTopicException.class, + () -> catalog.asTopicCatalog().alterTopic(topic, req1.topicChange())); + Assertions.assertEquals("topic not found", ex.getMessage()); + + // test RuntimeException + errResp = ErrorResponse.internalError("internal error"); + buildMockResource( + Method.PUT, + topicPath, + new TopicUpdatesRequest(ImmutableList.of(req1)), + errResp, + SC_SERVER_ERROR); + ex = + Assertions.assertThrows( + RuntimeException.class, + () -> catalog.asTopicCatalog().alterTopic(topic, req1.topicChange())); + Assertions.assertEquals("internal error", ex.getMessage()); + } + + @Test + public void testDropTopic() throws JsonProcessingException { + NameIdentifier topic = NameIdentifier.of(metalakeName, catalogName, "schema1", "topic1"); + String topicPath = + withSlash(MessagingCatalog.formatTopicRequestPath(topic.namespace()) + "/" + topic.name()); + DropResponse resp = new DropResponse(true); + buildMockResource(Method.DELETE, topicPath, null, resp, SC_OK); + + Assertions.assertTrue(catalog.asTopicCatalog().dropTopic(topic)); + + resp = new DropResponse(false); + buildMockResource(Method.DELETE, topicPath, null, resp, SC_OK); + Assertions.assertFalse(catalog.asTopicCatalog().dropTopic(topic)); + + // test RuntimeException + ErrorResponse errResp = ErrorResponse.internalError("internal error"); + buildMockResource(Method.DELETE, topicPath, null, errResp, SC_SERVER_ERROR); + Exception ex = + Assertions.assertThrows( + RuntimeException.class, () -> catalog.asTopicCatalog().dropTopic(topic)); + Assertions.assertEquals("internal error", ex.getMessage()); + } + + private TopicDTO mockTopicDTO( + String name, String comment, ImmutableMap properties) { + return TopicDTO.builder() + .withName(name) + .withComment(comment) + .withProperties(properties) + .withAudit(AuditDTO.builder().withCreator("creator").withCreateTime(Instant.now()).build()) + .build(); + } + + private void assertTopic(TopicDTO expected, Topic actual) { + Assertions.assertEquals(expected.name(), actual.name()); + Assertions.assertEquals(expected.comment(), actual.comment()); + Assertions.assertEquals(expected.properties(), actual.properties()); + } +} From 83ab3e7ca72c1bf06c523c912780900d937f17a7 Mon Sep 17 00:00:00 2001 From: Ziva Li Date: Tue, 2 Apr 2024 11:14:08 +0800 Subject: [PATCH 17/20] [#2598] Fix: Fix HDFS IP resolution for Docker Desktop (#2740) ### What changes were proposed in this pull request? Fix HDFS client IP address resolution when using Docker Desktop. Originally it may resolve to invalid domain name causing error. ### Why are the changes needed? Fix: #2598 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. --- .../gravitino/integration/test/container/ContainerSuite.java | 1 + 1 file changed, 1 insertion(+) diff --git a/integration-test-common/src/test/java/com/datastrato/gravitino/integration/test/container/ContainerSuite.java b/integration-test-common/src/test/java/com/datastrato/gravitino/integration/test/container/ContainerSuite.java index e2344ab0928..89c46194603 100644 --- a/integration-test-common/src/test/java/com/datastrato/gravitino/integration/test/container/ContainerSuite.java +++ b/integration-test-common/src/test/java/com/datastrato/gravitino/integration/test/container/ContainerSuite.java @@ -111,6 +111,7 @@ public void startTrinoContainer( .withExtraHosts( ImmutableMap.builder() .put("host.docker.internal", "host-gateway") + .put(HiveContainer.HOST_NAME, hiveContainerIp) .build()) .withFilesToMount( ImmutableMap.builder() From d5742c21c5daf100df84e53397c310488c7aa9fd Mon Sep 17 00:00:00 2001 From: Jerry Shao Date: Tue, 2 Apr 2024 11:58:28 +0800 Subject: [PATCH 18/20] [#2750]improve(doc): Split the metadata managing doc into several ones (#2760) ### What changes were proposed in this pull request? Split the "manage-metadata-using-gravitino.md" into several ones. ### Why are the changes needed? As we added more catalog types, it is not feasible to list all the APIs in one doc, so here propose to split the docs. Fix: #2750 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Just docs. --- docs/apache-hive-catalog.md | 10 +- docs/expression.md | 2 +- docs/index.md | 8 +- docs/jdbc-mysql-catalog.md | 12 +- docs/jdbc-postgresql-catalog.md | 12 +- docs/lakehouse-iceberg-catalog.md | 8 +- docs/manage-metalake-using-gravitino.md | 182 +++++++++++++++++ ...ge-relational-metadata-using-gravitino.md} | 188 ++---------------- docs/trino-connector/development.md | 2 +- docs/trino-connector/installation.md | 2 +- docs/trino-connector/supported-catalog.md | 4 +- docs/trino-connector/trino-connector.md | 4 +- docs/webui.md | 4 - 13 files changed, 233 insertions(+), 205 deletions(-) create mode 100644 docs/manage-metalake-using-gravitino.md rename docs/{manage-metadata-using-gravitino.md => manage-relational-metadata-using-gravitino.md} (89%) diff --git a/docs/apache-hive-catalog.md b/docs/apache-hive-catalog.md index c4e5985d7fd..2fe3da52748 100644 --- a/docs/apache-hive-catalog.md +++ b/docs/apache-hive-catalog.md @@ -42,7 +42,7 @@ The Hive catalog supports creating, updating, and deleting databases and tables ### Catalog operations -Refer to [Manage Metadata Using Gravitino](./manage-metadata-using-gravitino.md#catalog-operations) for more details. +Refer to [Manage Relational Metadata Using Gravitino](./manage-relational-metadata-using-gravitino.md#catalog-operations) for more details. ## Schema @@ -61,7 +61,7 @@ The following table lists predefined schema properties for the Hive database. Ad ### Schema operations -see [Manage Metadata Using Gravitino](./manage-metadata-using-gravitino.md#schema-operations). +see [Manage Relational Metadata Using Gravitino](./manage-relational-metadata-using-gravitino.md#schema-operations). ## Table @@ -117,7 +117,7 @@ The following table lists the data types mapped from the Hive catalog to Graviti | `uniontype` | `uniontype` | 0.2.0 | :::info -Since 0.5.0, the data types other than listed above are mapped to Gravitino **[Unparsed Type](./manage-metadata-using-gravitino.md#unparsed-type)** that represents an unresolvable data type from the Hive catalog. +Since 0.5.0, the data types other than listed above are mapped to Gravitino **[Unparsed Type](./manage-relational-metadata-using-gravitino.md#unparsed-type)** that represents an unresolvable data type from the Hive catalog. ::: ### Table properties @@ -151,11 +151,11 @@ Hive automatically adds and manages some reserved properties. Users aren't allow ### Table operations -Refer to [Manage Metadata Using Gravitino](./manage-metadata-using-gravitino.md#table-operations) for more details. +Refer to [Manage Relational Metadata Using Gravitino](./manage-relational-metadata-using-gravitino.md#table-operations) for more details. #### Alter operations -Gravitino has already defined a unified set of [metadata operation interfaces](./manage-metadata-using-gravitino.md#alter-a-table), and almost all [Hive Alter operations](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-AlterTable/Partition/Column) have corresponding table update requests which enable you to change the struct of an existing table. +Gravitino has already defined a unified set of [metadata operation interfaces](./manage-relational-metadata-using-gravitino.md#alter-a-table), and almost all [Hive Alter operations](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-AlterTable/Partition/Column) have corresponding table update requests which enable you to change the struct of an existing table. The following table lists the mapping relationship between Hive Alter operations and Gravitino table update requests. ##### Alter table diff --git a/docs/expression.md b/docs/expression.md index 3515a36f1e3..33ae51c723e 100644 --- a/docs/expression.md +++ b/docs/expression.md @@ -9,7 +9,7 @@ license: Copyright 2024 Datastrato Pvt Ltd. This software is licensed under the import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; -This page introduces the expression system of Gravitino. Expressions are vital component of metadata definition, through expressions, you can define [default values](./manage-metadata-using-gravitino.md#table-column-default-value) for columns, function arguments for [function partitioning](./table-partitioning-bucketing-sort-order-indexes.md#table-partitioning), [bucketing](./table-partitioning-bucketing-sort-order-indexes.md#table-bucketing), and sort term of [sort ordering](./table-partitioning-bucketing-sort-order-indexes.md#sort-ordering) in tables. +This page introduces the expression system of Gravitino. Expressions are vital component of metadata definition, through expressions, you can define [default values](./manage-relational-metadata-using-gravitino.md#table-column-default-value) for columns, function arguments for [function partitioning](./table-partitioning-bucketing-sort-order-indexes.md#table-partitioning), [bucketing](./table-partitioning-bucketing-sort-order-indexes.md#table-bucketing), and sort term of [sort ordering](./table-partitioning-bucketing-sort-order-indexes.md#sort-ordering) in tables. Gravitino expression system divides expressions into three basic parts: field reference, literal, and function. Function expressions can contain field references, literals, and other function expressions. ## Field reference diff --git a/docs/index.md b/docs/index.md index c2d14526170..d24782edfa6 100644 --- a/docs/index.md +++ b/docs/index.md @@ -48,7 +48,11 @@ To get started with Gravitino, see [Getting started](./getting-started.md) for t Gravitino provides two SDKs to manage metadata from different catalogs in a unified way: the REST API and the Java SDK. You can use either to manage metadata. See -[Manage metadata using Gravitino](./manage-metadata-using-gravitino.md) for details. + +* [Manage metalake using Gravitino](./manage-metalake-using-gravitino.md) to learn how to manage + metalakes. +* [Manage relational metadata using Gravitino](./manage-relational-metadata-using-gravitino.md) + to learn how to manage relational metadata. Also, you can find the complete REST API definition in [Gravitino Open API](./api/rest/gravitino-rest-api), and the @@ -58,6 +62,8 @@ Gravitino provides a web UI to manage the metadata. Visit the web UI in the brow Gravitino currently supports the following catalogs: +**Relational catalogs:** + * [**Iceberg catalog**](./lakehouse-iceberg-catalog.md) * [**Hive catalog**](./apache-hive-catalog.md) * [**MySQL catalog**](./jdbc-mysql-catalog.md) diff --git a/docs/jdbc-mysql-catalog.md b/docs/jdbc-mysql-catalog.md index 611abf970f9..0c3a38d9565 100644 --- a/docs/jdbc-mysql-catalog.md +++ b/docs/jdbc-mysql-catalog.md @@ -28,7 +28,7 @@ Gravitino saves some system information in schema and table comment, like `(From - Supports metadata management of MySQL (5.7, 8.0). - Supports DDL operation for MySQL databases and tables. - Supports table index. -- Supports [column default value](./manage-metadata-using-gravitino.md#table-column-default-value) and [auto-increment](./manage-metadata-using-gravitino.md#table-column-auto-increment). +- Supports [column default value](./manage-relational-metadata-using-gravitino.md#table-column-default-value) and [auto-increment](./manage-relational-metadata-using-gravitino.md#table-column-auto-increment). - Supports managing MySQL table features though table properties, like using `engine` to set MySQL storage engine. ### Catalog properties @@ -54,7 +54,7 @@ You must download the corresponding JDBC driver to the `catalogs/jdbc-mysql/libs ### Catalog operations -Refer to [Manage Metadata Using Gravitino](./manage-metadata-using-gravitino.md#catalog-operations) for more details. +Refer to [Manage Relational Metadata Using Gravitino](./manage-relational-metadata-using-gravitino.md#catalog-operations) for more details. ## Schema @@ -71,7 +71,7 @@ Refer to [Manage Metadata Using Gravitino](./manage-metadata-using-gravitino.md# ### Schema operations -Refer to [Manage Metadata Using Gravitino](./manage-metadata-using-gravitino.md#schema-operations) for more details. +Refer to [Manage Relational Metadata Using Gravitino](./manage-relational-metadata-using-gravitino.md#schema-operations) for more details. ## Table @@ -80,7 +80,7 @@ Refer to [Manage Metadata Using Gravitino](./manage-metadata-using-gravitino.md# - Gravitino's table concept corresponds to the MySQL table. - Supports DDL operation for MySQL tables. - Supports index. -- Supports [column default value](./manage-metadata-using-gravitino.md#table-column-default-value) and [auto-increment](./manage-metadata-using-gravitino.md#table-column-auto-increment).. +- Supports [column default value](./manage-relational-metadata-using-gravitino.md#table-column-default-value) and [auto-increment](./manage-relational-metadata-using-gravitino.md#table-column-auto-increment).. - Supports managing MySQL table features though table properties, like using `engine` to set MySQL storage engine. #### Table column types @@ -104,7 +104,7 @@ Refer to [Manage Metadata Using Gravitino](./manage-metadata-using-gravitino.md# :::info MySQL doesn't support Gravitino `Boolean` `Fixed` `Struct` `List` `Map` `Timestamp_tz` `IntervalDay` `IntervalYear` `Union` `UUID` type. -Meanwhile, the data types other than listed above are mapped to Gravitino **[Unparsed Type](./manage-metadata-using-gravitino.md#unparsed-type)** that represents an unresolvable data type since 0.5.0. +Meanwhile, the data types other than listed above are mapped to Gravitino **[Unparsed Type](./manage-relational-metadata-using-gravitino.md#unparsed-type)** that represents an unresolvable data type since 0.5.0. ::: #### Table column auto-increment @@ -215,7 +215,7 @@ Index[] indexes = new Index[] { ### Table operations -Refer to [Manage Metadata Using Gravitino](./manage-metadata-using-gravitino.md#table-operations) for more details. +Refer to [Manage Relational Metadata Using Gravitino](./manage-relational-metadata-using-gravitino.md#table-operations) for more details. #### Alter table operations diff --git a/docs/jdbc-postgresql-catalog.md b/docs/jdbc-postgresql-catalog.md index e3aeed73abf..22a667e187d 100644 --- a/docs/jdbc-postgresql-catalog.md +++ b/docs/jdbc-postgresql-catalog.md @@ -28,7 +28,7 @@ Gravitino saves some system information in schema and table comment, like `(From - Supports metadata management of PostgreSQL (12.x, 13.x, 14.x, 15.x, 16.x). - Supports DDL operation for PostgreSQL schemas and tables. - Supports table index. -- Supports [column default value](./manage-metadata-using-gravitino.md#table-column-default-value). and [auto-increment](./manage-metadata-using-gravitino.md#table-column-auto-increment). +- Supports [column default value](./manage-relational-metadata-using-gravitino.md#table-column-default-value). and [auto-increment](./manage-relational-metadata-using-gravitino.md#table-column-auto-increment). ### Catalog properties @@ -57,7 +57,7 @@ In PostgreSQL, the database corresponds to the Gravitino catalog, and the schema ### Catalog operations -Please refer to [Manage Metadata Using Gravitino](./manage-metadata-using-gravitino.md#catalog-operations) for more details. +Please refer to [Manage Relational Metadata Using Gravitino](./manage-relational-metadata-using-gravitino.md#catalog-operations) for more details. ## Schema @@ -74,7 +74,7 @@ Please refer to [Manage Metadata Using Gravitino](./manage-metadata-using-gravit ### Schema operations -Please refer to [Manage Metadata Using Gravitino](./manage-metadata-using-gravitino.md#schema-operations) for more details. +Please refer to [Manage Relational Metadata Using Gravitino](./manage-relational-metadata-using-gravitino.md#schema-operations) for more details. ## Table @@ -83,7 +83,7 @@ Please refer to [Manage Metadata Using Gravitino](./manage-metadata-using-gravit - The Gravitino table corresponds to the PostgreSQL table. - Supports DDL operation for PostgreSQL tables. - Supports index. -- Support [column default value](./manage-metadata-using-gravitino.md#table-column-default-value) and [auto-increment](./manage-metadata-using-gravitino.md#table-column-auto-increment). +- Support [column default value](./manage-relational-metadata-using-gravitino.md#table-column-default-value) and [auto-increment](./manage-relational-metadata-using-gravitino.md#table-column-auto-increment). - Doesn't support table property settings. #### Table column types @@ -108,7 +108,7 @@ Please refer to [Manage Metadata Using Gravitino](./manage-metadata-using-gravit :::info PostgreSQL doesn't support Gravitino `Fixed` `Struct` `List` `Map` `IntervalDay` `IntervalYear` `Union` `UUID` type. -Meanwhile, the data types other than listed above are mapped to Gravitino **[Unparsed Type](./manage-metadata-using-gravitino.md#unparsed-type)** that represents an unresolvable data type since 0.5.0. +Meanwhile, the data types other than listed above are mapped to Gravitino **[Unparsed Type](./manage-relational-metadata-using-gravitino.md#unparsed-type)** that represents an unresolvable data type since 0.5.0. ::: #### Table column auto-increment @@ -158,7 +158,7 @@ Index[] indexes = new Index[] { ### Table operations -Please refer to [Manage Metadata Using Gravitino](./manage-metadata-using-gravitino.md#table-operations) for more details. +Please refer to [Manage Relational Metadata Using Gravitino](./manage-relational-metadata-using-gravitino.md#table-operations) for more details. #### Alter table operations diff --git a/docs/lakehouse-iceberg-catalog.md b/docs/lakehouse-iceberg-catalog.md index bdaf7c4539e..1c8923b82f2 100644 --- a/docs/lakehouse-iceberg-catalog.md +++ b/docs/lakehouse-iceberg-catalog.md @@ -58,7 +58,7 @@ You must download the corresponding JDBC driver to the `catalogs/lakehouse-icebe ### Catalog operations -Please refer to [Manage Metadata Using Gravitino](./manage-metadata-using-gravitino.md#catalog-operations) for more details. +Please refer to [Manage Relational Metadata Using Gravitino](./manage-relational-metadata-using-gravitino.md#catalog-operations) for more details. ## Schema @@ -72,7 +72,7 @@ You could put properties except `comment`. ### Schema operations -Please refer to [Manage Metadata Using Gravitino](./manage-metadata-using-gravitino.md#schema-operations) for more details. +Please refer to [Manage Relational Metadata Using Gravitino](./manage-relational-metadata-using-gravitino.md#schema-operations) for more details. ## Table @@ -215,7 +215,7 @@ Apache Iceberg doesn't support Gravitino `EvenDistribution` type. :::info Apache Iceberg doesn't support Gravitino `Varchar` `Fixedchar` `Byte` `Short` `Union` type. -Meanwhile, the data types other than listed above are mapped to Gravitino **[Unparsed Type](./manage-metadata-using-gravitino.md#unparsed-type)** that represents an unresolvable data type since 0.5.0. +Meanwhile, the data types other than listed above are mapped to Gravitino **[Unparsed Type](./manage-relational-metadata-using-gravitino.md#unparsed-type)** that represents an unresolvable data type since 0.5.0. ::: ### Table properties @@ -241,7 +241,7 @@ The Gravitino server doesn't allow passing the following reserved fields. ### Table operations -Please refer to [Manage Metadata Using Gravitino](./manage-metadata-using-gravitino.md#table-operations) for more details. +Please refer to [Manage Relational Metadata Using Gravitino](./manage-relational-metadata-using-gravitino.md#table-operations) for more details. #### Alter table operations diff --git a/docs/manage-metalake-using-gravitino.md b/docs/manage-metalake-using-gravitino.md new file mode 100644 index 00000000000..335c6dbb675 --- /dev/null +++ b/docs/manage-metalake-using-gravitino.md @@ -0,0 +1,182 @@ +--- +title: "Manage metalake using Gravitino" +slug: /manage-metalake-using-gravitino +date: 2023-12-10 +keyword: Gravitino metalake manage +license: Copyright 2023 Datastrato Pvt Ltd. This software is licensed under the Apache License version 2. +--- + +import Tabs from '@theme/Tabs'; +import TabItem from '@theme/TabItem'; + +This page introduces how to manage metalake by Gravitino. Metalake is a tenant-like concept in +Gravitino, all the catalogs, users and roles are under a metalake. Typically, a metalake is +mapping to a organization or a company. + +Through Gravitino, you can create, edit, and delete metalake. This page includes the following +contents: + +Assuming Gravitino has just started, and the host and port is [http://localhost:8090](http://localhost:8090). + +## Metalake operations + +### Create a metalake + +You can create a metalake by sending a `POST` request to the `/api/metalakes` endpoint or just use the Gravitino Java client. +The following is an example of creating a metalake: + + + + +```shell +curl -X POST -H "Accept: application/vnd.gravitino.v1+json" \ +-H "Content-Type: application/json" -d '{"name":"metalake","comment":"comment","properties":{}}' \ +http://localhost:8090/api/metalakes +``` + + + + +```java +GravitinoClient gravitinoClient = GravitinoClient + .builder("http://127.0.0.1:8090") + .build(); +GravitinoMetaLake newMetalake = gravitinoClient.createMetalake( + NameIdentifier.of("metalake"), + "This is a new metalake", + new HashMap<>()); + // ... +``` + + + + +### Load a metalake + +You can create a metalake by sending a `GET` request to the `/api/metalakes/{metalake_name}` endpoint or just use the Gravitino Java client. The following is an example of loading a metalake: + + + + +```shell +curl -X GET -H "Accept: application/vnd.gravitino.v1+json" \ +-H "Content-Type: application/json" http://localhost:8090/api/metalakes/metalake +``` + + + + +```java +// ... +GravitinoMetaLake loaded = gravitinoClient.loadMetalake( + NameIdentifier.of("metalake")); +// ... +``` + + + + +### Alter a metalake + +You can modify a metalake by sending a `PUT` request to the `/api/metalakes/{metalake_name}` endpoint or just use the Gravitino Java client. The following is an example of altering a metalake: + + + + +```shell +curl -X PUT -H "Accept: application/vnd.gravitino.v1+json" \ +-H "Content-Type: application/json" -d '{ + "updates": [ + { + "@type": "rename", + "newName": "metalake" + }, + { + "@type": "setProperty", + "property": "key2", + "value": "value2" + } + ] +}' http://localhost:8090/api/metalakes/new_metalake +``` + + + + +```java +// ... +GravitinoMetaLake renamed = gravitinoClient.alterMetalake( + NameIdentifier.of("new_metalake"), + MetalakeChange.rename("new_metalake_renamed") +); +// ... +``` + + + + + +Currently, Gravitino supports the following changes to a metalake: + +| Supported modification | JSON | Java | +|------------------------|--------------------------------------------------------------|-------------------------------------------------| +| Rename metalake | `{"@type":"rename","newName":"metalake_renamed"}` | `MetalakeChange.rename("metalake_renamed")` | +| Update comment | `{"@type":"updateComment","newComment":"new_comment"}` | `MetalakeChange.updateComment("new_comment")` | +| Set a property | `{"@type":"setProperty","property":"key1","value":"value1"}` | `MetalakeChange.setProperty("key1", "value1")` | +| Remove a property | `{"@type":"removeProperty","property":"key1"}` | `MetalakeChange.removeProperty("key1")` | + + +### Drop a metalake + +You can remove a metalake by sending a `DELETE` request to the `/api/metalakes/{metalake_name}` endpoint or just use the Gravitino Java client. The following is an example of dropping a metalake: + + + + +```shell +curl -X DELETE -H "Accept: application/vnd.gravitino.v1+json" \ +-H "Content-Type: application/json" http://localhost:8090/api/metalakes/metalake +``` + + + + +```java +// ... +boolean success = gravitinoClient.dropMetalake( + NameIdentifier.of("metalake") +); +// ... +``` + + + + +:::note +Current Gravitino doesn't support dropping a metalake in cascade mode, which means all the +catalogs, schemas and tables under the metalake need to be removed before dropping the metalake. +::: + +### List all metalakes + +You can list metalakes by sending a `GET` request to the `/api/metalakes` endpoint or just use the Gravitino Java client. The following is an example of listing all metalake names: + + + + +```shell +curl -X GET -H "Accept: application/vnd.gravitino.v1+json" \ +-H "Content-Type: application/json" http://localhost:8090/api/metalakes +``` + + + + +```java +// ... +GravitinoMetaLake[] allMetalakes = gravitinoClient.listMetalakes(); +// ... +``` + + + diff --git a/docs/manage-metadata-using-gravitino.md b/docs/manage-relational-metadata-using-gravitino.md similarity index 89% rename from docs/manage-metadata-using-gravitino.md rename to docs/manage-relational-metadata-using-gravitino.md index d4580c5bb4b..b40a6274400 100644 --- a/docs/manage-metadata-using-gravitino.md +++ b/docs/manage-relational-metadata-using-gravitino.md @@ -1,198 +1,42 @@ --- -title: "Manage metadata using Gravitino" -slug: /manage-metadata-using-gravitino +title: "Manage relational metadata using Gravitino" +slug: /manage-relational-metadata-using-gravitino date: 2023-12-10 -keyword: Gravitino metadata manage +keyword: Gravitino relational metadata manage license: Copyright 2023 Datastrato Pvt Ltd. This software is licensed under the Apache License version 2. --- import Tabs from '@theme/Tabs'; import TabItem from '@theme/TabItem'; -This page introduces how to manage metadata by Gravitino. Through Gravitino, you can create, edit, and delete metadata -like metalakes, catalogs, schemas, and tables. This page includes the following contents: +This page introduces how to manage relational metadata by Gravitino, relational metadata refers +to relational catalog, schema, table and partitions. Through Gravitino, you can create, edit, and +delete relational metadata via unified REST APIs or Java client. -In this document, Gravitino uses Apache Hive catalog as an example to show how to manage metadata by Gravitino. Other catalogs are similar to Hive catalog, -but they may have some differences, especially in catalog property, table property, and column type. For more details, please refer to the related doc. +In this document, Gravitino uses Apache Hive catalog as an example to show how to manage +relational metadata by Gravitino. Other relational catalogs are similar to Hive catalog, +but they may have some differences, especially in catalog property, table property, and column type. +For more details, please refer to the related doc. - [**Apache Hive**](./apache-hive-catalog.md) - [**MySQL**](./jdbc-mysql-catalog.md) - [**PostgreSQL**](./jdbc-postgresql-catalog.md) - [**Apache Iceberg**](./lakehouse-iceberg-catalog.md) +Assuming: -Assuming Gravitino has just started, and the host and port is [http://localhost:8090](http://localhost:8090). - -## Metalake operations - -### Create a metalake - -You can create a metalake by sending a `POST` request to the `/api/metalakes` endpoint or just use the Gravitino Java client. -The following is an example of creating a metalake: - - - - -```shell -curl -X POST -H "Accept: application/vnd.gravitino.v1+json" \ --H "Content-Type: application/json" -d '{"name":"metalake","comment":"comment","properties":{}}' \ -http://localhost:8090/api/metalakes -``` - - - - -```java -GravitinoClient gravitinoClient = GravitinoClient - .builder("http://127.0.0.1:8090") - .build(); -GravitinoMetaLake newMetalake = gravitinoClient.createMetalake( - NameIdentifier.of("metalake"), - "This is a new metalake", - new HashMap<>()); - // ... -``` - - - - -### Load a metalake - -You can create a metalake by sending a `GET` request to the `/api/metalakes/{metalake_name}` endpoint or just use the Gravitino Java client. The following is an example of loading a metalake: - - - - -```shell -curl -X GET -H "Accept: application/vnd.gravitino.v1+json" \ --H "Content-Type: application/json" http://localhost:8090/api/metalakes/metalake -``` - - - - -```java -// ... -GravitinoMetaLake loaded = gravitinoClient.loadMetalake( - NameIdentifier.of("metalake")); -// ... -``` - - - - -### Alter a metalake - -You can modify a metalake by sending a `PUT` request to the `/api/metalakes/{metalake_name}` endpoint or just use the Gravitino Java client. The following is an example of altering a metalake: - - - - -```shell -curl -X PUT -H "Accept: application/vnd.gravitino.v1+json" \ --H "Content-Type: application/json" -d '{ - "updates": [ - { - "@type": "rename", - "newName": "metalake" - }, - { - "@type": "setProperty", - "property": "key2", - "value": "value2" - } - ] -}' http://localhost:8090/api/metalakes/new_metalake -``` - - - - -```java -// ... -GravitinoMetaLake renamed = gravitinoClient.alterMetalake( - NameIdentifier.of("new_metalake"), - MetalakeChange.rename("new_metalake_renamed") -); -// ... -``` - - - - - -Currently, Gravitino supports the following changes to a metalake: - -| Supported modification | JSON | Java | -|------------------------|--------------------------------------------------------------|-------------------------------------------------| -| Rename metalake | `{"@type":"rename","newName":"metalake_renamed"}` | `MetalakeChange.rename("metalake_renamed")` | -| Update comment | `{"@type":"updateComment","newComment":"new_comment"}` | `MetalakeChange.updateComment("new_comment")` | -| Set a property | `{"@type":"setProperty","property":"key1","value":"value1"}` | `MetalakeChange.setProperty("key1", "value1")` | -| Remove a property | `{"@type":"removeProperty","property":"key1"}` | `MetalakeChange.removeProperty("key1")` | - - -### Drop a metalake - -You can remove a metalake by sending a `DELETE` request to the `/api/metalakes/{metalake_name}` endpoint or just use the Gravitino Java client. The following is an example of dropping a metalake: - - - - -```shell -curl -X DELETE -H "Accept: application/vnd.gravitino.v1+json" \ --H "Content-Type: application/json" http://localhost:8090/api/metalakes/metalake -``` - - - - -```java -// ... -boolean success = gravitinoClient.dropMetalake( - NameIdentifier.of("metalake") -); -// ... -``` - - - - -:::note -Dropping a metalake only removes metadata about the metalake and catalogs, schemas, tables under the metalake in Gravitino, It doesn't remove the real schema and table data in Apache Hive. -::: - -### List all metalakes - -You can list metalakes by sending a `GET` request to the `/api/metalakes` endpoint or just use the Gravitino Java client. The following is an example of listing all metalake names: - - - - -```shell -curl -X GET -H "Accept: application/vnd.gravitino.v1+json" \ --H "Content-Type: application/json" http://localhost:8090/api/metalakes -``` - - - - -```java -// ... -GravitinoMetaLake[] allMetalakes = gravitinoClient.listMetalakes(); -// ... -``` - - - + - Gravitino has just started, and the host and port is [http://localhost:8090](http://localhost:8090). + - Metalake has been created. ## Catalog operations ### Create a catalog :::tip -Users should create a metalake before creating a catalog. +The code below is an example of creating a Hive catalog. For other relational catalogs, the code is +similar, but the catalog type, provider, and properties may be different. For more details, please refer to the related doc. -The code below is an example of creating a Hive catalog. For other catalogs, the code is similar, but the catalog type, provider, and properties may be different. For more details, please refer to the related doc. +For relational catalog, you must specify the catalog `type` as `RELATIONAL` when creating a catalog. ::: You can create a catalog by sending a `POST` request to the `/api/metalakes/{metalake_name}/catalogs` endpoint or just use the Gravitino Java client. The following is an example of creating a catalog: diff --git a/docs/trino-connector/development.md b/docs/trino-connector/development.md index 306c4da3b79..1c7a7b167af 100644 --- a/docs/trino-connector/development.md +++ b/docs/trino-connector/development.md @@ -13,7 +13,7 @@ This document is to guide users through the development of the Gravitino connect Before you start developing the Gravitino trino connector, you need to have the following prerequisites: 1. You need to start the Gravitino server locally, for more information, please refer to the [start Gravitino server](../how-to-install.md) -2. Create a catalog in the Gravitino server, for more information, please refer to the [Gravitino metadata management](../manage-metadata-using-gravitino.md). Assuming we have just created a MySQL catalog using the following command: +2. Create a catalog in the Gravitino server, for more information, please refer to the [Gravitino metadata management](../manage-relational-metadata-using-gravitino.md). Assuming we have just created a MySQL catalog using the following command: ```curl curl -X POST -H "Content-Type: application/json" -d '{"name":"test","comment":"comment","properties":{}}' http://localhost:8090/api/metalakes diff --git a/docs/trino-connector/installation.md b/docs/trino-connector/installation.md index b2391ec902b..707d1d3802b 100644 --- a/docs/trino-connector/installation.md +++ b/docs/trino-connector/installation.md @@ -113,7 +113,7 @@ system You can see the `gravitino` catalog in the result set. This signifies the successful installation of the Gravitino connector. -Assuming you have created a catalog named `test.jdbc-mysql` in the Gravitino server, or please refer to [Create a Catalog](../manage-metadata-using-gravitino.md#create-a-catalog). Then you can use the Trino CLI to connect to the Trino container and run a query like this. +Assuming you have created a catalog named `test.jdbc-mysql` in the Gravitino server, or please refer to [Create a Catalog](../manage-relational-metadata-using-gravitino.md#create-a-catalog). Then you can use the Trino CLI to connect to the Trino container and run a query like this. ```text docker exec -it trino-gravitino trino diff --git a/docs/trino-connector/supported-catalog.md b/docs/trino-connector/supported-catalog.md index ce2fd8b4c61..b578e668a51 100644 --- a/docs/trino-connector/supported-catalog.md +++ b/docs/trino-connector/supported-catalog.md @@ -120,7 +120,7 @@ call gravitino.system.alter_catalog( ``` if you need more information about catalog, please refer to: -[Create a Catalog](../manage-metadata-using-gravitino.md#create-a-catalog). +[Create a Catalog](../manage-relational-metadata-using-gravitino.md#create-a-catalog). ## Data type mapping between Trino and Gravitino @@ -147,4 +147,4 @@ Hive does not support `TIME` data type. | MapType | MAP | | StructType | ROW | -For more about Trino data types, please refer to [Trino data types](https://trino.io/docs/current/language/types.html) and Gravitino data types, please refer to [Gravitino data types](../manage-metadata-using-gravitino.md#gravitino-table-column-type). \ No newline at end of file +For more about Trino data types, please refer to [Trino data types](https://trino.io/docs/current/language/types.html) and Gravitino data types, please refer to [Gravitino data types](../manage-relational-metadata-using-gravitino.md#gravitino-table-column-type). diff --git a/docs/trino-connector/trino-connector.md b/docs/trino-connector/trino-connector.md index 6a5fbca9262..1858d695963 100644 --- a/docs/trino-connector/trino-connector.md +++ b/docs/trino-connector/trino-connector.md @@ -22,10 +22,10 @@ The loading of Gravitino's catalogs into Trino follows the naming convention: ``` Regarding `metalake` and `catalog`, -you can refer to [Create a Metalake](../manage-metadata-using-gravitino.md#create-a-metalake), [Create a Catalog](../manage-metadata-using-gravitino.md#create-a-catalog). +you can refer to [Create a Metalake](../manage-relational-metadata-using-gravitino.md#create-a-metalake), [Create a Catalog](../manage-relational-metadata-using-gravitino.md#create-a-catalog). Usage in queries is as follows: ```text SELECT * from "metalake.catalog".dbname.tabname -``` \ No newline at end of file +``` diff --git a/docs/webui.md b/docs/webui.md index 8678352cedf..0800f7c7125 100644 --- a/docs/webui.md +++ b/docs/webui.md @@ -16,10 +16,6 @@ This document primarily outlines how users can manage metadata within Gravitino Currently, you can integrate [OAuth settings](security.md) to view, add, modify, and delete metalakes, create catalogs, and view catalogs, schemas, and tables, among other functions. -:::caution -More features are under development. For the details, please refer to the [Manage metadata using Gravitino](manage-metadata-using-gravitino.md) document. -::: - [Build](how-to-build.md#quick-start) and [deploy](getting-started.md#getting-started-locally) the Gravitino Web UI and open it in a browser at `http://:`, by default is [http://localhost:8090](http://localhost:8090). ## Initial page From 441145996c81eb71e5734f509dc761b46614018e Mon Sep 17 00:00:00 2001 From: Yongjie Zhao Date: Tue, 2 Apr 2024 08:03:25 +0200 Subject: [PATCH 19/20] [#2113] feat(pyClient): initial Gravitino Python client module (#2676) ### What changes were proposed in this pull request? image The more detailed introductions for the PR has been uploaded on the [bilibili](https://www.bilibili.com/video/BV11i42197x3/?vd_source=b1397e6194e37e70828e961f41b7b9db), please jump in there to get a quick view. #2113 ### Follow the below command 1. Install the python-client in your local machine, notice that version of Python should be upper than 3.8. ```shell $ cd clients/client-python $ pip install -e . $ pip install ipython (optional) $ ipython (optional) ``` 2. Run the Python terminal and happy coding. ``` In [1]: from gravitino import GravitinoClient In [2]: client = GravitinoClient("http://localhost:8090") In [3]: client.get_metalakes() In [4]: metalake_demo = client.get_metalakes()[1] In [5]: metalake_demo.catalog_hive.sales.customers.info() ``` I will follow next steps to refactor/implement the initial Python Client, also heads up @Lanznx was the initial contributor for the Python Client. - [x] Remove `requests` since we are using really simple HTTP client to communicate between `Python script` and `Gravitino`, the `http.client` in Python3 is sufficient. - [x] Add Unit test - [ ] Add codes linter - [ ] Provide an approach for uploading PyPI ### Why are the changes needed? The motivation of the PR intend to introduce an interactivity and exploratory user-experience between Python terminal and Gravitino Server. ### Does this PR introduce _any_ user-facing change? N/A ### How was this patch tested? Unit tests will cover changes. --------- Co-authored-by: Yongjie Zhao Co-authored-by: Shaofeng Shi --- clients/client-python/.gitignore | 163 +++++++++++++++++- clients/client-python/Makefile | 15 ++ clients/client-python/README.md | 11 +- clients/client-python/gravitino/__init__.py | 13 ++ .../__init__.py => gravitino/constants.py} | 4 +- .../{__init__.py => gravitino/exceptions.py} | 2 +- .../gravitino/gravitino_client.py | 158 +++++++++++++++++ clients/client-python/gravitino/service.py | 77 +++++++++ clients/client-python/gravitino/typing.py | 11 ++ .../core => gravitino/utils}/__init__.py | 4 +- .../gravitino/utils/exceptions.py | 95 ++++++++++ .../gravitino/utils/http_client.py | 145 ++++++++++++++++ .../gravitino_client/core/dto.py | 11 -- .../gravitino_client/core/gravitino_client.py | 56 ------ clients/client-python/requirements-dev.txt | 2 + clients/client-python/requirements.txt | 4 +- clients/client-python/setup.py | 23 +++ clients/client-python/tests/__init__.py | 2 +- clients/client-python/tests/core/__init__.py | 4 - .../tests/core/test_gravitino_client.py | 45 ----- clients/client-python/tests/fixtures.py | 104 +++++++++++ .../tests/test_gravitino_client.py | 58 +++++++ clients/client-python/tests/utils.py | 49 ++++++ 23 files changed, 925 insertions(+), 131 deletions(-) create mode 100644 clients/client-python/Makefile create mode 100644 clients/client-python/gravitino/__init__.py rename clients/client-python/{gravitino_client/__init__.py => gravitino/constants.py} (84%) rename clients/client-python/{__init__.py => gravitino/exceptions.py} (96%) create mode 100644 clients/client-python/gravitino/gravitino_client.py create mode 100644 clients/client-python/gravitino/service.py create mode 100644 clients/client-python/gravitino/typing.py rename clients/client-python/{gravitino_client/core => gravitino/utils}/__init__.py (58%) create mode 100644 clients/client-python/gravitino/utils/exceptions.py create mode 100644 clients/client-python/gravitino/utils/http_client.py delete mode 100644 clients/client-python/gravitino_client/core/dto.py delete mode 100644 clients/client-python/gravitino_client/core/gravitino_client.py create mode 100644 clients/client-python/requirements-dev.txt create mode 100644 clients/client-python/setup.py delete mode 100644 clients/client-python/tests/core/__init__.py delete mode 100644 clients/client-python/tests/core/test_gravitino_client.py create mode 100644 clients/client-python/tests/fixtures.py create mode 100644 clients/client-python/tests/test_gravitino_client.py create mode 100644 clients/client-python/tests/utils.py diff --git a/clients/client-python/.gitignore b/clients/client-python/.gitignore index e0549b75b57..6769e21d99a 100644 --- a/clients/client-python/.gitignore +++ b/clients/client-python/.gitignore @@ -1,5 +1,160 @@ -# Copyright 2024 Datastrato Pvt Ltd. -# This software is licensed under the Apache License version 2. - +# Byte-compiled / optimized / DLL files __pycache__/ -.pytest_cache/ \ No newline at end of file +*.py[cod] +*$py.class + +# C extensions +*.so + +# Distribution / packaging +.Python +build/ +develop-eggs/ +dist/ +downloads/ +eggs/ +.eggs/ +lib/ +lib64/ +parts/ +sdist/ +var/ +wheels/ +share/python-wheels/ +*.egg-info/ +.installed.cfg +*.egg +MANIFEST + +# PyInstaller +# Usually these files are written by a python script from a template +# before PyInstaller builds the exe, so as to inject date/other infos into it. +*.manifest +*.spec + +# Installer logs +pip-log.txt +pip-delete-this-directory.txt + +# Unit test / coverage reports +htmlcov/ +.tox/ +.nox/ +.coverage +.coverage.* +.cache +nosetests.xml +coverage.xml +*.cover +*.py,cover +.hypothesis/ +.pytest_cache/ +cover/ + +# Translations +*.mo +*.pot + +# Django stuff: +*.log +local_settings.py +db.sqlite3 +db.sqlite3-journal + +# Flask stuff: +instance/ +.webassets-cache + +# Scrapy stuff: +.scrapy + +# Sphinx documentation +docs/_build/ + +# PyBuilder +.pybuilder/ +target/ + +# Jupyter Notebook +.ipynb_checkpoints + +# IPython +profile_default/ +ipython_config.py + +# pyenv +# For a library or package, you might want to ignore these files since the code is +# intended to run in multiple environments; otherwise, check them in: +# .python-version + +# pipenv +# According to pypa/pipenv#598, it is recommended to include Pipfile.lock in version control. +# However, in case of collaboration, if having platform-specific dependencies or dependencies +# having no cross-platform support, pipenv may install dependencies that don't work, or not +# install all needed dependencies. +#Pipfile.lock + +# poetry +# Similar to Pipfile.lock, it is generally recommended to include poetry.lock in version control. +# This is especially recommended for binary packages to ensure reproducibility, and is more +# commonly ignored for libraries. +# https://python-poetry.org/docs/basic-usage/#commit-your-poetrylock-file-to-version-control +#poetry.lock + +# pdm +# Similar to Pipfile.lock, it is generally recommended to include pdm.lock in version control. +#pdm.lock +# pdm stores project-wide configurations in .pdm.toml, but it is recommended to not include it +# in version control. +# https://pdm.fming.dev/#use-with-ide +.pdm.toml + +# PEP 582; used by e.g. github.com/David-OConnor/pyflow and github.com/pdm-project/pdm +__pypackages__/ + +# Celery stuff +celerybeat-schedule +celerybeat.pid + +# SageMath parsed files +*.sage.py + +# Environments +.env +.venv +env/ +venv/ +ENV/ +env.bak/ +venv.bak/ + +# Spyder project settings +.spyderproject +.spyproject + +# Rope project settings +.ropeproject + +# mkdocs documentation +/site + +# mypy +.mypy_cache/ +.dmypy.json +dmypy.json + +# Pyre type checker +.pyre/ + +# pytype static type analyzer +.pytype/ + +# Cython debug symbols +cython_debug/ + +# PyCharm +# JetBrains specific template is maintained in a separate JetBrains.gitignore that can +# be found at https://github.com/github/gitignore/blob/main/Global/JetBrains.gitignore +# and can be added to the global gitignore or merged into this file. For a more nuclear +# option (not recommended) you can uncomment the following to ignore the entire idea folder. +#.idea/ \ No newline at end of file diff --git a/clients/client-python/Makefile b/clients/client-python/Makefile new file mode 100644 index 00000000000..f769e5ac76a --- /dev/null +++ b/clients/client-python/Makefile @@ -0,0 +1,15 @@ +# Copyright 2024 Datastrato Pvt Ltd. +# This software is licensed under the Apache License version 2. +.PHONY: mkvenv install test clean + +mkvenv: + python -m venv venv && source ./venv/bin/activate && pip install -U pip + +install: + source ./venv/bin/activate && pip install -e . + +test: + source ./venv/bin/activate && python -m unittest + +clean: + rm -rf venv diff --git a/clients/client-python/README.md b/clients/client-python/README.md index 8515e255469..d69ba3e14ef 100644 --- a/clients/client-python/README.md +++ b/clients/client-python/README.md @@ -5,9 +5,14 @@ # Quick Start -1. Install dependency +1. Install current library in your local machine. ```bash - pip install -r requirements.txt + pip install -e . ``` -2. After the packages is installed, you can simply run `pytest` in any directory inside gravitino/ \ No newline at end of file +# Development Environment + +1. Install dependency + ```bash + pip install -e .[dev] + ``` \ No newline at end of file diff --git a/clients/client-python/gravitino/__init__.py b/clients/client-python/gravitino/__init__.py new file mode 100644 index 00000000000..fdd4d199d30 --- /dev/null +++ b/clients/client-python/gravitino/__init__.py @@ -0,0 +1,13 @@ +""" +Copyright 2024 Datastrato Pvt Ltd. +This software is licensed under the Apache License version 2. +""" + +from gravitino.gravitino_client import ( + GravitinoClient, + gravitino_metalake, + MetaLake, + Catalog, + Schema, + Table, +) diff --git a/clients/client-python/gravitino_client/__init__.py b/clients/client-python/gravitino/constants.py similarity index 84% rename from clients/client-python/gravitino_client/__init__.py rename to clients/client-python/gravitino/constants.py index c064f0b0f0c..bca1d178c49 100644 --- a/clients/client-python/gravitino_client/__init__.py +++ b/clients/client-python/gravitino/constants.py @@ -1,4 +1,6 @@ """ Copyright 2024 Datastrato Pvt Ltd. This software is licensed under the Apache License version 2. -""" \ No newline at end of file +""" + +TIMEOUT = 10 diff --git a/clients/client-python/__init__.py b/clients/client-python/gravitino/exceptions.py similarity index 96% rename from clients/client-python/__init__.py rename to clients/client-python/gravitino/exceptions.py index c064f0b0f0c..5779a3ad252 100644 --- a/clients/client-python/__init__.py +++ b/clients/client-python/gravitino/exceptions.py @@ -1,4 +1,4 @@ """ Copyright 2024 Datastrato Pvt Ltd. This software is licensed under the Apache License version 2. -""" \ No newline at end of file +""" diff --git a/clients/client-python/gravitino/gravitino_client.py b/clients/client-python/gravitino/gravitino_client.py new file mode 100644 index 00000000000..a8f97bb85c7 --- /dev/null +++ b/clients/client-python/gravitino/gravitino_client.py @@ -0,0 +1,158 @@ +""" +Copyright 2024 Datastrato Pvt Ltd. +This software is licensed under the Apache License version 2. +""" + +from gravitino.constants import TIMEOUT +from gravitino.service import initialize_service, service + + +class MetaLake: + def __init__(self, metalake_name: str): + self.name = metalake_name + self.service = service["service"] + self.metalake = self.service.get_metalake(self.name) + self.catalogs = self.service.list_catalogs(self.name) + + def __repr__(self): + return f"MetaLake<{self.name}>" + + def __getattr__(self, catalog_name): + if catalog_name in dir(self): + return Catalog(self.name, catalog_name) + + def __dir__(self): + return [catalog["name"] for catalog in self.catalogs] + + def __contains__(self, item): + return item in dir(self) + + +class Catalog: + def __init__(self, metalake_name: str, catalog_name: str): + self.metalake_name = metalake_name + self.catalog_name = catalog_name + self.name = catalog_name + self.service = service["service"] + self.schemas = self.service.list_schemas(metalake_name, catalog_name) + + def __repr__(self): + return f"Catalog<{self.name}>" + + def __getattr__(self, schema_name): + if schema_name in dir(self): + return Schema(self.metalake_name, self.catalog_name, schema_name) + + def __dir__(self): + return [schema["name"] for schema in self.schemas] + + def __contains__(self, item): + return item in dir(self) + + +class Schema: + def __init__(self, metalake_name: str, catalog_name: str, schema_name: str): + self.metalake_name = metalake_name + self.catalog_name = catalog_name + self.schema_name = schema_name + self.name = schema_name + self.service = service["service"] + self.tables = self.service.list_tables(metalake_name, catalog_name, schema_name) + + def __repr__(self): + return f"Schema<{self.name}>" + + def __getattr__(self, table_name): + if table_name in dir(self): + return Table( + self.metalake_name, self.catalog_name, self.schema_name, table_name + ) + + def __dir__(self): + return [table["name"] for table in self.tables] + + def __contains__(self, item): + return item in dir(self) + + +class Table: + def __init__( + self, metalake_name: str, catalog_name: str, schema_name: str, table_name: str + ): + self.metalake_name = metalake_name + self.catalog_name = catalog_name + self.schema_name = schema_name + self.table_name = table_name + self.name = schema_name + self.service = service["service"] + + def __repr__(self): + return f"Table<{self.name}>" + + def info(self): + return self.service.get_table( + self.metalake_name, self.catalog_name, self.schema_name, self.table_name + ) + + +class GravitinoClient: + def __init__( + self, + host: str, + *, + prefix: str = "/api", + timeout: int = TIMEOUT, + debug: bool = False, + ) -> None: + _base_url = f"{host.rstrip('/')}/{prefix.strip('/')}" + initialize_service(_base_url, timeout) + self.service = service["service"] + self.debug = debug + + @classmethod + def initialize_metalake( + cls, + host: str, + metalake_name: str, + *, + prefix: str = "/api", + timeout: int = TIMEOUT, + debug: bool = False, + ) -> MetaLake: + # keep in mind, all constructors should include same interface as __init__ function + client = cls( + host, + prefix=prefix, + timeout=timeout, + debug=debug, + ) + return client.get_metalake(metalake_name) + + @property + def version(self): + return self.service.get_version() + + def get_metalakes(self) -> [MetaLake]: + return [ + MetaLake(metalake.get("name")) for metalake in self.service.list_metalakes() + ] + + def get_metalake(self, metalake: str) -> MetaLake: + return MetaLake(metalake) + + +def gravitino_metalake( + host: str, + metalake_name: str, + *, + prefix: str = "/api", + timeout: int = TIMEOUT, + debug: bool = False, +) -> MetaLake: + return GravitinoClient.initialize_metalake( + host, + metalake_name, + prefix=prefix, + timeout=timeout, + debug=debug, + ) diff --git a/clients/client-python/gravitino/service.py b/clients/client-python/gravitino/service.py new file mode 100644 index 00000000000..9bd4cd6dfbe --- /dev/null +++ b/clients/client-python/gravitino/service.py @@ -0,0 +1,77 @@ +""" +Copyright 2024 Datastrato Pvt Ltd. +This software is licensed under the Apache License version 2. +""" + +from gravitino.utils import HTTPClient, unpack, Response +from gravitino.constants import TIMEOUT + + +class _Service: + def __init__( + self, + url: str, + timeout: int = TIMEOUT, + ) -> None: + self.http_client = HTTPClient(url, timeout=timeout) + + @unpack("version") + def get_version(self) -> Response: + return self.http_client.get("/version") + + @unpack("metalakes") + def list_metalakes(self) -> Response: + return self.http_client.get("/metalakes") + + @unpack("metalake") + def get_metalake(self, metalake: str) -> Response: + return self.http_client.get(f"/metalakes/{metalake}") + + @unpack("identifiers") + def list_catalogs(self, metalake: str) -> Response: + return self.http_client.get(f"/metalakes/{metalake}/catalogs/") + + @unpack("catalog") + def get_catalog(self, metalake: str, catalog: str) -> Response: + return self.http_client.get(f"/metalakes/{metalake}/catalogs/{catalog}") + + @unpack("identifiers") + def list_schemas(self, metalake: str, catalog: str) -> Response: + return self.http_client.get(f"/metalakes/{metalake}/catalogs/{catalog}/schemas") + + @unpack("schema") + def get_schema(self, metalake: str, catalog: str, schema: str) -> Response: + return self.http_client.get( + f"/metalakes/{metalake}/catalogs/{catalog}/schemas/{schema}" + ) + + @unpack("identifiers") + def list_tables(self, metalake: str, catalog: str, schema: str) -> Response: + return self.http_client.get( + f"/metalakes/{metalake}/catalogs/{catalog}/schemas/{schema}/tables" + ) + + @unpack("table") + def get_table( + self, metalake: str, catalog: str, schema: str, table: str + ) -> Response: + return self.http_client.get( + f"/metalakes/{metalake}/catalogs/{catalog}/schemas/{schema}/tables/{table}" + ) + + @unpack("names") + def list_partitions( + self, metalake: str, catalog: str, schema: str, table: str + ) -> Response: + return self.http_client.get( + f"/metalakes/{metalake}/catalogs/{catalog}/schemas/{schema}/tables/{table}/partitions" + ) + + +service = {} + + +def initialize_service(url: str, timeout: int = TIMEOUT): + global service + if not service: + service["service"] = _Service(url, timeout) diff --git a/clients/client-python/gravitino/typing.py b/clients/client-python/gravitino/typing.py new file mode 100644 index 00000000000..4dc7db893f4 --- /dev/null +++ b/clients/client-python/gravitino/typing.py @@ -0,0 +1,11 @@ +""" +Copyright 2024 Datastrato Pvt Ltd. +This software is licensed under the Apache License version 2. +""" + +from typing import Mapping, Sequence, Union + +# https://github.com/python/typing/issues/182#issuecomment-1320974824 +JSON_ro = Union[ + Mapping[str, "JSON_ro"], Sequence["JSON_ro"], str, int, float, bool, None +] diff --git a/clients/client-python/gravitino_client/core/__init__.py b/clients/client-python/gravitino/utils/__init__.py similarity index 58% rename from clients/client-python/gravitino_client/core/__init__.py rename to clients/client-python/gravitino/utils/__init__.py index 16d5ed38fd3..03295d89182 100644 --- a/clients/client-python/gravitino_client/core/__init__.py +++ b/clients/client-python/gravitino/utils/__init__.py @@ -2,5 +2,5 @@ Copyright 2024 Datastrato Pvt Ltd. This software is licensed under the Apache License version 2. """ -from .gravitino_client import GravitinoClient -from .dto import VersionDTO \ No newline at end of file + +from gravitino.utils.http_client import Response, HTTPClient, unpack diff --git a/clients/client-python/gravitino/utils/exceptions.py b/clients/client-python/gravitino/utils/exceptions.py new file mode 100644 index 00000000000..147a1698362 --- /dev/null +++ b/clients/client-python/gravitino/utils/exceptions.py @@ -0,0 +1,95 @@ +""" +Copyright 2024 Datastrato Pvt Ltd. +This software is licensed under the Apache License version 2. +""" + +import json + + +class HTTPError(Exception): + """Base of all other errors""" + + def __init__(self, error): + self.status_code = error.code + self.reason = error.reason + self.body = error.read() + self.headers = error.hdrs + + def json(self): + """ + :return: object of response error from the API + """ + try: + return json.loads(self.body.decode("utf-8")) + except json.decoder.JSONDecodeError: + return {"exception": self.body.decode("utf-8")} + + def __str__(self): + return self.json().get("exception") + + +class BadRequestsError(HTTPError): + pass + + +class UnauthorizedError(HTTPError): + pass + + +class ForbiddenError(HTTPError): + pass + + +class NotFoundError(HTTPError): + pass + + +class MethodNotAllowedError(HTTPError): + pass + + +class PayloadTooLargeError(HTTPError): + pass + + +class UnsupportedMediaTypeError(HTTPError): + pass + + +class TooManyRequestsError(HTTPError): + pass + + +class InternalServerError(HTTPError): + pass + + +class ServiceUnavailableError(HTTPError): + pass + + +class GatewayTimeoutError(HTTPError): + pass + + +err_dict = { + 400: BadRequestsError, + 401: UnauthorizedError, + 403: ForbiddenError, + 404: NotFoundError, + 405: MethodNotAllowedError, + 413: PayloadTooLargeError, + 415: UnsupportedMediaTypeError, + 429: TooManyRequestsError, + 500: InternalServerError, + 503: ServiceUnavailableError, + 504: GatewayTimeoutError, +} + + +def handle_error(error): + try: + exc = err_dict[error.code](error) + except KeyError: + return HTTPError(error) + return exc diff --git a/clients/client-python/gravitino/utils/http_client.py b/clients/client-python/gravitino/utils/http_client.py new file mode 100644 index 00000000000..870d2c2bc31 --- /dev/null +++ b/clients/client-python/gravitino/utils/http_client.py @@ -0,0 +1,145 @@ +""" +Copyright 2024 Datastrato Pvt Ltd. +This software is licensed under the Apache License version 2. +""" + +from urllib.request import Request, build_opener +from urllib.parse import urlencode +from urllib.error import HTTPError +import json as _json + +from gravitino.typing import JSON_ro +from gravitino.utils.exceptions import handle_error +from gravitino.constants import TIMEOUT + + +class Response: + def __init__(self, response): + self._status_code = response.getcode() + self._body = response.read() + self._headers = response.info() + self._url = response.url + + @property + def status_code(self): + return self._status_code + + @property + def url(self): + return self._url + + @property + def body(self): + return self._body + + @property + def headers(self): + return self._headers + + def json(self): + if self.body: + return _json.loads(self.body.decode("utf-8")) + else: + return None + + +class HTTPClient: + def __init__( + self, + host, + *, + request_headers=None, + timeout=TIMEOUT, + is_debug=False, + ) -> None: + self.host = host + self.request_headers = request_headers or {} + self.timeout = timeout + self.is_debug = is_debug + + def _build_url(self, endpoint=None, params=None): + url = self.host + + if endpoint: + url = "{}/{}".format(url.rstrip("/"), endpoint.lstrip("/")) + + if params: + params = {k: v for k, v in params.items() if v is not None} + url_values = urlencode(sorted(params.items()), True) + url = "{}?{}".format(url, url_values) + + return url + + def _update_headers(self, request_headers): + self.request_headers.update(request_headers) + + def _mask_auth_headers(self, headers): + if self.is_debug: + return headers + + _headers = {} + for key, value in headers.items(): + if key.lower() == "authorization": + _headers[key] = "******" + else: + _headers[key] = value + return _headers + + def _make_request(self, opener, request, timeout=None): + timeout = timeout or self.timeout + try: + return opener.open(request, timeout=timeout) + except HTTPError as err: + exc = handle_error(err) + exc.__cause__ = None + raise exc + + def _request( + self, method, endpoint, params=None, json=None, headers=None, timeout=None + ): + method = method.upper() + request_data = None + + if headers: + self._update_headers(headers) + if json: + request_data = _json.dumps(json).encode("utf-8") + + opener = build_opener() + request = Request(self._build_url(endpoint, params), data=request_data) + if self.request_headers: + for key, value in self.request_headers.items(): + request.add_header(key, value) + if request_data and ("Content-Type" not in self.request_headers): + request.add_header("Content-Type", "application/json") + + request.get_method = lambda: method + return Response(self._make_request(opener, request, timeout=timeout)) + + def get(self, endpoint, params=None, **kwargs): + return self._request("get", endpoint, params=params, **kwargs) + + def delete(self, endpoint, **kwargs): + return self._request("delete", endpoint, **kwargs) + + def post(self, endpoint, json=None, **kwargs): + return self._request("post", endpoint, json=json, **kwargs) + + def put(self, endpoint, json=None, **kwargs): + return self._request("put", endpoint, json=json, **kwargs) + + +def unpack(path: str): + def decorator(func): + def wrapper(*args, **kwargs) -> JSON_ro: + resp = func(*args, **kwargs) + rv = resp.json() + for p in path.split("."): + if p not in rv: + raise KeyError(f"The path '{path}' can't find in dict") + rv = rv.get(p) + return rv + + return wrapper + + return decorator diff --git a/clients/client-python/gravitino_client/core/dto.py b/clients/client-python/gravitino_client/core/dto.py deleted file mode 100644 index f6fc0f52c44..00000000000 --- a/clients/client-python/gravitino_client/core/dto.py +++ /dev/null @@ -1,11 +0,0 @@ -""" -Copyright 2024 Datastrato Pvt Ltd. -This software is licensed under the Apache License version 2. -""" -from dataclasses import dataclass - -@dataclass -class VersionDTO: - version: str - compile_date: str - git_commit: str diff --git a/clients/client-python/gravitino_client/core/gravitino_client.py b/clients/client-python/gravitino_client/core/gravitino_client.py deleted file mode 100644 index eed39431b0e..00000000000 --- a/clients/client-python/gravitino_client/core/gravitino_client.py +++ /dev/null @@ -1,56 +0,0 @@ -""" -Copyright 2024 Datastrato Pvt Ltd. -This software is licensed under the Apache License version 2. -""" -import requests -from requests.exceptions import HTTPError -from gravitino_client.core.dto import VersionDTO - - -class GravitinoClient: - """ - Gravitino Client for interacting with the Gravitino API, allowing the client to list, load, - create, and alter Metalakes. - - Attributes: - base_url (str): The base URL of the Gravitino API to which the client will make requests. - - Args: - base_url (str): The base URL for the Gravitino API. - """ - - def __init__(self, base_url): - """ - Initializes a new instance of the GravitinoClient. - - Args: - base_url (str): The base URL for the Gravitino API where the client will send requests. - """ - self.base_url = base_url - - def getVersion(self) -> VersionDTO: - """ - Retrieves the version information from the Gravitino API. - - This method makes a GET request to the Gravitino API and extracts the version information from the response, - wrapping it into a VersionDTO object. - - Returns: - VersionDTO: An object containing the version details, including version, compile date, and git commit hash. - - Raises: - HTTPError: An error from the requests library if the HTTP request returned an unsuccessful status code. - """ - try: - response = requests.get(f"{self.base_url}/api/version") - response.raise_for_status() - version_data = response.json() - version_info = version_data.get("version") - - return VersionDTO( - version=version_info['version'], - compile_date=version_info['compileDate'], - git_commit=version_info['gitCommit'] - ) - except HTTPError as e: - raise HTTPError(f"Failed to retrieve version information: {e}") diff --git a/clients/client-python/requirements-dev.txt b/clients/client-python/requirements-dev.txt new file mode 100644 index 00000000000..2cfa42afe23 --- /dev/null +++ b/clients/client-python/requirements-dev.txt @@ -0,0 +1,2 @@ +# Copyright 2024 Datastrato Pvt Ltd. +# This software is licensed under the Apache License version 2. \ No newline at end of file diff --git a/clients/client-python/requirements.txt b/clients/client-python/requirements.txt index 5f118436582..2cfa42afe23 100644 --- a/clients/client-python/requirements.txt +++ b/clients/client-python/requirements.txt @@ -1,4 +1,2 @@ # Copyright 2024 Datastrato Pvt Ltd. -# This software is licensed under the Apache License version 2. -pytest~=8.0.1 -requests~=2.31.0 \ No newline at end of file +# This software is licensed under the Apache License version 2. \ No newline at end of file diff --git a/clients/client-python/setup.py b/clients/client-python/setup.py new file mode 100644 index 00000000000..e130f2ed0fd --- /dev/null +++ b/clients/client-python/setup.py @@ -0,0 +1,23 @@ +""" +Copyright 2024 Datastrato Pvt Ltd. +This software is licensed under the Apache License version 2. +""" + +from setuptools import find_packages, setup + + +setup( + name="gravitino", + description="project description TBD", + version="0.0.1", + long_description=open("README.md").read(), + long_description_content_type="text/markdown", + url="https://github.com/datastrato/gravitino", + author="datastrato", + python_requires=">=3.8", + packages=find_packages(include=["gravitino", ".*"]), + install_requires=open("requirements.txt").read(), + extras_require={ + "dev": open("requirements-dev.txt").read(), + }, +) diff --git a/clients/client-python/tests/__init__.py b/clients/client-python/tests/__init__.py index c064f0b0f0c..5779a3ad252 100644 --- a/clients/client-python/tests/__init__.py +++ b/clients/client-python/tests/__init__.py @@ -1,4 +1,4 @@ """ Copyright 2024 Datastrato Pvt Ltd. This software is licensed under the Apache License version 2. -""" \ No newline at end of file +""" diff --git a/clients/client-python/tests/core/__init__.py b/clients/client-python/tests/core/__init__.py deleted file mode 100644 index c064f0b0f0c..00000000000 --- a/clients/client-python/tests/core/__init__.py +++ /dev/null @@ -1,4 +0,0 @@ -""" -Copyright 2024 Datastrato Pvt Ltd. -This software is licensed under the Apache License version 2. -""" \ No newline at end of file diff --git a/clients/client-python/tests/core/test_gravitino_client.py b/clients/client-python/tests/core/test_gravitino_client.py deleted file mode 100644 index 7d5ffba9eca..00000000000 --- a/clients/client-python/tests/core/test_gravitino_client.py +++ /dev/null @@ -1,45 +0,0 @@ -""" -Copyright 2024 Datastrato Pvt Ltd. -This software is licensed under the Apache License version 2. -""" -import pytest -import requests -from unittest.mock import MagicMock -from gravitino_client.core import GravitinoClient, VersionDTO - - -@pytest.fixture -def mock_get(monkeypatch): - mock = MagicMock() - monkeypatch.setattr("requests.get", mock) - return mock - - -def test_get_version_success(mock_get): - expected_version_dto = VersionDTO( - version="0.3.2-SNAPSHOT", - compile_date="25/01/2024 00:04:59", - git_commit="cb7a604bf19b6f992f00529e938cdd1d37af0187" - ) - mock_get.return_value.json.return_value = { - "code": 0, - "version": { - "version": "0.3.2-SNAPSHOT", - "compileDate": "25/01/2024 00:04:59", - "gitCommit": "cb7a604bf19b6f992f00529e938cdd1d37af0187" - } - } - - client = GravitinoClient(base_url="http://localhost:8090") - version_data = client.getVersion() - - assert version_data == expected_version_dto - - -def test_get_version_http_error(mock_get): - mock_get.side_effect = requests.exceptions.HTTPError - - client = GravitinoClient(base_url="http://localhost:8090") - - with pytest.raises(requests.exceptions.HTTPError): - client.getVersion() diff --git a/clients/client-python/tests/fixtures.py b/clients/client-python/tests/fixtures.py new file mode 100644 index 00000000000..dbf47969934 --- /dev/null +++ b/clients/client-python/tests/fixtures.py @@ -0,0 +1,104 @@ +""" +Copyright 2024 Datastrato Pvt Ltd. +This software is licensed under the Apache License version 2. +""" + +services_version = { + "version": "0.4.0", + "compileDate": "06/02/2024 08:37:11", + "gitCommit": "ae87dbdef5a749cdbed66d7f0e841cc809ad2510", +} +services_list_metalakes = [ + { + "name": "metalake_demo", + "comment": "comment", + "audit": {"creator": "anonymous", "createTime": "2024-03-30T13:49:53.382Z"}, + } +] +services_get_metalake = { + "name": "metalake_demo", + "comment": "comment", + "audit": {"creator": "anonymous", "createTime": "2024-03-30T13:49:53.382Z"}, +} +services_list_catalogs = [ + {"namespace": ["metalake_demo"], "name": "catalog_hive"}, + {"namespace": ["metalake_demo"], "name": "catalog_iceberg"}, + {"namespace": ["metalake_demo"], "name": "catalog_postgres"}, +] +services_get_catalog = {"namespace": ["metalake_demo"], "name": "catalog_hive"} +services_list_schemas = [ + {"namespace": ["metalake_demo", "catalog_hive"], "name": "default"}, + {"namespace": ["metalake_demo", "catalog_hive"], "name": "sales"}, +] +services_get_schema = {"namespace": ["metalake_demo", "catalog_hive"], "name": "sales"} +services_list_tables = [ + {"namespace": ["metalake_demo", "catalog_hive", "sales"], "name": "categories"}, + {"namespace": ["metalake_demo", "catalog_hive", "sales"], "name": "customers"}, + {"namespace": ["metalake_demo", "catalog_hive", "sales"], "name": "products"}, + {"namespace": ["metalake_demo", "catalog_hive", "sales"], "name": "sales"}, + {"namespace": ["metalake_demo", "catalog_hive", "sales"], "name": "stores"}, +] +services_get_table = { + "name": "sales", + "comment": "", + "columns": [ + { + "name": "sale_id", + "type": "integer", + "nullable": True, + "autoIncrement": False, + }, + { + "name": "employee_id", + "type": "integer", + "nullable": True, + "autoIncrement": False, + }, + { + "name": "store_id", + "type": "integer", + "nullable": True, + "autoIncrement": False, + }, + { + "name": "product_id", + "type": "integer", + "nullable": True, + "autoIncrement": False, + }, + { + "name": "customer_id", + "type": "integer", + "nullable": True, + "autoIncrement": False, + }, + {"name": "sold", "type": "date", "nullable": True, "autoIncrement": False}, + { + "name": "quantity", + "type": "integer", + "nullable": True, + "autoIncrement": False, + }, + { + "name": "total_amount", + "type": "decimal(10,2)", + "nullable": True, + "autoIncrement": False, + }, + ], + "properties": { + "input-format": "org.apache.hadoop.mapred.TextInputFormat", + "transient_lastDdlTime": "1711806631", + "output-format": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat", + "location": "hdfs://hive:9000/user/hive/warehouse/sales.db/sales", + "table-type": "MANAGED_TABLE", + "serde-lib": "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe", + "STATS_GENERATED_VIA_STATS_TASK": "workaround for potential lack of HIVE-12730", + "serde-name": "sales", + }, + "audit": {"creator": "anonymous", "createTime": "2024-03-30T13:50:31.289Z"}, + "distribution": {"strategy": "none", "number": 0, "funcArgs": []}, + "sortOrders": [], + "partitioning": [], + "indexes": [], +} diff --git a/clients/client-python/tests/test_gravitino_client.py b/clients/client-python/tests/test_gravitino_client.py new file mode 100644 index 00000000000..ad497b2ba7f --- /dev/null +++ b/clients/client-python/tests/test_gravitino_client.py @@ -0,0 +1,58 @@ +""" +Copyright 2024 Datastrato Pvt Ltd. +This software is licensed under the Apache License version 2. +""" + +import unittest + +from gravitino import GravitinoClient, gravitino_metalake +from .utils import services_fixtures + + +@services_fixtures +class TestGravitinoClient(unittest.TestCase): + def setUp(self): + self.client = GravitinoClient("http://localhost:9000") + + def test_version(self, *args): + self.assertIn("version", list(self.client.version.keys())) + + def test_get_metalakes(self, *args): + metalakes = self.client.get_metalakes() + self.assertEqual(len(metalakes), 1) + self.assertEqual(metalakes[0].name, "metalake_demo") + + def test_get_metalake(self, *args): + metalake = self.client.get_metalake("metalake_demo") + self.assertEqual(metalake.name, "metalake_demo") + self.assertIn("catalog_hive", metalake) + + def test_get_catalog(self, *args): + catalog = self.client.get_metalake("metalake_demo").catalog_hive + self.assertEqual(catalog.name, "catalog_hive") + self.assertIn("sales", catalog) + + def test_get_schema(self, *args): + schema = self.client.get_metalake("metalake_demo").catalog_hive.sales + self.assertEqual(schema.name, "sales") + self.assertIn("sales", schema) + + def test_get_table(self, *args): + table = self.client.get_metalake("metalake_demo").catalog_hive.sales.sales + self.assertEqual(table.name, "sales") + self.assertEqual(table.info().get("name"), "sales") + + def test_dynamic_properties(self, *args): + metalake = self.client.get_metalake("metalake_demo") + self.assertIn("catalog_hive", dir(metalake)) + self.assertIn("catalog_iceberg", dir(metalake)) + self.assertIn("catalog_postgres", dir(metalake)) + self.assertEqual(metalake.catalog_hive.name, "catalog_hive") + self.assertEqual(metalake.catalog_hive.sales.name, "sales") + + +@services_fixtures +class TestGravitinoMetalake(unittest.TestCase): + def test_gravitino_metalake(self, *args): + metalake = gravitino_metalake("http://localhost:9000", "metalake_demo") + self.assertEqual(metalake.name, "metalake_demo") diff --git a/clients/client-python/tests/utils.py b/clients/client-python/tests/utils.py new file mode 100644 index 00000000000..9953e929b71 --- /dev/null +++ b/clients/client-python/tests/utils.py @@ -0,0 +1,49 @@ +""" +Copyright 2024 Datastrato Pvt Ltd. +This software is licensed under the Apache License version 2. +""" + +from unittest.mock import patch +from . import fixtures + + +def services_fixtures(cls): + @patch( + "gravitino.service._Service.get_version", return_value=fixtures.services_version + ) + @patch( + "gravitino.service._Service.list_metalakes", + return_value=fixtures.services_list_metalakes, + ) + @patch( + "gravitino.service._Service.get_metalake", + return_value=fixtures.services_get_metalake, + ) + @patch( + "gravitino.service._Service.list_catalogs", + return_value=fixtures.services_list_catalogs, + ) + @patch( + "gravitino.service._Service.get_catalog", + return_value=fixtures.services_get_catalog, + ) + @patch( + "gravitino.service._Service.list_schemas", + return_value=fixtures.services_list_schemas, + ) + @patch( + "gravitino.service._Service.get_schema", + return_value=fixtures.services_get_schema, + ) + @patch( + "gravitino.service._Service.list_tables", + return_value=fixtures.services_list_tables, + ) + @patch( + "gravitino.service._Service.get_table", + return_value=fixtures.services_get_table, + ) + class Wrapper(cls): + pass + + return Wrapper From 46ebaf6c55a81bef6e3c35be8a62e71caf687879 Mon Sep 17 00:00:00 2001 From: cai can <94670132+caican00@users.noreply.github.com> Date: Tue, 2 Apr 2024 15:12:52 +0800 Subject: [PATCH 20/20] [#2541] feat(spark-connector): support basic DDL and DML operations to iceberg catalog (#2544) ### What changes were proposed in this pull request? 1. support DDL operations to iceberg catalog. 2. support read and write operations to iceberg Table. ### Why are the changes needed? support basic DDL and DML operations for iceberg table using sparksql. Fix: https://github.com/datastrato/gravitino/issues/2541 ### Does this PR introduce _any_ user-facing change? Yes, users can use sparksql to do iceberg table ddl and read&write operations. ### How was this patch tested? New Iceberg ITs. --- .../iceberg/IcebergCatalogOperations.java | 5 + integration-test/build.gradle.kts | 1 - .../integration/test/spark/SparkCommonIT.java | 86 +++++++++-- .../integration/test/spark/SparkEnvIT.java | 28 +++- .../test/spark/hive/SparkHiveCatalogIT.java | 5 + .../spark/iceberg/SparkIcebergCatalogIT.java | 34 ++++ .../GravitinoCatalogAdaptorFactory.java | 3 + .../catalog/GravitinoCatalogManager.java | 1 + .../connector/iceberg/IcebergAdaptor.java | 145 ++++++++++++++++++ .../iceberg/IcebergPropertiesConstants.java | 40 +++++ .../iceberg/IcebergPropertiesConverter.java | 23 +++ .../connector/iceberg/SparkIcebergTable.java | 23 +++ 12 files changed, 368 insertions(+), 26 deletions(-) create mode 100644 integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java create mode 100644 spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/IcebergAdaptor.java create mode 100644 spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/IcebergPropertiesConstants.java create mode 100644 spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/IcebergPropertiesConverter.java create mode 100644 spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/SparkIcebergTable.java diff --git a/catalogs/catalog-lakehouse-iceberg/src/main/java/com/datastrato/gravitino/catalog/lakehouse/iceberg/IcebergCatalogOperations.java b/catalogs/catalog-lakehouse-iceberg/src/main/java/com/datastrato/gravitino/catalog/lakehouse/iceberg/IcebergCatalogOperations.java index bec07775f7a..642c256dc8c 100644 --- a/catalogs/catalog-lakehouse-iceberg/src/main/java/com/datastrato/gravitino/catalog/lakehouse/iceberg/IcebergCatalogOperations.java +++ b/catalogs/catalog-lakehouse-iceberg/src/main/java/com/datastrato/gravitino/catalog/lakehouse/iceberg/IcebergCatalogOperations.java @@ -314,6 +314,11 @@ public boolean dropSchema(NameIdentifier ident, boolean cascade) throws NonEmpty */ @Override public NameIdentifier[] listTables(Namespace namespace) throws NoSuchSchemaException { + NameIdentifier schemaIdent = NameIdentifier.of(namespace.levels()); + if (!schemaExists(schemaIdent)) { + throw new NoSuchSchemaException("Schema (database) does not exist %s", namespace); + } + try { ListTablesResponse listTablesResponse = icebergTableOps.listTable(IcebergTableOpsHelper.getIcebergNamespace(namespace)); diff --git a/integration-test/build.gradle.kts b/integration-test/build.gradle.kts index ffdc62e653b..d7253f5112a 100644 --- a/integration-test/build.gradle.kts +++ b/integration-test/build.gradle.kts @@ -30,7 +30,6 @@ dependencies { testImplementation(project(":server")) testImplementation(project(":server-common")) testImplementation(project(":spark-connector")) { - exclude("org.apache.iceberg") exclude("org.apache.hadoop", "hadoop-client-api") exclude("org.apache.hadoop", "hadoop-client-runtime") } 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 731836370fe..04f1c0dbc05 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 @@ -8,6 +8,7 @@ import com.datastrato.gravitino.integration.test.util.spark.SparkTableInfo.SparkColumnInfo; import com.datastrato.gravitino.integration.test.util.spark.SparkTableInfoChecker; import com.google.common.collect.ImmutableMap; +import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -15,6 +16,7 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import org.apache.commons.io.FileUtils; import org.apache.hadoop.fs.Path; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; @@ -22,14 +24,17 @@ 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.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.condition.EnabledIf; -import org.junit.platform.commons.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public abstract class SparkCommonIT extends SparkEnvIT { + private static final Logger LOG = LoggerFactory.getLogger(SparkCommonIT.class); // To generate test data for write&read table. protected static final Map typeConstant = @@ -61,14 +66,36 @@ private static String getInsertWithPartitionSql( // Whether supports [CLUSTERED BY col_name3 SORTED BY col_name INTO num_buckets BUCKETS] protected abstract boolean supportsSparkSQLClusteredBy(); - // Use a custom database not the original default database because SparkIT couldn't read&write - // data to tables in default database. The main reason is default database location is + protected abstract boolean supportsPartition(); + + // Use a custom database not the original default database because SparkCommonIT couldn't + // read&write data to tables in default database. The main reason is default database location is // determined by `hive.metastore.warehouse.dir` in hive-site.xml which is local HDFS address // not real HDFS address. The location of tables created under default database is like // hdfs://localhost:9000/xxx which couldn't read write data from SparkCommonIT. Will use default // database after spark connector support Alter database xx set location command. @BeforeAll - void initDefaultDatabase() { + void initDefaultDatabase() throws IOException { + // In embedded mode, derby acts as the backend database for the hive metastore + // and creates a directory named metastore_db to store metadata, + // supporting only one connection at a time. + // Previously, only SparkHiveCatalogIT accessed derby without any exceptions. + // Now, SparkIcebergCatalogIT exists at the same time. + // This exception about `ERROR XSDB6: Another instance of Derby may have already + // booted the database {GRAVITINO_HOME}/integration-test/metastore_db` will occur when + // SparkIcebergCatalogIT is initialized after the Sparkhivecatalogit is executed. + // The main reason is that the lock file in the metastore_db directory is not cleaned so that a + // new connection cannot be created, + // so a clean operation is done here to ensure that a new connection can be created. + File hiveLocalMetaStorePath = new File("metastore_db"); + try { + if (hiveLocalMetaStorePath.exists()) { + FileUtils.deleteDirectory(hiveLocalMetaStorePath); + } + } catch (IOException e) { + LOG.error(String.format("delete director %s failed.", hiveLocalMetaStorePath), e); + throw e; + } sql("USE " + getCatalogName()); createDatabaseIfNotExists(getDefaultDatabase()); } @@ -79,6 +106,26 @@ void init() { sql("USE " + getDefaultDatabase()); } + @AfterAll + void cleanUp() { + sql("USE " + getCatalogName()); + getDatabases() + .forEach(database -> sql(String.format("DROP DATABASE IF EXISTS %s CASCADE", database))); + } + + @Test + void testListTables() { + String tableName = "t_list"; + dropTableIfExists(tableName); + Set tableNames = listTableNames(); + Assertions.assertFalse(tableNames.contains(tableName)); + createSimpleTable(tableName); + tableNames = listTableNames(); + Assertions.assertTrue(tableNames.contains(tableName)); + Assertions.assertThrowsExactly( + NoSuchNamespaceException.class, () -> sql("SHOW TABLES IN nonexistent_schema")); + } + @Test void testLoadCatalogs() { Set catalogs = getCatalogs(); @@ -89,20 +136,20 @@ void testLoadCatalogs() { void testCreateAndLoadSchema() { String testDatabaseName = "t_create1"; dropDatabaseIfExists(testDatabaseName); - sql("CREATE DATABASE " + testDatabaseName); + sql("CREATE DATABASE " + testDatabaseName + " WITH DBPROPERTIES (ID=001);"); Map databaseMeta = getDatabaseMetadata(testDatabaseName); Assertions.assertFalse(databaseMeta.containsKey("Comment")); Assertions.assertTrue(databaseMeta.containsKey("Location")); Assertions.assertEquals("datastrato", databaseMeta.get("Owner")); String properties = databaseMeta.get("Properties"); - Assertions.assertTrue(StringUtils.isBlank(properties)); + Assertions.assertTrue(properties.contains("(ID,001)")); testDatabaseName = "t_create2"; dropDatabaseIfExists(testDatabaseName); String testDatabaseLocation = "/tmp/" + testDatabaseName; sql( String.format( - "CREATE DATABASE %s COMMENT 'comment' LOCATION '%s'\n" + " WITH DBPROPERTIES (ID=001);", + "CREATE DATABASE %s COMMENT 'comment' LOCATION '%s'\n" + " WITH DBPROPERTIES (ID=002);", testDatabaseName, testDatabaseLocation)); databaseMeta = getDatabaseMetadata(testDatabaseName); String comment = databaseMeta.get("Comment"); @@ -111,19 +158,22 @@ void testCreateAndLoadSchema() { // underlying catalog may change /tmp/t_create2 to file:/tmp/t_create2 Assertions.assertTrue(databaseMeta.get("Location").contains(testDatabaseLocation)); properties = databaseMeta.get("Properties"); - Assertions.assertEquals("((ID,001))", properties); + Assertions.assertTrue(properties.contains("(ID,002)")); } @Test void testAlterSchema() { String testDatabaseName = "t_alter"; dropDatabaseIfExists(testDatabaseName); - sql("CREATE DATABASE " + testDatabaseName); + sql("CREATE DATABASE " + testDatabaseName + " WITH DBPROPERTIES (ID=001);"); Assertions.assertTrue( - StringUtils.isBlank(getDatabaseMetadata(testDatabaseName).get("Properties"))); + getDatabaseMetadata(testDatabaseName).get("Properties").contains("(ID,001)")); - sql(String.format("ALTER DATABASE %s SET DBPROPERTIES ('ID'='001')", testDatabaseName)); - Assertions.assertEquals("((ID,001))", getDatabaseMetadata(testDatabaseName).get("Properties")); + sql(String.format("ALTER DATABASE %s SET DBPROPERTIES ('ID'='002')", testDatabaseName)); + Assertions.assertFalse( + getDatabaseMetadata(testDatabaseName).get("Properties").contains("(ID,001)")); + Assertions.assertTrue( + getDatabaseMetadata(testDatabaseName).get("Properties").contains("(ID,002)")); // Hive metastore doesn't support alter database location, therefore this test method // doesn't verify ALTER DATABASE database_name SET LOCATION 'new_location'. @@ -334,9 +384,9 @@ void testAlterTableUpdateColumnType() { checkTableColumns(tableName, simpleTableColumns, getTableInfo(tableName)); sql(String.format("ALTER TABLE %S ADD COLUMNS (col1 int)", tableName)); - sql(String.format("ALTER TABLE %S CHANGE COLUMN col1 col1 string", tableName)); + sql(String.format("ALTER TABLE %S CHANGE COLUMN col1 col1 bigint", tableName)); ArrayList updateColumns = new ArrayList<>(simpleTableColumns); - updateColumns.add(SparkColumnInfo.of("col1", DataTypes.StringType, null)); + updateColumns.add(SparkColumnInfo.of("col1", DataTypes.LongType, null)); checkTableColumns(tableName, updateColumns, getTableInfo(tableName)); } @@ -354,7 +404,7 @@ void testAlterTableRenameColumn() { sql(String.format("ALTER TABLE %S ADD COLUMNS (col1 int)", tableName)); sql( String.format( - "ALTER TABLE %S RENAME COLUMN %S TO %S", tableName, oldColumnName, newColumnName)); + "ALTER TABLE %s RENAME COLUMN %s TO %s", tableName, oldColumnName, newColumnName)); ArrayList renameColumns = new ArrayList<>(simpleTableColumns); renameColumns.add(SparkColumnInfo.of(newColumnName, DataTypes.IntegerType, null)); checkTableColumns(tableName, renameColumns, getTableInfo(tableName)); @@ -373,7 +423,7 @@ void testUpdateColumnPosition() { sql( String.format( - "CREATE TABLE %s (id STRING COMMENT '', name STRING COMMENT '', age STRING COMMENT '') USING PARQUET", + "CREATE TABLE %s (id STRING COMMENT '', name STRING COMMENT '', age STRING COMMENT '')", tableName)); checkTableColumns(tableName, simpleTableColumns, getTableInfo(tableName)); @@ -456,12 +506,13 @@ void testComplexType() { } @Test + @EnabledIf("supportsPartition") void testCreateDatasourceFormatPartitionTable() { String tableName = "datasource_partition_table"; dropTableIfExists(tableName); String createTableSQL = getCreateSimpleTableString(tableName); - createTableSQL = createTableSQL + "USING PARQUET PARTITIONED BY (name, age)"; + createTableSQL = createTableSQL + " USING PARQUET PARTITIONED BY (name, age)"; sql(createTableSQL); SparkTableInfo tableInfo = getTableInfo(tableName); SparkTableInfoChecker checker = @@ -558,6 +609,7 @@ void testInsertTableAsSelect() { } @Test + @EnabledIf("supportsPartition") void testInsertDatasourceFormatPartitionTableAsSelect() { String tableName = "insert_select_partition_table"; String newTableName = "new_" + tableName; diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/SparkEnvIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/SparkEnvIT.java index 373fbdca4a6..096402e6121 100644 --- a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/SparkEnvIT.java +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/SparkEnvIT.java @@ -12,6 +12,7 @@ import com.datastrato.gravitino.integration.test.container.HiveContainer; import com.datastrato.gravitino.integration.test.util.spark.SparkUtilIT; import com.datastrato.gravitino.spark.connector.GravitinoSparkConfig; +import com.datastrato.gravitino.spark.connector.iceberg.IcebergPropertiesConstants; import com.datastrato.gravitino.spark.connector.plugin.GravitinoSparkPlugin; import com.google.common.collect.Maps; import java.io.IOException; @@ -37,6 +38,7 @@ public abstract class SparkEnvIT extends SparkUtilIT { private SparkSession sparkSession; private String hiveMetastoreUri = "thrift://127.0.0.1:9083"; private String gravitinoUri = "http://127.0.0.1:8090"; + private String warehouse; protected abstract String getCatalogName(); @@ -79,8 +81,18 @@ private void initMetalakeAndCatalogs() { client.createMetalake(NameIdentifier.of(metalakeName), "", Collections.emptyMap()); GravitinoMetalake metalake = client.loadMetalake(NameIdentifier.of(metalakeName)); Map properties = Maps.newHashMap(); - properties.put(GravitinoSparkConfig.GRAVITINO_HIVE_METASTORE_URI, hiveMetastoreUri); - + switch (getProvider()) { + case "hive": + properties.put(GravitinoSparkConfig.GRAVITINO_HIVE_METASTORE_URI, hiveMetastoreUri); + break; + case "lakehouse-iceberg": + properties.put(IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_BACKEND, "hive"); + properties.put(IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_WAREHOUSE, warehouse); + properties.put(IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_URI, hiveMetastoreUri); + break; + default: + throw new IllegalArgumentException("Unsupported provider: " + getProvider()); + } metalake.createCatalog( NameIdentifier.of(metalakeName, getCatalogName()), Catalog.Type.RELATIONAL, @@ -102,6 +114,11 @@ private void initHiveEnv() { "thrift://%s:%d", containerSuite.getHiveContainer().getContainerIpAddress(), HiveContainer.HIVE_METASTORE_PORT); + warehouse = + String.format( + "hdfs://%s:%d/user/hive/warehouse", + containerSuite.getHiveContainer().getContainerIpAddress(), + HiveContainer.HDFS_DEFAULTFS_PORT); } private void initHdfsFileSystem() { @@ -129,12 +146,7 @@ private void initSparkEnv() { .config(GravitinoSparkConfig.GRAVITINO_URI, gravitinoUri) .config(GravitinoSparkConfig.GRAVITINO_METALAKE, metalakeName) .config("hive.exec.dynamic.partition.mode", "nonstrict") - .config( - "spark.sql.warehouse.dir", - String.format( - "hdfs://%s:%d/user/hive/warehouse", - containerSuite.getHiveContainer().getContainerIpAddress(), - HiveContainer.HDFS_DEFAULTFS_PORT)) + .config("spark.sql.warehouse.dir", warehouse) .enableHiveSupport() .getOrCreate(); } 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 bc513eafa79..91bea87a2aa 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 @@ -40,6 +40,11 @@ protected boolean supportsSparkSQLClusteredBy() { return true; } + @Override + protected boolean supportsPartition() { + return true; + } + @Test public void testCreateHiveFormatPartitionTable() { String tableName = "hive_partition_table"; diff --git a/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java new file mode 100644 index 00000000000..53cd78db2d1 --- /dev/null +++ b/integration-test/src/test/java/com/datastrato/gravitino/integration/test/spark/iceberg/SparkIcebergCatalogIT.java @@ -0,0 +1,34 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ +package com.datastrato.gravitino.integration.test.spark.iceberg; + +import com.datastrato.gravitino.integration.test.spark.SparkCommonIT; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.TestInstance; + +@Tag("gravitino-docker-it") +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public class SparkIcebergCatalogIT extends SparkCommonIT { + + @Override + protected String getCatalogName() { + return "iceberg"; + } + + @Override + protected String getProvider() { + return "lakehouse-iceberg"; + } + + @Override + protected boolean supportsSparkSQLClusteredBy() { + return false; + } + + @Override + protected boolean supportsPartition() { + return false; + } +} diff --git a/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/GravitinoCatalogAdaptorFactory.java b/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/GravitinoCatalogAdaptorFactory.java index 23a13ceeec2..0599f5cad1b 100644 --- a/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/GravitinoCatalogAdaptorFactory.java +++ b/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/GravitinoCatalogAdaptorFactory.java @@ -6,6 +6,7 @@ package com.datastrato.gravitino.spark.connector; import com.datastrato.gravitino.spark.connector.hive.HiveAdaptor; +import com.datastrato.gravitino.spark.connector.iceberg.IcebergAdaptor; import java.util.Locale; /** @@ -17,6 +18,8 @@ public static GravitinoCatalogAdaptor createGravitinoAdaptor(String provider) { switch (provider.toLowerCase(Locale.ROOT)) { case "hive": return new HiveAdaptor(); + case "lakehouse-iceberg": + return new IcebergAdaptor(); default: throw new RuntimeException(String.format("Provider:%s is not supported yet", provider)); } diff --git a/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/catalog/GravitinoCatalogManager.java b/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/catalog/GravitinoCatalogManager.java index c933cb6d2d7..5b610201817 100644 --- a/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/catalog/GravitinoCatalogManager.java +++ b/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/catalog/GravitinoCatalogManager.java @@ -58,6 +58,7 @@ public void close() { Preconditions.checkState(!isClosed, "Gravitino Catalog is already closed"); isClosed = true; gravitinoClient.close(); + gravitinoCatalogManager = null; } public Catalog getGravitinoCatalogInfo(String name) { diff --git a/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/IcebergAdaptor.java b/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/IcebergAdaptor.java new file mode 100644 index 00000000000..cf73dfb0427 --- /dev/null +++ b/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/IcebergAdaptor.java @@ -0,0 +1,145 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ + +package com.datastrato.gravitino.spark.connector.iceberg; + +import com.datastrato.gravitino.rel.Table; +import com.datastrato.gravitino.spark.connector.GravitinoCatalogAdaptor; +import com.datastrato.gravitino.spark.connector.PropertiesConverter; +import com.datastrato.gravitino.spark.connector.table.SparkBaseTable; +import com.google.common.base.Preconditions; +import java.util.HashMap; +import java.util.Locale; +import java.util.Map; +import org.apache.commons.lang3.StringUtils; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +/** IcebergAdaptor provides specific operations for Iceberg Catalog to adapt to GravitinoCatalog. */ +public class IcebergAdaptor implements GravitinoCatalogAdaptor { + + private void initHiveProperties( + String catalogBackend, + Map gravitinoProperties, + HashMap icebergProperties) { + String metastoreUri = + gravitinoProperties.get(IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_URI); + Preconditions.checkArgument( + StringUtils.isNotBlank(metastoreUri), + "Couldn't get " + + IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_URI + + " from Iceberg Catalog properties"); + String hiveWarehouse = + gravitinoProperties.get(IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_WAREHOUSE); + Preconditions.checkArgument( + StringUtils.isNotBlank(hiveWarehouse), + "Couldn't get " + + IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_WAREHOUSE + + " from Iceberg Catalog properties"); + icebergProperties.put( + IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_TYPE, + catalogBackend.toLowerCase(Locale.ENGLISH)); + icebergProperties.put(IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_URI, metastoreUri); + icebergProperties.put( + IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_WAREHOUSE, hiveWarehouse); + } + + private void initJdbcProperties( + String catalogBackend, + Map gravitinoProperties, + HashMap icebergProperties) { + String jdbcUri = + gravitinoProperties.get(IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_URI); + Preconditions.checkArgument( + StringUtils.isNotBlank(jdbcUri), + "Couldn't get " + + IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_URI + + " from Iceberg Catalog properties"); + String jdbcWarehouse = + gravitinoProperties.get(IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_WAREHOUSE); + Preconditions.checkArgument( + StringUtils.isNotBlank(jdbcWarehouse), + "Couldn't get " + + IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_WAREHOUSE + + " from Iceberg Catalog properties"); + String jdbcUser = gravitinoProperties.get(IcebergPropertiesConstants.GRAVITINO_JDBC_USER); + Preconditions.checkArgument( + StringUtils.isNotBlank(jdbcUser), + "Couldn't get " + + IcebergPropertiesConstants.GRAVITINO_JDBC_USER + + " from Iceberg Catalog properties"); + String jdbcPassword = + gravitinoProperties.get(IcebergPropertiesConstants.GRAVITINO_JDBC_PASSWORD); + Preconditions.checkArgument( + StringUtils.isNotBlank(jdbcPassword), + "Couldn't get " + + IcebergPropertiesConstants.GRAVITINO_JDBC_PASSWORD + + " from Iceberg Catalog properties"); + String jdbcDriver = + gravitinoProperties.get(IcebergPropertiesConstants.GRAVITINO_ICEBERG_JDBC_DRIVER); + Preconditions.checkArgument( + StringUtils.isNotBlank(jdbcDriver), + "Couldn't get " + + IcebergPropertiesConstants.GRAVITINO_ICEBERG_JDBC_DRIVER + + " from Iceberg Catalog properties"); + icebergProperties.put( + IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_TYPE, + catalogBackend.toLowerCase(Locale.ROOT)); + icebergProperties.put(IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_URI, jdbcUri); + icebergProperties.put( + IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_WAREHOUSE, jdbcWarehouse); + icebergProperties.put(IcebergPropertiesConstants.GRAVITINO_ICEBERG_JDBC_USER, jdbcUser); + icebergProperties.put(IcebergPropertiesConstants.GRAVITINO_ICEBERG_JDBC_PASSWORD, jdbcPassword); + icebergProperties.put(IcebergPropertiesConstants.GRAVITINO_ICEBERG_JDBC_DRIVER, jdbcDriver); + } + + @Override + public PropertiesConverter getPropertiesConverter() { + return new IcebergPropertiesConverter(); + } + + @Override + public SparkBaseTable createSparkTable( + Identifier identifier, + Table gravitinoTable, + TableCatalog sparkCatalog, + PropertiesConverter propertiesConverter) { + return new SparkIcebergTable(identifier, gravitinoTable, sparkCatalog, propertiesConverter); + } + + @Override + public TableCatalog createAndInitSparkCatalog( + String name, CaseInsensitiveStringMap options, Map properties) { + Preconditions.checkArgument( + properties != null, "Iceberg Catalog properties should not be null"); + + String catalogBackend = + properties.get(IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_BACKEND); + Preconditions.checkArgument( + StringUtils.isNotBlank(catalogBackend), "Iceberg Catalog backend should not be empty."); + + HashMap all = new HashMap<>(options); + + switch (catalogBackend.toLowerCase(Locale.ENGLISH)) { + case IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_BACKEND_HIVE: + initHiveProperties(catalogBackend, properties, all); + break; + case IcebergPropertiesConstants.GRAVITINO_ICEBERG_CATALOG_BACKEND_JDBC: + initJdbcProperties(catalogBackend, properties, all); + break; + default: + // SparkCatalog does not support Memory type catalog + throw new IllegalArgumentException( + "Unsupported Iceberg Catalog backend: " + catalogBackend); + } + + TableCatalog icebergCatalog = new SparkCatalog(); + icebergCatalog.initialize(name, new CaseInsensitiveStringMap(all)); + + return icebergCatalog; + } +} diff --git a/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/IcebergPropertiesConstants.java b/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/IcebergPropertiesConstants.java new file mode 100644 index 00000000000..d69964785ab --- /dev/null +++ b/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/IcebergPropertiesConstants.java @@ -0,0 +1,40 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ + +package com.datastrato.gravitino.spark.connector.iceberg; + +import com.datastrato.gravitino.catalog.lakehouse.iceberg.IcebergCatalogPropertiesMetadata; +import com.google.common.annotations.VisibleForTesting; + +public class IcebergPropertiesConstants { + + @VisibleForTesting + public static final String GRAVITINO_ICEBERG_CATALOG_BACKEND = + IcebergCatalogPropertiesMetadata.CATALOG_BACKEND_NAME; + + @VisibleForTesting + public static final String GRAVITINO_ICEBERG_CATALOG_WAREHOUSE = + IcebergCatalogPropertiesMetadata.WAREHOUSE; + + @VisibleForTesting + public static final String GRAVITINO_ICEBERG_CATALOG_URI = IcebergCatalogPropertiesMetadata.URI; + + public static final String GRAVITINO_JDBC_USER = + IcebergCatalogPropertiesMetadata.GRAVITINO_JDBC_USER; + public static final String GRAVITINO_ICEBERG_JDBC_USER = + IcebergCatalogPropertiesMetadata.ICEBERG_JDBC_USER; + public static final String GRAVITINO_JDBC_PASSWORD = + IcebergCatalogPropertiesMetadata.GRAVITINO_JDBC_PASSWORD; + public static final String GRAVITINO_ICEBERG_JDBC_PASSWORD = + IcebergCatalogPropertiesMetadata.ICEBERG_JDBC_PASSWORD; + public static final String GRAVITINO_ICEBERG_JDBC_DRIVER = + IcebergCatalogPropertiesMetadata.GRAVITINO_JDBC_DRIVER; + + public static final String GRAVITINO_ICEBERG_CATALOG_TYPE = "type"; + public static final String GRAVITINO_ICEBERG_CATALOG_BACKEND_HIVE = "hive"; + public static final String GRAVITINO_ICEBERG_CATALOG_BACKEND_JDBC = "jdbc"; + + private IcebergPropertiesConstants() {} +} diff --git a/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/IcebergPropertiesConverter.java b/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/IcebergPropertiesConverter.java new file mode 100644 index 00000000000..f96107c814d --- /dev/null +++ b/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/IcebergPropertiesConverter.java @@ -0,0 +1,23 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ + +package com.datastrato.gravitino.spark.connector.iceberg; + +import com.datastrato.gravitino.spark.connector.PropertiesConverter; +import java.util.HashMap; +import java.util.Map; + +/** Transform Iceberg catalog properties between Spark and Gravitino. */ +public class IcebergPropertiesConverter implements PropertiesConverter { + @Override + public Map toGravitinoTableProperties(Map properties) { + return new HashMap<>(properties); + } + + @Override + public Map toSparkTableProperties(Map properties) { + return new HashMap<>(properties); + } +} diff --git a/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/SparkIcebergTable.java b/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/SparkIcebergTable.java new file mode 100644 index 00000000000..aabdf149efa --- /dev/null +++ b/spark-connector/src/main/java/com/datastrato/gravitino/spark/connector/iceberg/SparkIcebergTable.java @@ -0,0 +1,23 @@ +/* + * Copyright 2024 Datastrato Pvt Ltd. + * This software is licensed under the Apache License version 2. + */ + +package com.datastrato.gravitino.spark.connector.iceberg; + +import com.datastrato.gravitino.rel.Table; +import com.datastrato.gravitino.spark.connector.PropertiesConverter; +import com.datastrato.gravitino.spark.connector.table.SparkBaseTable; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; + +public class SparkIcebergTable extends SparkBaseTable { + + public SparkIcebergTable( + Identifier identifier, + Table gravitinoTable, + TableCatalog sparkIcebergCatalog, + PropertiesConverter propertiesConverter) { + super(identifier, gravitinoTable, sparkIcebergCatalog, propertiesConverter); + } +}