From bd7c8bed54cb21b80f156326e10c26fa8c445c03 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Mon, 27 Jun 2022 19:26:28 -0700 Subject: [PATCH 01/22] Druid catalog basics Catalog object model for tables, columns Druid metadata DB storage (as an exension) REST API to update the catalog (as an extension) Integration tests Model only: no planner integration yet --- .travis.yml | 13 +- .../apache/druid/data/input/InputFormat.java | 10 +- .../apache/druid/data/input/InputSource.java | 12 +- .../druid/data/input/impl/CsvInputFormat.java | 2 + .../data/input/impl/DelimitedInputFormat.java | 3 +- .../data/input/impl/HttpInputSource.java | 2 + .../data/input/impl/InlineInputSource.java | 7 +- .../data/input/impl/JsonInputFormat.java | 6 +- .../data/input/impl/LocalInputSource.java | 1 + .../metadata/DefaultPasswordProvider.java | 2 + .../EnvironmentVariablePasswordProvider.java | 2 + .../druid/metadata/PasswordProvider.java | 5 +- .../druid/segment/column/ColumnType.java | 1 + distribution/pom.xml | 3 + extensions-core/druid-catalog/pom.xml | 328 ++++++++ .../apache/druid/catalog/guice/Catalog.java | 34 + .../guice/CatalogCoordinatorModule.java | 95 +++ .../catalog/http/CatalogListenerResource.java | 126 +++ .../druid/catalog/http/CatalogResource.java | 718 ++++++++++++++++++ .../apache/druid/catalog/storage/Actions.java | 104 +++ .../catalog/storage/CatalogAuthorizer.java | 100 +++ .../druid/catalog/storage/CatalogStorage.java | 148 ++++ .../druid/catalog/storage/HideColumns.java | 96 +++ .../catalog/storage/MetastoreManager.java | 60 ++ .../catalog/storage/MetastoreManagerImpl.java | 94 +++ .../druid/catalog/storage/MoveColumn.java | 116 +++ .../catalog/storage/sql/CatalogManager.java | 167 ++++ .../druid/catalog/storage/sql/DbUtils.java | 47 ++ .../storage/sql/SQLCatalogManager.java | 578 ++++++++++++++ .../druid/catalog/sync/CacheNotifier.java | 118 +++ .../catalog/sync/CachedMetadataCatalog.java | 229 ++++++ .../druid/catalog/sync/CatalogClient.java | 154 ++++ .../catalog/sync/CatalogUpdateNotifier.java | 121 +++ .../catalog/sync/ListeningNodeSupplier.java | 67 ++ .../catalog/sync/LocalMetadataCatalog.java | 91 +++ .../druid/catalog/sync/MetadataCatalog.java | 80 ++ .../druid/catalog/sync/RestUpdateSender.java | 193 +++++ ...rg.apache.druid.initialization.DruidModule | 16 + .../druid/catalog/storage/CatalogTests.java | 127 ++++ .../catalog/storage/TableManagerTest.java | 207 +++++ .../druid/catalog/sync/CacheNotifierTest.java | 122 +++ .../catalog/sync/CatalogMetadataTest.java | 346 +++++++++ .../druid/catalog/sync/MockCatalogSync.java | 100 +++ .../http/catalog/CatalogResourceTest.java | 619 +++++++++++++++ .../server/http/catalog/CommandTest.java | 156 ++++ .../server/http/catalog/DummyRequest.java | 500 ++++++++++++ integration-tests-ex/cases/cluster.sh | 7 +- .../Common/environment-configs/common.env | 2 +- integration-tests-ex/cases/pom.xml | 11 +- .../testsEx/catalog/ITCatalogRestTest.java | 194 +++++ .../druid/testsEx/categories/Catalog.java | 27 + .../druid/testsEx/cluster/CatalogClient.java | 200 +++++ .../testsEx/cluster/DruidClusterClient.java | 94 ++- pom.xml | 1 + .../druid/segment/column/RowSignature.java | 1 + server/pom.xml | 9 +- .../druid/catalog/model/CatalogUtils.java | 242 ++++++ .../druid/catalog/model/ColumnDefn.java | 110 +++ .../druid/catalog/model/ColumnSpec.java | 141 ++++ .../apache/druid/catalog/model/Columns.java | 105 +++ .../druid/catalog/model/ObjectDefn.java | 138 ++++ .../druid/catalog/model/ObjectFacade.java | 62 ++ .../druid/catalog/model/Parameterized.java | 62 ++ .../druid/catalog/model/Properties.java | 302 ++++++++ .../druid/catalog/model/ResolvedTable.java | 91 +++ .../druid/catalog/model/SchemaRegistry.java | 42 + .../catalog/model/SchemaRegistryImpl.java | 140 ++++ .../apache/druid/catalog/model/TableDefn.java | 171 +++++ .../catalog/model/TableDefnRegistry.java | 78 ++ .../apache/druid/catalog/model/TableId.java | 113 +++ .../druid/catalog/model/TableMetadata.java | 241 ++++++ .../apache/druid/catalog/model/TableSpec.java | 154 ++++ .../catalog/model/facade/ColumnFacade.java | 80 ++ .../model/facade/DatasourceFacade.java | 88 +++ .../model/facade/InputTableFacade.java | 51 ++ .../catalog/model/facade/TableFacade.java | 58 ++ .../catalog/model/table/ClusterKeySpec.java | 83 ++ .../catalog/model/table/DatasourceDefn.java | 278 +++++++ .../catalog/model/table/ExternalSpec.java | 61 ++ .../model/table/ExternalTableDefn.java | 272 +++++++ .../catalog/model/table/HttpTableDefn.java | 190 +++++ .../catalog/model/table/InlineTableDefn.java | 91 +++ .../catalog/model/table/InputFormats.java | 301 ++++++++ .../catalog/model/table/LocalTableDefn.java | 123 +++ .../catalog/model/table/MeasureTypes.java | 213 ++++++ .../catalog/model/table/TableBuilder.java | 289 +++++++ .../storage/derby/DerbyMetadataStorage.java | 21 +- .../apache/druid/server/security/Access.java | 1 + .../org/apache/druid/catalog/CatalogTest.java | 27 + .../druid/catalog/model/MeasureTypesTest.java | 98 +++ .../druid/catalog/model/PropertyDefnTest.java | 140 ++++ .../catalog/model/TableMetadataTest.java | 153 ++++ .../model/table/DatasourceTableTest.java | 672 ++++++++++++++++ .../model/table/HttpInputTableTest.java | 217 ++++++ .../catalog/model/table/InlineTableTest.java | 95 +++ .../catalog/model/table/InputFormatTest.java | 284 +++++++ .../catalog/model/table/LocalTableTest.java | 209 +++++ .../sql/calcite/aggregation/Aggregation.java | 2 - .../schema/SegmentMetadataCacheTest.java | 2 +- 99 files changed, 12925 insertions(+), 38 deletions(-) create mode 100644 extensions-core/druid-catalog/pom.xml create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/guice/Catalog.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/guice/CatalogCoordinatorModule.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogListenerResource.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/Actions.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogAuthorizer.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetastoreManager.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetastoreManagerImpl.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MoveColumn.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/DbUtils.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CacheNotifier.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogClient.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateNotifier.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/ListeningNodeSupplier.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/LocalMetadataCatalog.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/MetadataCatalog.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/RestUpdateSender.java create mode 100644 extensions-core/druid-catalog/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule create mode 100644 extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/CatalogTests.java create mode 100644 extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java create mode 100644 extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CacheNotifierTest.java create mode 100644 extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java create mode 100644 extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/MockCatalogSync.java create mode 100644 extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java create mode 100644 extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java create mode 100644 extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/DummyRequest.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/Catalog.java create mode 100644 integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/CatalogClient.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/Columns.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/ObjectFacade.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/Parameterized.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/Properties.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/ResolvedTable.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/SchemaRegistry.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/TableDefn.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/TableId.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/TableSpec.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/facade/ColumnFacade.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/facade/DatasourceFacade.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/facade/InputTableFacade.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/facade/TableFacade.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/table/ClusterKeySpec.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/table/DatasourceDefn.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/table/ExternalSpec.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/table/InlineTableDefn.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/table/InputFormats.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/table/LocalTableDefn.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/table/MeasureTypes.java create mode 100644 server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java create mode 100644 server/src/test/java/org/apache/druid/catalog/CatalogTest.java create mode 100644 server/src/test/java/org/apache/druid/catalog/model/MeasureTypesTest.java create mode 100644 server/src/test/java/org/apache/druid/catalog/model/PropertyDefnTest.java create mode 100644 server/src/test/java/org/apache/druid/catalog/model/TableMetadataTest.java create mode 100644 server/src/test/java/org/apache/druid/catalog/model/table/DatasourceTableTest.java create mode 100644 server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java create mode 100644 server/src/test/java/org/apache/druid/catalog/model/table/InlineTableTest.java create mode 100644 server/src/test/java/org/apache/druid/catalog/model/table/InputFormatTest.java create mode 100644 server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java diff --git a/.travis.yml b/.travis.yml index 00fbea1a0632..a76ecbbd74b1 100644 --- a/.travis.yml +++ b/.travis.yml @@ -687,12 +687,17 @@ jobs: jdk: openjdk8 services: *integration_test_services env: JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='middleManager' - # Uses the installation defined above. Then, builds the test tools and docker image, - # and runs one IT. If tests fail, echos log lines of any of - # the Druid services that did not exit normally. script: ./it.sh travis MultiStageQuery - # Disabling BatchIndex test as it is failing with due to timeout, fixing it will be taken in a separate PR. + - &integration_tests_ex + name: "(Compile=openjdk8, Run=openjdk8) catalog integration tests" + stage: Tests - phase 2 + jdk: openjdk8 + services: *integration_test_services + env: JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='middleManager' + script: ./it.sh travis Catalog + + # Disabling BatchIndex test as it is failing with due to timeout, fixing it will be taken in a separate PR. #- <<: *integration_tests_ex # name: "(Compile=openjdk8, Run=openjdk8) batch index integration test with Indexer (new)" # env: JVM_RUNTIME='-Djvm.runtime=8' USE_INDEXER='indexer' diff --git a/core/src/main/java/org/apache/druid/data/input/InputFormat.java b/core/src/main/java/org/apache/druid/data/input/InputFormat.java index 485a76ba6662..0ee6276675e9 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/InputFormat.java @@ -41,15 +41,17 @@ * See {@link NestedInputFormat} for nested input formats such as JSON. */ @UnstableApi -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = InputFormat.TYPE_PROPERTY) @JsonSubTypes(value = { - @Type(name = "csv", value = CsvInputFormat.class), - @Type(name = "json", value = JsonInputFormat.class), + @Type(name = CsvInputFormat.TYPE_KEY, value = CsvInputFormat.class), + @Type(name = JsonInputFormat.TYPE_KEY, value = JsonInputFormat.class), @Type(name = "regex", value = RegexInputFormat.class), - @Type(name = "tsv", value = DelimitedInputFormat.class) + @Type(name = DelimitedInputFormat.TYPE_KEY, value = DelimitedInputFormat.class) }) public interface InputFormat { + String TYPE_PROPERTY = "type"; + /** * Trait to indicate that a file can be split into multiple {@link InputSplit}s. *

diff --git a/core/src/main/java/org/apache/druid/data/input/InputSource.java b/core/src/main/java/org/apache/druid/data/input/InputSource.java index 0a3cda250f43..b7a50a8dc634 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/InputSource.java @@ -19,6 +19,7 @@ package org.apache.druid.data.input; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonSubTypes.Type; import com.fasterxml.jackson.annotation.JsonTypeInfo; @@ -47,19 +48,22 @@ * } */ @UnstableApi -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = InputSource.TYPE_PROPERTY) @JsonSubTypes(value = { - @Type(name = "local", value = LocalInputSource.class), - @Type(name = "http", value = HttpInputSource.class), - @Type(name = "inline", value = InlineInputSource.class), + @Type(name = LocalInputSource.TYPE_KEY, value = LocalInputSource.class), + @Type(name = HttpInputSource.TYPE_KEY, value = HttpInputSource.class), + @Type(name = InlineInputSource.TYPE_KEY, value = InlineInputSource.class), @Type(name = "combining", value = CombiningInputSource.class) }) public interface InputSource { + String TYPE_PROPERTY = "type"; + /** * Returns true if this inputSource can be processed in parallel using ParallelIndexSupervisorTask. It must be * castable to SplittableInputSource and the various SplittableInputSource methods must work as documented. */ + @JsonIgnore boolean isSplittable(); /** diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java index edc514f5d278..ec4c0084ff7b 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/CsvInputFormat.java @@ -37,6 +37,7 @@ public class CsvInputFormat extends FlatTextInputFormat { + public static final String TYPE_KEY = "csv"; private static final char SEPARATOR = ','; @JsonCreator @@ -59,6 +60,7 @@ public String getDelimiter() } @Override + @JsonIgnore public boolean isSplittable() { return true; diff --git a/core/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java index 8562cdeea608..599bd2e634c0 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/DelimitedInputFormat.java @@ -36,10 +36,11 @@ import java.util.List; /** - * InputFormat for customized Delimitor Separate Value format of input data(default is TSV). + * InputFormat for customized Delimiter Separate Value format of input data (default is TSV). */ public class DelimitedInputFormat extends FlatTextInputFormat { + public static final String TYPE_KEY = "tsv"; private static final String DEFAULT_DELIMITER = "\t"; @JsonCreator diff --git a/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java index fe162bc71eb1..5283c2bb65d7 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/HttpInputSource.java @@ -44,6 +44,8 @@ public class HttpInputSource extends AbstractInputSource implements SplittableInputSource { + public static final String TYPE_KEY = "http"; + private final List uris; @Nullable private final String httpAuthenticationUsername; diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InlineInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/InlineInputSource.java index 4cc6d0f9cf4c..eafe13aaec39 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/InlineInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/InlineInputSource.java @@ -20,8 +20,10 @@ package org.apache.druid.data.input.impl; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import com.google.common.base.Strings; import org.apache.druid.data.input.AbstractInputSource; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputRowSchema; @@ -35,12 +37,14 @@ public class InlineInputSource extends AbstractInputSource { + public static final String TYPE_KEY = "inline"; + private final String data; @JsonCreator public InlineInputSource(@JsonProperty("data") String data) { - Preconditions.checkArgument(data != null && !data.isEmpty(), "empty data"); + Preconditions.checkArgument(!Strings.isNullOrEmpty(data), "empty data"); this.data = data; } @@ -51,6 +55,7 @@ public String getData() } @Override + @JsonIgnore public boolean isSplittable() { return false; diff --git a/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java index 041942c9e626..821c5bf4355e 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/JsonInputFormat.java @@ -39,20 +39,20 @@ public class JsonInputFormat extends NestedInputFormat { + public static final String TYPE_KEY = "json"; + private final Map featureSpec; private final ObjectMapper objectMapper; private final boolean keepNullColumns; /** - * - * This parameter indicates whether or not the given InputEntity should be split by lines before parsing it. + * Indicates whether or not the given InputEntity should be split by lines before parsing it. * If it is set to true, the InputEntity must be split by lines first. * If it is set to false, unlike what you could imagine, it means that the InputEntity doesn't have to be split by lines first, but it can still contain multiple lines. * A created InputEntityReader from this format will determine by itself if line splitting is necessary. * * This parameter should always be true for batch ingestion and false for streaming ingestion. * For more information, see: https://github.com/apache/druid/pull/10383. - * */ private final boolean lineSplittable; diff --git a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java index 534c88aa3a36..688f8b1140bf 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/LocalInputSource.java @@ -57,6 +57,7 @@ public class LocalInputSource extends AbstractInputSource implements SplittableInputSource> { private static final Logger log = new Logger(LocalInputSource.class); + public static final String TYPE_KEY = "local"; @Nullable private final File baseDir; diff --git a/core/src/main/java/org/apache/druid/metadata/DefaultPasswordProvider.java b/core/src/main/java/org/apache/druid/metadata/DefaultPasswordProvider.java index 46feb3d5bb8a..d49f156883ce 100644 --- a/core/src/main/java/org/apache/druid/metadata/DefaultPasswordProvider.java +++ b/core/src/main/java/org/apache/druid/metadata/DefaultPasswordProvider.java @@ -24,6 +24,8 @@ public class DefaultPasswordProvider implements PasswordProvider { + public static final String TYPE_KEY = "default"; + private final String password; @JsonCreator diff --git a/core/src/main/java/org/apache/druid/metadata/EnvironmentVariablePasswordProvider.java b/core/src/main/java/org/apache/druid/metadata/EnvironmentVariablePasswordProvider.java index 14e71f8314c6..4cdc8492de3f 100644 --- a/core/src/main/java/org/apache/druid/metadata/EnvironmentVariablePasswordProvider.java +++ b/core/src/main/java/org/apache/druid/metadata/EnvironmentVariablePasswordProvider.java @@ -26,6 +26,8 @@ public class EnvironmentVariablePasswordProvider implements PasswordProvider { + public static final String TYPE_KEY = "environment"; + private final String variable; @JsonCreator diff --git a/core/src/main/java/org/apache/druid/metadata/PasswordProvider.java b/core/src/main/java/org/apache/druid/metadata/PasswordProvider.java index ab7ec7c22872..c443e97b979e 100644 --- a/core/src/main/java/org/apache/druid/metadata/PasswordProvider.java +++ b/core/src/main/java/org/apache/druid/metadata/PasswordProvider.java @@ -36,9 +36,8 @@ @ExtensionPoint @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultPasswordProvider.class) @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "default", value = DefaultPasswordProvider.class), - @JsonSubTypes.Type(name = "environment", value = EnvironmentVariablePasswordProvider.class), - + @JsonSubTypes.Type(name = DefaultPasswordProvider.TYPE_KEY, value = DefaultPasswordProvider.class), + @JsonSubTypes.Type(name = EnvironmentVariablePasswordProvider.TYPE_KEY, value = EnvironmentVariablePasswordProvider.class), }) public interface PasswordProvider { diff --git a/core/src/main/java/org/apache/druid/segment/column/ColumnType.java b/core/src/main/java/org/apache/druid/segment/column/ColumnType.java index 5876d65c0545..16673cc62cbf 100644 --- a/core/src/main/java/org/apache/druid/segment/column/ColumnType.java +++ b/core/src/main/java/org/apache/druid/segment/column/ColumnType.java @@ -120,6 +120,7 @@ public static ColumnType ofArray(ColumnType elementType) { return ColumnTypeFactory.getInstance().ofArray(elementType); } + public static ColumnType ofComplex(@Nullable String complexTypeName) { return ColumnTypeFactory.getInstance().ofComplex(complexTypeName); diff --git a/distribution/pom.xml b/distribution/pom.xml index 729d6f7bcda9..1c319e84eb58 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -46,6 +46,7 @@ ${project.parent.version} + --clean @@ -255,6 +256,8 @@ org.apache.druid.extensions:druid-ranger-security -c org.apache.druid.extensions:druid-kubernetes-extensions + -c + org.apache.druid.extensions:druid-catalog ${druid.distribution.pulldeps.opts} diff --git a/extensions-core/druid-catalog/pom.xml b/extensions-core/druid-catalog/pom.xml new file mode 100644 index 000000000000..72d427c474ad --- /dev/null +++ b/extensions-core/druid-catalog/pom.xml @@ -0,0 +1,328 @@ + + + + + + 4.0.0 + + org.apache.druid.extensions + druid-catalog + druid-catalog + Druid Catalog + + + org.apache.druid + druid + 25.0.0-SNAPSHOT + ../../pom.xml + + + + + org.apache.druid + druid-core + ${project.parent.version} + provided + + + org.apache.druid + druid-processing + ${project.parent.version} + provided + + + org.apache.druid + druid-server + ${project.parent.version} + provided + + + org.apache.druid + druid-indexing-service + ${project.parent.version} + provided + + + org.apache.druid + druid-sql + ${project.parent.version} + provided + + + org.apache.druid + druid-services + ${project.parent.version} + provided + + + com.google.inject + guice + provided + + + com.google.inject.extensions + guice-multibindings + provided + + + com.google.guava + guava + provided + + + com.opencsv + opencsv + provided + + + com.fasterxml.jackson.core + jackson-databind + provided + + + com.fasterxml.jackson.core + jackson-core + provided + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + javax.validation + validation-api + provided + + + com.google.code.findbugs + jsr305 + provided + + + jakarta.inject + jakarta.inject-api + provided + + + joda-time + joda-time + provided + + + io.netty + netty + provided + + + org.apache.calcite + calcite-core + provided + + + javax.inject + javax.inject + 1 + provided + + + org.apache.curator + curator-client + provided + + + com.fasterxml.jackson.dataformat + jackson-dataformat-smile + provided + + + org.jdbi + jdbi + provided + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-smile-provider + provided + + + javax.ws.rs + jsr311-api + provided + + + org.apache.commons + commons-lang3 + provided + + + javax.servlet + javax.servlet-api + provided + + + com.sun.jersey + jersey-server + provided + + + com.google.errorprone + error_prone_annotations + provided + + + org.lz4 + lz4-java + provided + + + org.apache.datasketches + datasketches-java + provided + + + org.apache.datasketches + datasketches-memory + provided + + + it.unimi.dsi + fastutil-core + provided + + + commons-io + commons-io + provided + + + + + org.easymock + easymock + test + + + org.hamcrest + hamcrest-all + test + + + org.hamcrest + hamcrest-core + test + + + junit + junit + test + + + org.mockito + mockito-core + test + + + nl.jqno.equalsverifier + equalsverifier + test + + + org.apache.druid + druid-core + ${project.parent.version} + test + test-jar + + + org.apache.druid + druid-processing + ${project.parent.version} + test + test-jar + + + org.apache.druid + druid-server + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-sql + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-hll + ${project.parent.version} + + provided + + + + + + + + + + + surefire-java9 + + (1.9,) + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + + + @{jacocoArgLine} + -Xmx1500m + -XX:MaxDirectMemorySize=512m + -Duser.language=en + -Duser.GroupByQueryRunnerTest.javacountry=US + -Dfile.encoding=UTF-8 + -Duser.timezone=UTC + -Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager + -Daws.region=us-east-1 + -Ddruid.indexing.doubleStorage=double + --add-exports java.base/jdk.internal.ref=ALL-UNNAMED + + false + + true + + + + + + + + diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/guice/Catalog.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/guice/Catalog.java new file mode 100644 index 000000000000..7400e2f0454a --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/guice/Catalog.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.guice; + +import com.google.inject.BindingAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@Target({ElementType.METHOD}) +@Retention(RetentionPolicy.RUNTIME) +@BindingAnnotation +public @interface Catalog +{ +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/guice/CatalogCoordinatorModule.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/guice/CatalogCoordinatorModule.java new file mode 100644 index 000000000000..3a04d678cf3f --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/guice/CatalogCoordinatorModule.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.guice; + +import com.fasterxml.jackson.databind.Module; +import com.google.inject.Binder; +import org.apache.druid.catalog.http.CatalogResource; +import org.apache.druid.catalog.model.SchemaRegistry; +import org.apache.druid.catalog.model.SchemaRegistryImpl; +import org.apache.druid.catalog.storage.CatalogStorage; +import org.apache.druid.catalog.storage.MetastoreManager; +import org.apache.druid.catalog.storage.MetastoreManagerImpl; +import org.apache.druid.catalog.storage.sql.CatalogManager; +import org.apache.druid.catalog.storage.sql.SQLCatalogManager; +import org.apache.druid.catalog.sync.CatalogUpdateNotifier; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.guice.Jerseys; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.guice.LifecycleModule; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.annotations.LoadScope; +import org.apache.druid.initialization.DruidModule; + +import java.util.Collections; +import java.util.List; + +/** + * Configures the catalog database on the Coordinator, along + * with its REST resource for CRUD updates and the notifier + * for push updates. + */ +@LoadScope(roles = NodeRole.COORDINATOR_JSON_NAME) +public class CatalogCoordinatorModule implements DruidModule +{ + @Override + public void configure(Binder binder) + { + // Database layer: only the SQL version is supported at present. + binder + .bind(CatalogManager.class) + .to(SQLCatalogManager.class) + .in(LazySingleton.class); + + // Storage abstraction used by the REST API, sits on top of the + // database layer. + binder + .bind(CatalogStorage.class) + .in(LazySingleton.class); + binder + .bind(MetastoreManager.class) + .to(MetastoreManagerImpl.class) + .in(LazySingleton.class); + + // At present, the set of schemas is fixed. + binder + .bind(SchemaRegistry.class) + .to(SchemaRegistryImpl.class) + .in(LazySingleton.class); + + // Push update notifier, which is lifecycle managed. No references, + // so force Guice to create the instance. (Lifecycle will also, if + // Guice hasn't done so.) + binder + .bind(CatalogUpdateNotifier.class) + .in(ManageLifecycle.class); + LifecycleModule.register(binder, CatalogUpdateNotifier.class); + + // Public REST API and private cache sync API. + Jerseys.addResource(binder, CatalogResource.class); + } + + @Override + public List getJacksonModules() + { + return Collections.emptyList(); + } +} + diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogListenerResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogListenerResource.java new file mode 100644 index 000000000000..bdbeda600cb2 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogListenerResource.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.http; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.sync.CatalogUpdateNotifier; +import org.apache.druid.catalog.sync.MetadataCatalog.CatalogListener; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.Action; +import org.apache.druid.server.security.AuthorizationUtils; +import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.server.security.Resource; +import org.apache.druid.server.security.ResourceAction; +import org.apache.druid.server.security.ResourceType; + +import javax.inject.Inject; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; + +import java.io.IOException; +import java.io.InputStream; + +@Path(CatalogListenerResource.BASE_URL) +public class CatalogListenerResource +{ + public static final String BASE_URL = "/druid/broker/v1/catalog"; + public static final String SYNC_URL = "/sync"; + + private final CatalogListener listener; + private final AuthorizerMapper authorizerMapper; + private final ObjectMapper smileMapper; + private final ObjectMapper jsonMapper; + + @Inject + public CatalogListenerResource( + final CatalogListener listener, + @Smile final ObjectMapper smileMapper, + @Json final ObjectMapper jsonMapper, + final AuthorizerMapper authorizerMapper) + { + this.listener = listener; + this.authorizerMapper = authorizerMapper; + this.smileMapper = smileMapper; + this.jsonMapper = jsonMapper; + } + + @POST + @Path(SYNC_URL) + @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + public Response syncTable( + final InputStream inputStream, + @Context final HttpServletRequest req) + { + Response resp = checkAuth(req); + if (resp != null) { + return resp; + } + final String reqContentType = req.getContentType(); + final boolean isSmile = SmileMediaTypes.APPLICATION_JACKSON_SMILE.equals(reqContentType); + final ObjectMapper mapper = isSmile ? smileMapper : jsonMapper; + TableMetadata tableSpec; + try { + tableSpec = mapper.readValue(inputStream, TableMetadata.class); + } + catch (IOException e) { + return Response.serverError().entity(e.getMessage()).build(); + } + TableSpec spec = tableSpec.spec(); + if (CatalogUpdateNotifier.TOMBSTONE_TABLE_TYPE.equals(spec.type())) { + listener.deleted(tableSpec.id()); + } else { + listener.updated(tableSpec); + } + return Response.status(Response.Status.ACCEPTED).build(); + } + + private Response checkAuth(final HttpServletRequest request) + { + final ResourceAction resourceAction = new ResourceAction( + new Resource("CONFIG", ResourceType.CONFIG), + Action.WRITE + ); + + final Access authResult = AuthorizationUtils.authorizeResourceAction( + request, + resourceAction, + authorizerMapper + ); + + if (authResult.isAllowed()) { + return null; + } + return Response.status(Response.Status.FORBIDDEN) + .type(MediaType.TEXT_PLAIN) + .entity(StringUtils.format("Access-Check-Result: %s", authResult.toString())) + .build(); + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java new file mode 100644 index 000000000000..b1491c52e35e --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java @@ -0,0 +1,718 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.http; + +import com.google.common.base.Strings; +import org.apache.curator.shaded.com.google.common.collect.Lists; +import org.apache.druid.catalog.model.CatalogUtils; +import org.apache.druid.catalog.model.SchemaRegistry.SchemaSpec; +import org.apache.druid.catalog.model.TableDefnRegistry; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.catalog.storage.Actions; +import org.apache.druid.catalog.storage.CatalogStorage; +import org.apache.druid.catalog.storage.HideColumns; +import org.apache.druid.catalog.storage.MoveColumn; +import org.apache.druid.catalog.storage.MoveColumn.Position; +import org.apache.druid.catalog.storage.sql.CatalogManager.DuplicateKeyException; +import org.apache.druid.catalog.storage.sql.CatalogManager.NotFoundException; +import org.apache.druid.catalog.storage.sql.CatalogManager.OutOfDateException; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.server.security.Action; +import org.apache.druid.server.security.AuthorizationUtils; +import org.apache.druid.server.security.ForbiddenException; +import org.apache.druid.server.security.ResourceType; + +import javax.inject.Inject; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +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.QueryParam; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; + +import java.util.Collections; +import java.util.List; +import java.util.function.Function; + +/** + * REST endpoint for user and internal catalog actions. Catalog actions + * occur at the global level (all schemas), the schema level, or the + * table level. + * + * @see {@link CatalogListenerResource} for the client-side API. + */ +@Path(CatalogResource.ROOT_PATH) +public class CatalogResource +{ + public static final String ROOT_PATH = "/druid/coordinator/v1/catalog"; + + private final CatalogStorage catalog; + + @Inject + public CatalogResource(final CatalogStorage catalog) + { + this.catalog = catalog; + } + + private enum PostAction + { + NEW, + IFNEW, + REPLACE, + FORCE; + } + + /** + * Create a new table containing the given table specification. + * + * @param dbSchema The name of the Druid schema, which must be writable + * and the user must have at least read access. + * @param name The name of the table definition to modify. The user must + * have write access to the table. + * @param spec The new table definition. + * @param action What to do if the table already exists. + * {@code ifNew} is the same as the SQL IF NOT EXISTS clause. If {@code new}, + * then an error is raised if the table exists. If {@code ifNew}, then + * the action silently does nothing if the table exists. Primarily for + * use in scripts. The other two options are primarily for use in tests. + * @param req the HTTP request used for authorization. + * @return the version number of the table + */ + @POST + @Path("/tables/{dbSchema}/{name}") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response postTable( + @PathParam("dbSchema") String dbSchema, + @PathParam("name") String name, + TableSpec spec, + @QueryParam("action") String actionParam, + @QueryParam("version") long version, + @Context final HttpServletRequest req + ) + { + final PostAction action; + if (actionParam == null) { + action = PostAction.NEW; + } else { + action = PostAction.valueOf(StringUtils.toUpperCase(actionParam)); + if (action == null) { + return Actions.badRequest( + Actions.INVALID, + StringUtils.format( + "Not a valid action: [%s]. Valid actions are new, ifNew, replace, force", + actionParam + ) + ); + } + } + TableId tableId = TableId.of(dbSchema, name); + Response response = authorizeTable(tableId, spec, req); + if (response != null) { + return response; + } + TableMetadata table = TableMetadata.newTable(tableId, spec); + try { + catalog.validate(table); + } + catch (IAE e) { + return Actions.badRequest(Actions.INVALID, e.getMessage()); + } + + switch (action) { + case NEW: + return insertTableSpec(table, false); + case IFNEW: + return insertTableSpec(table, true); + case REPLACE: + return updateTableSpec(table, version); + case FORCE: + return addOrUpdateTableSpec(table); + default: + throw new ISE("Unknown action."); + } + } + + private Response authorizeTable(TableId tableId, TableSpec spec, final HttpServletRequest req) + { + // Druid has a fixed set of schemas. Ensure the one provided is valid. + Pair result = validateSchema(tableId.schema()); + if (result.lhs != null) { + return result.lhs; + } + SchemaSpec schema = result.rhs; + + // The schema has to be one that allows table definitions. + if (!schema.writable()) { + return Actions.badRequest( + Actions.INVALID, + StringUtils.format("Cannot modify schema %s", tableId.schema()) + ); + } + + // Table name can't be blank or have spaces + if (Strings.isNullOrEmpty(tableId.name())) { + return Actions.badRequest(Actions.INVALID, "Table name is required"); + } + if (!tableId.name().equals(tableId.name().trim())) { + return Actions.badRequest(Actions.INVALID, "Table name cannot start or end with spaces"); + } + + // The given table spec has to be valid for the given schema. + if (spec != null && !schema.accepts(spec.type())) { + return Actions.badRequest( + Actions.INVALID, + StringUtils.format( + "Cannot create tables of type %s in schema %s", + spec == null ? "null" : spec.getClass().getSimpleName(), + tableId.schema()) + ); + } + + // The user has to have permission to modify the table. + try { + catalog.authorizer().authorizeTable(schema, tableId.name(), Action.WRITE, req); + } + catch (ForbiddenException e) { + return Actions.forbidden(e); + } + + // Everything checks out, let the request proceed. + return null; + } + + private Response insertTableSpec(TableMetadata table, boolean ifNew) + { + try { + long createVersion = catalog.tables().create(table); + return Actions.okWithVersion(createVersion); + } + catch (DuplicateKeyException e) { + if (!ifNew) { + return Actions.badRequest( + Actions.DUPLICATE_ERROR, + StringUtils.format( + "A table of name %s already exists", + table.id().sqlName() + ) + ); + } else { + return Actions.okWithVersion(0); + } + } + catch (Exception e) { + return Actions.exception(e); + } + } + + private Response updateTableSpec(TableMetadata table, long version) + { + try { + long newVersion = catalog.tables().update(table, version); + return Actions.okWithVersion(newVersion); + } + catch (NotFoundException e) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + catch (OutOfDateException e) { + return Response + .status(Response.Status.BAD_REQUEST) + .entity( + Actions.error( + Actions.DUPLICATE_ERROR, + "The table entry not found or is older than the given version: reload and retry")) + .build(); + } + catch (Exception e) { + return Actions.exception(e); + } + } + + private Response addOrUpdateTableSpec(TableMetadata table) + { + try { + long newVersion = catalog.tables().create(table); + return Actions.okWithVersion(newVersion); + } + catch (DuplicateKeyException e) { + // Fall through + } + catch (Exception e) { + return Actions.exception(e); + } + try { + long newVersion = catalog.tables().update(table, 0); + return Actions.okWithVersion(newVersion); + } + catch (Exception e) { + return Actions.exception(e); + } + } + + /** + * Update a table within the given schema. + * + * @param dbSchema The name of the Druid schema, which must be writable + * and the user must have at least read access. + * @param name The name of the table definition to modify. The user must + * have write access to the table. + * @param spec The new table definition. + * @param version An optional table version. If provided, the metadata DB + * entry for the table must be at this exact version or the update + * will fail. (Provides "optimistic locking.") If omitted (that is, + * if zero), then no update conflict change is done. + * @param req the HTTP request used for authorization. + * @return the new version number of the table + */ + @PUT + @Path("/tables/{dbSchema}/{name}") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response updateTableDefn( + @PathParam("dbSchema") String dbSchema, + @PathParam("name") String name, + TableSpec spec, + @QueryParam("version") long version, + @Context final HttpServletRequest req + ) + { + + TableDefnRegistry tableRegistry = catalog.tableRegistry(); + return incrementalUpdate( + TableId.of(dbSchema, name), + spec, + req, + (existing) -> tableRegistry.resolve(existing).merge(spec).spec() + ); + } + + private Response incrementalUpdate( + TableId tableId, + TableSpec newSpec, + @Context final HttpServletRequest req, + Function action + ) + { + Response response = authorizeTable(tableId, newSpec, req); + if (response != null) { + return response; + } + try { + long newVersion = catalog.tables().updatePayload(tableId, action); + return Actions.okWithVersion(newVersion); + } + catch (NotFoundException e) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + catch (Exception e) { + return Actions.exception(e); + } + } + + /** + * Move a single column to the start end of the column list, or before or after + * another column. Both columns must exist. Returns the version of the table + * after the update. + *

+ * The operation is done atomically so no optimistic locking is required. + * + * @param dbSchema + * @param name + * @param command + * @param req + * @return + */ + @POST + @Path("/tables/{dbSchema}/{name}/moveColumn") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response moveColumn( + @PathParam("dbSchema") final String dbSchema, + @PathParam("name") final String name, + final MoveColumn command, + @Context final HttpServletRequest req + ) + { + if (command == null) { + return Actions.badRequest(Actions.INVALID, "A MoveColumn object is required"); + } + if (Strings.isNullOrEmpty(command.column)) { + return Actions.badRequest(Actions.INVALID, "A column name is required"); + } + if (command.where == null) { + return Actions.badRequest(Actions.INVALID, "A target location is required"); + } + if ((command.where == Position.BEFORE || command.where == Position.AFTER) && Strings.isNullOrEmpty(command.anchor)) { + return Actions.badRequest(Actions.INVALID, "A anchor column is required for BEFORE or AFTER"); + } + return incrementalUpdate( + TableId.of(dbSchema, name), + null, + req, + (spec) -> spec.withColumns(command.perform(spec.columns())) + ); + } + + /** + * Hide or unhide columns. If both appear, hide takes precedence. Returns the + * new table version. + */ + @POST + @Path("/tables/{dbSchema}/{name}/hideColumns") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response hideColumns( + @PathParam("dbSchema") final String dbSchema, + @PathParam("name") final String name, + final HideColumns command, + @Context final HttpServletRequest req + ) + { + return incrementalUpdate( + TableId.of(dbSchema, name), + null, + req, + (spec) -> { + if (!DatasourceDefn.isDatasource(spec.type())) { + throw new ISE("hideColumns is supported only for data source specs"); + } + @SuppressWarnings("unchecked") + List hiddenProps = (List) spec.properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY); + return spec.withProperty( + DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, + command.perform(hiddenProps) + ); + } + ); + } + + /** + * Drop column metadata. Only removes metadata entries, has no effect on the + * physical segments. Returns the new table version. + */ + @POST + @Path("/tables/{dbSchema}/{name}/dropColumns") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response dropColumns( + @PathParam("dbSchema") final String dbSchema, + @PathParam("name") final String name, + final List columns, + @Context final HttpServletRequest req + ) + { + return incrementalUpdate( + TableId.of(dbSchema, name), + null, + req, + (spec) -> spec.withColumns(CatalogUtils.dropColumns(spec.columns(), columns)) + ); + } + + /** + * Retrieves the definition of the given table. + *

+ * Returns a 404 (NOT FOUND) error if the table definition does not exist. + * Note that this check is only for the definition; the table (or + * datasource) itself may exist. Similarly, this call may return a definition + * even if there is no datasource of the same name (typically occurs when + * the definition is created before the datasource itself.) + * + * @param dbSchema The Druid schema. The user must have read access. + * @param name The name of the table within the schema. The user must have + * read access. + * @param req the HTTP request used for authorization. + * @return the definition for the table, if any. + */ + @GET + @Path("/tables/{dbSchema}/{name}") + @Produces(MediaType.APPLICATION_JSON) + public Response getTable( + @PathParam("dbSchema") String dbSchema, + @PathParam("name") String name, + @Context final HttpServletRequest req + ) + { + Pair result = validateSchema(dbSchema); + if (result.lhs != null) { + return result.lhs; + } + if (Strings.isNullOrEmpty(name)) { + return Actions.badRequest(Actions.INVALID, "Table name is required"); + } + try { + catalog.authorizer().authorizeTable(result.rhs, name, Action.READ, req); + } + catch (ForbiddenException e) { + return Actions.forbidden(e); + } + try { + TableId tableId = new TableId(dbSchema, name); + TableMetadata table = catalog.tables().read(tableId); + if (table == null) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + return Response.ok().entity(table).build(); + } + catch (Exception e) { + return Actions.exception(e); + } + } + + /** + * Retrieves the list of all Druid schema names. At present, Druid does + * not impose security on schemas, only tables within schemas. + */ + @GET + @Path("/list/schemas/names") + @Produces(MediaType.APPLICATION_JSON) + public Response listSchemas( + @Context final HttpServletRequest req + ) + { + // No good resource to use: we really need finer-grain control. + catalog.authorizer().authorizeAccess(ResourceType.STATE, "schemas", Action.READ, req); + return Response.ok().entity(catalog.schemaRegistry().names()).build(); + } + + /** + * Retrieves the list of all Druid table names for which the user has at + * least read access. + */ + @GET + @Path("/list/tables/names") + @Produces(MediaType.APPLICATION_JSON) + public Response listTables( + @Context final HttpServletRequest req + ) + { + List tables = catalog.tables().list(); + Iterable filtered = AuthorizationUtils.filterAuthorizedResources( + req, + tables, + tableId -> { + SchemaSpec schema = catalog.resolveSchema(tableId.schema()); + if (schema == null) { + // Should never occur. + return null; + } + return Collections.singletonList( + catalog.authorizer().resourceAction(schema, tableId.name(), Action.READ)); + }, + catalog.authorizer().mapper()); + return Response.ok().entity(Lists.newArrayList(filtered)).build(); + } + + /** + * Retrieves the list of table names within the given schema for which the + * user has at least read access. This returns the list of table definitions + * which will probably differ from the list of actual tables. For example, for + * the read-only schemas, there will be no table definitions. + * + * @param dbSchema The Druid schema to query. The user must have read access. + */ + @GET + @Path("/schemas/{dbSchema}/names") + @Produces(MediaType.APPLICATION_JSON) + public Response listTables( + @PathParam("dbSchema") String dbSchema, + @Context final HttpServletRequest req + ) + { + Pair result = validateSchema(dbSchema); + if (result.lhs != null) { + return result.lhs; + } + SchemaSpec schema = result.rhs; + List tables = catalog.tables().list(dbSchema); + Iterable filtered = AuthorizationUtils.filterAuthorizedResources( + req, + tables, + name -> + Collections.singletonList( + catalog.authorizer().resourceAction(schema, name, Action.READ)), + catalog.authorizer().mapper()); + return Response.ok().entity(Lists.newArrayList(filtered)).build(); + } + + /** + * Retrieves the list of all Druid table metadata for which the user has at + * least read access. + */ + @GET + @Path("/schemas/{dbSchema}/tables") + @Produces(MediaType.APPLICATION_JSON) + public Response listTableDetails( + @PathParam("dbSchema") String dbSchema, + @Context final HttpServletRequest req + ) + { + Pair result = validateSchema(dbSchema); + if (result.lhs != null) { + return result.lhs; + } + SchemaSpec schema = result.rhs; + List tables = catalog.tables().listDetails(schema.name()); + Iterable filtered = AuthorizationUtils.filterAuthorizedResources( + req, + tables, + table -> { + TableId tableId = table.id(); + return Collections.singletonList( + catalog.authorizer().resourceAction(schema, tableId.name(), Action.READ)); + }, + catalog.authorizer().mapper()); + + return Response.ok().entity(Lists.newArrayList(filtered)).build(); + } + + /** + * Deletes the table definition (but not the underlying table or datasource) + * for the given schema and table. + * + * @param dbSchema The name of the schema that holds the table. + * @param name The name of the table definition to delete. The user must have + * write access. + * @param ifExists Optional flag. If {@code false} (the default), 404 (NOT FOUND) + * error is returned if the table does not exist. If {@code true}, + * then acts like the SQL IF EXISTS clause and does not return an + * error if the table does not exist, + */ + @DELETE + @Path("/tables/{dbSchema}/{name}") + @Produces(MediaType.APPLICATION_JSON) + public Response deleteTable( + @PathParam("dbSchema") String dbSchema, + @PathParam("name") String name, + @QueryParam("ifExists") boolean ifExists, + @Context final HttpServletRequest req + ) + { + TableId tableId = new TableId(dbSchema, name); + Pair result = validateSchema(tableId.schema()); + if (result.lhs != null) { + return result.lhs; + } + SchemaSpec schema = result.rhs; + if (!schema.writable()) { + return Actions.badRequest( + Actions.INVALID, + StringUtils.format("Cannot delete tables from schema %s", tableId.schema())); + } + if (Strings.isNullOrEmpty(name)) { + return Actions.badRequest(Actions.INVALID, "Table name is required"); + } + try { + catalog.authorizer().authorizeTable(schema, tableId.name(), Action.WRITE, req); + } + catch (ForbiddenException e) { + return Actions.forbidden(e); + } + try { + if (!catalog.tables().delete(tableId) && !ifExists) { + return Actions.notFound(tableId.sqlName()); + } + } + catch (Exception e) { + return Actions.exception(e); + } + return Actions.ok(); + } + + public static final String SCHEMA_SYNC = "/schemas/{dbSchema}/sync"; + + /** + * Synchronization request from the Broker for a database schema. Requests all + * table definitions known to the catalog. Used to prime a cache on first access. + * After that, the Coordinator will push updates to Brokers. Returns the full + * list of table details. + * + * It is expected that the number of table definitions will be of small or moderate + * size, so no provision is made to handle very large lists. + */ + @GET + @Path(SCHEMA_SYNC) + @Produces(MediaType.APPLICATION_JSON) + public Response syncSchema( + @PathParam("dbSchema") String dbSchema, + @Context final HttpServletRequest req + ) + { + // Same as the user-command for now. This endpoint reserves the right to change + // over time as needed, while the user endpoint cannot easily change. + return listTableDetails(dbSchema, req); + } + + public static final String TABLE_SYNC = "/tables/{dbSchema}/{name}/sync"; + + /** + * Synchronization request from the Broker for information about a specific table + * (datasource). Done on first access to the table by any query. After that, the + * Coordinator pushes updates to the Broker on any changes. + */ + @GET + @Path(TABLE_SYNC) + @Produces(MediaType.APPLICATION_JSON) + public Response syncTable( + @PathParam("dbSchema") String dbSchema, + @PathParam("name") String name, + @Context final HttpServletRequest req + ) + { + return getTable(dbSchema, name, req); + } + + @POST + @Path("/flush") + public Response flush( + @Context final HttpServletRequest req + ) + { + // Nothing to do yet. + return Actions.ok(); + } + + private Pair validateSchema(String dbSchema) + { + if (Strings.isNullOrEmpty(dbSchema)) { + return Pair.of(Actions.badRequest(Actions.INVALID, "Schema name is required"), null); + } + SchemaSpec schema = catalog.resolveSchema(dbSchema); + if (schema == null) { + return Pair.of(Actions.notFound( + StringUtils.format("Unknown schema %s", dbSchema)), + null); + } + return Pair.of(null, schema); + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/Actions.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/Actions.java new file mode 100644 index 000000000000..08a561b7517e --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/Actions.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.storage; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.server.security.ForbiddenException; + +import javax.ws.rs.core.Response; + +import java.util.Map; + +/** + * Helper functions for the catalog REST API actions. + */ +public class Actions +{ + public static final String DUPLICATE_ERROR = "Already exists"; + public static final String FAILED_ERROR = "Failed"; + public static final String INVALID = "Invalid"; + public static final String FORBIDDEN = "Forbidden"; + public static final String NOT_FOUND = "Not found"; + + public static final String ERROR_KEY = "error"; + public static final String ERR_MSG_KEY = "errorMessage"; + + public static Map error(String code, String msg) + { + return ImmutableMap.of(ERROR_KEY, code, ERR_MSG_KEY, msg); + } + + public static Response exception(Exception e) + { + return Response + .serverError() + .entity(error(FAILED_ERROR, e.getMessage())) + .build(); + } + + public static Response badRequest(String code, String msg) + { + return Response + .status(Response.Status.BAD_REQUEST) + .entity(error(code, msg)) + .build(); + } + + public static Response notFound(String msg) + { + return Response + .status(Response.Status.NOT_FOUND) + .entity(error(NOT_FOUND, msg)) + .build(); + } + + public static Response ok() + { + return Response.ok().build(); + } + + public static Response forbidden() + { + return forbidden("Unauthorized"); + } + + public static Response forbidden(ForbiddenException e) + { + return forbidden(e.getMessage()); + } + + public static Response forbidden(String msg) + { + // Like ForbiddenExceptionMapper, but in the standard error + // format. Used instead of throwing ForbiddenException + return Response.status(Response.Status.FORBIDDEN) + .entity(error(FORBIDDEN, msg)) + .build(); + } + + public static Response okWithVersion(long version) + { + return Response + .ok() + .entity(ImmutableMap.of("version", version)) + .build(); + + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogAuthorizer.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogAuthorizer.java new file mode 100644 index 000000000000..558463c2bf02 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogAuthorizer.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.storage; + +import org.apache.druid.catalog.model.SchemaRegistry.SchemaSpec; +import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.Action; +import org.apache.druid.server.security.AuthorizationUtils; +import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.server.security.ForbiddenException; +import org.apache.druid.server.security.Resource; +import org.apache.druid.server.security.ResourceAction; + +import javax.inject.Inject; +import javax.servlet.http.HttpServletRequest; + +/** + * Encapsulates the details of catalog authorization. + */ +public class CatalogAuthorizer +{ + private final AuthorizerMapper authorizerMapper; + + @Inject + public CatalogAuthorizer( + AuthorizerMapper authorizerMapper) + { + this.authorizerMapper = authorizerMapper; + } + + public AuthorizerMapper mapper() + { + return authorizerMapper; + } + + public void authorizeTable(SchemaSpec schema, String name, Action action, HttpServletRequest request) + { + if (action == Action.WRITE && !schema.writable()) { + throw new ForbiddenException( + "Cannot create table definitions in schema: " + schema.name()); + } + authorize(schema.securityResource(), name, action, request); + } + + public void authorize(String resource, String key, Action action, HttpServletRequest request) + { + final Access authResult = authorizeAccess(resource, key, action, request); + if (!authResult.isAllowed()) { + throw new ForbiddenException(authResult.toString()); + } + } + + public boolean isAuthorized(String resource, String key, Action action, HttpServletRequest request) + { + final Access authResult = authorizeAccess(resource, key, action, request); + return authResult.isAllowed(); + } + + public Access authorizeAccess(String resource, String key, Action action, HttpServletRequest request) + { + return AuthorizationUtils.authorizeResourceAction( + request, + new ResourceAction(new Resource(key, resource), action), + authorizerMapper + ); + } + + public ResourceAction resourceAction(SchemaSpec schema, String name, Action action) + { + return new ResourceAction(new Resource(name, schema.securityResource()), action); + } + + public Action inferAction(HttpServletRequest request) + { + switch (request.getMethod()) { + case "GET": + case "HEAD": + return Action.READ; + default: + return Action.WRITE; + } + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java new file mode 100644 index 000000000000..061cd856a846 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.storage; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.SchemaRegistry; +import org.apache.druid.catalog.model.SchemaRegistry.SchemaSpec; +import org.apache.druid.catalog.model.SchemaRegistryImpl; +import org.apache.druid.catalog.model.TableDefnRegistry; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.storage.sql.CatalogManager; +import org.apache.druid.catalog.sync.MetadataCatalog.CatalogListener; +import org.apache.druid.catalog.sync.MetadataCatalog.CatalogSource; +import org.apache.druid.catalog.sync.MetadataCatalog.CatalogUpdateProvider; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.server.security.AuthorizerMapper; + +import javax.inject.Inject; + +import java.util.List; + +/** + * Facade over the three internal components used to manage the metadata + * catalog from the REST API. + */ +public class CatalogStorage implements CatalogUpdateProvider, CatalogSource +{ + public static class ListenerAdapter implements CatalogManager.Listener + { + private final CatalogListener dest; + + public ListenerAdapter(CatalogListener dest) + { + this.dest = dest; + } + + @Override + public void added(TableMetadata table) + { + dest.updated(table); + } + + @Override + public void updated(TableMetadata table) + { + dest.updated(table); + } + + @Override + public void deleted(TableId id) + { + dest.deleted(id); + } + } + + protected final SchemaRegistry schemaRegistry; + protected final TableDefnRegistry tableRegistry; + protected final CatalogManager catalogMgr; + protected final CatalogAuthorizer authorizer; + + @Inject + public CatalogStorage( + CatalogManager catalogMgr, + AuthorizerMapper authorizerMapper, + @Json ObjectMapper jsonMapper + ) + { + this.schemaRegistry = new SchemaRegistryImpl(); + this.tableRegistry = new TableDefnRegistry(jsonMapper); + this.catalogMgr = catalogMgr; + this.authorizer = new CatalogAuthorizer(authorizerMapper); + } + + public CatalogAuthorizer authorizer() + { + return authorizer; + } + + public CatalogManager tables() + { + return catalogMgr; + } + + public SchemaRegistry schemaRegistry() + { + return schemaRegistry; + } + + public SchemaSpec resolveSchema(String dbSchema) + { + return schemaRegistry.schema(dbSchema); + } + + @Override + public void register(CatalogListener listener) + { + tables().register(new ListenerAdapter(listener)); + } + + @Override + public List tablesForSchema(String dbSchema) + { + return tables().listDetails(dbSchema); + } + + @Override + public TableMetadata table(TableId id) + { + return tables().read(id); + } + + public void validate(TableMetadata table) + { + table.validate(); + tableRegistry.resolve(table.spec()).validate(); + } + + public TableDefnRegistry tableRegistry() + { + return tableRegistry; + } + + @Override + public ResolvedTable resolveTable(TableId id) + { + TableMetadata table = table(id); + return table == null ? null : tableRegistry.resolve(table.spec()); + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java new file mode 100644 index 000000000000..5d1e1db39ccc --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.storage; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; + +public class HideColumns +{ + @JsonProperty + public final List hide; + @JsonProperty + public final List unhide; + + @JsonCreator + public HideColumns( + @JsonProperty("hide") @Nullable final List hide, + @JsonProperty("unhide") @Nullable final List unhide + ) + { + this.hide = hide; + this.unhide = unhide; + } + + @JsonIgnore + public boolean isEmpty() + { + return (hide == null || hide.isEmpty()) + && (unhide == null || unhide.isEmpty()); + } + + public List perform(List hiddenColumns) + { + if (hiddenColumns == null) { + hiddenColumns = Collections.emptyList(); + } + Set existing = new HashSet<>(hiddenColumns); + if (unhide != null) { + for (String col : unhide) { + existing.remove(col); + } + } + List revised = new ArrayList<>(); + for (String col : hiddenColumns) { + if (existing.contains(col)) { + revised.add(col); + } + } + if (hide != null) { + for (String col : hide) { + if (!existing.contains(col)) { + revised.add(col); + } + } + } + return revised.isEmpty() ? null : revised; + } + + @Override + public boolean equals(Object o) + { + if (o == null || o.getClass() != getClass()) { + return false; + } + HideColumns other = (HideColumns) o; + return Objects.equals(this.hide, other.hide) + && Objects.equals(this.unhide, other.unhide); + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetastoreManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetastoreManager.java new file mode 100644 index 000000000000..15ae898176e6 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetastoreManager.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.storage; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.metadata.MetadataStorageConnector; +import org.apache.druid.metadata.MetadataStorageConnectorConfig; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.SQLMetadataConnector; + +/** + * Represents the metastore manager database and its implementation. + * Abstracts away the various kick-knacks used to define the metastore. + * The metastore operations are defined via table-specific classes. + */ +public interface MetastoreManager +{ + MetadataStorageConnector connector(); + MetadataStorageConnectorConfig config(); + MetadataStorageTablesConfig tablesConfig(); + + /** + * Whether to create tables if they do not exist. + */ + boolean createTables(); + + /** + * Object mapper to use for serializing and deserializing + * JSON objects stored in the metastore DB. + */ + ObjectMapper jsonMapper(); + + /** + * Is the implementation SQL-based? + */ + boolean isSql(); + + /** + * If SQL based, return the SQL version of the metastore + * connector. Throws an exception if not SQL-based. + */ + SQLMetadataConnector sqlConnector(); +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetastoreManagerImpl.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetastoreManagerImpl.java new file mode 100644 index 000000000000..ad1248a5ee52 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetastoreManagerImpl.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.storage; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Supplier; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.metadata.MetadataStorageConnector; +import org.apache.druid.metadata.MetadataStorageConnectorConfig; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.SQLMetadataConnector; + +import javax.inject.Inject; + +public class MetastoreManagerImpl implements MetastoreManager +{ + private final ObjectMapper jsonMapper; + private final MetadataStorageConnector connector; + private final MetadataStorageConnectorConfig config; + private final MetadataStorageTablesConfig tablesConfig; + + @Inject + public MetastoreManagerImpl( + @Json ObjectMapper jsonMapper, + MetadataStorageConnector connector, + Supplier configSupplier, + Supplier tablesConfigSupplier + ) + { + this.jsonMapper = jsonMapper; + this.connector = connector; + this.config = configSupplier.get(); + this.tablesConfig = tablesConfigSupplier.get(); + } + + @Override + public MetadataStorageConnector connector() + { + return connector; + } + + @Override + public MetadataStorageConnectorConfig config() + { + return config; + } + + @Override + public MetadataStorageTablesConfig tablesConfig() + { + return tablesConfig; + } + + @Override + public boolean createTables() + { + return config.isCreateTables(); + } + + @Override + public ObjectMapper jsonMapper() + { + return jsonMapper; + } + + @Override + public boolean isSql() + { + return connector instanceof SQLMetadataConnector; + } + + @Override + public SQLMetadataConnector sqlConnector() + { + return (SQLMetadataConnector) connector; + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MoveColumn.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MoveColumn.java new file mode 100644 index 000000000000..c91d68ae86a2 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MoveColumn.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.storage; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.catalog.model.CatalogUtils; +import org.apache.druid.catalog.model.ColumnSpec; +import org.apache.druid.java.util.common.ISE; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +/** + * JSON payload for the reorder column API. + */ +public class MoveColumn +{ + public enum Position + { + FIRST, + LAST, + BEFORE, + AFTER + } + + @JsonProperty + public final String column; + @JsonProperty + public final Position where; + @Nullable + @JsonProperty + public final String anchor; + + @JsonCreator + public MoveColumn( + @JsonProperty("column") final String column, + @JsonProperty("where") final Position where, + @JsonProperty("anchor") @Nullable final String anchor + ) + { + this.column = column; + this.where = where; + this.anchor = anchor; + } + + public List perform(List columns) + { + List revised = new ArrayList<>(columns); + final int colPosn = CatalogUtils.findColumn(columns, column); + if (colPosn == -1) { + throw new ISE("Column [%s] is not defined", column); + } + int anchorPosn; + if (where == Position.BEFORE || where == Position.AFTER) { + anchorPosn = CatalogUtils.findColumn(columns, anchor); + if (anchorPosn == -1) { + throw new ISE("Anchor [%s] is not defined", column); + } + if (anchorPosn > colPosn) { + anchorPosn--; + } + } else { + anchorPosn = -1; + } + + ColumnSpec col = revised.remove(colPosn); + switch (where) { + case FIRST: + revised.add(0, col); + break; + case LAST: + revised.add(col); + break; + case BEFORE: + revised.add(anchorPosn, col); + break; + case AFTER: + revised.add(anchorPosn + 1, col); + break; + } + return revised; + } + + @Override + public boolean equals(Object o) + { + if (o == null || o.getClass() != getClass()) { + return false; + } + MoveColumn other = (MoveColumn) o; + return Objects.equals(this.column, other.column) + && this.where == other.where + && Objects.equals(this.anchor, other.anchor); + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java new file mode 100644 index 000000000000..a137ae413610 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.storage.sql; + +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.model.TableSpec; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.function.Function; + +/** + * Manages catalog data. Used in Coordinator, which will be in either + * an leader or standby state. The Coordinator calls the {@link #start()} + * method when it becomes the leader, and calls {@link #stop()} when + * it loses leadership, or shuts down. + * + * Performs detailed CRUD operations on the catalog tables table. + * Higher-level operations appear elsewhere. + */ +public interface CatalogManager +{ + /** + * Thrown with an "optimistic lock" fails: the version of a + * catalog object being updated is not the same as that of + * the expected version. + */ + class OutOfDateException extends Exception + { + public OutOfDateException(String msg) + { + super(msg); + } + } + + class NotFoundException extends Exception + { + public NotFoundException(String msg) + { + super(msg); + } + } + + /** + * Indicates an attempt to insert a duplicate key into a table. + * This could indicate a logic error, or a race condition. It is + * generally not retryable: it us unrealistic to expect the other + * thread to helpfully delete the record it just added. + */ + class DuplicateKeyException extends Exception + { + public DuplicateKeyException(String msg, Exception e) + { + super(msg, e); + } + } + + interface Listener + { + void added(TableMetadata table); + void updated(TableMetadata table); + void deleted(TableId id); + } + + void start(); + + + void register(Listener listener); + void createTableDefnTable(); + + /** + * Create a table entry. + * + * @return the version of the newly created table. Call + * {@link TableMetadata#asUpdate(long)} if you want a new + * {@link TableMetadata} with the new version. + * @throws {@link DuplicateKeyException} if the row is a duplicate + * (schema, name) pair. This generally indicates a code error, + * or since our code is perfect, a race condition or a DB + * update outside of Druid. In any event, the error is not + * retryable: the user should pick another name, or update the + * existing table + */ + long create(TableMetadata table) throws DuplicateKeyException; + + /** + * Update a table definition. + *

+ * If {@code oldVersion == 0}, overwrites any current content. + * This is a potential race conditions if this is a partial update + * because of the possibility of another user doing an update since the + * read. Fine when the goal is to replace the entire definition. + * Else, only does the update is at the given version. + *

+ * Retryable only if the version is given, and an + * {@code OutOfDateException} is thrown. + */ + long update(TableMetadata table, long oldVersion) throws OutOfDateException, NotFoundException; + + /** + * Update the table spec incrementally using the transform provided. Performs the update + * in a transaction to ensure the read and write are atomic. + * + * @param id the table to update + * @param transform the transform to apply to the table spec + * @return the update timestamp (version) of the updated record + */ + long updatePayload(TableId id, Function transform) throws NotFoundException; + + /** + * Move the table to the deleting state. No version check: fine + * if the table is already in the deleting state. Does nothing if the + * table does not exist. + * + * @return new table update timestamp, or 0 if the table does not + * exist + */ + long markDeleting(TableId id); + + /** + * Read the table record for the given ID. + * + * @return the table record, or {@code null} if the entry is not + * found in the DB. + */ + @Nullable TableMetadata read(TableId id); + + /** + * Delete the table record for the given ID. Essentially does a + * "DELETE IF EXISTS". There is no version check. Delete should be + * called only when there are no segments left for the table: use + * {@link #markDeleting(TableId)} to indicates that the segments are + * being deleted. Call this method after deletion is complete. + *

+ * Does not cascade deletes yet. Eventually, should delete all entries + * for the table. + * + * @return {@code true} if the table exists and was deleted, + * {@code false} if the table did not exist. + */ + boolean delete(TableId id); + + List list(); + List list(String dbSchema); + List listDetails(String dbSchema); + + void stop(); +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/DbUtils.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/DbUtils.java new file mode 100644 index 000000000000..b421653f1c19 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/DbUtils.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.storage.sql; + +import org.skife.jdbi.v2.exceptions.UnableToExecuteStatementException; + +public class DbUtils +{ + // Move to SqlMetadataConnector and its subclasses + public static boolean isDuplicateRecordException(UnableToExecuteStatementException e) + { + Throwable cause = e.getCause(); + if (e.getCause() == null) { + return false; + } + + // Use class names to avoid compile-time dependencies. + // Derby implementation + if (cause.getClass().getSimpleName().equals("DerbySQLIntegrityConstraintViolationException")) { + return true; + } + // MySQL implementation + if (cause.getClass().getSimpleName().equals("MySQLIntegrityConstraintViolationException")) { + return true; + } + + // Don't know. + return false; + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java new file mode 100644 index 000000000000..a7888d2270f3 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java @@ -0,0 +1,578 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.storage.sql; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.inject.Inject; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.storage.MetastoreManager; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.metadata.SQLMetadataConnector; +import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.IDBI; +import org.skife.jdbi.v2.Query; +import org.skife.jdbi.v2.ResultIterator; +import org.skife.jdbi.v2.Update; +import org.skife.jdbi.v2.exceptions.CallbackFailedException; +import org.skife.jdbi.v2.exceptions.UnableToExecuteStatementException; +import org.skife.jdbi.v2.tweak.HandleCallback; + +import java.util.Deque; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.function.Function; + +@ManageLifecycle +public class SQLCatalogManager implements CatalogManager +{ + public static final String TABLES_TABLE = "tableDefs"; + + private static final String INSERT_TABLE = + "INSERT INTO %s\n" + + " (schemaName, name, creationTime, updateTime, state, payload)\n" + + " VALUES(:schemaName, :name, :creationTime, :updateTime, :state, :payload)"; + + private static final String UPDATE_HEAD = + "UPDATE %s\n SET\n"; + + private static final String WHERE_TABLE_ID = + "WHERE schemaName = :schemaName\n" + + " AND name = :name\n"; + + private static final String SAFETY_CHECK = + " AND updateTime = :oldVersion"; + + private static final String UPDATE_DEFN_UNSAFE = + UPDATE_HEAD + + " payload = :payload,\n" + + " updateTime = :updateTime\n" + + WHERE_TABLE_ID; + + private static final String UPDATE_DEFN_SAFE = + UPDATE_DEFN_UNSAFE + + SAFETY_CHECK; + + private static final String UPDATE_STATE = + UPDATE_HEAD + + " state = :state,\n" + + " updateTime = :updateTime\n" + + WHERE_TABLE_ID; + + private static final String SELECT_TABLE = + "SELECT creationTime, updateTime, state, payload\n" + + "FROM %s\n" + + WHERE_TABLE_ID; + + private static final String SELECT_PAYLOAD = + "SELECT state, payload\n" + + "FROM %s\n" + + WHERE_TABLE_ID; + + private static final String SELECT_ALL_TABLES = + "SELECT schemaName, name\n" + + "FROM %s\n" + + "ORDER BY schemaName, name"; + + private static final String SELECT_TABLES_IN_SCHEMA = + "SELECT name\n" + + "FROM %s\n" + + "WHERE schemaName = :schemaName\n" + + "ORDER BY name"; + + private static final String SELECT_TABLE_DETAILS_IN_SCHEMA = + "SELECT name, creationTime, updateTime, state, payload\n" + + "FROM %s\n" + + "WHERE schemaName = :schemaName\n" + + "ORDER BY name"; + + private static final String DELETE_TABLE = + "DELETE FROM %s\n" + + WHERE_TABLE_ID; + + private final MetastoreManager metastoreManager; + private final SQLMetadataConnector connector; + private final ObjectMapper jsonMapper; + private final IDBI dbi; + private final String tableName; + private final Deque listeners = new ConcurrentLinkedDeque<>(); + + @Inject + public SQLCatalogManager(MetastoreManager metastoreManager) + { + if (!metastoreManager.isSql()) { + throw new ISE("SQLCatalogManager only works with SQL based metadata store at this time"); + } + this.metastoreManager = metastoreManager; + this.connector = metastoreManager.sqlConnector(); + this.dbi = connector.getDBI(); + this.jsonMapper = metastoreManager.jsonMapper(); + this.tableName = getTableDefnTable(); + } + + @Override + @LifecycleStart + public void start() + { + createTableDefnTable(); + } + + @Override + public void stop() + { + } + + // Mimics what MetadataStorageTablesConfig should do. + public String getTableDefnTable() + { + final String base = metastoreManager.tablesConfig().getBase(); + if (Strings.isNullOrEmpty(base)) { + return TABLES_TABLE; + } else { + return StringUtils.format("%s_%s", base, TABLES_TABLE); + } + } + + // TODO: Move to SqlMetadataConnector + @Override + public void createTableDefnTable() + { + if (!metastoreManager.createTables()) { + return; + } + connector.createTable( + tableName, + ImmutableList.of( + StringUtils.format( + "CREATE TABLE %s (\n" + + " schemaName VARCHAR(255) NOT NULL,\n" + + " name VARCHAR(255) NOT NULL,\n" + + " creationTime BIGINT NOT NULL,\n" + + " updateTime BIGINT NOT NULL,\n" + + " state CHAR(1) NOT NULL,\n" + + " payload %s,\n" + + " PRIMARY KEY(schemaName, name)\n" + + ")", + tableName, + connector.getPayloadType()))); + } + + @Override + public long create(TableMetadata table) throws DuplicateKeyException + { + try { + return dbi.withHandle( + new HandleCallback() + { + @Override + public Long withHandle(Handle handle) throws DuplicateKeyException + { + long updateTime = System.currentTimeMillis(); + Update stmt = handle.createStatement( + StringUtils.format(INSERT_TABLE, tableName) + ) + .bind("schemaName", table.id().schema()) + .bind("name", table.id().name()) + .bind("creationTime", updateTime) + .bind("updateTime", updateTime) + .bind("state", TableMetadata.TableState.ACTIVE.code()) + .bind("payload", table.spec().toBytes(jsonMapper)); + try { + stmt.execute(); + } + catch (UnableToExecuteStatementException e) { + if (DbUtils.isDuplicateRecordException(e)) { + throw new DuplicateKeyException( + "Tried to insert a duplicate table: " + table.sqlName(), + e); + } else { + throw e; + } + } + sendAddition(table, updateTime); + return updateTime; + } + } + ); + } + catch (CallbackFailedException e) { + if (e.getCause() instanceof DuplicateKeyException) { + throw (DuplicateKeyException) e.getCause(); + } + throw e; + } + } + + @Override + public TableMetadata read(TableId id) + { + return dbi.withHandle( + new HandleCallback() + { + @Override + public TableMetadata withHandle(Handle handle) + { + Query> query = handle.createQuery( + StringUtils.format(SELECT_TABLE, tableName) + ) + .setFetchSize(connector.getStreamingFetchSize()) + .bind("schemaName", id.schema()) + .bind("name", id.name()); + final ResultIterator resultIterator = + query.map((index, r, ctx) -> + new TableMetadata( + id, + r.getLong(1), + r.getLong(2), + TableMetadata.TableState.fromCode(r.getString(3)), + TableSpec.fromBytes(jsonMapper, r.getBytes(4)) + )) + .iterator(); + if (resultIterator.hasNext()) { + return resultIterator.next(); + } + return null; + } + } + ); + } + + @Override + public long update(TableMetadata table, long oldVersion) throws OutOfDateException, NotFoundException + { + if (oldVersion == 0) { + return updateUnsafe(table.id(), table.spec()); + } else { + return updateSafe(table.id(), table.spec(), oldVersion); + } + } + + private long updateSafe(TableId id, TableSpec defn, long oldVersion) throws OutOfDateException + { + try { + return dbi.withHandle( + new HandleCallback() + { + @Override + public Long withHandle(Handle handle) throws OutOfDateException + { + long updateTime = System.currentTimeMillis(); + int updateCount = handle.createStatement( + StringUtils.format(UPDATE_DEFN_SAFE, tableName)) + .bind("schemaName", id.schema()) + .bind("name", id.name()) + .bind("payload", defn.toBytes(jsonMapper)) + .bind("updateTime", updateTime) + .bind("oldVersion", oldVersion) + .execute(); + if (updateCount == 0) { + throw new OutOfDateException( + StringUtils.format( + "Table %s: not found or update version does not match DB version", + id.sqlName())); + } + sendUpdate(id); + return updateTime; + } + } + ); + } + catch (CallbackFailedException e) { + if (e.getCause() instanceof OutOfDateException) { + throw (OutOfDateException) e.getCause(); + } + throw e; + } + } + + private long updateUnsafe(TableId id, TableSpec defn) throws NotFoundException + { + try { + return dbi.withHandle( + new HandleCallback() + { + @Override + public Long withHandle(Handle handle) throws NotFoundException + { + long updateTime = System.currentTimeMillis(); + int updateCount = handle.createStatement( + StringUtils.format(UPDATE_DEFN_UNSAFE, tableName)) + .bind("schemaName", id.schema()) + .bind("name", id.name()) + .bind("payload", defn.toBytes(jsonMapper)) + .bind("updateTime", updateTime) + .execute(); + if (updateCount == 0) { + throw new NotFoundException( + StringUtils.format("Table %s: not found", id.sqlName()) + ); + } + sendUpdate(id); + return updateTime; + } + } + ); + } + catch (CallbackFailedException e) { + if (e.getCause() instanceof NotFoundException) { + throw (NotFoundException) e.getCause(); + } + throw e; + } + } + + @Override + public long updatePayload(TableId id, Function transform) throws NotFoundException + { + try { + return dbi.withHandle( + new HandleCallback() + { + @Override + public Long withHandle(Handle handle) throws NotFoundException + { + handle.begin(); + try { + Query> query = handle.createQuery( + StringUtils.format(SELECT_PAYLOAD, tableName) + ) + .setFetchSize(connector.getStreamingFetchSize()) + .bind("schemaName", id.schema()) + .bind("name", id.name()); + + final ResultIterator resultIterator = + query.map((index, r, ctx) -> + new TableMetadata( + id, + 0, + 0, + TableMetadata.TableState.fromCode(r.getString(1)), + TableSpec.fromBytes(jsonMapper, r.getBytes(2)) + )) + .iterator(); + TableMetadata table; + if (resultIterator.hasNext()) { + table = resultIterator.next(); + } else { + handle.rollback(); + throw new NotFoundException( + StringUtils.format("Table %s: not found", id.sqlName()) + ); + } + if (table.state() != TableMetadata.TableState.ACTIVE) { + throw new ISE("Table is in state [%s] and cannot be updated", table.state()); + } + TableSpec revised = transform.apply(table.spec()); + long updateTime = System.currentTimeMillis(); + int updateCount = handle.createStatement( + StringUtils.format(UPDATE_DEFN_UNSAFE, tableName)) + .bind("schemaName", id.schema()) + .bind("name", id.name()) + .bind("payload", revised.toBytes(jsonMapper)) + .bind("updateTime", updateTime) + .execute(); + if (updateCount == 0) { + // Should never occur because we're holding a lock. + throw new ISE("Table %s: not found", id.sqlName()); + } + handle.commit(); + sendUpdate(id); + return updateTime; + } + catch (RuntimeException e) { + handle.rollback(); + throw e; + } + } + } + ); + } + catch (CallbackFailedException e) { + if (e.getCause() instanceof NotFoundException) { + throw (NotFoundException) e.getCause(); + } + throw e; + } + } + + @Override + public long markDeleting(TableId id) + { + return dbi.withHandle( + new HandleCallback() + { + @Override + public Long withHandle(Handle handle) + { + long updateTime = System.currentTimeMillis(); + int updateCount = handle.createStatement( + StringUtils.format(UPDATE_STATE, tableName)) + .bind("schemaName", id.schema()) + .bind("name", id.name()) + .bind("updateTime", updateTime) + .bind("state", TableMetadata.TableState.DELETING.code()) + .execute(); + sendDeletion(id); + return updateCount == 1 ? updateTime : 0; + } + } + ); + } + + @Override + public boolean delete(TableId id) + { + return dbi.withHandle( + new HandleCallback() + { + @Override + public Boolean withHandle(Handle handle) + { + int updateCount = handle.createStatement( + StringUtils.format(DELETE_TABLE, tableName)) + .bind("schemaName", id.schema()) + .bind("name", id.name()) + .execute(); + sendDeletion(id); + return updateCount > 0; + } + } + ); + } + + @Override + public List list() + { + return dbi.withHandle( + new HandleCallback>() + { + @Override + public List withHandle(Handle handle) + { + Query> query = handle.createQuery( + StringUtils.format(SELECT_ALL_TABLES, tableName) + ) + .setFetchSize(connector.getStreamingFetchSize()); + final ResultIterator resultIterator = + query.map((index, r, ctx) -> + new TableId(r.getString(1), r.getString(2))) + .iterator(); + return Lists.newArrayList(resultIterator); + } + } + ); + } + + @Override + public List list(String dbSchema) + { + return dbi.withHandle( + new HandleCallback>() + { + @Override + public List withHandle(Handle handle) + { + Query> query = handle.createQuery( + StringUtils.format(SELECT_TABLES_IN_SCHEMA, tableName) + ) + .bind("schemaName", dbSchema) + .setFetchSize(connector.getStreamingFetchSize()); + final ResultIterator resultIterator = + query.map((index, r, ctx) -> + r.getString(1)) + .iterator(); + return Lists.newArrayList(resultIterator); + } + } + ); + } + + @Override + public List listDetails(String dbSchema) + { + return dbi.withHandle( + new HandleCallback>() + { + @Override + public List withHandle(Handle handle) + { + Query> query = handle.createQuery( + StringUtils.format(SELECT_TABLE_DETAILS_IN_SCHEMA, tableName) + ) + .bind("schemaName", dbSchema) + .setFetchSize(connector.getStreamingFetchSize()); + final ResultIterator resultIterator = + query.map((index, r, ctx) -> + new TableMetadata( + TableId.of(dbSchema, r.getString(1)), + r.getLong(2), + r.getLong(3), + TableMetadata.TableState.fromCode(r.getString(4)), + TableSpec.fromBytes(jsonMapper, r.getBytes(5)))) + .iterator(); + return Lists.newArrayList(resultIterator); + } + } + ); + } + + @Override + public synchronized void register(Listener listener) + { + listeners.add(listener); + } + + protected synchronized void sendAddition(TableMetadata table, long updateTime) + { + if (listeners.isEmpty()) { + return; + } + TableMetadata newTable = table.fromInsert(updateTime); + for (Listener listener : listeners) { + listener.added(newTable); + } + } + + protected synchronized void sendUpdate(TableId id) + { + if (listeners.isEmpty()) { + return; + } + TableMetadata updatedTable = read(id); + for (Listener listener : listeners) { + listener.updated(updatedTable); + } + } + + protected synchronized void sendDeletion(TableId id) + { + for (Listener listener : listeners) { + listener.deleted(id); + } + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CacheNotifier.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CacheNotifier.java new file mode 100644 index 000000000000..3b74432338b4 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CacheNotifier.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.sync; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.druid.concurrent.Threads; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.emitter.EmittingLogger; + +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; + +/** + * Push style notifications that allow propagation of data from whatever server is + * running this notifier to whoever might be listening. Notifications arrive + * via a queue, then are dispatched via a configured sender. Details of the + * source of the updates, and how updates are sent, are handled external + * to this class. + * + * The algorithm is simple: each update is processed entirely before the + * next one is processed. As a result, this class is suitable for + * low-frequency updates: where the worst-case send times are less than + * the worst-case update frequency. If updates are faster, they will back + * up, and the class should be redesigned to allow healthy receivers to + * continue to get updates while laggards block only themselves. + * + * Events can be queued before startup. They will be send once the notifier + * is started. Events left in the queue at shutdown will be lost. + * + * Defined by composition so it can be tested and reused in other + * contexts. + */ +public class CacheNotifier +{ + private static final EmittingLogger LOG = new EmittingLogger(CacheNotifier.class); + + private final ExecutorService exec; + private final String callerName; + private final BlockingQueue updates = new LinkedBlockingQueue<>(); + private final Consumer sender; + + public CacheNotifier( + final String callerName, + final Consumer sender + ) + { + this.callerName = callerName; + this.sender = sender; + + this.exec = Execs.singleThreaded( + StringUtils.format( + "%s-notifierThread-", + StringUtils.encodeForFormat(callerName)) + "%d" + ); + } + + public void start() + { + exec.submit(() -> { + while (!Thread.interrupted()) { + try { + sender.accept(updates.take()); + } + catch (InterruptedException e) { + return; + } + catch (Throwable t) { + LOG.makeAlert(t, callerName + ": Error occured while handling updates.").emit(); + } + } + }); + } + + public void send(byte[] update) + { + updates.add(update); + } + + @VisibleForTesting + public void stopGracefully() + { + try { + while (!updates.isEmpty()) { + Threads.sleepFor(100, TimeUnit.MILLISECONDS); + } + } + catch (InterruptedException e) { + // Ignore + } + stop(); + } + + public void stop() + { + exec.shutdownNow(); + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java new file mode 100644 index 000000000000..980e1bc5f309 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.sync; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.SchemaRegistry; +import org.apache.druid.catalog.model.SchemaRegistry.SchemaSpec; +import org.apache.druid.catalog.model.TableDefnRegistry; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.sync.MetadataCatalog.CatalogListener; +import org.apache.druid.guice.annotations.Json; + +import javax.inject.Inject; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Caching version of the metadata catalog. Draws information + * from a base catalog. Fetches from the base if: + *

+ * + * Both tables and schemas are cached. In particular, if a table or + * schema is requested, and does not exist in the base catalog, then + * that schema is marked as not existing and won't be fetched again. + * + * The cache is updated via an update facility which either flushes + * the cache (crude) or listens to the base catalog for updates and + * populates the cache with updates. For a local cache, the DB layer + * provides the updates. For a remote cache, the DB host pushes updates. + */ +public class CachedMetadataCatalog implements MetadataCatalog, CatalogListener +{ + public static final int NOT_FETCHED = -1; + public static final int UNDEFINED = 0; + + private static class TableEntry + { + private final TableMetadata table; + + protected TableEntry(SchemaSpec schema, TableMetadata table) + { + this.table = table; + } + + protected long version() + { + return table == null ? UNDEFINED : table.updateTime(); + } + } + + private class SchemaEntry + { + private final SchemaSpec schema; + private long version = NOT_FETCHED; + private final ConcurrentHashMap cache = new ConcurrentHashMap<>(); + + protected SchemaEntry(SchemaSpec schema) + { + this.schema = schema; + } + + protected TableMetadata resolveTable(TableId tableId) + { + TableEntry entry = cache.computeIfAbsent( + tableId.name(), + key -> new TableEntry(schema, base.table(tableId)) + ); + return entry.table; + } + + public synchronized List tables() + { + if (version == UNDEFINED) { + return Collections.emptyList(); + } + if (version == NOT_FETCHED) { + List catalogTables = base.tablesForSchema(schema.name()); + for (TableMetadata table : catalogTables) { + update(table); + } + } + List orderedTables = new ArrayList<>(); + + // Get the list of actual tables; excluding any cached "misses". + cache.forEach((k, v) -> { + if (v.table != null) { + orderedTables.add(v.table); + } + }); + orderedTables.sort((e1, e2) -> e1.id().name().compareTo(e2.id().name())); + return orderedTables; + } + + public synchronized void update(TableMetadata table) + { + cache.compute( + table.id().name(), + (k, v) -> v == null || v.version() < table.updateTime() + ? new TableEntry(schema, table) + : v + ); + version = Math.max(version, table.updateTime()); + } + + public void remove(String name) + { + cache.remove(name); + } + + public Set tableNames() + { + Set tables = new HashSet<>(); + cache.forEach((k, v) -> { + if (v.table != null) { + tables.add(k); + } + }); + return tables; + } + } + + private final ConcurrentHashMap schemaCache = new ConcurrentHashMap<>(); + private final CatalogSource base; + private final SchemaRegistry schemaRegistry; + private final TableDefnRegistry tableRegistry; + + @Inject + public CachedMetadataCatalog( + CatalogSource catalog, + SchemaRegistry schemaRegistry, + @Json ObjectMapper jsonMapper + ) + { + this.base = catalog; + this.schemaRegistry = schemaRegistry; + this.tableRegistry = new TableDefnRegistry(jsonMapper); + } + + @Override + public TableMetadata getTable(TableId tableId) + { + SchemaEntry schemaEntry = entryFor(tableId.schema()); + return schemaEntry == null ? null : schemaEntry.resolveTable(tableId); + } + + @Override + public ResolvedTable resolveTable(TableId tableId) + { + TableMetadata table = getTable(tableId); + return table == null ? null : tableRegistry.resolve(table.spec()); + } + + @Override + public List tables(String schemaName) + { + SchemaEntry schemaEntry = entryFor(schemaName); + return schemaEntry == null ? null : schemaEntry.tables(); + } + + @Override + public void updated(TableMetadata table) + { + SchemaEntry schemaEntry = entryFor(table.id().schema()); + if (schemaEntry != null) { + schemaEntry.update(table); + } + } + + @Override + public void deleted(TableId tableId) + { + SchemaEntry schemaEntry = entryFor(tableId.schema()); + if (schemaEntry != null) { + schemaEntry.remove(tableId.name()); + } + } + + @Override + public Set tableNames(String schemaName) + { + SchemaEntry schemaEntry = entryFor(schemaName); + return schemaEntry == null ? Collections.emptySet() : schemaEntry.tableNames(); + } + + public void flush() + { + schemaCache.clear(); + } + + private SchemaEntry entryFor(String schemaName) + { + return schemaCache.computeIfAbsent( + schemaName, + k -> { + SchemaSpec schema = schemaRegistry.schema(k); + return schema == null ? null : new SchemaEntry(schema); + }); + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogClient.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogClient.java new file mode 100644 index 000000000000..a06addfce710 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogClient.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.sync; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.catalog.http.CatalogResource; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableDefnRegistry; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.sync.MetadataCatalog.CatalogSource; +import org.apache.druid.client.coordinator.Coordinator; +import org.apache.druid.discovery.DruidLeaderClient; +import org.apache.druid.guice.annotations.Json; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.StringFullResponseHolder; +import org.jboss.netty.handler.codec.http.HttpHeaders; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; + +import javax.inject.Inject; +import javax.ws.rs.core.MediaType; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +/** + * Guice-injected client for the catalog update sync process. Requests + * tables and schemas from the catalog component on the Coordinator. + * + * This class handles any recoverable error case. If this class throws + * an exception, then something went very wrong and there is little the + * caller can do to make things better. All the caller can do is try + * again later and hope things improve. + */ +public class CatalogClient implements CatalogSource +{ + public static final String SCHEMA_SYNC_PATH = CatalogResource.ROOT_PATH + CatalogResource.SCHEMA_SYNC; + public static final String TABLE_SYNC_PATH = CatalogResource.ROOT_PATH + CatalogResource.TABLE_SYNC; + private static final TypeReference> LIST_OF_TABLE_METADATA_TYPE = new TypeReference>() + { + }; + // Not strictly needed as a TypeReference, but doing so makes the code simpler. + private static final TypeReference TABLE_METADATA_TYPE = new TypeReference() + { + }; + + private final DruidLeaderClient coordClient; + private final ObjectMapper smileMapper; + private final TableDefnRegistry tableRegistry; + + @Inject + public CatalogClient( + @Coordinator DruidLeaderClient coordClient, + @Smile ObjectMapper smileMapper, + @Json ObjectMapper jsonMapper + ) + { + this.coordClient = coordClient; + this.smileMapper = smileMapper; + this.tableRegistry = new TableDefnRegistry(jsonMapper); + } + + @Override + public List tablesForSchema(String dbSchema) + { + String url = StringUtils.replace(SCHEMA_SYNC_PATH, "{dbSchema}", dbSchema); + List results = send(url, LIST_OF_TABLE_METADATA_TYPE); + + // Not found for a list is an empty list. + return results == null ? Collections.emptyList() : results; + } + + @Override + public TableMetadata table(TableId id) + { + String url = StringUtils.replace(TABLE_SYNC_PATH, "{dbSchema}", id.schema()); + url = StringUtils.replace(url, "{name}", id.name()); + return send(url, TABLE_METADATA_TYPE); + } + + @Override + public ResolvedTable resolveTable(TableId id) + { + TableMetadata table = table(id); + return table == null ? null : tableRegistry.resolve(table.spec()); + } + + /** + * Send the update. Exceptions are "unexpected": they should never occur in a + * working system. If they occur, something is broken. + * + * @return the requested update, or null if the item was not found in the + * catalog. + */ + private T send(String url, TypeReference typeRef) + { + final Request request; + try { + request = coordClient.makeRequest(HttpMethod.GET, url) + .addHeader(HttpHeaders.Names.ACCEPT, MediaType.APPLICATION_JSON); + } + catch (IOException e) { + throw new ISE("Cannot create catalog sync request"); + } + final StringFullResponseHolder responseHolder; + try { + responseHolder = coordClient.go(request); + } + catch (IOException e) { + throw new ISE(e, "Failed to send catalog sync"); + } + catch (InterruptedException e1) { + // Treat as a not-found: the only way this exception should occur + // is during shutdown. + return null; + } + if (responseHolder.getStatus().getCode() == HttpResponseStatus.NOT_FOUND.getCode()) { + // Not found means the item disappeared. Returning null means "not found". + return null; + } + if (responseHolder.getStatus().getCode() != HttpResponseStatus.OK.getCode()) { + throw new ISE("Unexpected status from catalog sync: " + responseHolder.getStatus()); + } + try { + return smileMapper.readValue(responseHolder.getContent(), typeRef); + } + catch (IOException e) { + throw new ISE(e, "Could not decode the JSON response from catalog sync."); + } + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateNotifier.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateNotifier.java new file mode 100644 index 000000000000..55b2e848df4b --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateNotifier.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.sync; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.catalog.http.CatalogListenerResource; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.storage.CatalogStorage; +import org.apache.druid.catalog.sync.MetadataCatalog.CatalogListener; +import org.apache.druid.catalog.sync.RestUpdateSender.RestSender; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.annotations.EscalatedClient; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.server.DruidNode; +import org.joda.time.Duration; + +import javax.inject.Inject; + +import java.util.Collections; +import java.util.function.Supplier; + +/** + * Global update notifier for the catalog. Registers itself as a catalog + * listener, then uses the common cache notifier to send Smile-encoded JSON + * updates to broker nodes discovered from node discovery (typically ZooKeeper.) + *

+ * Deletes are encoded as a table update with a table definition of a special + * tombstone type. This saves having the need for two endpoints, or having + * a wrapper class to handle deletes. + */ +@ManageLifecycle +public class CatalogUpdateNotifier implements CatalogListener +{ + private static final String CALLER_NAME = "Catalog Sync"; + private static final long TIMEOUT_MS = 5000; + + /** + * Internal table type used in updates to notify listeners that a table has + * been deleted. Avoids the need for a special "table deleted" message. + */ + public static final String TOMBSTONE_TABLE_TYPE = "tombstone"; + private static final TableSpec TABLE_TOMBSTONE = new TableSpec(TOMBSTONE_TABLE_TYPE, null, null); + + private final CacheNotifier notifier; + private final ObjectMapper smileMapper; + + @Inject + public CatalogUpdateNotifier( + CatalogStorage catalog, + DruidNodeDiscoveryProvider discoveryProvider, + @EscalatedClient HttpClient httpClient, + @Smile ObjectMapper smileMapper + ) + { + long timeoutMs = TIMEOUT_MS; + this.smileMapper = smileMapper; + Supplier> nodeSupplier = new ListeningNodeSupplier( + Collections.singletonList(NodeRole.BROKER), + discoveryProvider); + RestSender restSender = RestUpdateSender.httpClientSender(httpClient, Duration.millis(timeoutMs)); + RestUpdateSender sender = new RestUpdateSender( + CALLER_NAME, + nodeSupplier, + restSender, + CatalogListenerResource.BASE_URL + CatalogListenerResource.SYNC_URL, + timeoutMs); + this.notifier = new CacheNotifier( + CALLER_NAME, + sender); + catalog.register(this); + } + + @LifecycleStart + public void start() + { + notifier.start(); + } + + @LifecycleStop + public void stop() + { + notifier.stop(); + } + + @Override + public void updated(TableMetadata update) + { + notifier.send(update.toBytes(smileMapper)); + } + + @Override + public void deleted(TableId tableId) + { + TableMetadata spec = TableMetadata.newTable(tableId, TABLE_TOMBSTONE); + notifier.send(spec.toBytes(smileMapper)); + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/ListeningNodeSupplier.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/ListeningNodeSupplier.java new file mode 100644 index 000000000000..e08ffce77505 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/ListeningNodeSupplier.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.sync; + +import org.apache.druid.discovery.DiscoveryDruidNode; +import org.apache.druid.discovery.DruidNodeDiscovery; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.NodeRole; +import org.apache.druid.server.DruidNode; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.function.Supplier; + +/** + * Provides an up-to-date list of Druid nodes of the given types each + * time the list is requested. + * + * The algorithm could be improved to cache the list and update it only + * when the set of nodes changes. For the catalog, the rate of change is + * likely to be low, so creating the list each time is fine. If this code + * is used for high-speed updates, then caching would be desirable. + */ +public class ListeningNodeSupplier implements Supplier> +{ + private final List nodeTypes; + private final DruidNodeDiscoveryProvider discoveryProvider; + + public ListeningNodeSupplier( + List nodeTypes, + DruidNodeDiscoveryProvider discoveryProvider + ) + { + this.nodeTypes = nodeTypes; + this.discoveryProvider = discoveryProvider; + } + + @Override + public Iterable get() + { + List druidNodes = new ArrayList<>(); + for (NodeRole nodeRole : nodeTypes) { + DruidNodeDiscovery nodeDiscovery = discoveryProvider.getForNodeRole(nodeRole); + Collection nodes = nodeDiscovery.getAllNodes(); + nodes.forEach(node -> druidNodes.add(node.getDruidNode())); + } + return druidNodes; + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/LocalMetadataCatalog.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/LocalMetadataCatalog.java new file mode 100644 index 000000000000..98e2eea516fd --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/LocalMetadataCatalog.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.sync; + +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.SchemaRegistry; +import org.apache.druid.catalog.model.SchemaRegistry.SchemaSpec; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; + +import javax.inject.Inject; + +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Metadata catalog which reads from the catalog storage. No caching. + * For testing, and as the Coordinator-side implementation of the remote + * synchronization protocol. + */ +public class LocalMetadataCatalog implements MetadataCatalog +{ + private final CatalogSource catalog; + private final SchemaRegistry schemaRegistry; + + @Inject + public LocalMetadataCatalog( + CatalogSource catalog, + SchemaRegistry schemaRegistry + ) + { + this.catalog = catalog; + this.schemaRegistry = schemaRegistry; + } + + @Override + public TableMetadata getTable(TableId tableId) + { + return catalog.table(tableId); + } + + @Override + public ResolvedTable resolveTable(TableId tableId) + { + return catalog.resolveTable(tableId); + } + + @Override + public List tables(String schemaName) + { + SchemaSpec schema = schemaRegistry.schema(schemaName); + if (schema == null || !schema.writable()) { + return Collections.emptyList(); + } + return catalog.tablesForSchema(schemaName); + } + + @Override + public Set tableNames(String schemaName) + { + SchemaSpec schema = schemaRegistry.schema(schemaName); + if (schema == null || !schema.writable()) { + return Collections.emptySet(); + } + List catalogTables = catalog.tablesForSchema(schemaName); + Set tables = new HashSet<>(); + for (TableMetadata table : catalogTables) { + tables.add(table.id().name()); + } + return tables; + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/MetadataCatalog.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/MetadataCatalog.java new file mode 100644 index 000000000000..ed3861cce4e8 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/MetadataCatalog.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.sync; + +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; + +import java.util.List; +import java.util.Set; + +/** + * Client view of the metadata catalog. Implementations can be local + * (with the DB on the same node), or remote (if the DB is on another + * node.) Any caching that is desired is done behind this interface. + *

+ * This interface does not interpolate physical data from + * segments. That work is done by a layer on top of this one: a + * layer which also has visibility to the segment caching logic. + */ +public interface MetadataCatalog +{ + interface CatalogSource + { + List tablesForSchema(String dbSchema); + TableMetadata table(TableId id); + ResolvedTable resolveTable(TableId id); + } + + interface CatalogListener + { + void updated(TableMetadata update); + void deleted(TableId tableId); + } + + interface CatalogUpdateProvider + { + void register(CatalogListener listener); + } + + /** + * Resolves a table given a {@link TableId} with the schema and + * table name. Does not do security checks: the caller is responsible. + * + * @return the table metadata, if any exists, else {@code null} if + * no metadata is available. Note that a datasource can exist without + * metadata. Views and input sources exist only if their + * metadata exists. System tables never have metadata. + */ + TableMetadata getTable(TableId tableId); + ResolvedTable resolveTable(TableId tableId); + + /** + * List of tables defined within the given schema. Does not filter the + * tables by permissions: the caller is responsible for that. + * + * @param schemaName + * @return + */ + List tables(String schemaName); + + Set tableNames(String schemaName); +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/RestUpdateSender.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/RestUpdateSender.java new file mode 100644 index 000000000000..78495a9e241e --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/RestUpdateSender.java @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.sync; + +import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.StatusResponseHandler; +import org.apache.druid.java.util.http.client.response.StatusResponseHolder; +import org.apache.druid.server.DruidNode; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.joda.time.Duration; + +import java.net.MalformedURLException; +import java.net.URL; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.Consumer; +import java.util.function.Supplier; + +/** + * Sends updates to a set of Druid nodes provided by the given supplier. + * The update is provided by the caller in serialized form. The class sends + * updates concurrently, and returns futures for all the requests. + * + * Updates are processed one by one, but each updates is sent concurrently. + * All responses from all receivers must arrive (or a timeout must occur) + * before the next updates can be sent. As a result, this class is suitable for + * low-frequency updates: where the worst-case send times are less than + * the worst-case update frequency. If updates are faster, they will back + * up, and the class should be redesigned to allow healthy receivers to + * continue to get updates while laggards block only themselves. + * + * Defined by composition so it can be tested and reused in other + * contexts. + */ +public class RestUpdateSender implements Consumer +{ + private static final EmittingLogger LOG = new EmittingLogger(RestUpdateSender.class); + + public interface RestSender + { + ListenableFuture send(URL listenerURL, byte[] serializedEntity); + } + + private static class HttpClientSender implements RestSender + { + private final HttpClient httpClient; + private final Duration cacheNotificationsTimeout; + + private HttpClientSender( + HttpClient httpClient, + Duration cacheNotificationsTimeout) + { + this.httpClient = httpClient; + this.cacheNotificationsTimeout = cacheNotificationsTimeout; + } + + @Override + public ListenableFuture send(URL listenerURL, byte[] serializedEntity) + { + // Best effort, if this fails, remote node will poll + // and pick up the update eventually. + return httpClient.go( + new Request(HttpMethod.POST, listenerURL) + .setContent(SmileMediaTypes.APPLICATION_JACKSON_SMILE, serializedEntity), + StatusResponseHandler.getInstance(), + cacheNotificationsTimeout + ); + } + } + + private final String callerName; + private final Supplier> destinationSupplier; + private final String baseUrl; + private final RestSender sender; + private final long cacheNotificationsTimeoutMs; + + public RestUpdateSender( + final String callerName, + final Supplier> destinationSupplier, + final RestSender sender, + final String baseUrl, + final long cacheNotificationsTimeoutMs + ) + { + this.callerName = callerName; + this.destinationSupplier = destinationSupplier; + this.sender = sender; + this.baseUrl = baseUrl; + this.cacheNotificationsTimeoutMs = cacheNotificationsTimeoutMs; + } + + public static RestSender httpClientSender(HttpClient httpClient, Duration cacheNotificationsTimeou) + { + return new HttpClientSender(httpClient, cacheNotificationsTimeou); + } + + @Override + public void accept(byte[] serializedEntity) + { + LOG.debug(callerName + ": Sending update notifications"); + + // Best effort, if a notification fails, the remote node will eventually poll to update its state + // We wait for responses however, to avoid flooding remote nodes with notifications. + List> futures = new ArrayList<>(); + for (DruidNode node : destinationSupplier.get()) { + futures.add( + sender.send( + getListenerURL(node, baseUrl), + serializedEntity)); + } + + try { + List responses = getResponsesFromFutures(futures); + + for (StatusResponseHolder response : responses) { + if (response == null) { + LOG.error("Got null future response from update request."); + continue; + } + HttpResponseStatus status = response.getStatus(); + if (HttpResponseStatus.OK.equals(status) || + HttpResponseStatus.ACCEPTED.equals(status)) { + LOG.debug("Got status [%s]", status); + } else { + LOG.error("Got error status [%s], content [%s]", status, response.getContent()); + } + } + } + catch (Exception e) { + LOG.makeAlert(e, callerName + ": Failed to get response for cache notification.").emit(); + } + + LOG.debug(callerName + ": Received responses for cache update notifications."); + } + + @VisibleForTesting + List getResponsesFromFutures( + List> futures + ) throws InterruptedException, ExecutionException, TimeoutException + { + return Futures.successfulAsList(futures) + .get( + cacheNotificationsTimeoutMs, + TimeUnit.MILLISECONDS + ); + } + + private URL getListenerURL(DruidNode druidNode, String baseUrl) + { + try { + return new URL( + druidNode.getServiceScheme(), + druidNode.getHost(), + druidNode.getPortToUse(), + baseUrl + ); + } + catch (MalformedURLException mue) { + String msg = StringUtils.format(callerName + ": Malformed url for DruidNode [%s] and baseUrl [%s]", druidNode, baseUrl); + LOG.error(msg); + throw new RE(mue, msg); + } + } +} diff --git a/extensions-core/druid-catalog/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-core/druid-catalog/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule new file mode 100644 index 000000000000..3f9cd5ca725e --- /dev/null +++ b/extensions-core/druid-catalog/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.druid.catalog.guice.CatalogCoordinatorModule diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/CatalogTests.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/CatalogTests.java new file mode 100644 index 000000000000..fce8fa87f67e --- /dev/null +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/CatalogTests.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.storage; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.catalog.storage.sql.CatalogManager; +import org.apache.druid.catalog.storage.sql.SQLCatalogManager; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.metadata.TestDerbyConnector.DerbyConnectorRule; +import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.Action; +import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.Authorizer; +import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.server.security.Resource; +import org.apache.druid.server.security.ResourceType; + +import java.util.Arrays; + +public class CatalogTests +{ + public static final String TEST_AUTHORITY = "test"; + + public static final String SUPER_USER = "super"; + public static final String READER_USER = "reader"; + public static final String WRITER_USER = "writer"; + public static final String DENY_USER = "denyAll"; + + protected static final AuthorizerMapper AUTH_MAPPER = new AuthorizerMapper( + ImmutableMap.of(TEST_AUTHORITY, new TestAuthorizer())); + + private static class TestAuthorizer implements Authorizer + { + @Override + public Access authorize( + AuthenticationResult authenticationResult, + Resource resource, + Action action + ) + { + final String userName = authenticationResult.getIdentity(); + if (SUPER_USER.equals(userName)) { + return Access.OK; + } + if (ResourceType.DATASOURCE.equals(resource.getType())) { + if ("forbidden".equals(resource.getName())) { + return Access.DENIED; + } + return new Access( + WRITER_USER.equals(userName) || + READER_USER.equals(userName) && action == Action.READ); + } + return Access.OK; + } + } + + public static InputFormat csvFormat() + { + return new CsvInputFormat( + Arrays.asList("x", "y", "z"), + null, // listDelimiter + false, // hasHeaderRow + false, // findColumnsFromHeader + 0 // skipHeaderRows + ); + } + + public static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); + + public static class DbFixture + { + public CatalogManager manager; + public CatalogStorage storage; + + public DbFixture(DerbyConnectorRule derbyConnectorRule) + { + MetastoreManager metastoreMgr = new MetastoreManagerImpl( + JSON_MAPPER, + derbyConnectorRule.getConnector(), + () -> derbyConnectorRule.getMetadataConnectorConfig(), + derbyConnectorRule.metadataTablesConfigSupplier() + ); + manager = new SQLCatalogManager(metastoreMgr); + manager.start(); + storage = new CatalogStorage( + manager, + AUTH_MAPPER, + JSON_MAPPER + ); + } + + public void tearDown() + { + if (manager != null) { + manager.stop(); + manager = null; + } + } + } + + public static void tearDown(DbFixture fixture) + { + if (fixture != null) { + fixture.tearDown(); + } + } +} diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java new file mode 100644 index 000000000000..132c23b43017 --- /dev/null +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java @@ -0,0 +1,207 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.storage; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.catalog.CatalogTest; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.catalog.storage.sql.CatalogManager; +import org.apache.druid.catalog.storage.sql.CatalogManager.DuplicateKeyException; +import org.apache.druid.catalog.storage.sql.CatalogManager.NotFoundException; +import org.apache.druid.catalog.storage.sql.CatalogManager.OutOfDateException; +import org.apache.druid.catalog.storage.sql.SQLCatalogManager; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.metadata.TestDerbyConnector; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +@Category(CatalogTest.class) +public class TableManagerTest +{ + private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); + + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = + new TestDerbyConnector.DerbyConnectorRule(); + private CatalogManager manager; + + @Before + public void setUp() + { + MetastoreManager metastoreMgr = new MetastoreManagerImpl( + JSON_MAPPER, + derbyConnectorRule.getConnector(), + () -> derbyConnectorRule.getMetadataConnectorConfig(), + derbyConnectorRule.metadataTablesConfigSupplier() + ); + manager = new SQLCatalogManager(metastoreMgr); + manager.start(); + } + + @After + public void tearDown() + { + if (manager != null) { + manager.stop(); + manager = null; + } + } + + @Test + public void testCreate() throws DuplicateKeyException + { + Map props = ImmutableMap.of( + DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", + DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY, "PT1M", + DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000 + ); + TableSpec spec = new TableSpec(DatasourceDefn.ROLLUP_DATASOURCE_TYPE, props, null); + TableMetadata table = TableMetadata.newTable(TableId.datasource("table1"), spec); + + // Table does not exist, read returns nothing. + assertNull(manager.read(table.id())); + + // Create the table + long version = manager.create(table); + TableMetadata created = table.fromInsert(version); + + // Read the record + TableMetadata read = manager.read(table.id()); + assertEquals(created, read); + + // Try to create a second time + assertThrows(DuplicateKeyException.class, () -> manager.create(table)); + } + + @Test + public void testUpdate() throws DuplicateKeyException, OutOfDateException, NotFoundException + { + Map props = ImmutableMap.of( + DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", + DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY, "PT1M", + DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000 + ); + TableSpec spec = new TableSpec(DatasourceDefn.ROLLUP_DATASOURCE_TYPE, props, null); + TableMetadata table = TableMetadata.newTable(TableId.datasource("table1"), spec); + long version = manager.create(table); + + // Change the definition + props = ImmutableMap.of( + DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", + DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY, "PT1H", + DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 2_000_000 + ); + TableSpec spec2 = spec.withProperties(props); + TableMetadata table2 = table.withSpec(spec2); + assertThrows(OutOfDateException.class, () -> manager.update(table2, 3)); + + assertEquals(version, manager.read(table.id()).updateTime()); + long newVersion = manager.update(table2, version); + TableMetadata table3 = manager.read(table.id()); + assertEquals(spec2, table3.spec()); + assertEquals(newVersion, table3.updateTime()); + + // Changing the state requires no version check + assertEquals(TableMetadata.TableState.ACTIVE, table3.state()); + newVersion = manager.markDeleting(table.id()); + TableMetadata table4 = manager.read(table.id()); + assertEquals(TableMetadata.TableState.DELETING, table4.state()); + assertEquals(newVersion, table4.updateTime()); + + // Update: no version check) + TableMetadata table5 = table.withSpec(spec2); + long newerVersion = manager.update(table5, 0); + assertTrue(newerVersion > newVersion); + } + + @Test + public void testDelete() throws DuplicateKeyException + { + Map props = ImmutableMap.of( + DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", + DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY, "PT1M", + DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000 + ); + TableSpec spec = new TableSpec(DatasourceDefn.ROLLUP_DATASOURCE_TYPE, props, null); + TableMetadata table = TableMetadata.newTable(TableId.datasource("table1"), spec); + + assertFalse(manager.delete(table.id())); + manager.create(table); + assertTrue(manager.delete(table.id())); + assertFalse(manager.delete(table.id())); + } + + @Test + public void testList() throws DuplicateKeyException + { + List list = manager.list(); + assertTrue(list.isEmpty()); + + Map props = ImmutableMap.of( + DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H", + DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY, "PT1M", + DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000 + ); + TableSpec spec = new TableSpec(DatasourceDefn.ROLLUP_DATASOURCE_TYPE, props, null); + + // Create tables in inverse order + TableMetadata table2 = TableMetadata.newTable(TableId.datasource("table2"), spec); + long version = manager.create(table2); + table2 = table2.fromInsert(version); + TableMetadata table1 = TableMetadata.newTable(TableId.datasource("table1"), spec); + version = manager.create(table1); + table1 = table1.fromInsert(version); + + list = manager.list(); + assertEquals(2, list.size()); + TableId id = list.get(0); + assertEquals(TableId.DRUID_SCHEMA, id.schema()); + assertEquals("table1", id.name()); + id = list.get(1); + assertEquals(TableId.DRUID_SCHEMA, id.schema()); + assertEquals("table2", id.name()); + + List names = manager.list(TableId.DRUID_SCHEMA); + assertEquals(2, names.size()); + + names = manager.list(TableId.SYSTEM_SCHEMA); + assertEquals(0, names.size()); + + List details = manager.listDetails(TableId.DRUID_SCHEMA); + assertEquals(Arrays.asList(table1, table2), details); + } +} diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CacheNotifierTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CacheNotifierTest.java new file mode 100644 index 000000000000..7922c951cf67 --- /dev/null +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CacheNotifierTest.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.sync; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.catalog.sync.RestUpdateSender.RestSender; +import org.apache.druid.java.util.http.client.response.StatusResponseHolder; +import org.apache.druid.server.DruidNode; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.junit.Test; + +import java.net.URL; +import java.util.Arrays; +import java.util.List; +import java.util.function.Consumer; +import java.util.function.Supplier; + +import static org.junit.Assert.assertEquals; + +public class CacheNotifierTest +{ + private static class MockSender implements Consumer + { + int sendCount; + + @Override + public void accept(byte[] update) + { + assertEquals(sendCount++, update[0]); + } + } + + @Test + public void testNotifier() + { + MockSender sender = new MockSender(); + CacheNotifier notifier = new CacheNotifier("test", sender); + notifier.start(); + for (int i = 0; i < 100; i++) { + byte[] msg = new byte[] {(byte) i}; + notifier.send(msg); + } + notifier.stopGracefully(); + assertEquals(100, sender.sendCount); + } + + private static class MockRestSender implements RestSender + { + int sendCount; + + @Override + public ListenableFuture send(URL listenerURL, byte[] serializedEntity) + { + sendCount++; + StatusResponseHolder holder = new StatusResponseHolder(HttpResponseStatus.ACCEPTED, new StringBuilder()); + return Futures.immediateFuture(holder); + } + } + + @Test + public void testRestUpdateSender() + { + DruidNode node1 = new DruidNode("service", "host1", true, 1000, 0, true, false); + DruidNode node2 = new DruidNode("service", "host2", true, 1000, 0, true, false); + List nodes = Arrays.asList(node1, node2); + Supplier> nodeSupplier = () -> nodes; + MockRestSender restSender = new MockRestSender(); + RestUpdateSender updateSender = new RestUpdateSender( + "test", + nodeSupplier, + restSender, + "/test/foo", + 1000); + for (int i = 0; i < 100; i++) { + byte[] msg = new byte[] {(byte) i}; + updateSender.accept(msg); + } + assertEquals(200, restSender.sendCount); + } + + @Test + public void testStack() + { + DruidNode node1 = new DruidNode("service", "host1", true, 1000, 0, true, false); + DruidNode node2 = new DruidNode("service", "host2", true, 1000, 0, true, false); + List nodes = Arrays.asList(node1, node2); + Supplier> nodeSupplier = () -> nodes; + MockRestSender restSender = new MockRestSender(); + RestUpdateSender updateSender = new RestUpdateSender( + "test", + nodeSupplier, + restSender, + "/test/foo", + 1000); + CacheNotifier notifier = new CacheNotifier("test", updateSender); + notifier.start(); + for (int i = 0; i < 100; i++) { + byte[] msg = new byte[] {(byte) i}; + notifier.send(msg); + } + notifier.stopGracefully(); + assertEquals(200, restSender.sendCount); + } +} diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java new file mode 100644 index 000000000000..899279c40b8b --- /dev/null +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java @@ -0,0 +1,346 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.sync; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.smile.SmileFactory; +import org.apache.druid.catalog.model.ColumnSpec; +import org.apache.druid.catalog.model.Columns; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.model.facade.DatasourceFacade; +import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.catalog.model.table.ExternalTableDefn; +import org.apache.druid.catalog.model.table.InlineTableDefn; +import org.apache.druid.catalog.model.table.InputFormats; +import org.apache.druid.catalog.model.table.TableBuilder; +import org.apache.druid.catalog.storage.CatalogStorage; +import org.apache.druid.catalog.storage.CatalogTests; +import org.apache.druid.catalog.storage.sql.CatalogManager.DuplicateKeyException; +import org.apache.druid.catalog.storage.sql.CatalogManager.NotFoundException; +import org.apache.druid.catalog.storage.sql.CatalogManager.OutOfDateException; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.metadata.TestDerbyConnector; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +public class CatalogMetadataTest +{ + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + + private CatalogTests.DbFixture dbFixture; + private CatalogStorage storage; + private ObjectMapper jsonMapper; + private ObjectMapper smileMapper; + + @Before + public void setUp() + { + dbFixture = new CatalogTests.DbFixture(derbyConnectorRule); + storage = dbFixture.storage; + jsonMapper = new ObjectMapper(); + smileMapper = new ObjectMapper(new SmileFactory()); + } + + @After + public void tearDown() + { + CatalogTests.tearDown(dbFixture); + } + + /** + * Checks validation via the storage API. Detailed error checks + * are done elsewhere: here we just ensure that they are, in fact, done. + */ + @Test + public void testInputValidation() + { + // Valid definition + { + TableMetadata table = TableBuilder.externalTable(InlineTableDefn.TABLE_TYPE, "externTable") + .format(InputFormats.CSV_FORMAT_TYPE) + .data("a", "c") + .column("a", Columns.VARCHAR) + .build(); + storage.validate(table); + } + + // No columns + { + TableMetadata table = TableBuilder.externalTable(InlineTableDefn.TABLE_TYPE, "externTable") + .format(InputFormats.CSV_FORMAT_TYPE) + .data("a", "c") + .build(); + assertThrows(IAE.class, () -> storage.validate(table)); + } + + // No format + { + TableMetadata table = TableBuilder.externalTable(InlineTableDefn.TABLE_TYPE, "externTable") + .data("a", "c") + .column("a", Columns.VARCHAR) + .build(); + assertThrows(IAE.class, () -> storage.validate(table)); + } + } + + @Test + public void testDirect() throws DuplicateKeyException, OutOfDateException, NotFoundException + { + populateCatalog(); + MetadataCatalog catalog = new LocalMetadataCatalog(storage, storage.schemaRegistry()); + verifyInitial(catalog); + alterCatalog(); + verifyAltered(catalog); + } + + @Test + public void testCached() throws DuplicateKeyException, OutOfDateException, NotFoundException + { + populateCatalog(); + CachedMetadataCatalog catalog = new CachedMetadataCatalog(storage, storage.schemaRegistry(), jsonMapper); + storage.register(catalog); + verifyInitial(catalog); + alterCatalog(); + verifyAltered(catalog); + + // Also test the deletion case + TableId table2 = TableId.datasource("table2"); + storage.tables().delete(table2); + assertNull(storage.tables().read(table2)); + + List tables = catalog.tables(TableId.DRUID_SCHEMA); + assertEquals(2, tables.size()); + assertEquals("table1", tables.get(0).id().name()); + assertEquals("table3", tables.get(1).id().name()); + } + + @Test + public void testRemoteWithJson() throws DuplicateKeyException, OutOfDateException, NotFoundException + { + doTestRemote(false); + } + + @Test + public void testRemoteWithSmile() throws DuplicateKeyException, OutOfDateException, NotFoundException + { + doTestRemote(true); + } + + private void doTestRemote(boolean useSmile) throws DuplicateKeyException, OutOfDateException, NotFoundException + { + populateCatalog(); + MockCatalogSync sync = new MockCatalogSync(storage, jsonMapper, smileMapper, useSmile); + MetadataCatalog catalog = sync.catalog(); + storage.register(sync); + verifyInitial(catalog); + alterCatalog(); + verifyAltered(catalog); + + // Also test the deletion case + TableId table2 = TableId.datasource("table2"); + storage.tables().delete(table2); + assertNull(storage.tables().read(table2)); + + List tables = catalog.tables(TableId.DRUID_SCHEMA); + assertEquals(2, tables.size()); + assertEquals("table1", tables.get(0).id().name()); + assertEquals("table3", tables.get(1).id().name()); + } + + /** + * Populate the catalog with a few items using the REST resource. + * @throws DuplicateKeyException + */ + private void populateCatalog() throws DuplicateKeyException + { + TableMetadata table1 = TableBuilder.detailTable("table1", "P1D") + .timeColumn() + .column("a", Columns.VARCHAR) + .build(); + storage.validate(table1); + storage.tables().create(table1); + + TableMetadata table2 = TableBuilder.rollupTable("table2", "P1D") + .rollupGranularity("PT1H") + .timeColumn() + .column("dim", Columns.VARCHAR) + .measure("measure", "SUM(BIGINT)") + .build(); + storage.validate(table2); + storage.tables().create(table2); + + TableMetadata table3 = TableBuilder.externalTable(InlineTableDefn.TABLE_TYPE, "table3") + .format(InputFormats.CSV_FORMAT_TYPE) + .data("a", "c") + .column("a", Columns.VARCHAR) + .build(); + storage.validate(table3); + storage.tables().create(table3); + } + + private void verifyInitial(MetadataCatalog catalog) + { + { + TableId id = TableId.datasource("table1"); + TableMetadata table = catalog.getTable(id); + assertEquals(id, table.id()); + assertTrue(table.updateTime() > 0); + + TableSpec dsSpec = table.spec(); + assertEquals(DatasourceDefn.DETAIL_DATASOURCE_TYPE, dsSpec.type()); + List cols = dsSpec.columns(); + assertEquals(2, cols.size()); + assertEquals(Columns.TIME_COLUMN, cols.get(0).name()); + assertEquals(Columns.TIMESTAMP, cols.get(0).sqlType()); + assertEquals(DatasourceDefn.DETAIL_COLUMN_TYPE, cols.get(0).type()); + assertEquals("a", cols.get(1).name()); + assertEquals(Columns.VARCHAR, cols.get(1).sqlType()); + assertEquals(DatasourceDefn.DETAIL_COLUMN_TYPE, cols.get(0).type()); + + DatasourceFacade ds = new DatasourceFacade(catalog.resolveTable(id)); + assertEquals("P1D", ds.segmentGranularityString()); + assertTrue(ds.isDetail()); + assertFalse(ds.isRollup()); + assertNull(ds.rollupGranularity()); + } + { + TableId id = TableId.datasource("table2"); + TableMetadata table = catalog.getTable(id); + assertEquals(id, table.id()); + assertTrue(table.updateTime() > 0); + + TableSpec dsSpec = table.spec(); + assertEquals(DatasourceDefn.ROLLUP_DATASOURCE_TYPE, table.spec().type()); + List cols = dsSpec.columns(); + assertEquals(3, cols.size()); + assertEquals("__time", cols.get(0).name()); + assertEquals(Columns.TIME_COLUMN, cols.get(0).name()); + assertEquals(Columns.TIMESTAMP, cols.get(0).sqlType()); + assertEquals("dim", cols.get(1).name()); + assertEquals(Columns.VARCHAR, cols.get(1).sqlType()); + assertEquals(DatasourceDefn.DIMENSION_TYPE, cols.get(1).type()); + assertEquals("measure", cols.get(2).name()); + assertEquals("SUM(BIGINT)", cols.get(2).sqlType()); + assertEquals(DatasourceDefn.MEASURE_TYPE, cols.get(2).type()); + + DatasourceFacade ds = new DatasourceFacade(catalog.resolveTable(id)); + assertEquals("P1D", ds.segmentGranularityString()); + assertFalse(ds.isDetail()); + assertTrue(ds.isRollup()); + assertEquals("P1D", ds.segmentGranularityString()); + } + + assertNull(catalog.getTable(TableId.datasource("table3"))); + assertNull(catalog.resolveTable(TableId.datasource("table3"))); + + { + TableId id = TableId.external("table3"); + TableMetadata table = catalog.getTable(id); + assertEquals(id, table.id()); + assertTrue(table.updateTime() > 0); + + TableSpec inputSpec = table.spec(); + assertEquals(InlineTableDefn.TABLE_TYPE, inputSpec.type()); + List cols = inputSpec.columns(); + assertEquals(1, cols.size()); + assertEquals("a", cols.get(0).name()); + assertEquals(Columns.VARCHAR, cols.get(0).sqlType()); + assertEquals(ExternalTableDefn.EXTERNAL_COLUMN_TYPE, cols.get(0).type()); + + assertNotNull(inputSpec.properties()); + } + + List tables = catalog.tables(TableId.DRUID_SCHEMA); + assertEquals(2, tables.size()); + assertEquals("table1", tables.get(0).id().name()); + assertEquals("table2", tables.get(1).id().name()); + + tables = catalog.tables(TableId.EXTERNAL_SCHEMA); + assertEquals(1, tables.size()); + assertEquals("table3", tables.get(0).id().name()); + } + + private void alterCatalog() throws DuplicateKeyException, OutOfDateException, NotFoundException + { + // Add a column to table 1 + TableId id1 = TableId.datasource("table1"); + TableMetadata table1 = storage.tables().read(id1); + assertNotNull(table1); + + TableSpec update1 = TableBuilder.copyOf(table1) + .column("b", Columns.DOUBLE) + .buildSpec(); + storage.tables().update(table1.withSpec(update1), table1.updateTime()); + + // Create a table 3 + TableMetadata table3 = TableBuilder.detailTable("table3", "P1D") + .timeColumn() + .column("x", "FLOAT") + .build(); + storage.tables().create(table3); + } + + private void verifyAltered(MetadataCatalog catalog) + { + { + TableId id = TableId.datasource("table1"); + TableMetadata table = catalog.getTable(id); + + TableSpec dsSpec = table.spec(); + List cols = dsSpec.columns(); + assertEquals(3, cols.size()); + assertEquals(Columns.TIME_COLUMN, cols.get(0).name()); + assertEquals("a", cols.get(1).name()); + assertEquals("b", cols.get(2).name()); + assertEquals(Columns.DOUBLE, cols.get(2).sqlType()); + assertEquals(DatasourceDefn.DETAIL_COLUMN_TYPE, cols.get(2).type()); + } + { + TableId id = TableId.datasource("table3"); + TableMetadata table = catalog.getTable(id); + + TableSpec dsSpec = table.spec(); + List cols = dsSpec.columns(); + assertEquals(2, cols.size()); + assertEquals(Columns.TIME_COLUMN, cols.get(0).name()); + assertEquals("x", cols.get(1).name()); + } + + List tables = catalog.tables(TableId.DRUID_SCHEMA); + assertEquals(3, tables.size()); + assertEquals("table1", tables.get(0).id().name()); + assertEquals("table2", tables.get(1).id().name()); + assertEquals("table3", tables.get(2).id().name()); + } +} diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/MockCatalogSync.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/MockCatalogSync.java new file mode 100644 index 000000000000..45464085fd55 --- /dev/null +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/MockCatalogSync.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.sync; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import org.apache.druid.catalog.http.CatalogListenerResource; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.storage.CatalogStorage; +import org.apache.druid.catalog.storage.CatalogTests; +import org.apache.druid.catalog.sync.MetadataCatalog.CatalogListener; +import org.apache.druid.server.http.catalog.DummyRequest; + +import javax.ws.rs.core.MediaType; + +import java.io.ByteArrayInputStream; + +/** + * Simulates a network sync from catalog (Coordinator) to consumer (Broker). + */ +public class MockCatalogSync implements CatalogListener +{ + private final CatalogListenerResource listenerResource; + private final CachedMetadataCatalog catalog; + private final boolean useSmile; + private final ObjectMapper smileMapper; + private final ObjectMapper jsonMapper; + + public MockCatalogSync( + CatalogStorage storage, + final ObjectMapper smileMapper, + final ObjectMapper jsonMapper, + boolean useSmile + ) + { + this.catalog = new CachedMetadataCatalog(storage, storage.schemaRegistry(), jsonMapper); + this.listenerResource = new CatalogListenerResource( + catalog, + smileMapper, + jsonMapper, + storage.authorizer().mapper() + ); + this.useSmile = useSmile; + this.smileMapper = smileMapper; + this.jsonMapper = jsonMapper; + } + + @Override + public void updated(TableMetadata update) + { + doSync(update); + } + + private void doSync(TableMetadata update) + { + byte[] encoded = update.toBytes(useSmile ? smileMapper : jsonMapper); + listenerResource.syncTable( + new ByteArrayInputStream(encoded), + new DummyRequest( + DummyRequest.POST, + CatalogTests.SUPER_USER, + useSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON + ) + ); + } + + @Override + public void deleted(TableId tableId) + { + TableMetadata spec = TableMetadata.newTable( + tableId, + new TableSpec(CatalogUpdateNotifier.TOMBSTONE_TABLE_TYPE, null, null) + ); + doSync(spec); + } + + public MetadataCatalog catalog() + { + return catalog; + } +} diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java new file mode 100644 index 000000000000..9d61b911bec6 --- /dev/null +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java @@ -0,0 +1,619 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.http.catalog; + +import org.apache.druid.catalog.http.CatalogResource; +import org.apache.druid.catalog.model.CatalogUtils; +import org.apache.druid.catalog.model.Columns; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.catalog.model.table.InlineTableDefn; +import org.apache.druid.catalog.model.table.InputFormats; +import org.apache.druid.catalog.model.table.TableBuilder; +import org.apache.druid.catalog.storage.CatalogTests; +import org.apache.druid.catalog.storage.HideColumns; +import org.apache.druid.catalog.storage.MoveColumn; +import org.apache.druid.metadata.TestDerbyConnector; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import javax.ws.rs.core.Response; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.apache.druid.server.http.catalog.DummyRequest.deleteBy; +import static org.apache.druid.server.http.catalog.DummyRequest.getBy; +import static org.apache.druid.server.http.catalog.DummyRequest.postBy; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +/** + * Test of REST API operations for the table catalog. + */ +public class CatalogResourceTest +{ + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + + private CatalogTests.DbFixture dbFixture; + private CatalogResource resource; + + @Before + public void setUp() + { + dbFixture = new CatalogTests.DbFixture(derbyConnectorRule); + resource = new CatalogResource(dbFixture.storage); + } + + @After + public void tearDown() + { + CatalogTests.tearDown(dbFixture); + } + + private static long getVersion(Response resp) + { + @SuppressWarnings("unchecked") + Map result = (Map) resp.getEntity(); + return (Long) result.get("version"); + } + + @Test + public void testCreate() + { + final String tableName = "create"; + TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D").buildSpec(); + + // Blank schema name: infer the schema. + Response resp = resource.postTable("", tableName, dsSpec, null, 0, postBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Missing table name + resp = resource.postTable(TableId.DRUID_SCHEMA, "", dsSpec, null, 0, postBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Invalid table name + resp = resource.postTable(TableId.DRUID_SCHEMA, " bogus ", dsSpec, null, 0, postBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Unknown schema + resp = resource.postTable("bogus", tableName, dsSpec, null, 0, postBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // Immutable schema + resp = resource.postTable(TableId.CATALOG_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Wrong definition type. + resp = resource.postTable(TableId.EXTERNAL_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // No permissions + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.DENY_USER)); + assertEquals(Response.Status.FORBIDDEN.getStatusCode(), resp.getStatus()); + + // Read permission + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.FORBIDDEN.getStatusCode(), resp.getStatus()); + + // Write permission + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertTrue(getVersion(resp) > 0); + + // Duplicate + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Duplicate, "if not exists" + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "ifnew", 0, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertEquals(0, getVersion(resp)); + + // Inline input source + TableSpec inputSpec = TableBuilder.externalTable(InlineTableDefn.TABLE_TYPE, "inline") + .format(InputFormats.CSV_FORMAT_TYPE) + .data("a,b,1", "c,d,2") + .column("a", Columns.VARCHAR) + .column("b", Columns.VARCHAR) + .column("c", Columns.BIGINT) + .buildSpec(); + resp = resource.postTable(TableId.EXTERNAL_SCHEMA, "inline", inputSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + // Wrong spec type + resp = resource.postTable(TableId.DRUID_SCHEMA, "invalid", inputSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + } + + @Test + public void testUpdate() + { + final String tableName = "update"; + TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D").buildSpec(); + + // Does not exist + Response resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "replace", 0, postBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // Create the table + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + long version = getVersion(resp); + + // No update permission + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "replace", 0, postBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.FORBIDDEN.getStatusCode(), resp.getStatus()); + + // Out-of-date version + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "replace", 10, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Valid version + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "replace", version, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertTrue(getVersion(resp) > version); + version = getVersion(resp); + + // Overwrite + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "replace", 0, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertTrue(getVersion(resp) > version); + } + + @Test + public void testForce() + { + final String tableName = "force"; + TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D").buildSpec(); + + // Create the table + Response resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "force", 0, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + long version = getVersion(resp); + + // Overwrite + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "force", 0, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertTrue(getVersion(resp) > version); + } + + @Test + public void testRead() + { + final String tableName = "read"; + TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D").buildSpec(); + + // Missing schema name + Response resp = resource.getTable("", tableName, getBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Missing table name + resp = resource.getTable(TableId.DRUID_SCHEMA, null, getBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Unknown schema + resp = resource.getTable("bogus", tableName, getBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // Does not exist + resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, getBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // Create the table + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + long version = getVersion(resp); + + // No read permission + resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, getBy(CatalogTests.DENY_USER)); + assertEquals(Response.Status.FORBIDDEN.getStatusCode(), resp.getStatus()); + + // Valid + resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + TableMetadata read = (TableMetadata) resp.getEntity(); + assertEquals(version, read.updateTime()); + assertEquals(dsSpec, read.spec()); + + // Internal sync API + resp = resource.syncTable(TableId.DRUID_SCHEMA, tableName, getBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + read = (TableMetadata) resp.getEntity(); + assertEquals(version, read.updateTime()); + assertEquals(dsSpec, read.spec()); + } + + @SuppressWarnings("unchecked") + private List getTableIdList(Response resp) + { + return (List) resp.getEntity(); + } + + @SuppressWarnings("unchecked") + private List getTableList(Response resp) + { + return (List) resp.getEntity(); + } + + @SuppressWarnings("unchecked") + private List getDetailsList(Response resp) + { + return (List) resp.getEntity(); + } + + @Test + public void testList() + { + // No entries + Response resp = resource.listTables(getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + List tableIds = getTableIdList(resp); + assertTrue(tableIds.isEmpty()); + + resp = resource.listTables(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + List tables = getTableList(resp); + assertTrue(tables.isEmpty()); + + // Missing schema + resp = resource.listTables(null, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Invalid schema + resp = resource.listTables("bogus", getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // Create a table + final String tableName = "list"; + TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D").buildSpec(); + resp = resource.postTable(TableId.DRUID_SCHEMA, "list", dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + // No read access + resp = resource.listTables(getBy(CatalogTests.DENY_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + tableIds = getTableIdList(resp); + assertTrue(tableIds.isEmpty()); + + resp = resource.listTables(TableId.DRUID_SCHEMA, getBy(CatalogTests.DENY_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + tables = getTableList(resp); + assertTrue(tables.isEmpty()); + + // Read access + resp = resource.listTables(getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + tableIds = getTableIdList(resp); + assertEquals(1, tableIds.size()); + + resp = resource.listTables(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + tables = getTableList(resp); + assertEquals(1, tables.size()); + + resp = resource.listTables(TableId.SYSTEM_SCHEMA, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + tables = getTableList(resp); + assertTrue(tables.isEmpty()); + + // Internal sync schema API + resp = resource.syncSchema(TableId.SYSTEM_SCHEMA, getBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertTrue(getDetailsList(resp).isEmpty()); + + resp = resource.syncSchema(TableId.DRUID_SCHEMA, getBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + List details = getDetailsList(resp); + assertEquals(1, details.size()); + } + + @Test + public void testDelete() + { + // Missing schema name + String tableName = "delete"; + Response resp = resource.deleteTable("", tableName, false, deleteBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Missing table name + resp = resource.deleteTable(TableId.DRUID_SCHEMA, null, false, deleteBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Unknown schema + resp = resource.deleteTable("bogus", tableName, false, deleteBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // Immutable schema + resp = resource.deleteTable(TableId.CATALOG_SCHEMA, tableName, false, deleteBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Does not exist + resp = resource.deleteTable(TableId.DRUID_SCHEMA, tableName, false, deleteBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + resp = resource.deleteTable(TableId.DRUID_SCHEMA, tableName, true, deleteBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + // Create the table + TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D").buildSpec(); + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + // No write permission + resp = resource.deleteTable(TableId.DRUID_SCHEMA, tableName, false, deleteBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.FORBIDDEN.getStatusCode(), resp.getStatus()); + + // Write permission + resp = resource.deleteTable(TableId.DRUID_SCHEMA, tableName, false, deleteBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + resp = resource.deleteTable(TableId.DRUID_SCHEMA, tableName, false, deleteBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + resp = resource.deleteTable(TableId.DRUID_SCHEMA, tableName, true, deleteBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + } + + @Test + public void testLifecycle() + { + // Operations for one table - create + String table1Name = "lifecycle1"; + TableSpec dsSpec = TableBuilder.detailTable(table1Name, "P1D").buildSpec(); + Response resp = resource.postTable(TableId.DRUID_SCHEMA, table1Name, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + long version = getVersion(resp); + + // read + resp = resource.getTable(TableId.DRUID_SCHEMA, table1Name, postBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + TableMetadata read1 = (TableMetadata) resp.getEntity(); + TableId id1 = TableId.of(TableId.DRUID_SCHEMA, table1Name); + assertEquals(id1, read1.id()); + assertEquals(version, read1.updateTime()); + assertEquals(dsSpec, read1.spec()); + + // list + resp = resource.listTables(getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + List tableIds = getTableIdList(resp); + assertEquals(1, tableIds.size()); + assertEquals(id1, tableIds.get(0)); + + resp = resource.listTables(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + List tables = getTableList(resp); + assertEquals(1, tables.size()); + assertEquals(id1.name(), tables.get(0)); + + // update + TableSpec table2Spec = TableBuilder.detailTable(table1Name, "PT1H").buildSpec(); + resp = resource.postTable(TableId.DRUID_SCHEMA, table1Name, table2Spec, "replace", version, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertTrue(getVersion(resp) > version); + version = getVersion(resp); + + // verify update + resp = resource.getTable(TableId.DRUID_SCHEMA, table1Name, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + TableMetadata read = (TableMetadata) resp.getEntity(); + assertEquals(read1.creationTime(), read.creationTime()); + assertEquals(version, read.updateTime()); + assertEquals(table2Spec, read.spec()); + + // add second table + String table2Name = "lifecycle2"; + resp = resource.postTable(TableId.DRUID_SCHEMA, table2Name, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + TableId id2 = TableId.of(TableId.DRUID_SCHEMA, table2Name); + + // verify lists + resp = resource.listTables(getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + tableIds = getTableIdList(resp); + assertEquals(2, tableIds.size()); + assertEquals(id1, tableIds.get(0)); + assertEquals(id2, tableIds.get(1)); + + resp = resource.listTables(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + tables = getTableList(resp); + assertEquals(2, tables.size()); + assertEquals(id1.name(), tables.get(0)); + assertEquals(id2.name(), tables.get(1)); + + // delete and verify + resp = resource.deleteTable(TableId.DRUID_SCHEMA, table1Name, false, deleteBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + resp = resource.listTables(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + tables = getTableList(resp); + assertEquals(1, tables.size()); + + resp = resource.deleteTable(TableId.DRUID_SCHEMA, table2Name, false, deleteBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + resp = resource.listTables(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + tables = getTableList(resp); + assertEquals(0, tables.size()); + } + + @Test + public void testMoveColumn() + { + String tableName = "move"; + TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D") + .column("a", "VARCHAR") + .column("b", "BIGINT") + .column("c", "FLOAT") + .buildSpec(); + Response resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + long version = getVersion(resp); + + // Bad schema + MoveColumn cmd = new MoveColumn("foo", MoveColumn.Position.FIRST, null); + resp = resource.moveColumn("bogus", tableName, cmd, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // Bad table + resp = resource.moveColumn(TableId.DRUID_SCHEMA, "bogus", cmd, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // No target column + cmd = new MoveColumn(null, MoveColumn.Position.FIRST, null); + resp = resource.moveColumn(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // No anchor column + cmd = new MoveColumn("a", MoveColumn.Position.BEFORE, null); + resp = resource.moveColumn(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Move first + cmd = new MoveColumn("c", MoveColumn.Position.FIRST, null); + resp = resource.moveColumn(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertTrue(getVersion(resp) > version); + + resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, postBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + TableMetadata read = (TableMetadata) resp.getEntity(); + assertEquals( + Arrays.asList("c", "a", "b"), + CatalogUtils.columnNames(read.spec().columns()) + ); + + // Other cases are tested in CommandTest since all the REST plumbing is the same + } + + @Test + public void testHideColumns() + { + String tableName = "hide"; + TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D") + .buildSpec(); + Response resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + long version = getVersion(resp); + + // Bad schema + HideColumns cmd = new HideColumns(null, null); + resp = resource.hideColumns("bogus", tableName, cmd, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // Bad table + resp = resource.hideColumns(TableId.DRUID_SCHEMA, "bogus", cmd, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // Nothing to do + resp = resource.hideColumns(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, postBy(CatalogTests.READER_USER)); + TableMetadata read = (TableMetadata) resp.getEntity(); + assertNull(read.spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY)); + + // Hide + cmd = new HideColumns(Arrays.asList("a", "b"), null); + resp = resource.hideColumns(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, postBy(CatalogTests.READER_USER)); + read = (TableMetadata) resp.getEntity(); + assertEquals( + Arrays.asList("a", "b"), + read.spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + assertTrue(read.updateTime() > version); + + // Unhide + hide + cmd = new HideColumns(Arrays.asList("b", "c"), Arrays.asList("a", "e")); + resp = resource.hideColumns(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, postBy(CatalogTests.READER_USER)); + read = (TableMetadata) resp.getEntity(); + assertEquals( + Arrays.asList("b", "c"), + read.spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + assertTrue(read.updateTime() > version); + + // Other cases are tested in CommandTest + } + + @Test + public void testDropColumns() + { + String tableName = "drop"; + TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D") + .column("a", "VARCHAR") + .column("b", "BIGINT") + .column("c", "FLOAT") + .buildSpec(); + + Response resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + long version = getVersion(resp); + + // Bad schema + resp = resource.dropColumns("bogus", tableName, Collections.emptyList(), postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // Bad table + resp = resource.dropColumns(TableId.DRUID_SCHEMA, "bogus", Collections.emptyList(), postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // Nothing to do + resp = resource.dropColumns(TableId.DRUID_SCHEMA, tableName, Collections.emptyList(), postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, postBy(CatalogTests.READER_USER)); + TableMetadata read = (TableMetadata) resp.getEntity(); + assertEquals( + CatalogUtils.columnNames(dsSpec.columns()), + CatalogUtils.columnNames(read.spec().columns()) + ); + + // Drop + resp = resource.dropColumns(TableId.DRUID_SCHEMA, tableName, Arrays.asList("a", "c"), postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, postBy(CatalogTests.READER_USER)); + read = (TableMetadata) resp.getEntity(); + assertTrue(read.updateTime() > version); + assertEquals( + Collections.singletonList("b"), + CatalogUtils.columnNames(read.spec().columns()) + ); + } +} diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java new file mode 100644 index 000000000000..8640515dc60f --- /dev/null +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.http.catalog; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.catalog.model.CatalogUtils; +import org.apache.druid.catalog.model.ColumnSpec; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.model.table.TableBuilder; +import org.apache.druid.catalog.storage.HideColumns; +import org.apache.druid.catalog.storage.MoveColumn; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +public class CommandTest +{ + private ObjectMapper mapper = new ObjectMapper(); + + @Test + public void testMoveColumn() + { + TableSpec dsSpec = TableBuilder.detailTable("foo", "P1D") + .column("a", "VARCHAR") + .column("b", "BIGINT") + .column("c", "FLOAT") + .buildSpec(); + + // Move first + MoveColumn cmd = new MoveColumn("c", MoveColumn.Position.FIRST, null); + List revised = cmd.perform(dsSpec.columns()); + assertEquals( + Arrays.asList("c", "a", "b"), + CatalogUtils.columnNames(revised) + ); + + // Move last + cmd = new MoveColumn("a", MoveColumn.Position.LAST, null); + revised = cmd.perform(dsSpec.columns()); + assertEquals( + Arrays.asList("b", "c", "a"), + CatalogUtils.columnNames(revised) + ); + + // Move before, earlier anchor + cmd = new MoveColumn("c", MoveColumn.Position.BEFORE, "b"); + revised = cmd.perform(dsSpec.columns()); + assertEquals( + Arrays.asList("a", "c", "b"), + CatalogUtils.columnNames(revised) + ); + + // Move before, later anchor + cmd = new MoveColumn("a", MoveColumn.Position.BEFORE, "c"); + revised = cmd.perform(dsSpec.columns()); + assertEquals( + Arrays.asList("b", "a", "c"), + CatalogUtils.columnNames(revised) + ); + + // Move after, earlier anchor + cmd = new MoveColumn("c", MoveColumn.Position.AFTER, "a"); + revised = cmd.perform(dsSpec.columns()); + assertEquals( + Arrays.asList("a", "c", "b"), + CatalogUtils.columnNames(revised) + ); + + // Move after, later anchor + cmd = new MoveColumn("a", MoveColumn.Position.AFTER, "b"); + revised = cmd.perform(dsSpec.columns()); + assertEquals( + Arrays.asList("b", "a", "c"), + CatalogUtils.columnNames(revised) + ); + + byte[] bytes = CatalogUtils.toBytes(mapper, cmd); + MoveColumn cmd2 = CatalogUtils.fromBytes(mapper, bytes, MoveColumn.class); + assertEquals(cmd, cmd2); + } + + @Test + public void testHideColumns() + { + // Everything is null + HideColumns cmd = new HideColumns(null, null); + List revised = cmd.perform(null); + assertNull(revised); + + // Unhide from null list + cmd = new HideColumns(null, Collections.singletonList("a")); + revised = cmd.perform(null); + assertNull(revised); + + // And from an empty list + cmd = new HideColumns(null, Collections.singletonList("a")); + revised = cmd.perform(Collections.emptyList()); + assertNull(revised); + + // Hide starting from a null list. + cmd = new HideColumns(Arrays.asList("a", "b"), null); + revised = cmd.perform(null); + assertEquals(Arrays.asList("a", "b"), revised); + + // Hide starting from an empty list. + cmd = new HideColumns(Arrays.asList("a", "b"), Collections.emptyList()); + revised = cmd.perform(Collections.emptyList()); + assertEquals(Arrays.asList("a", "b"), revised); + + // Hide with existing columns + cmd = new HideColumns(Arrays.asList("b", "d"), null); + revised = cmd.perform(Arrays.asList("a", "b", "c")); + assertEquals(Arrays.asList("a", "b", "c", "d"), revised); + + // Hide with existing columns + cmd = new HideColumns(Arrays.asList("b", "d"), null); + revised = cmd.perform(Arrays.asList("a", "b", "c")); + assertEquals(Arrays.asList("a", "b", "c", "d"), revised); + + // Unhide existing columns + cmd = new HideColumns(null, Arrays.asList("b", "d")); + revised = cmd.perform(Arrays.asList("a", "b", "c")); + assertEquals(Arrays.asList("a", "c"), revised); + + // Both hide and unhide. Hide takes precedence. + cmd = new HideColumns(Arrays.asList("b", "d", "e"), Arrays.asList("c", "d")); + revised = cmd.perform(Arrays.asList("a", "b", "c")); + assertEquals(Arrays.asList("a", "b", "d", "e"), revised); + + byte[] bytes = CatalogUtils.toBytes(mapper, cmd); + HideColumns cmd2 = CatalogUtils.fromBytes(mapper, bytes, HideColumns.class); + assertEquals(cmd, cmd2); + } +} diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/DummyRequest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/DummyRequest.java new file mode 100644 index 000000000000..8316392c0eb6 --- /dev/null +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/DummyRequest.java @@ -0,0 +1,500 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.http.catalog; + +import org.apache.druid.catalog.storage.CatalogTests; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthenticationResult; + +import javax.servlet.AsyncContext; +import javax.servlet.DispatcherType; +import javax.servlet.RequestDispatcher; +import javax.servlet.ServletContext; +import javax.servlet.ServletInputStream; +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; +import javax.servlet.http.Cookie; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import javax.servlet.http.HttpSession; +import javax.servlet.http.HttpUpgradeHandler; +import javax.servlet.http.Part; + +import java.io.BufferedReader; +import java.security.Principal; +import java.util.Collection; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.Locale; +import java.util.Map; + +/** + * Test-only implementation of an HTTP request. Allows us to control + * aspects of the request without resorting to mocks. + */ +public class DummyRequest implements HttpServletRequest +{ + public static final String GET = "GET"; + public static final String POST = "POST"; + public static final String DELETE = "DELETE"; + + private final String method; + private final Map attribs = new HashMap<>(); + private final String contentType; + + public DummyRequest(String method, String userName) + { + this(method, userName, null); + } + + public DummyRequest(String method, String userName, String contentType) + { + this.method = method; + AuthenticationResult authResult = + new AuthenticationResult(userName, CatalogTests.TEST_AUTHORITY, null, null); + attribs.put(AuthConfig.DRUID_AUTHENTICATION_RESULT, authResult); + this.contentType = contentType; + } + + public static HttpServletRequest postBy(String user) + { + return new DummyRequest(DummyRequest.POST, user); + } + + public static HttpServletRequest getBy(String user) + { + return new DummyRequest(DummyRequest.GET, user); + } + + public static HttpServletRequest deleteBy(String user) + { + return new DummyRequest(DummyRequest.DELETE, user); + } + + @Override + public Object getAttribute(String name) + { + return attribs.get(name); + } + + @Override + public Enumeration getAttributeNames() + { + return null; + } + + @Override + public String getCharacterEncoding() + { + return null; + } + + @Override + public void setCharacterEncoding(String env) + { + } + + @Override + public int getContentLength() + { + return 0; + } + + @Override + public long getContentLengthLong() + { + return 0; + } + + @Override + public String getContentType() + { + return contentType; + } + + @Override + public ServletInputStream getInputStream() + { + return null; + } + + @Override + public String getParameter(String name) + { + return null; + } + + @Override + public Enumeration getParameterNames() + { + return null; + } + + @Override + public String[] getParameterValues(String name) + { + return null; + } + + @Override + public Map getParameterMap() + { + return null; + } + + @Override + public String getProtocol() + { + return null; + } + + @Override + public String getScheme() + { + return null; + } + + @Override + public String getServerName() + { + return null; + } + + @Override + public int getServerPort() + { + return 0; + } + + @Override + public BufferedReader getReader() + { + return null; + } + + @Override + public String getRemoteAddr() + { + return null; + } + + @Override + public String getRemoteHost() + { + return null; + } + + @Override + public void setAttribute(String name, Object o) + { + attribs.put(name, o); + } + + @Override + public void removeAttribute(String name) + { + } + + @Override + public Locale getLocale() + { + return null; + } + + @Override + public Enumeration getLocales() + { + return null; + } + + @Override + public boolean isSecure() + { + return false; + } + + @Override + public RequestDispatcher getRequestDispatcher(String path) + { + return null; + } + + @Override + public String getRealPath(String path) + { + return null; + } + + @Override + public int getRemotePort() + { + return 0; + } + + @Override + public String getLocalName() + { + return null; + } + + @Override + public String getLocalAddr() + { + return null; + } + + @Override + public int getLocalPort() + { + return 0; + } + + @Override + public ServletContext getServletContext() + { + return null; + } + + @Override + public AsyncContext startAsync() + { + return null; + } + + @Override + public AsyncContext startAsync(ServletRequest servletRequest, ServletResponse servletResponse) + { + return null; + } + + @Override + public boolean isAsyncStarted() + { + return false; + } + + @Override + public boolean isAsyncSupported() + { + return false; + } + + @Override + public AsyncContext getAsyncContext() + { + return null; + } + + @Override + public DispatcherType getDispatcherType() + { + return null; + } + + @Override + public String getAuthType() + { + return null; + } + + @Override + public Cookie[] getCookies() + { + return null; + } + + @Override + public long getDateHeader(String name) + { + return 0; + } + + @Override + public String getHeader(String name) + { + return null; + } + + @Override + public Enumeration getHeaders(String name) + { + return null; + } + + @Override + public Enumeration getHeaderNames() + { + return null; + } + + @Override + public int getIntHeader(String name) + { + return 0; + } + + @Override + public String getMethod() + { + return method; + } + + @Override + public String getPathInfo() + { + return null; + } + + @Override + public String getPathTranslated() + { + return null; + } + + @Override + public String getContextPath() + { + return null; + } + + @Override + public String getQueryString() + { + return null; + } + + @Override + public String getRemoteUser() + { + return null; + } + + @Override + public boolean isUserInRole(String role) + { + return false; + } + + @Override + public Principal getUserPrincipal() + { + return null; + } + + @Override + public String getRequestedSessionId() + { + return null; + } + + @Override + public String getRequestURI() + { + return null; + } + + @Override + public StringBuffer getRequestURL() + { + return null; + } + + @Override + public String getServletPath() + { + return null; + } + + @Override + public HttpSession getSession(boolean create) + { + return null; + } + + @Override + public HttpSession getSession() + { + return null; + } + + @Override + public String changeSessionId() + { + return null; + } + + @Override + public boolean isRequestedSessionIdValid() + { + return false; + } + + @Override + public boolean isRequestedSessionIdFromCookie() + { + return false; + } + + @Override + public boolean isRequestedSessionIdFromURL() + { + return false; + } + + @Override + public boolean isRequestedSessionIdFromUrl() + { + return false; + } + + @Override + public boolean authenticate(HttpServletResponse response) + { + return false; + } + + @Override + public void login(String username, String password) + { + } + + @Override + public void logout() + { + } + + @Override + public Collection getParts() + { + return null; + } + + @Override + public Part getPart(String name) + { + return null; + } + + @Override + public T upgrade(Class handlerClass) + { + return null; + } +} diff --git a/integration-tests-ex/cases/cluster.sh b/integration-tests-ex/cases/cluster.sh index 0b19b478fcb8..fde648b0c5ff 100755 --- a/integration-tests-ex/cases/cluster.sh +++ b/integration-tests-ex/cases/cluster.sh @@ -82,6 +82,9 @@ function category { "InputFormat") export DRUID_INTEGRATION_TEST_GROUP=BatchIndex ;; + "Catalog") + export DRUID_INTEGRATION_TEST_GROUP=BatchIndex + ;; *) export DRUID_INTEGRATION_TEST_GROUP=$CATEGORY ;; @@ -178,7 +181,7 @@ case $CMD in cd $CLUSTER_DIR docker-compose up -d # Enable the following for debugging - show_status + #show_status ;; "status" ) category $* @@ -188,7 +191,7 @@ case $CMD in "down" ) category $* # Enable the following for debugging - show_status + #show_status cd $CLUSTER_DIR echo OVERRIDE_ENV="$ENV_FILE" docker-compose $CMD OVERRIDE_ENV="$ENV_FILE" docker-compose $CMD diff --git a/integration-tests-ex/cases/cluster/Common/environment-configs/common.env b/integration-tests-ex/cases/cluster/Common/environment-configs/common.env index 535412832620..8b58cd949e31 100644 --- a/integration-tests-ex/cases/cluster/Common/environment-configs/common.env +++ b/integration-tests-ex/cases/cluster/Common/environment-configs/common.env @@ -49,7 +49,7 @@ DRUID_INSTANCE= # variables: druid_standard_loadList defined here, and druid_test_loadList, defined # in a docker-compose.yaml file, for any test-specific extensions. # See compose.md for more details. -druid_standard_loadList=mysql-metadata-storage,druid-it-tools,druid-lookups-cached-global,druid-histogram,druid-datasketches,druid-parquet-extensions,druid-avro-extensions,druid-protobuf-extensions,druid-orc-extensions,druid-kafka-indexing-service,druid-s3-extensions,druid-multi-stage-query +druid_standard_loadList=mysql-metadata-storage,druid-it-tools,druid-lookups-cached-global,druid-histogram,druid-datasketches,druid-parquet-extensions,druid-avro-extensions,druid-protobuf-extensions,druid-orc-extensions,druid-kafka-indexing-service,druid-s3-extensions,druid-multi-stage-query,druid-catalog # Location of Hadoop dependencies provided at runtime in the shared directory. druid_extensions_hadoopDependenciesDir=/shared/hadoop-dependencies diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml index ae2c80d0d3e5..540b662c53f7 100644 --- a/integration-tests-ex/cases/pom.xml +++ b/integration-tests-ex/cases/pom.xml @@ -122,22 +122,18 @@ com.github.docker-java docker-java-core - 3.2.0 com.github.docker-java docker-java-api - 3.2.0 com.github.docker-java docker-java-transport-netty - 3.2.11 com.github.docker-java docker-java - 3.2.0 com.github.docker-java @@ -186,6 +182,13 @@ org.apache.druid.extensions druid-multi-stage-query ${project.parent.version} + provided + + + org.apache.druid.extensions + druid-catalog + ${project.parent.version} + provided org.apache.commons diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java new file mode 100644 index 000000000000..298ea62acae0 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.catalog; + +import com.google.inject.Inject; +import org.apache.druid.catalog.model.CatalogUtils; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.catalog.model.table.TableBuilder; +import org.apache.druid.catalog.storage.HideColumns; +import org.apache.druid.catalog.storage.MoveColumn; +import org.apache.druid.testsEx.categories.Catalog; +import org.apache.druid.testsEx.cluster.CatalogClient; +import org.apache.druid.testsEx.cluster.DruidClusterClient; +import org.apache.druid.testsEx.config.DruidTestRunner; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +/** + * Light sanity check of the Catalog REST API. Functional testing is + * done via a unit test. Here we simply ensure that the Jersey plumbing + * works as intended. + */ +@RunWith(DruidTestRunner.class) +@Category(Catalog.class) +public class ITCatalogRestTest +{ + @Inject + private DruidClusterClient clusterClient; + + /** + * Sample a few error cases to ensure the plumbing works. + * Complete error testing appears in unit tests. + */ + @Test + public void testErrors() + { + CatalogClient client = new CatalogClient(clusterClient); + + // Bogus schema + { + final TableMetadata table = new TableBuilder() + .id(TableId.of("bogus", "foo")) + .build(); + + assertThrows( + Exception.class, + () -> client.createTable(table) + ); + } + + // Read-only schema + { + final TableMetadata table = new TableBuilder() + .id(TableId.of(TableId.SYSTEM_SCHEMA, "foo")) + .property(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D") + .build(); + assertThrows( + Exception.class, + () -> client.createTable(table) + ); + } + + // Malformed table name + { + final TableMetadata table = TableBuilder.detailTable(" foo ", "P1D") + .build(); + assertThrows( + Exception.class, + () -> client.createTable(table) + ); + } + } + + /** + * Run though a table lifecycle to sanity check each API. Thorough + * testing of each API appears in unit tests. + */ + @Test + public void testLifecycle() + { + CatalogClient client = new CatalogClient(clusterClient); + + // Create a datasource + TableMetadata table = TableBuilder.detailTable("example", "P1D") + .column("a", "VARCHAR") + .column("b", "BIGINT") + .column("c", "FLOAT") + .build(); + + // Use force action so test is reentrant if it fails part way through + // when debugging. + long version = client.createTable(table, "force"); + + // Update the datasource + TableSpec dsSpec2 = TableBuilder.copyOf(table) + .property(DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 3_000_000) + .column("d", "DOUBLE") + .buildSpec(); + + // First, optimistic locking, wrong version + client.updateTable(table.id(), dsSpec2, 1); + + // Optimistic locking, correct version + long newVersion = client.updateTable(table.id(), dsSpec2, version); + assertTrue(newVersion > version); + + // Verify the update + TableMetadata read = client.readTable(table.id()); + assertEquals(dsSpec2, read.spec()); + + // Move a column + MoveColumn moveCmd = new MoveColumn("d", MoveColumn.Position.BEFORE, "a"); + client.moveColumn(table.id(), moveCmd); + + // Drop a column + client.dropColumns(table.id(), Collections.singletonList("b")); + read = client.readTable(table.id()); + assertEquals(Arrays.asList("d", "a", "c"), CatalogUtils.columnNames(read.spec().columns())); + + // Hide columns + HideColumns hideCmd = new HideColumns( + Arrays.asList("e", "f"), + Collections.singletonList("g") + ); + client.hideColumns(table.id(), hideCmd); + read = client.readTable(table.id()); + assertEquals( + Arrays.asList("e", "f"), + read.spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + + // Unhide + hideCmd = new HideColumns( + null, + Collections.singletonList("e") + ); + client.hideColumns(table.id(), hideCmd); + read = client.readTable(table.id()); + assertEquals( + Collections.singletonList("f"), + read.spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + + // List schemas + List schemaNames = client.listSchemas(); + assertTrue(schemaNames.contains(TableId.DRUID_SCHEMA)); + assertTrue(schemaNames.contains(TableId.EXTERNAL_SCHEMA)); + assertTrue(schemaNames.contains(TableId.SYSTEM_SCHEMA)); + assertTrue(schemaNames.contains(TableId.CATALOG_SCHEMA)); + + // List table names in schema + List tableNames = client.listTableNamesInSchema(TableId.DRUID_SCHEMA); + assertTrue(tableNames.contains(table.id().name())); + + // List tables + List tables = client.listTables(); + assertTrue(tables.contains(table.id())); + + // Drop the table + client.dropTable(table.id()); + tableNames = client.listTableNamesInSchema(TableId.DRUID_SCHEMA); + assertFalse(tableNames.contains(table.id().name())); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/Catalog.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/Catalog.java new file mode 100644 index 000000000000..133b6303c891 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/categories/Catalog.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.categories; + +import org.apache.druid.testsEx.config.Cluster; + +@Cluster(BatchIndex.class) +public class Catalog +{ +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/CatalogClient.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/CatalogClient.java new file mode 100644 index 000000000000..b4f64577dc05 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/CatalogClient.java @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testsEx.cluster; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import org.apache.druid.catalog.http.CatalogResource; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.storage.HideColumns; +import org.apache.druid.catalog.storage.MoveColumn; +import org.apache.druid.java.util.common.StringUtils; +import org.jboss.netty.handler.codec.http.HttpMethod; + +import java.util.List; + +public class CatalogClient +{ + public static class VersionResponse + { + public final long version; + + @JsonCreator + public VersionResponse( + @JsonProperty("version") long version + ) + { + this.version = version; + } + } + + private final DruidClusterClient clusterClient; + + public CatalogClient(final DruidClusterClient clusterClient) + { + this.clusterClient = clusterClient; + } + + public long createTable(TableMetadata table) + { + return createTable(table, null); + } + + public long createTable(TableMetadata table, String action) + { + // Use action= + String url = StringUtils.format( + "%s%s/tables/%s/%s", + clusterClient.leadCoordinatorUrl(), + CatalogResource.ROOT_PATH, + table.id().schema(), + table.id().name() + ); + if (action != null) { + url += "?action=" + action; + } + VersionResponse response = clusterClient.post(url, table.spec(), VersionResponse.class); + return response.version; + } + + public long updateTable(TableId tableId, TableSpec tableSpec, long version) + { + String url = StringUtils.format( + "%s%s/tables/%s/%s", + clusterClient.leadCoordinatorUrl(), + CatalogResource.ROOT_PATH, + tableId.schema(), + tableId.name() + ); + if (version > 0) { + url += "?version=" + version; + } + VersionResponse response = clusterClient.put(url, tableSpec, VersionResponse.class); + return response.version; + } + + public TableMetadata readTable(TableId tableId) + { + String url = StringUtils.format( + "%s%s/tables/%s/%s", + clusterClient.leadCoordinatorUrl(), + CatalogResource.ROOT_PATH, + tableId.schema(), + tableId.name() + ); + return clusterClient.getAs(url, TableMetadata.class); + } + + public void dropTable(TableId tableId) + { + String url = StringUtils.format( + "%s%s/tables/%s/%s", + clusterClient.leadCoordinatorUrl(), + CatalogResource.ROOT_PATH, + tableId.schema(), + tableId.name() + ); + clusterClient.send(HttpMethod.DELETE, url); + } + + public long moveColumn(TableId tableId, MoveColumn cmd) + { + String url = StringUtils.format( + "%s%s/tables/%s/%s/moveColumn", + clusterClient.leadCoordinatorUrl(), + CatalogResource.ROOT_PATH, + tableId.schema(), + tableId.name() + ); + VersionResponse response = clusterClient.post(url, cmd, VersionResponse.class); + return response.version; + } + + public long hideColumns(TableId tableId, HideColumns cmd) + { + String url = StringUtils.format( + "%s%s/tables/%s/%s/hideColumns", + clusterClient.leadCoordinatorUrl(), + CatalogResource.ROOT_PATH, + tableId.schema(), + tableId.name() + ); + VersionResponse response = clusterClient.post(url, cmd, VersionResponse.class); + return response.version; + } + + public long dropColumns(TableId tableId, List columns) + { + String url = StringUtils.format( + "%s%s/tables/%s/%s/dropColumns", + clusterClient.leadCoordinatorUrl(), + CatalogResource.ROOT_PATH, + tableId.schema(), + tableId.name() + ); + VersionResponse response = clusterClient.post(url, columns, VersionResponse.class); + return response.version; + } + + public List listSchemas() + { + String url = StringUtils.format( + "%s%s/list/schemas/names", + clusterClient.leadCoordinatorUrl(), + CatalogResource.ROOT_PATH + ); + return clusterClient.getAs(url, new TypeReference>() { }); + } + + public List listTables() + { + String url = StringUtils.format( + "%s%s/list/tables/names", + clusterClient.leadCoordinatorUrl(), + CatalogResource.ROOT_PATH + ); + return clusterClient.getAs(url, new TypeReference>() { }); + } + + public List listTableNamesInSchema(String schemaName) + { + String url = StringUtils.format( + "%s%s/schemas/%s/names", + clusterClient.leadCoordinatorUrl(), + CatalogResource.ROOT_PATH, + schemaName + ); + return clusterClient.getAs(url, new TypeReference>() { }); + } + + public List listTablesInSchema(String schemaName) + { + String url = StringUtils.format( + "%s%s/schemas/%s/tables", + clusterClient.leadCoordinatorUrl(), + CatalogResource.ROOT_PATH, + schemaName + ); + return clusterClient.getAs(url, new TypeReference>() { }); + } +} diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/DruidClusterClient.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/DruidClusterClient.java index 88ee6c0afd7d..31b2e203945e 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/DruidClusterClient.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/DruidClusterClient.java @@ -35,10 +35,12 @@ import org.apache.druid.testsEx.config.ResolvedConfig; import org.apache.druid.testsEx.config.ResolvedDruidService; import org.apache.druid.testsEx.config.ResolvedService.ResolvedInstance; +import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; import javax.inject.Inject; +import javax.ws.rs.core.MediaType; import java.io.IOException; import java.net.URL; @@ -170,16 +172,22 @@ public Map routerCluster() * Low-level HTTP get for the given URL. */ public StatusResponseHolder get(String url) + { + return send(HttpMethod.GET, url); + } + + public StatusResponseHolder send(HttpMethod method, String url) { try { StatusResponseHolder response = httpClient.go( - new Request(HttpMethod.GET, new URL(url)), + new Request(method, new URL(url)), StatusResponseHandler.getInstance() ).get(); if (!response.getStatus().equals(HttpResponseStatus.OK)) { throw new ISE( - "Error from GET [%s] status [%s] content [%s]", + "Error from %s [%s] status [%s] content [%s]", + method, url, response.getStatus(), response.getContent() @@ -192,6 +200,44 @@ public StatusResponseHolder get(String url) } } + public StatusResponseHolder post(String url, Object body) + { + return sendPayload(HttpMethod.POST, url, body); + } + + public StatusResponseHolder put(String url, Object body) + { + return sendPayload(HttpMethod.PUT, url, body); + } + + public StatusResponseHolder sendPayload(HttpMethod method, String url, Object body) + { + final StatusResponseHolder response; + try { + final byte[] payload = jsonMapper.writeValueAsBytes(body); + response = httpClient.go( + new Request(method, new URL(url)) + .addHeader(HttpHeaders.Names.ACCEPT, MediaType.APPLICATION_JSON) + .addHeader(HttpHeaders.Names.CONTENT_TYPE, MediaType.APPLICATION_JSON) + .setContent(payload), + StatusResponseHandler.getInstance() + ).get(); + } + catch (Exception e) { + throw new RuntimeException(e); + } + + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error from POST [%s] status [%s] content [%s]", + url, + response.getStatus(), + response.getContent() + ); + } + return response; + } + /** * Issue a GET command and deserialize the JSON result to the given class. */ @@ -220,6 +266,50 @@ public T getAs(String url, TypeReference typeRef) } } + public R post(String url, M body, TypeReference typeRef) + { + StatusResponseHolder response = post(url, body); + try { + return jsonMapper.readValue(response.getContent(), typeRef); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + public R post(String url, M body, Class responseClass) + { + StatusResponseHolder response = post(url, body); + try { + return jsonMapper.readValue(response.getContent(), responseClass); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + public R put(String url, M body, Class responseClass) + { + StatusResponseHolder response = put(url, body); + try { + return jsonMapper.readValue(response.getContent(), responseClass); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + public T delete(String url, Class clazz) + { + StatusResponseHolder response = send(HttpMethod.DELETE, url); + try { + return jsonMapper.readValue(response.getContent(), clazz); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + /** * Call the `/status/selfDiscovered` given a node URL. */ diff --git a/pom.xml b/pom.xml index b663c89eba07..4d660216bf1b 100644 --- a/pom.xml +++ b/pom.xml @@ -184,6 +184,7 @@ extensions-core/druid-basic-security extensions-core/google-extensions extensions-core/druid-ranger-security + extensions-core/druid-catalog extensions-core/testing-tools extensions-contrib/compressed-bigdecimal diff --git a/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java b/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java index a0e4dd029534..93b7f06f81fd 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java +++ b/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java @@ -32,6 +32,7 @@ import org.apache.druid.segment.ColumnInspector; import javax.annotation.Nullable; + import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; diff --git a/server/pom.xml b/server/pom.xml index 74db961540f5..77644b7f8aa4 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -313,6 +313,10 @@ com.fasterxml.jackson.module jackson-module-guice + + org.apache.commons + commons-lang3 + @@ -320,11 +324,6 @@ junit test - - org.apache.commons - commons-lang3 - test - org.mockito mockito-core diff --git a/server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java b/server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java new file mode 100644 index 000000000000..3d558ddb9ba8 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.granularity.PeriodGranularity; +import org.joda.time.Period; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +public class CatalogUtils +{ + // Amazing that a parser doesn't already exist... + private static final Map GRANULARITIES = new HashMap<>(); + + static { + GRANULARITIES.put("millisecond", Granularities.SECOND); + GRANULARITIES.put("second", Granularities.SECOND); + GRANULARITIES.put("minute", Granularities.MINUTE); + GRANULARITIES.put("5 minute", Granularities.FIVE_MINUTE); + GRANULARITIES.put("5 minutes", Granularities.FIVE_MINUTE); + GRANULARITIES.put("five_minute", Granularities.FIVE_MINUTE); + GRANULARITIES.put("10 minute", Granularities.TEN_MINUTE); + GRANULARITIES.put("10 minutes", Granularities.TEN_MINUTE); + GRANULARITIES.put("ten_minute", Granularities.TEN_MINUTE); + GRANULARITIES.put("15 minute", Granularities.FIFTEEN_MINUTE); + GRANULARITIES.put("15 minutes", Granularities.FIFTEEN_MINUTE); + GRANULARITIES.put("fifteen_minute", Granularities.FIFTEEN_MINUTE); + GRANULARITIES.put("30 minute", Granularities.THIRTY_MINUTE); + GRANULARITIES.put("30 minutes", Granularities.THIRTY_MINUTE); + GRANULARITIES.put("thirty_minute", Granularities.THIRTY_MINUTE); + GRANULARITIES.put("hour", Granularities.HOUR); + GRANULARITIES.put("6 hour", Granularities.SIX_HOUR); + GRANULARITIES.put("6 hours", Granularities.SIX_HOUR); + GRANULARITIES.put("six_hour", Granularities.SIX_HOUR); + GRANULARITIES.put("day", Granularities.DAY); + GRANULARITIES.put("week", Granularities.WEEK); + GRANULARITIES.put("month", Granularities.MONTH); + GRANULARITIES.put("quarter", Granularities.QUARTER); + GRANULARITIES.put("year", Granularities.YEAR); + GRANULARITIES.put("all", Granularities.ALL); + } + + public static Granularity toGranularity(String value) + { + return GRANULARITIES.get(StringUtils.toLowerCase(value)); + } + + public static int findColumn(List columns, String colName) + { + for (int i = 0; i < columns.size(); i++) { + if (columns.get(i).name().equals(colName)) { + return i; + } + } + return -1; + } + + public static List columnNames(List columns) + { + return columns + .stream() + .map(col -> col.name()) + .collect(Collectors.toList()); + } + + public static List dropColumns( + final List columns, + final List toDrop) + { + if (toDrop == null || toDrop.isEmpty()) { + return columns; + } + Set drop = new HashSet(toDrop); + List revised = new ArrayList<>(); + for (T col : columns) { + if (!drop.contains(col.name())) { + revised.add(col); + } + } + return revised; + } + + /** + * Convert a catalog granularity string to the Druid form. Catalog granularities + * are either the usual descriptive strings (in any case), or an ISO period. + * For the odd interval, the interval name is also accepted (for the other + * intervals, the interval name is the descriptive string). + */ + public static Granularity asDruidGranularity(String value) + { + if (Strings.isNullOrEmpty(value)) { + return Granularities.ALL; + } + Granularity gran = toGranularity(value); + if (gran != null) { + return gran; + } + + try { + return new PeriodGranularity(new Period(value), null, null); + } + catch (IllegalArgumentException e) { + throw new IAE(StringUtils.format("%s is an invalid period string", value)); + } + } + + /** + * {@code String}-to-{@code List} conversion. The string can contain zero items, + * one items, or a list. The list items are separated by a comma and optional + * whitespace. + */ + public static List stringToList(String value) + { + if (value == null) { + return null; + } + return Arrays.asList(value.split(",\\s*")); + } + + public static T safeCast(Object value, Class type, String propertyName) + { + if (value == null) { + return null; + } + try { + return type.cast(value); + } + catch (ClassCastException e) { + throw new IAE("Value [%s] is not valid for property %s, expected type %s", + value, + propertyName, + type.getSimpleName() + ); + } + } + + public static T safeGet(Map map, String propertyName, Class type) + { + return safeCast(map.get(propertyName), type, propertyName); + } + + public static String stringListToLines(List lines) + { + if (lines.isEmpty()) { + return ""; + } + return String.join("\n", lines) + "\n"; + } + + public static Set setOf(String...items) + { + if (items.length == 0) { + return null; + } + return new HashSet<>(Arrays.asList(items)); + } + + public static byte[] toBytes(ObjectMapper jsonMapper, Object obj) + { + try { + return jsonMapper.writeValueAsBytes(obj); + } + catch (JsonProcessingException e) { + throw new ISE("Failed to serialize " + obj.getClass().getSimpleName()); + } + } + + public static T fromBytes(ObjectMapper jsonMapper, byte[] bytes, Class clazz) + { + try { + return jsonMapper.readValue(bytes, clazz); + } + catch (IOException e) { + throw new ISE(e, "Failed to deserialize a " + clazz.getSimpleName()); + } + } + + public static String toString(Object obj) + { + ObjectMapper jsonMapper = new ObjectMapper(); + try { + return jsonMapper.writeValueAsString(obj); + } + catch (JsonProcessingException e) { + throw new ISE("Failed to serialize TableDefn"); + } + } + + public static List concatLists( + final List base, + final List additions + ) + { + if (base == null && additions != null) { + return additions; + } + if (base != null && additions == null) { + return base; + } + List extended = new ArrayList<>(); + if (base != null) { + extended.addAll(base); + } + if (additions != null) { + extended.addAll(additions); + } + return extended; + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.java b/server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.java new file mode 100644 index 000000000000..797abf5f83b7 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.catalog.model.Properties.PropertyDefn; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; + +import java.util.List; +import java.util.Map; + +public class ColumnDefn extends ObjectDefn +{ + /** + * Convenience class that holds a column specification and its corresponding + * definition. This allows the spec to be a pure "data object" without knowledge + * of the metadata representation given by the column definition. + */ + public static class ResolvedColumn + { + private final ColumnDefn defn; + private final ColumnSpec spec; + + public ResolvedColumn(ColumnDefn defn, ColumnSpec spec) + { + this.defn = defn; + this.spec = spec; + } + + public ColumnDefn defn() + { + return defn; + } + + public ColumnSpec spec() + { + return spec; + } + + public ResolvedColumn merge(ColumnSpec update) + { + return new ResolvedColumn(defn, defn.merge(spec, update)); + } + + public void validate(ObjectMapper jsonMapper) + { + defn.validate(spec, jsonMapper); + } + } + + public ColumnDefn( + final String name, + final String typeValue, + final List fields + ) + { + super(name, typeValue, fields); + } + + public ColumnSpec merge(ColumnSpec spec, ColumnSpec update) + { + String updateType = update.type(); + if (updateType != null && !spec.type().equals(updateType)) { + throw new IAE("The update type must be null or [%s]", spec.type()); + } + String revisedType = update.sqlType() == null ? spec.sqlType() : update.sqlType(); + Map revisedProps = mergeProperties( + spec.properties(), + update.properties() + ); + return new ColumnSpec(spec.type(), spec.name(), revisedType, revisedProps); + } + + public void validate(ColumnSpec spec, ObjectMapper jsonMapper) + { + spec.validate(); + } + + public void validateScalarColumn(ColumnSpec spec) + { + Columns.validateScalarColumn(spec.name(), spec.sqlType()); + if (Columns.isTimeColumn(spec.name())) { + if (spec.sqlType() != null && !Columns.TIMESTAMP.equalsIgnoreCase(spec.sqlType())) { + throw new IAE(StringUtils.format( + "%s column must have no SQL type or SQL type %s", + Columns.TIME_COLUMN, + Columns.TIMESTAMP + )); + } + } + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java b/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java new file mode 100644 index 000000000000..f56e6c871c3b --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonInclude.Include; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import org.apache.druid.guice.annotations.UnstableApi; +import org.apache.druid.java.util.common.IAE; + +import javax.annotation.Nullable; + +import java.util.Collections; +import java.util.Map; +import java.util.Objects; + +/** + * Specification of table columns. Columns have multiple types + * represented via the type field. + */ +@UnstableApi +public class ColumnSpec +{ + private final String type; + private final String name; + private final String sqlType; + private final Map properties; + + @JsonCreator + public ColumnSpec( + @JsonProperty("type")final String type, + @JsonProperty("name")final String name, + @JsonProperty("sqlType") @Nullable final String sqlType, + @JsonProperty("properties") @Nullable final Map properties + ) + { + this.type = type; + this.name = name; + this.sqlType = sqlType; + this.properties = properties == null ? Collections.emptyMap() : properties; + } + + @JsonProperty("type") + public String type() + { + return type; + } + + @JsonProperty("name") + public String name() + { + return name; + } + + @JsonProperty("sqlType") + @JsonInclude(Include.NON_NULL) + public String sqlType() + { + return sqlType; + } + + @JsonProperty("properties") + @JsonInclude(Include.NON_EMPTY) + public Map properties() + { + return properties; + } + + public void validate() + { + if (Strings.isNullOrEmpty(type)) { + throw new IAE("Column type is required"); + } + if (Strings.isNullOrEmpty(name)) { + throw new IAE("Column name is required"); + } + } + + public byte[] toBytes(ObjectMapper jsonMapper) + { + return CatalogUtils.toBytes(jsonMapper, this); + } + + public static ColumnSpec fromBytes(ObjectMapper jsonMapper, byte[] bytes) + { + return CatalogUtils.fromBytes(jsonMapper, bytes, ColumnSpec.class); + } + + @Override + public String toString() + { + return CatalogUtils.toString(this); + } + + @Override + public boolean equals(Object o) + { + if (o == this) { + return true; + } + if (o == null || o.getClass() != getClass()) { + return false; + } + ColumnSpec other = (ColumnSpec) o; + return Objects.equals(this.type, other.type) + && Objects.equals(this.name, other.name) + && Objects.equals(this.sqlType, other.sqlType) + && Objects.equals(this.properties, other.properties); + } + + @Override + public int hashCode() + { + return Objects.hash( + type, + name, + sqlType, + properties + ); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/Columns.java b/server/src/main/java/org/apache/druid/catalog/model/Columns.java new file mode 100644 index 000000000000..23f2db9c9e88 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/Columns.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class Columns +{ + public static final String TIME_COLUMN = "__time"; + + public static final String VARCHAR = "VARCHAR"; + public static final String BIGINT = "BIGINT"; + public static final String FLOAT = "FLOAT"; + public static final String DOUBLE = "DOUBLE"; + public static final String TIMESTAMP = "TIMESTAMP"; + + public static final Set NUMERIC_TYPES = ImmutableSet.of(BIGINT, FLOAT, DOUBLE); + public static final Set SCALAR_TYPES = ImmutableSet.of(VARCHAR, BIGINT, FLOAT, DOUBLE); + + public static final Map SQL_TO_DRUID_TYPES = + new ImmutableMap.Builder() + .put(TIMESTAMP, ColumnType.LONG) + .put(BIGINT, ColumnType.LONG) + .put(FLOAT, ColumnType.FLOAT) + .put(DOUBLE, ColumnType.DOUBLE) + .put(VARCHAR, ColumnType.STRING) + .build(); + + public static boolean isTimestamp(String type) + { + return TIMESTAMP.equalsIgnoreCase(type.trim()); + } + + public static boolean isScalar(String type) + { + return SCALAR_TYPES.contains(StringUtils.toUpperCase(type.trim())); + } + + public static ColumnType druidType(String sqlType) + { + if (sqlType == null) { + return null; + } + return SQL_TO_DRUID_TYPES.get(StringUtils.toUpperCase(sqlType)); + } + + public static void validateScalarColumn(String name, String type) + { + if (type == null) { + return; + } + if (Columns.TIME_COLUMN.equals(name)) { + if (!Columns.isTimestamp(type)) { + throw new IAE("__time column must have type TIMESTAMP"); + } + } else if (!Columns.isScalar(type)) { + throw new IAE("Not a supported SQL type: " + type); + } + } + + public static boolean isTimeColumn(String name) + { + return TIME_COLUMN.equals(name); + } + + public static RowSignature convertSignature(TableSpec spec) + { + List columns = spec.columns(); + RowSignature.Builder builder = RowSignature.builder(); + for (ColumnSpec col : columns) { + ColumnType druidType = Columns.SQL_TO_DRUID_TYPES.get(StringUtils.toUpperCase(col.sqlType())); + if (druidType == null) { + druidType = ColumnType.STRING; + } + builder.add(col.name(), druidType); + } + return builder.build(); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java b/server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java new file mode 100644 index 000000000000..b1fb5ed5834c --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.catalog.model.Properties.PropertyDefn; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Metadata definition of the metadata objects stored in the catalog. (Yes, + * that means that this is meta-meta-data.) Objects consist of a map of + * property values (and perhaps other items defined in subclasses.) Each + * property is defined by a column metadata object. Objects allow extended + * properties which have no definition: the meaning of such properties is + * defined elsewhere. + */ +public class ObjectDefn +{ + private final String name; + private final String typeValue; + private final Map properties; + + public ObjectDefn( + final String name, + final String typeValue, + final List fields + ) + { + this.name = name; + this.typeValue = typeValue; + this.properties = toPropertyMap(fields); + } + + protected static Map toPropertyMap(final List props) + { + ImmutableMap.Builder builder = ImmutableMap.builder(); + if (props != null) { + for (PropertyDefn prop : props) { + builder.put(prop.name(), prop); + } + } + return builder.build(); + } + + public String name() + { + return name; + } + + /** + * The type value is the value of the {@code "type"} field written into the + * object's Java or JSON representation. It is akin to the type used by + * Jackson. + */ + public String typeValue() + { + return typeValue; + } + + public Map properties() + { + return properties; + } + + public PropertyDefn property(String key) + { + return properties.get(key); + } + + /** + * Merge the properties for an object using a set of updates in a map. If the + * update value is null, then remove the property in the revised set. If the + * property is known, use the column definition to merge the values. Else, the + * update replaces any existing value. + *

+ * This method does not validate the properties, except as needed to do a + * merge. A separate validation step is done on the final, merged object. + */ + protected Map mergeProperties( + final Map source, + final Map update + ) + { + if (update == null) { + return source; + } + if (source == null) { + return update; + } + Map merged = new HashMap<>(source); + for (Map.Entry entry : update.entrySet()) { + if (entry.getValue() == null) { + merged.remove(entry.getKey()); + } else { + PropertyDefn propDefn = property(entry.getKey()); + Object value = entry.getValue(); + if (propDefn != null) { + value = propDefn.merge(merged.get(entry.getKey()), entry.getValue()); + } + merged.put(entry.getKey(), value); + } + } + return merged; + } + + /** + * Validate the property values using the property definitions defined in + * this class. The list may contain "custom" properties which are accepted + * as-is. + */ + public void validate(Map spec, ObjectMapper jsonMapper) + { + for (PropertyDefn propDefn : properties.values()) { + propDefn.validate(spec.get(propDefn.name()), jsonMapper); + } + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/ObjectFacade.java b/server/src/main/java/org/apache/druid/catalog/model/ObjectFacade.java new file mode 100644 index 000000000000..99d432ef6afa --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/ObjectFacade.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import java.util.List; +import java.util.Map; + +/** + * Utility class to simplify typed access to catalog object properties. + */ +public abstract class ObjectFacade +{ + public abstract Map properties(); + + public Object property(String key) + { + return properties().get(key); + } + + public boolean hasProperty(String key) + { + return properties().containsKey(key); + } + + public boolean booleanProperty(String key) + { + return (Boolean) property(key); + } + + public String stringProperty(String key) + { + return (String) property(key); + } + + public Integer intProperty(String key) + { + return (Integer) property(key); + } + + @SuppressWarnings("unchecked") + public List stringListProperty(String key) + { + return (List) property(key); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/Parameterized.java b/server/src/main/java/org/apache/druid/catalog/model/Parameterized.java new file mode 100644 index 000000000000..74531d9cced3 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/Parameterized.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import org.apache.druid.catalog.model.table.ExternalSpec; + +import java.util.List; +import java.util.Map; + +public interface Parameterized +{ + interface ParameterDefn + { + String name(); + Class valueClass(); + } + + class ParameterImpl implements ParameterDefn + { + private final String name; + private final Class type; + + public ParameterImpl(final String name, final Class type) + { + this.name = name; + this.type = type; + } + + @Override + public String name() + { + return name; + } + + @Override + public Class valueClass() + { + return type; + } + } + + List parameters(); + ParameterDefn parameter(String name); + ExternalSpec applyParameters(ResolvedTable table, Map parameters); +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/Properties.java b/server/src/main/java/org/apache/druid/catalog/model/Properties.java new file mode 100644 index 000000000000..eaf98317e5da --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/Properties.java @@ -0,0 +1,302 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.PeriodGranularity; +import org.joda.time.Period; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Definition of a top-level property in a catalog object. + * Provides a set of typical property definitions. Others can be + * created case-by-case. + *

+ * Property definitions define the property name, validate the value, + * and merge updates. Properties have a type: but the type is implicit + * via the validation, as is needed when the type is actually a map + * which represents a Java object, or when the value is a list. + */ +public interface Properties +{ + public interface PropertyDefn + { + String name(); + String typeName(); + void validate(Object value, ObjectMapper jsonMapper); + Object merge(Object existing, Object update); + } + + abstract class BasePropertyDefn implements PropertyDefn + { + protected final String name; + + public BasePropertyDefn(final String name) + { + this.name = name; + } + + @Override + public String name() + { + return name; + } + + @Override + public Object merge(Object existing, Object update) + { + return update == null ? existing : update; + } + + @Override + public String toString() + { + return getClass().getSimpleName() + "{" + + "name: " + name + + ", type: " + typeName() + + "}"; + } + } + + class SimplePropertyDefn extends BasePropertyDefn + { + public final Class valueClass; + + public SimplePropertyDefn( + final String name, + final Class valueClass + ) + { + super(name); + this.valueClass = valueClass; + } + + @Override + public String typeName() + { + return valueClass.getSimpleName(); + } + + /** + * Convert the value from the deserialized JSON format to the type + * required by this field data type. Also used to decode values from + * SQL parameters. As a side effect, verifies that the value is of + * the correct type. + */ + public T decode(Object value, ObjectMapper jsonMapper) + { + if (value == null) { + return null; + } + try { + return jsonMapper.convertValue(value, valueClass); + } + catch (Exception e) { + throw new IAE( + "Value [%s] is not valid for property [%s], expected %s", + value, + name, + typeName() + ); + } + } + + /** + * Validate that the given value is valid for this property. + * By default, does a value conversion and discards the value. + */ + @Override + public void validate(Object value, ObjectMapper jsonMapper) + { + decode(value, jsonMapper); + } + } + + class TypeRefPropertyDefn extends BasePropertyDefn + { + public final String typeName; + public final TypeReference valueType; + + public TypeRefPropertyDefn( + final String name, + final String typeName, + final TypeReference valueType + ) + { + super(name); + this.typeName = Preconditions.checkNotNull(typeName); + this.valueType = valueType; + } + + @Override + public String typeName() + { + return typeName; + } + + public T decode(Object value, ObjectMapper jsonMapper) + { + if (value == null) { + return null; + } + try { + return jsonMapper.convertValue(value, valueType); + } + catch (Exception e) { + throw new IAE( + "Value [%s] is not valid for property [%s], expected %s", + value, + name, + typeName() + ); + } + } + + /** + * Convert the value from the deserialized JSON format to the type + * required by this field data type. Also used to decode values from + * SQL parameters. As a side effect, verifies that the value is of + * the correct type. + */ + @Override + public void validate(Object value, ObjectMapper jsonMapper) + { + decode(value, jsonMapper); + } + } + + class StringPropertyDefn extends SimplePropertyDefn + { + public StringPropertyDefn(String name) + { + super(name, String.class); + } + } + + class GranularityPropertyDefn extends StringPropertyDefn + { + public GranularityPropertyDefn(String name) + { + super(name); + } + + @Override + public void validate(Object value, ObjectMapper jsonMapper) + { + String gran = decode(value, jsonMapper); + validateGranularity(gran); + } + + public void validateGranularity(String value) + { + if (value == null) { + return; + } + try { + //noinspection ResultOfObjectAllocationIgnored + new PeriodGranularity(new Period(value), null, null); + } + catch (IllegalArgumentException e) { + throw new IAE(StringUtils.format("[%s] is an invalid granularity string", value)); + } + } + } + + class IntPropertyDefn extends SimplePropertyDefn + { + public IntPropertyDefn(String name) + { + super(name, Integer.class); + } + } + + class BooleanPropertyDefn extends SimplePropertyDefn + { + public BooleanPropertyDefn(String name) + { + super(name, Boolean.class); + } + } + + class ListPropertyDefn extends TypeRefPropertyDefn> + { + public ListPropertyDefn( + final String name, + final String typeName, + final TypeReference> valueType + ) + { + super(name, typeName, valueType); + } + + @SuppressWarnings("unchecked") + @Override + public Object merge(Object existing, Object updates) + { + if (updates == null) { + return existing; + } + if (existing == null) { + return updates; + } + List existingList; + List updatesList; + try { + existingList = (List) existing; + updatesList = (List) updates; + } + catch (ClassCastException e) { + throw new IAE( + "Value of field %s must be a list, found %s", + name, + updates.getClass().getSimpleName() + ); + } + Set existingSet = new HashSet<>(existingList); + List revised = new ArrayList<>(existingList); + for (T col : updatesList) { + if (!existingSet.contains(col)) { + revised.add(col); + } + } + return revised; + } + } + + class StringListPropertyDefn extends ListPropertyDefn + { + public StringListPropertyDefn(String name) + { + super( + name, + "string list", + new TypeReference>() {} + ); + } + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/ResolvedTable.java b/server/src/main/java/org/apache/druid/catalog/model/ResolvedTable.java new file mode 100644 index 000000000000..cfb0ca83e6d2 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/ResolvedTable.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.util.Map; + +/** + * Handle to a table specification along with its definition + * and the object mapper used to serialize/deserialize its data. + * This class provides the semantic view of the table specification, + * without the operational aspects such as version, name, etc. + * + * @see {@link TableMetadata} for the operational representation + */ +public class ResolvedTable extends ObjectFacade +{ + private final TableDefn defn; + private final TableSpec spec; + private final ObjectMapper jsonMapper; + + public ResolvedTable( + final TableDefn defn, + final TableSpec spec, + final ObjectMapper jsonMapper + ) + { + this.defn = defn; + this.spec = spec; + this.jsonMapper = jsonMapper; + } + + public TableDefn defn() + { + return defn; + } + + public TableSpec spec() + { + return spec; + } + + public ResolvedTable merge(TableSpec update) + { + return new ResolvedTable( + defn, + defn.merge(spec, update, jsonMapper), + jsonMapper + ); + } + + public ResolvedTable withProperties(Map props) + { + return new ResolvedTable(defn, spec.withProperties(props), jsonMapper); + } + + public void validate() + { + spec.validate(); + defn.validate(this); + } + + @Override + public Map properties() + { + return spec.properties(); + } + + public ObjectMapper jsonMapper() + { + return jsonMapper; + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistry.java b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistry.java new file mode 100644 index 000000000000..582c280d8226 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistry.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import java.util.Set; + +/** + * Defines the set of schemas available in Druid and their properties. + * Since Druid has a fixed set of schemas, this registry is currently + * hard-coded. That will change if/when Druid allows user-defined + * schemas. + */ +public interface SchemaRegistry +{ + interface SchemaSpec + { + String name(); + String securityResource(); + boolean writable(); + boolean accepts(String tableType); + } + + SchemaSpec schema(String name); + Set names(); +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java new file mode 100644 index 000000000000..3ca91d6fbc8a --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.catalog.model.table.ExternalTableDefn; +import org.apache.druid.server.security.ResourceType; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; + +/** + * Hard-coded schema registry that knows about the well-known, and + * a few obscure, Druid schemas. Does not allow for user-defined + * schemas, which the rest of Druid would not be able to support. + */ +public class SchemaRegistryImpl implements SchemaRegistry +{ + // Mimics the definition in ExternalOperatorConvertion + // TODO: Change this when ExternalOperatorConvertion changes + private String EXTERNAL_RESOURCE = "EXTERNAL"; + + public static class SchemaDefnImpl implements SchemaSpec + { + private final String name; + private final String resource; + private final Set accepts; + + public SchemaDefnImpl( + String name, + String resource, + Set accepts + ) + { + this.name = name; + this.resource = resource; + this.accepts = accepts; + } + + @Override + public String name() + { + return name; + } + + @Override + public String securityResource() + { + return resource; + } + + @Override + public boolean writable() + { + return accepts != null && !accepts.isEmpty(); + } + + @Override + public boolean accepts(String tableType) + { + if (accepts == null) { + return false; + } + return accepts.contains(tableType); + } + } + + private final Map builtIns; + + public SchemaRegistryImpl() + { + builtIns = new HashMap<>(); + register(new SchemaDefnImpl( + TableId.DRUID_SCHEMA, + ResourceType.DATASOURCE, + DatasourceDefn.tableTypes() + )); + register(new SchemaDefnImpl( + TableId.LOOKUP_SCHEMA, + ResourceType.CONFIG, + null // TODO + )); + register(new SchemaDefnImpl( + TableId.CATALOG_SCHEMA, + ResourceType.SYSTEM_TABLE, + null + )); + register(new SchemaDefnImpl( + TableId.SYSTEM_SCHEMA, + ResourceType.SYSTEM_TABLE, + null + )); + register(new SchemaDefnImpl( + TableId.EXTERNAL_SCHEMA, + EXTERNAL_RESOURCE, + ExternalTableDefn.tableTypes() + )); + register(new SchemaDefnImpl( + TableId.VIEW_SCHEMA, + ResourceType.VIEW, + null // TODO + )); + } + + private void register(SchemaSpec schemaDefn) + { + builtIns.put(schemaDefn.name(), schemaDefn); + } + + @Override + public SchemaSpec schema(String name) + { + return builtIns.get(name); + } + + @Override + public Set names() + { + return new TreeSet(builtIns.keySet()); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java new file mode 100644 index 000000000000..3ec5de4a65de --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.catalog.model.Properties.PropertyDefn; +import org.apache.druid.java.util.common.IAE; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Definition for all tables in the catalog. All tables have both + * properties and a schema. Subclasses define specific table types + * such as datasources or input tables. Some tables may be parameterized + * to allow the table to appear in a SQL table function by implementing + * the {@link Parameterized} interface. + */ +public class TableDefn extends ObjectDefn +{ + /** + * Human-readable description of the datasource. + */ + public static final String DESCRIPTION_PROPERTY = "description"; + + private final Map columnDefns; + + public TableDefn( + final String name, + final String typeValue, + final List properties, + final List columnDefns + ) + { + super( + name, + typeValue, + CatalogUtils.concatLists( + Collections.singletonList( + new Properties.StringPropertyDefn(DESCRIPTION_PROPERTY) + ), + properties + ) + ); + this.columnDefns = columnDefns == null ? Collections.emptyMap() : toColumnMap(columnDefns); + } + + public static Map toColumnMap(final List colTypes) + { + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (ColumnDefn colType : colTypes) { + builder.put(colType.typeValue(), colType); + } + return builder.build(); + } + + /** + * Validate a table spec using the table, field and column definitions defined + * here. The column definitions validate the type of each property value using + * the object mapper. + */ + public void validate(ResolvedTable table) + { + validate(table.properties(), table.jsonMapper()); + if (table.spec().columns() == null) { + return; + } + Set names = new HashSet<>(); + for (ColumnSpec colSpec : table.spec().columns()) { + if (!names.add(colSpec.name())) { + throw new IAE("Duplicate column name: " + colSpec.name()); + } + ColumnDefn.ResolvedColumn resolvedCol = resolveColumn(colSpec); + resolvedCol.validate(table.jsonMapper()); + } + } + + /** + * Resolve the column type to produce a composite object that holds + * both the definition and the column spec. + */ + public ColumnDefn.ResolvedColumn resolveColumn(ColumnSpec spec) + { + String type = spec.type(); + if (Strings.isNullOrEmpty(type)) { + throw new IAE("The column type is required."); + } + ColumnDefn defn = columnDefns.get(type); + if (defn == null) { + throw new IAE("Column type [%s] is not valid for tables of type [%s].", type, typeValue()); + } + return new ColumnDefn.ResolvedColumn(defn, spec); + } + + /** + * Merge a table spec with an update. The merge affects both the properties and + * the list of columns. + */ + public TableSpec merge(TableSpec spec, TableSpec update, ObjectMapper jsonMapper) + { + String updateType = update.type(); + if (updateType != null && !spec.type().equals(updateType)) { + throw new IAE("The update type must be null or [%s]", spec.type()); + } + Map revisedProps = mergeProperties(spec.properties(), update.properties()); + List revisedColumns = mergeColumns(spec.columns(), update.columns()); + TableSpec revisedSpec = new TableSpec(spec.type(), revisedProps, revisedColumns); + validate(new ResolvedTable(this, revisedSpec, jsonMapper)); + return revisedSpec; + } + + /** + * Merge the set of columns from an existing spec and an update. + * Columns are matched by name. If the column exists, then it is updated. If + * the column does not exist, then the new column is appended to the existing + * list. This merge operation cannot remove columns or change order. + */ + private List mergeColumns(List columns, List update) + { + if (update == null || update.isEmpty()) { + return columns; + } + Map original = new HashMap<>(); + for (int i = 0; i < columns.size(); i++) { + original.put(columns.get(i).name(), i); + } + List merged = new ArrayList<>(columns); + for (int i = 0; i < update.size(); i++) { + ColumnSpec col = update.get(i); + String colName = col.name(); + if (Strings.isNullOrEmpty(colName)) { + throw new IAE("Column %d must have a name", i + 1); + } + Integer index = original.get(col.name()); + if (index == null) { + if (Strings.isNullOrEmpty(col.type())) { + throw new IAE("Column %d must have a type", i + 1); + } + merged.add(col); + } else { + ColumnDefn.ResolvedColumn resolvedCol = resolveColumn(columns.get(index)); + merged.set(index, resolvedCol.merge(col).spec()); + } + } + return merged; + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java b/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java new file mode 100644 index 000000000000..bcf1d088b4a2 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.catalog.model.table.HttpTableDefn; +import org.apache.druid.catalog.model.table.InlineTableDefn; +import org.apache.druid.catalog.model.table.LocalTableDefn; +import org.apache.druid.java.util.common.IAE; + +import java.util.Map; + +public class TableDefnRegistry +{ + private static final TableDefn[] TABLE_DEFNS = { + new DatasourceDefn.DetailDatasourceDefn(), + new DatasourceDefn.RollupDatasourceDefn(), + new InlineTableDefn(), + new HttpTableDefn(), + new LocalTableDefn() + }; + + private final Map defns; + private final ObjectMapper jsonMapper; + + public TableDefnRegistry( + final TableDefn[] defns, + final ObjectMapper jsonMapper + ) + { + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (TableDefn defn : defns) { + builder.put(defn.typeValue(), defn); + } + this.defns = builder.build(); + this.jsonMapper = jsonMapper; + } + + public TableDefnRegistry( + final ObjectMapper jsonMapper + ) + { + this(TABLE_DEFNS, jsonMapper); + } + + public ResolvedTable resolve(TableSpec spec) + { + String type = spec.type(); + if (Strings.isNullOrEmpty(type)) { + throw new IAE("The table type is required."); + } + TableDefn defn = defns.get(type); + if (defn == null) { + throw new IAE("Table type [%s] is not valid.", type); + } + return new ResolvedTable(defn, spec, jsonMapper); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableId.java b/server/src/main/java/org/apache/druid/catalog/model/TableId.java new file mode 100644 index 000000000000..42832296ae2d --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/TableId.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Objects; +import org.apache.druid.java.util.common.StringUtils; + +/** + * SQL-like compound table ID with schema and table name. + */ +public class TableId +{ + // Well-known Druid schemas + public static final String DRUID_SCHEMA = "druid"; + public static final String LOOKUP_SCHEMA = "lookups"; + public static final String SYSTEM_SCHEMA = "sys"; + public static final String CATALOG_SCHEMA = "INFORMATION_SCHEMA"; + + // Extra for MSQE + public static final String EXTERNAL_SCHEMA = "extern"; + + // Extra for views + public static final String VIEW_SCHEMA = "view"; + + private final String schema; + private final String name; + + @JsonCreator + public TableId( + @JsonProperty("schema") String schema, + @JsonProperty("name") String name) + { + this.schema = schema; + this.name = name; + } + + public static TableId datasource(String name) + { + return new TableId(DRUID_SCHEMA, name); + } + + public static TableId external(String name) + { + return new TableId(EXTERNAL_SCHEMA, name); + } + + public static TableId of(String schema, String table) + { + return new TableId(schema, table); + } + + @JsonProperty("schema") + public String schema() + { + return schema; + } + + @JsonProperty("name") + public String name() + { + return name; + } + + public String sqlName() + { + return StringUtils.format("\"%s\".\"%s\"", schema, name); + } + + @Override + public String toString() + { + return sqlName(); + } + + @Override + public boolean equals(Object o) + { + if (o == this) { + return true; + } + if (o == null || o.getClass() != getClass()) { + return false; + } + TableId other = (TableId) o; + return Objects.equal(this.schema, other.schema) + && Objects.equal(this.name, other.name); + } + + @Override + public int hashCode() + { + return Objects.hashCode(schema, name); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java new file mode 100644 index 000000000000..5fbc69c26330 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java @@ -0,0 +1,241 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import org.apache.druid.guice.annotations.PublicApi; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; + +import java.util.Objects; + +/** + * REST API level description of a table. Tables have multiple types + * as described by subclasses. Stores the operational aspects of a + * table, such as its name, creation time, state and spec. + * + * @see {@link ResolvedTable} for the semantic representation. + */ +@PublicApi +public class TableMetadata +{ + public enum TableState + { + ACTIVE("A"), + DELETING("D"); + + private final String code; + + TableState(String code) + { + this.code = code; + } + + public String code() + { + return code; + } + + public static TableState fromCode(String code) + { + for (TableState state : values()) { + if (state.code.equals(code)) { + return state; + } + } + throw new ISE("Unknown TableState code: " + code); + } + } + + private final TableId id; + private final long creationTime; + private final long updateTime; + private final TableState state; + private final TableSpec spec; + + public TableMetadata( + @JsonProperty("id") TableId tableId, + @JsonProperty("creationTime") long creationTime, + @JsonProperty("updateTime") long updateTime, + @JsonProperty("state") TableState state, + @JsonProperty("spec") TableSpec spec) + { + this.id = tableId; + this.creationTime = creationTime; + this.updateTime = updateTime; + this.state = state; + this.spec = spec; + } + + public static TableMetadata newTable( + TableId id, + TableSpec defn + ) + { + return new TableMetadata( + id, + 0, + 0, + TableState.ACTIVE, + defn + ); + } + + public static TableMetadata newSegmentTable( + String name, + TableSpec defn + ) + { + return newTable( + TableId.datasource(name), + defn); + } + + public TableMetadata fromInsert(long updateTime) + { + return new TableMetadata( + id, + updateTime, + updateTime, + state, + spec + ); + } + + public TableMetadata asUpdate(long updateTime) + { + return new TableMetadata( + id, + creationTime, + updateTime, + state, + spec); + } + + public TableMetadata withSpec(TableSpec spec) + { + return new TableMetadata( + id, + creationTime, + updateTime, + state, + spec + ); + } + + @JsonProperty("id") + public TableId id() + { + return id; + } + + public String sqlName() + { + return id.sqlName(); + } + + @JsonProperty("state") + public TableState state() + { + return state; + } + + @JsonProperty("creationTime") + public long creationTime() + { + return creationTime; + } + + @JsonProperty("updateTime") + public long updateTime() + { + return updateTime; + } + + @JsonProperty("spec") + public TableSpec spec() + { + return spec; + } + + /** + * Syntactic validation of a table object. Validates only that which + * can be checked from this table object. + */ + public void validate() + { + if (Strings.isNullOrEmpty(id.schema())) { + throw new IAE("Database schema is required"); + } + if (Strings.isNullOrEmpty(id.name())) { + throw new IAE("Table name is required"); + } + if (spec == null) { + throw new IAE("A table definition must include a table spec."); + } + } + + public byte[] toBytes(ObjectMapper jsonMapper) + { + return CatalogUtils.toBytes(jsonMapper, this); + } + + public static TableMetadata fromBytes(ObjectMapper jsonMapper, byte[] bytes) + { + return CatalogUtils.fromBytes(jsonMapper, bytes, TableMetadata.class); + } + + @Override + public String toString() + { + return CatalogUtils.toString(this); + } + + @Override + public boolean equals(Object o) + { + if (o == this) { + return true; + } + if (o == null || o.getClass() != getClass()) { + return false; + } + TableMetadata other = (TableMetadata) o; + return Objects.equals(id, other.id) + && creationTime == other.creationTime + && updateTime == other.updateTime + && state == other.state + && Objects.equals(spec, other.spec); + } + + @Override + public int hashCode() + { + return Objects.hash( + id, + creationTime, + updateTime, + state, + spec + ); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java b/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java new file mode 100644 index 000000000000..be93046a616a --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonInclude.Include; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import org.apache.druid.java.util.common.IAE; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Definition of a table "hint" in the metastore, between client and + * Druid, and between Druid nodes. + *

+ * This class is a simple holder of values. Semantics on top of the values + * are provided separately: by the catalog object model for writes, and + * by various Java classes for reads. + */ +public class TableSpec +{ + private final String type; + private final Map properties; + private final List columns; + + @JsonCreator + public TableSpec( + @JsonProperty("type") final String type, + @JsonProperty("properties") final Map properties, + @JsonProperty("columns") final List columns + ) + { + this.type = type; + this.properties = properties == null ? Collections.emptyMap() : properties; + this.columns = columns == null ? Collections.emptyList() : columns; + + // Note: no validation here. If a bad definition got into the + // DB, don't prevent deserialization. + } + + public TableSpec withProperties(final Map properties) + { + return new TableSpec(type, properties, columns); + } + + public TableSpec withColumns(final List columns) + { + return new TableSpec(type, properties, columns); + } + + public TableSpec withProperty(String key, Object value) + { + Map revised = new HashMap<>(properties); + revised.put(key, value); + return withProperties(revised); + } + + @JsonProperty("type") + public String type() + { + return type; + } + + @JsonProperty("properties") + @JsonInclude(Include.NON_NULL) + public Map properties() + { + return properties; + } + + @JsonProperty("columns") + @JsonInclude(Include.NON_NULL) + public List columns() + { + return columns; + } + + /** + * Validate the final spec. Updates use this same class, but allow + * the spec to be partial (and thus inconsistent). Validation should + * be done on the merged result, not on the updates themselves. + */ + public void validate() + { + if (Strings.isNullOrEmpty(type)) { + throw new IAE("Table type is required"); + } + } + + public byte[] toBytes(ObjectMapper jsonMapper) + { + return CatalogUtils.toBytes(jsonMapper, this); + } + + public static TableSpec fromBytes(ObjectMapper jsonMapper, byte[] bytes) + { + return CatalogUtils.fromBytes(jsonMapper, bytes, TableSpec.class); + } + + @Override + public String toString() + { + return CatalogUtils.toString(this); + } + + @Override + public boolean equals(Object o) + { + if (o == this) { + return true; + } + if (o == null || o.getClass() != getClass()) { + return false; + } + TableSpec other = (TableSpec) o; + return Objects.equals(this.type, other.type) + && Objects.equals(this.columns, other.columns) + && Objects.equals(this.properties, other.properties); + } + + @Override + public int hashCode() + { + return Objects.hash( + type, + columns, + properties + ); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/facade/ColumnFacade.java b/server/src/main/java/org/apache/druid/catalog/model/facade/ColumnFacade.java new file mode 100644 index 000000000000..783c5a7e41d6 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/facade/ColumnFacade.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.facade; + +import org.apache.druid.catalog.model.ColumnDefn.ResolvedColumn; +import org.apache.druid.catalog.model.Columns; +import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.catalog.model.table.MeasureTypes; +import org.apache.druid.catalog.model.table.MeasureTypes.MeasureType; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.column.ColumnType; + +public class ColumnFacade +{ + public static class DatasourceColumnFacade extends ColumnFacade + { + public DatasourceColumnFacade(ResolvedColumn column) + { + super(column); + } + + @Override + public ColumnType druidType() + { + if (Columns.isTimeColumn(column.spec().name())) { + return ColumnType.LONG; + } + return super.druidType(); + } + + public boolean isMeasure() + { + return DatasourceDefn.MEASURE_TYPE.equals(column.spec().type()); + } + + public MeasureType measureType() + { + String sqlType = column.spec().sqlType(); + if (sqlType == null) { + return null; + } + try { + return MeasureTypes.parse(sqlType); + } + catch (ISE e) { + return null; + } + } + } + + protected final ResolvedColumn column; + + public ColumnFacade(ResolvedColumn column) + { + this.column = column; + } + + public ColumnType druidType() + { + String sqlType = column.spec().sqlType(); + return sqlType == null ? null : Columns.druidType(sqlType); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/facade/DatasourceFacade.java b/server/src/main/java/org/apache/druid/catalog/model/facade/DatasourceFacade.java new file mode 100644 index 000000000000..5fa77342e589 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/facade/DatasourceFacade.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.facade; + +import org.apache.druid.catalog.model.CatalogUtils; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.table.ClusterKeySpec; +import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.java.util.common.granularity.Granularity; + +import java.util.Collections; +import java.util.List; + +/** + * Convenience wrapper on top of a resolved table (a table spec + * and its corresponding definition.) To be used by consumers + * of catalog objects that work with specific datasource properties + * rather than layers that work with specs generically. + */ +public class DatasourceFacade extends TableFacade +{ + public DatasourceFacade(ResolvedTable resolved) + { + super(resolved); + } + + public boolean isRollup() + { + return DatasourceDefn.ROLLUP_DATASOURCE_TYPE.equals(spec().type()); + } + + public boolean isDetail() + { + return !isRollup(); + } + + public String segmentGranularityString() + { + return stringProperty(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY); + } + + public Granularity segmentGranularity() + { + String value = stringProperty(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY); + return value == null ? null : CatalogUtils.asDruidGranularity(value); + } + + public Integer targetSegmentRows() + { + return intProperty(DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY); + } + + @SuppressWarnings("unchecked") + public List clusterKeys() + { + return (List) property(DatasourceDefn.CLUSTER_KEYS_PROPERTY); + } + + @SuppressWarnings("unchecked") + public List hiddenColumns() + { + Object value = property(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY); + return value == null ? Collections.emptyList() : (List) value; + } + + public Granularity rollupGranularity() + { + String value = stringProperty(DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY); + return value == null ? null : CatalogUtils.asDruidGranularity(value); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/facade/InputTableFacade.java b/server/src/main/java/org/apache/druid/catalog/model/facade/InputTableFacade.java new file mode 100644 index 000000000000..c7416caf1145 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/facade/InputTableFacade.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.facade; + +import org.apache.druid.catalog.model.ColumnSpec; +import org.apache.druid.catalog.model.Columns; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; + +import java.util.List; + +public class InputTableFacade extends TableFacade +{ + public InputTableFacade(ResolvedTable resolved) + { + super(resolved); + } + + public RowSignature rowSignature() + { + List columns = spec().columns(); + RowSignature.Builder builder = RowSignature.builder(); + for (ColumnSpec col : columns) { + ColumnType druidType = Columns.SQL_TO_DRUID_TYPES.get(StringUtils.toUpperCase(col.sqlType())); + if (druidType == null) { + druidType = ColumnType.STRING; + } + builder.add(col.name(), druidType); + } + return builder.build(); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/facade/TableFacade.java b/server/src/main/java/org/apache/druid/catalog/model/facade/TableFacade.java new file mode 100644 index 000000000000..0c0dce79c1ba --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/facade/TableFacade.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.facade; + +import org.apache.druid.catalog.model.ColumnSpec; +import org.apache.druid.catalog.model.ObjectFacade; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableSpec; + +import java.util.List; +import java.util.Map; + +/** + * Convenience wrapper on top of a resolved table (a table spec + * and its corresponding definition.) + */ +public class TableFacade extends ObjectFacade +{ + protected final ResolvedTable resolved; + + public TableFacade(ResolvedTable resolved) + { + this.resolved = resolved; + } + + public TableSpec spec() + { + return resolved.spec(); + } + + @Override + public Map properties() + { + return spec().properties(); + } + + public List columns() + { + return spec().columns(); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/ClusterKeySpec.java b/server/src/main/java/org/apache/druid/catalog/model/table/ClusterKeySpec.java new file mode 100644 index 000000000000..2e8eb2181880 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/ClusterKeySpec.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.table; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.annotation.Nullable; + +import java.util.Objects; + +public class ClusterKeySpec +{ + private final String expr; + private final boolean desc; + + @JsonCreator + public ClusterKeySpec( + @JsonProperty("column") String expr, + @JsonProperty("desc") @Nullable Boolean desc + ) + { + this.expr = expr; + this.desc = desc != null && desc == true; + } + + public ClusterKeySpec(String expr) + { + this(expr, false); + } + + @JsonProperty("column") + public String expr() + { + return expr; + } + + @JsonProperty("desc") + public boolean desc() + { + return desc; + } + + @Override + public String toString() + { + return expr + (desc ? " DESC" : ""); + } + + @Override + public boolean equals(Object o) + { + if (o == null || o.getClass() != getClass()) { + return false; + } + ClusterKeySpec other = (ClusterKeySpec) o; + return Objects.equals(this.expr, other.expr) + && this.desc == other.desc; + } + + @Override + public int hashCode() + { + return Objects.hash(expr, desc); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/DatasourceDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/DatasourceDefn.java new file mode 100644 index 000000000000..0d8c4d7a95a7 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/DatasourceDefn.java @@ -0,0 +1,278 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.table; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import org.apache.druid.catalog.model.CatalogUtils; +import org.apache.druid.catalog.model.ColumnDefn; +import org.apache.druid.catalog.model.ColumnSpec; +import org.apache.druid.catalog.model.Columns; +import org.apache.druid.catalog.model.Properties; +import org.apache.druid.catalog.model.Properties.GranularityPropertyDefn; +import org.apache.druid.catalog.model.Properties.PropertyDefn; +import org.apache.druid.catalog.model.Properties.StringListPropertyDefn; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableDefn; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Set; + +public class DatasourceDefn extends TableDefn +{ + /** + * Segment grain at ingestion and initial compaction. Aging rules + * may override the value as segments age. If not provided here, + * then it must be provided at ingestion time. + */ + public static final String SEGMENT_GRANULARITY_PROPERTY = "segmentGranularity"; + + /** + * The target segment size at ingestion and initial compaction. + * If unset, then the system setting is used. + */ + public static final String TARGET_SEGMENT_ROWS_PROPERTY = "targetSegmentRows"; + public static final String CLUSTER_KEYS_PROPERTY = "clusterKeys"; + public static final String HIDDEN_COLUMNS_PROPERTY = "hiddenColumns"; + + /** + * Ingestion and auto-compaction rollup granularity. If null, then no + * rollup is enabled. Same as {@code queryGranularity} in and ingest spec, + * but renamed since this granularity affects rollup, not queries. Can be + * overridden at ingestion time. The grain may change as segments evolve: + * this is the grain only for ingest. + */ + public static final String ROLLUP_GRANULARITY_PROPERTY = "rollupGranularity"; + + public static final String DETAIL_DATASOURCE_TYPE = "detail"; + public static final String ROLLUP_DATASOURCE_TYPE = "rollup"; + + public static final String DETAIL_COLUMN_TYPE = "detail"; + public static final String DIMENSION_TYPE = "dimension"; + public static final String MEASURE_TYPE = "measure"; + public static final String INPUT_COLUMN_TYPE = "input"; + + public static class SegmentGranularityFieldDefn extends GranularityPropertyDefn + { + public SegmentGranularityFieldDefn() + { + super(SEGMENT_GRANULARITY_PROPERTY); + } + + @Override + public void validate(Object value, ObjectMapper jsonMapper) + { + String gran = decode(value, jsonMapper); + if (Strings.isNullOrEmpty(gran)) { + throw new IAE("Segment granularity is required."); + } + validateGranularity(gran); + } + } + + public static class HiddenColumnsDefn extends StringListPropertyDefn + { + public HiddenColumnsDefn() + { + super(HIDDEN_COLUMNS_PROPERTY); + } + + @Override + public void validate(Object value, ObjectMapper jsonMapper) + { + if (value == null) { + return; + } + List hiddenColumns = decode(value, jsonMapper); + for (String col : hiddenColumns) { + if (Columns.TIME_COLUMN.equals(col)) { + throw new IAE( + StringUtils.format("Cannot hide column %s", col) + ); + } + } + } + } + + /** + * Definition of a column in a detail (non-rollup) datasource. + */ + public static class DetailColumnDefn extends ColumnDefn + { + public DetailColumnDefn() + { + super( + "Column", + DETAIL_COLUMN_TYPE, + null + ); + } + + @Override + public void validate(ColumnSpec spec, ObjectMapper jsonMapper) + { + super.validate(spec, jsonMapper); + validateScalarColumn(spec); + } + } + + /** + * Definition of a dimension in a rollup datasource. + */ + public static class DimensionDefn extends ColumnDefn + { + public DimensionDefn() + { + super( + "Dimension", + DIMENSION_TYPE, + null + ); + } + + @Override + public void validate(ColumnSpec spec, ObjectMapper jsonMapper) + { + super.validate(spec, jsonMapper); + validateScalarColumn(spec); + } + } + + /** + * Definition of a measure (metric) column. + * Types are expressed as compound types: "AGG_FN(ARG_TYPE,...)" + * where "AGG_FN" is one of the supported aggregate functions, + * and "ARG_TYPE" is zero or more argument types. + */ + public static class MeasureDefn extends ColumnDefn + { + public MeasureDefn() + { + super( + "Measure", + MEASURE_TYPE, + null + ); + } + + @Override + public void validate(ColumnSpec spec, ObjectMapper jsonMapper) + { + super.validate(spec, jsonMapper); + if (spec.sqlType() == null) { + throw new IAE("A type is required for measure column " + spec.name()); + } + if (Columns.isTimeColumn(spec.name())) { + throw new IAE(StringUtils.format( + "%s column cannot be a measure", + Columns.TIME_COLUMN + )); + } + MeasureTypes.parse(spec.sqlType()); + } + } + + public static class DetailDatasourceDefn extends DatasourceDefn + { + public DetailDatasourceDefn() + { + super( + "Detail datasource", + DETAIL_DATASOURCE_TYPE, + null, + Collections.singletonList(new DetailColumnDefn()) + ); + } + } + + public static class RollupDatasourceDefn extends DatasourceDefn + { + public RollupDatasourceDefn() + { + super( + "Rollup datasource", + ROLLUP_DATASOURCE_TYPE, + Collections.singletonList( + new Properties.GranularityPropertyDefn(ROLLUP_GRANULARITY_PROPERTY) + ), + Arrays.asList( + new DimensionDefn(), + new MeasureDefn() + ) + ); + } + } + + public DatasourceDefn( + final String name, + final String typeValue, + final List properties, + final List columnDefns + ) + { + super( + name, + typeValue, + CatalogUtils.concatLists( + Arrays.asList( + new SegmentGranularityFieldDefn(), + new Properties.IntPropertyDefn(TARGET_SEGMENT_ROWS_PROPERTY), + new Properties.ListPropertyDefn( + CLUSTER_KEYS_PROPERTY, + "cluster keys", + new TypeReference>() { } + ), + new HiddenColumnsDefn() + ), + properties + ), + columnDefns + ); + } + + public static boolean isDatasource(String tableType) + { + return DETAIL_DATASOURCE_TYPE.equals(tableType) + || ROLLUP_DATASOURCE_TYPE.equals(tableType); + } + + public static boolean isDatasource(ResolvedTable table) + { + return table.defn() instanceof DatasourceDefn; + } + + public static boolean isMeasure(ColumnSpec col) + { + return DatasourceDefn.MEASURE_TYPE.equals(col.type()); + } + + public static Set tableTypes() + { + return CatalogUtils.setOf( + DatasourceDefn.DETAIL_DATASOURCE_TYPE, + DatasourceDefn.ROLLUP_DATASOURCE_TYPE + ); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/ExternalSpec.java b/server/src/main/java/org/apache/druid/catalog/model/table/ExternalSpec.java new file mode 100644 index 000000000000..7e0921110216 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/ExternalSpec.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.table; + +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.segment.column.RowSignature; + +/** + * Catalog form of an external table specification used to + * pass along the three components needed for an external table + * in MSQ ingest. + */ +public class ExternalSpec +{ + protected final InputSource inputSource; + protected final InputFormat inputFormat; + protected final RowSignature signature; + + public ExternalSpec( + final InputSource inputSource, + final InputFormat inputFormat, + final RowSignature signature) + { + this.inputSource = inputSource; + this.inputFormat = inputFormat; + this.signature = signature; + } + + public InputSource inputSource() + { + return inputSource; + } + + public InputFormat inputFormat() + { + return inputFormat; + } + + public RowSignature signature() + { + return signature; + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java new file mode 100644 index 000000000000..4d65e91046b1 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java @@ -0,0 +1,272 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.table; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.catalog.model.CatalogUtils; +import org.apache.druid.catalog.model.ColumnDefn; +import org.apache.druid.catalog.model.ColumnSpec; +import org.apache.druid.catalog.model.Columns; +import org.apache.druid.catalog.model.Parameterized; +import org.apache.druid.catalog.model.Parameterized.ParameterDefn; +import org.apache.druid.catalog.model.Properties.PropertyDefn; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableDefn; +import org.apache.druid.catalog.model.table.InputFormats.InputFormatDefn; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.utils.CollectionUtils; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Definition of an external input source, primarily for ingestion. + * The components are derived from those for Druid ingestion: an + * input source, a format and a set of columns. Also provides + * properties, as do all table definitions. + *

+ * The external table implements the mechanism for parameterized tables, + * but does not implement the {@link Parameterized} interface itself. + * Tables which are parameterized implement that interface to expose + * methods defined here. + */ +public abstract class ExternalTableDefn extends TableDefn +{ + public static final String EXTERNAL_COLUMN_TYPE = "extern"; + + public abstract static class FormattedExternalTableDefn extends ExternalTableDefn + { + public static final String FORMAT_PROPERTY = "format"; + + private Map formats; + + public FormattedExternalTableDefn( + final String name, + final String typeValue, + final List properties, + final List columnDefns, + final List formats, + final List parameters + ) + { + super( + name, + typeValue, + addFormatProperties(properties, formats), + columnDefns, + parameters + ); + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (InputFormatDefn format : formats) { + builder.put(format.typeTag(), format); + } + this.formats = builder.build(); + } + + /** + * Add format properties to the base set, in the order of the formats, + * in the order defined by the format. Allow same-named properties across + * formats, as long as the types are the same. + */ + private static List addFormatProperties( + final List properties, + final List formats + ) + { + List toAdd = new ArrayList<>(); + Map formatProps = new HashMap<>(); + for (InputFormatDefn format : formats) { + for (PropertyDefn prop : format.properties()) { + PropertyDefn existing = formatProps.putIfAbsent(prop.name(), prop); + if (existing == null) { + toAdd.add(prop); + } else if (existing.getClass() != prop.getClass()) { + throw new ISE( + "Format %s, property %s of class %s conflicts with another format property of class %s", + format.name(), + prop.name(), + prop.getClass().getSimpleName(), + existing.getClass().getSimpleName() + ); + } + } + } + return CatalogUtils.concatLists(properties, toAdd); + } + + @Override + protected InputFormat convertFormat(ResolvedTable table) + { + return formatDefn(table).convert(table); + } + + protected InputFormatDefn formatDefn(ResolvedTable table) + { + String formatTag = table.stringProperty(FORMAT_PROPERTY); + if (formatTag == null) { + throw new IAE("%s property must be set", FORMAT_PROPERTY); + } + InputFormatDefn formatDefn = formats.get(formatTag); + if (formatDefn == null) { + throw new IAE( + "Format type [%s] for property %s is not valid", + formatTag, + FORMAT_PROPERTY + ); + } + return formatDefn; + } + + @Override + public void validate(ResolvedTable table) + { + super.validate(table); + formatDefn(table).validate(table); + List columns = table.spec().columns(); + if (columns == null || columns.isEmpty()) { + throw new IAE( + "An external table of type %s must specify one or more columns", + table.spec().type() + ); + } + } + } + + /** + * Definition of a column in a detail (non-rollup) datasource. + */ + public static class ExternalColumnDefn extends ColumnDefn + { + public ExternalColumnDefn() + { + super( + "Column", + EXTERNAL_COLUMN_TYPE, + null + ); + } + + @Override + public void validate(ColumnSpec spec, ObjectMapper jsonMapper) + { + super.validate(spec, jsonMapper); + validateScalarColumn(spec); + } + } + + protected static final ExternalColumnDefn INPUT_COLUMN_DEFN = new ExternalColumnDefn(); + private final List parameterList; + private final Map parameterMap; + + + public ExternalTableDefn( + final String name, + final String typeValue, + final List fields, + final List columnDefns, + final List parameters + ) + { + super(name, typeValue, fields, columnDefns); + if (CollectionUtils.isNullOrEmpty(parameters)) { + this.parameterMap = null; + this.parameterList = null; + } else { + this.parameterList = parameters; + Map params = new HashMap<>(); + for (ParameterDefn param : parameters) { + if (params.put(param.name(), param) != null) { + throw new ISE("Duplicate parameter: ", param.name()); + } + } + this.parameterMap = ImmutableMap.copyOf(params); + } + } + + public List parameters() + { + return parameterList; + } + + public ParameterDefn parameter(String key) + { + return parameterMap.get(key); + } + + public abstract ResolvedTable mergeParameters(ResolvedTable table, Map values); + + public ExternalSpec convertToExtern(ResolvedTable table) + { + return new ExternalSpec( + convertSource(table), + convertFormat(table), + Columns.convertSignature(table.spec()) + ); + } + + protected InputFormat convertFormat(ResolvedTable table) + { + return null; + } + + protected abstract InputSource convertSource(ResolvedTable table); + + protected InputSource convertObject( + final ObjectMapper jsonMapper, + final Map jsonMap, + final Class targetClass + ) + { + try { + return jsonMapper.convertValue(jsonMap, targetClass); + } + catch (Exception e) { + throw new IAE(e, "Invalid table specification"); + } + } + + public ExternalSpec applyParameters(ResolvedTable table, Map parameters) + { + ResolvedTable revised = mergeParameters(table, parameters); + return convertToExtern(revised); + } + + public static boolean isInputTable(ResolvedTable table) + { + return table.defn() instanceof ExternalTableDefn; + } + + public static Set tableTypes() + { + // Known input tables. Get this from a registry later. + return CatalogUtils.setOf( + InlineTableDefn.TABLE_TYPE, + HttpTableDefn.TABLE_TYPE, + LocalTableDefn.TABLE_TYPE + ); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.java new file mode 100644 index 000000000000..6b4f5f1cbd34 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.java @@ -0,0 +1,190 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.table; + +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.catalog.model.CatalogUtils; +import org.apache.druid.catalog.model.Parameterized; +import org.apache.druid.catalog.model.Properties.StringListPropertyDefn; +import org.apache.druid.catalog.model.Properties.StringPropertyDefn; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.table.ExternalTableDefn.FormattedExternalTableDefn; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.impl.HttpInputSource; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.metadata.DefaultPasswordProvider; +import org.apache.druid.metadata.EnvironmentVariablePasswordProvider; +import org.apache.druid.utils.CollectionUtils; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Definition of an input table for an HTTP data source. Provides the same + * properties as the {@link HttpInputSource}, but as top-level properties + * that can be mapped to SQL function parameters. Property names are + * cleaned up for ease-of-use. The HTTP input source has multiple quirks, + * the conversion method smooths over those quirks for a simpler catalog + * experience. Provides a parameterized + * form where the user provides the partial URLs to use for a particular + * query. + */ +public class HttpTableDefn extends FormattedExternalTableDefn implements Parameterized +{ + public static final String TABLE_TYPE = HttpInputSource.TYPE_KEY; + public static final String URI_TEMPLATE_PROPERTY = "template"; + public static final String USER_PROPERTY = "user"; + public static final String PASSWORD_PROPERTY = "password"; + public static final String PASSWORD_ENV_VAR_PROPERTY = "passwordEnvVar"; + public static final String URIS_PROPERTY = "uris"; + public static final String URIS_PARAMETER = "uris"; + + public HttpTableDefn() + { + super( + "HTTP input table", + TABLE_TYPE, + Arrays.asList( + new StringListPropertyDefn(URIS_PROPERTY), + new StringPropertyDefn(USER_PROPERTY), + new StringPropertyDefn(PASSWORD_PROPERTY), + new StringPropertyDefn(PASSWORD_ENV_VAR_PROPERTY), + new StringPropertyDefn(URI_TEMPLATE_PROPERTY) + ), + Collections.singletonList(INPUT_COLUMN_DEFN), + InputFormats.ALL_FORMATS, + Collections.singletonList( + new ParameterImpl(URIS_PARAMETER, String.class) + ) + ); + } + + @Override + public ResolvedTable mergeParameters(ResolvedTable table, Map values) + { + String urisValue = CatalogUtils.safeGet(values, URIS_PARAMETER, String.class); + List uriValues = CatalogUtils.stringToList(urisValue); + if (CollectionUtils.isNullOrEmpty(uriValues)) { + throw new IAE("One or more values are required for parameter %s", URIS_PARAMETER); + } + String uriTemplate = table.stringProperty(URI_TEMPLATE_PROPERTY); + if (Strings.isNullOrEmpty(uriTemplate)) { + throw new IAE("Property %s must provide a URI template.", URI_TEMPLATE_PROPERTY); + } + Pattern p = Pattern.compile("\\{}"); + Matcher m = p.matcher(uriTemplate); + if (!m.find()) { + throw new IAE( + "Value [%s] for property %s must include a '{}' placeholder.", + uriTemplate, + URI_TEMPLATE_PROPERTY + ); + } + List uris = new ArrayList<>(); + for (String uri : uriValues) { + uris.add(m.replaceFirst(uri)); + } + + Map revisedProps = new HashMap<>(table.properties()); + revisedProps.remove(URI_TEMPLATE_PROPERTY); + revisedProps.put("uris", uris); + return table.withProperties(revisedProps); + } + + @Override + protected InputSource convertSource(ResolvedTable table) + { + Map jsonMap = new HashMap<>(); + jsonMap.put(InputSource.TYPE_PROPERTY, HttpInputSource.TYPE_KEY); + jsonMap.put("httpAuthenticationUsername", table.stringProperty(USER_PROPERTY)); + String password = table.stringProperty(PASSWORD_PROPERTY); + String passwordEnvVar = table.stringProperty(PASSWORD_ENV_VAR_PROPERTY); + if (password != null && passwordEnvVar != null) { + throw new ISE( + "Specify only one of %s or %s", + PASSWORD_PROPERTY, + PASSWORD_ENV_VAR_PROPERTY + ); + } + if (password != null) { + jsonMap.put( + "httpAuthenticationPassword", + ImmutableMap.of("type", DefaultPasswordProvider.TYPE_KEY, "password", password) + ); + } else if (passwordEnvVar != null) { + jsonMap.put( + "httpAuthenticationPassword", + ImmutableMap.of("type", EnvironmentVariablePasswordProvider.TYPE_KEY, "variable", passwordEnvVar) + ); + } + jsonMap.put("uris", convertUriList(table.stringListProperty(URIS_PROPERTY))); + return convertObject(table.jsonMapper(), jsonMap, HttpInputSource.class); + } + + @SuppressWarnings("unchecked") + public static List convertUriList(Object value) + { + if (value == null) { + return null; + } + List list; + try { + list = (List) value; + } + catch (ClassCastException e) { + throw new IAE("Value [%s] must be a list of strings", value); + } + List uris = new ArrayList<>(); + for (String strValue : list) { + try { + uris.add(new URI(strValue)); + } + catch (URISyntaxException e) { + throw new IAE(StringUtils.format("Argument [%s] is not a valid URI", value)); + } + } + return uris; + } + + @Override + public void validate(ResolvedTable table) + { + super.validate(table); + + // Validate the HTTP properties only if we don't have a template. + // If we do have a template, then we don't know how to form + // a valid parameter for that template. + // TODO: plug in a dummy URL so we can validate other properties. + if (!table.hasProperty(URI_TEMPLATE_PROPERTY)) { + convertSource(table); + } + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/InlineTableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/InlineTableDefn.java new file mode 100644 index 000000000000..703d1ae4a314 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/InlineTableDefn.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.table; + +import org.apache.druid.catalog.model.CatalogUtils; +import org.apache.druid.catalog.model.Properties.StringListPropertyDefn; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.table.ExternalTableDefn.FormattedExternalTableDefn; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.impl.InlineInputSource; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.utils.CollectionUtils; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Describes an inline table: one where the data is provided in the + * table spec as a series of text lines. + */ +public class InlineTableDefn extends FormattedExternalTableDefn +{ + public static final String TABLE_TYPE = InlineInputSource.TYPE_KEY; + public static final String DATA_PROPERTY = "data"; + + public InlineTableDefn() + { + super( + "Inline input table", + TABLE_TYPE, + Collections.singletonList( + new StringListPropertyDefn(DATA_PROPERTY) + ), + Collections.singletonList(INPUT_COLUMN_DEFN), + InputFormats.ALL_FORMATS, + null + ); + } + + @Override + protected InputSource convertSource(ResolvedTable table) + { + Map jsonMap = new HashMap<>(); + jsonMap.put(InputSource.TYPE_PROPERTY, InlineInputSource.TYPE_KEY); + List dataList = table.stringListProperty(DATA_PROPERTY); + + // Would be nice, from a completeness perspective, for the inline data + // source to allow zero rows of data. However, such is not the case. + if (CollectionUtils.isNullOrEmpty(dataList)) { + throw new IAE( + "An inline table requires one or more rows of data in the '%s' property", + DATA_PROPERTY + ); + } + jsonMap.put("data", CatalogUtils.stringListToLines(dataList)); + return convertObject(table.jsonMapper(), jsonMap, InlineInputSource.class); + } + + @Override + public ResolvedTable mergeParameters(ResolvedTable spec, Map values) + { + throw new UOE("Inline table does not support parameters"); + } + + @Override + public void validate(ResolvedTable table) + { + super.validate(table); + convertSource(table); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/InputFormats.java b/server/src/main/java/org/apache/druid/catalog/model/table/InputFormats.java new file mode 100644 index 000000000000..658f1139c794 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/InputFormats.java @@ -0,0 +1,301 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.table; + +import org.apache.curator.shaded.com.google.common.collect.ImmutableList; +import org.apache.druid.catalog.model.CatalogUtils; +import org.apache.druid.catalog.model.Properties.BooleanPropertyDefn; +import org.apache.druid.catalog.model.Properties.IntPropertyDefn; +import org.apache.druid.catalog.model.Properties.PropertyDefn; +import org.apache.druid.catalog.model.Properties.SimplePropertyDefn; +import org.apache.druid.catalog.model.Properties.StringPropertyDefn; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DelimitedInputFormat; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Definition of the input formats which converts from property + * lists in table specs to subclasses of {@link InputFormat}. + */ +public class InputFormats +{ + public interface InputFormatDefn + { + String name(); + String typeTag(); + List properties(); + void validate(ResolvedTable table); + InputFormat convert(ResolvedTable table); + } + + public abstract static class BaseFormatDefn implements InputFormatDefn + { + private final String name; + private final String typeTag; + private final List properties; + + public BaseFormatDefn( + final String name, + final String typeTag, + final List properties + ) + { + this.name = name; + this.typeTag = typeTag; + this.properties = properties; + } + + @Override + public String name() + { + return name; + } + + @Override + public String typeTag() + { + return typeTag; + } + + @Override + public List properties() + { + return properties; + } + + @Override + public void validate(ResolvedTable table) + { + convert(table); + } + } + + /** + * Definition of a flat text (CSV and delimited text) input format. + *

+ * Note that not all the fields in + * {@link org.apache.druid.data.input.impl.FlatTextInputFormat + * FlatTextInputFormat} appear here: + *

    + *
  • {@code findColumnsFromHeader} - not yet supported in MSQ.
  • + *
  • {@code hasHeaderRow} - Always set to false since we don't bother to read + * it. {@code skipHeaderRows} is used to specify the number of header + * rows to skip.
  • + *
+ */ + public abstract static class FlatTextFormatDefn extends BaseFormatDefn + { + public static final String LIST_DELIMITER_PROPERTY = "listDelimiter"; + public static final String SKIP_ROWS_PROPERTY = "skipRows"; + + public FlatTextFormatDefn( + final String name, + final String typeTag, + final List properties + ) + { + super( + name, + typeTag, + CatalogUtils.concatLists( + Arrays.asList( + new StringPropertyDefn(LIST_DELIMITER_PROPERTY), + new IntPropertyDefn(SKIP_ROWS_PROPERTY) + ), + properties + ) + ); + } + + protected Map gatherFields(ResolvedTable table) + { + Map jsonMap = new HashMap<>(); + jsonMap.put(InputFormat.TYPE_PROPERTY, CsvInputFormat.TYPE_KEY); + jsonMap.put("listDelimiter", table.property(LIST_DELIMITER_PROPERTY)); + // hasHeaderRow is required, even though we don't infer headers. + jsonMap.put("hasHeaderRow", false); + jsonMap.put("findColumnsFromHeader", false); + // Column list is required. Infer from schema. + List cols = table.spec().columns() + .stream() + .map(col -> col.name()) + .collect(Collectors.toList()); + jsonMap.put("columns", cols); + Object value = table.property(SKIP_ROWS_PROPERTY); + jsonMap.put("skipHeaderRows", value == null ? 0 : value); + return jsonMap; + } + } + + public static final String CSV_FORMAT_TYPE = CsvInputFormat.TYPE_KEY; + + public static class CsvFormatDefn extends FlatTextFormatDefn + { + public CsvFormatDefn() + { + super( + "CSV", + CSV_FORMAT_TYPE, + null + ); + } + + @Override + protected Map gatherFields(ResolvedTable table) + { + Map jsonMap = super.gatherFields(table); + jsonMap.put(InputFormat.TYPE_PROPERTY, CsvInputFormat.TYPE_KEY); + return jsonMap; + } + + @Override + public InputFormat convert(ResolvedTable table) + { + try { + return table.jsonMapper().convertValue(gatherFields(table), CsvInputFormat.class); + } + catch (Exception e) { + throw new IAE(e, "Invalid format specification"); + } + } + } + + public static final String DELIMITED_FORMAT_TYPE = DelimitedInputFormat.TYPE_KEY; + + public static class DelimitedFormatDefn extends FlatTextFormatDefn + { + public static final String DELIMITER_PROPERTY = "delimiter"; + + public DelimitedFormatDefn() + { + super( + "Delimited Text", + DELIMITED_FORMAT_TYPE, + Collections.singletonList( + new StringPropertyDefn(DELIMITER_PROPERTY) + ) + ); + } + + @Override + protected Map gatherFields(ResolvedTable table) + { + Map jsonMap = super.gatherFields(table); + jsonMap.put(InputFormat.TYPE_PROPERTY, DelimitedInputFormat.TYPE_KEY); + Object value = table.property(DELIMITER_PROPERTY); + if (value != null) { + jsonMap.put("delimiter", value); + } + return jsonMap; + } + + @Override + public InputFormat convert(ResolvedTable table) + { + return table.jsonMapper().convertValue(gatherFields(table), DelimitedInputFormat.class); + } + } + + public static final String JSON_FORMAT_TYPE = JsonInputFormat.TYPE_KEY; + + public static class JsonFormatDefn extends BaseFormatDefn + { + public static final String KEEP_NULLS_PROPERTY = "keepNulls"; + + public JsonFormatDefn() + { + super( + "JSON", + JSON_FORMAT_TYPE, + Collections.singletonList( + new BooleanPropertyDefn(KEEP_NULLS_PROPERTY) + ) + ); + } + + @Override + public InputFormat convert(ResolvedTable table) + { + // TODO flatten & feature specs + Map jsonMap = new HashMap<>(); + jsonMap.put(InputFormat.TYPE_PROPERTY, JsonInputFormat.TYPE_KEY); + jsonMap.put("keepNullColumns", table.property(KEEP_NULLS_PROPERTY)); + return table.jsonMapper().convertValue(jsonMap, JsonInputFormat.class); + } + } + + /** + * Generic format which allows a literal input spec. Allows the user to + * specify any input format and any options directly as JSON. The + * drawback is that the user must repeat the columns. + */ + public static class GenericFormatDefn extends BaseFormatDefn + { + public static final String INPUT_FORMAT_SPEC_PROPERTY = "inputFormatSpec"; + public static final String FORMAT_KEY = "generic"; + + public GenericFormatDefn() + { + super( + "Generic", + FORMAT_KEY, + Collections.singletonList( + new SimplePropertyDefn(INPUT_FORMAT_SPEC_PROPERTY, InputFormat.class) + ) + ); + } + + @Override + public InputFormat convert(ResolvedTable table) + { + Object value = table.property(INPUT_FORMAT_SPEC_PROPERTY); + if (value == null) { + throw new ISE( + "An input format must be provided in the %s property when input type is %s", + INPUT_FORMAT_SPEC_PROPERTY, + name() + ); + } + return table.jsonMapper().convertValue(value, InputFormat.class); + } + } + + public static final InputFormatDefn CSV_FORMAT_DEFN = new CsvFormatDefn(); + public static final InputFormatDefn DELIMITED_FORMAT_DEFN = new DelimitedFormatDefn(); + public static final InputFormatDefn JSON_FORMAT_DEFN = new JsonFormatDefn(); + public static final GenericFormatDefn GENERIC_FORMAT_DEFN = new GenericFormatDefn(); + public static final List ALL_FORMATS = ImmutableList.of( + CSV_FORMAT_DEFN, + DELIMITED_FORMAT_DEFN, + JSON_FORMAT_DEFN, + GENERIC_FORMAT_DEFN + ); +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/LocalTableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/LocalTableDefn.java new file mode 100644 index 000000000000..3877c977d81d --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/LocalTableDefn.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.table; + +import org.apache.druid.catalog.model.CatalogUtils; +import org.apache.druid.catalog.model.Parameterized; +import org.apache.druid.catalog.model.Properties.StringListPropertyDefn; +import org.apache.druid.catalog.model.Properties.StringPropertyDefn; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.table.ExternalTableDefn.FormattedExternalTableDefn; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.utils.CollectionUtils; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class LocalTableDefn extends FormattedExternalTableDefn implements Parameterized +{ + public static final String TABLE_TYPE = LocalInputSource.TYPE_KEY; + + /** + * Base directory for file or filter operations. If not provided, + * then the servers current working directory is assumed, which is + * typically valid only for sample data. + */ + public static final String BASE_DIR_PROPERTY = "baseDir"; + + // Note name "fileFilter", not "filter". These properties mix in with + // others and "filter" is a bit too generic in that context. + public static final String FILE_FILTER_PROPERTY = "fileFilter"; + public static final String FILES_PROPERTY = "files"; + + public LocalTableDefn() + { + super( + "Local file input table", + TABLE_TYPE, + Arrays.asList( + new StringPropertyDefn(BASE_DIR_PROPERTY), + new StringPropertyDefn(FILE_FILTER_PROPERTY), + new StringListPropertyDefn(FILES_PROPERTY) + ), + Collections.singletonList(INPUT_COLUMN_DEFN), + InputFormats.ALL_FORMATS, + Arrays.asList( + new ParameterImpl(FILE_FILTER_PROPERTY, String.class), + new ParameterImpl(FILES_PROPERTY, String.class) + ) + ); + } + + @Override + public ResolvedTable mergeParameters(ResolvedTable table, Map values) + { + // The safe get can only check + String filesParam = CatalogUtils.safeGet(values, FILES_PROPERTY, String.class); + String filterParam = CatalogUtils.safeGet(values, FILE_FILTER_PROPERTY, String.class); + Map revisedProps = new HashMap<>(table.properties()); + if (filesParam != null) { + revisedProps.put(FILES_PROPERTY, CatalogUtils.stringToList(filesParam)); + } + if (filterParam != null) { + revisedProps.put(FILE_FILTER_PROPERTY, filterParam); + } + return table.withProperties(revisedProps); + } + + @Override + protected InputSource convertSource(ResolvedTable table) + { + Map jsonMap = new HashMap<>(); + jsonMap.put(InputSource.TYPE_PROPERTY, LocalInputSource.TYPE_KEY); + String baseDir = table.stringProperty(BASE_DIR_PROPERTY); + jsonMap.put("baseDir", baseDir); + List files = table.stringListProperty(FILES_PROPERTY); + jsonMap.put("files", files); + + // Note the odd semantics of this class. + // If we give a base directory, and explicitly state files, we must + // also provide a file filter which presumably matches the very files + // we list. Take pity on the user and provide a filter in this case. + String filter = table.stringProperty(FILE_FILTER_PROPERTY); + if (baseDir != null && !CollectionUtils.isNullOrEmpty(files) && filter == null) { + filter = "*"; + } + jsonMap.put("filter", filter); + return convertObject(table.jsonMapper(), jsonMap, LocalInputSource.class); + } + + @Override + public void validate(ResolvedTable table) + { + super.validate(table); + formatDefn(table).validate(table); + + // Validate the source if it is complete enough; else we need + // parameters later. + if (table.hasProperty(BASE_DIR_PROPERTY) || table.hasProperty(FILES_PROPERTY)) { + convertSource(table); + } + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/MeasureTypes.java b/server/src/main/java/org/apache/druid/catalog/model/table/MeasureTypes.java new file mode 100644 index 000000000000..e40749ec431e --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/MeasureTypes.java @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.table; + +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.column.ColumnType; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +public class MeasureTypes +{ + public static final String OPTIONAL = "--"; + + public enum BaseType + { + VARCHAR(ColumnType.STRING), + BIGINT(ColumnType.LONG), + FLOAT(ColumnType.FLOAT), + DOUBLE(ColumnType.DOUBLE); + + public final ColumnType nativeType; + + BaseType(ColumnType nativeType) + { + this.nativeType = nativeType; + } + } + + public static class MeasureType + { + public final String name; + public final List argTypes; + public final String sqlSeedFn; + public final String sqlReducerFn; + public final String nativeType; + public final String nativeAggFn; + public final ColumnType storageType; + public final String nativeReducerFn; + + public MeasureType( + final String name, + final List argTypes, + final String sqlSeedFn, + final String sqlReducerFn, + final String nativeType, + final ColumnType storageType, + final String nativeAggFn, + final String nativeReducerFn + ) + { + this.name = name; + this.argTypes = argTypes == null ? Collections.emptyList() : argTypes; + this.sqlSeedFn = sqlSeedFn; + this.sqlReducerFn = sqlReducerFn; + this.nativeType = nativeType; + this.storageType = storageType; + this.nativeAggFn = nativeAggFn; + this.nativeReducerFn = nativeReducerFn; + } + + @Override + public String toString() + { + StringBuilder buf = new StringBuilder() + .append(name) + .append("("); + for (int i = 0; i < argTypes.size(); i++) { + if (i > 0) { + buf.append(", "); + } + buf.append(argTypes.get(i).name()); + } + return buf.append(")").toString(); + } + } + + // See: https://druid.apache.org/docs/latest/querying/aggregations.html + public static final MeasureType COUNT_TYPE = new MeasureType( + "COUNT", + null, + null, + "SUM", + "longSum", + ColumnType.LONG, + "count", + "longSum" + ); + + public static final MeasureType SUM_BIGINT_TYPE = simpleAggType("sum", BaseType.BIGINT); + public static final MeasureType SUM_FLOAT_TYPE = simpleAggType("sum", BaseType.FLOAT); + public static final MeasureType SUM_DOUBLE_TYPE = simpleAggType("sum", BaseType.DOUBLE); + public static final MeasureType MIN_BIGINT_TYPE = simpleAggType("min", BaseType.BIGINT); + public static final MeasureType MIN_FLOAT_TYPE = simpleAggType("min", BaseType.FLOAT); + public static final MeasureType MIN_DOUBLE_TYPE = simpleAggType("min", BaseType.DOUBLE); + public static final MeasureType MAX_BIGINT_TYPE = simpleAggType("max", BaseType.BIGINT); + public static final MeasureType MAX_FLOAT_TYPE = simpleAggType("max", BaseType.FLOAT); + public static final MeasureType MAX_DOUBLE_TYPE = simpleAggType("max", BaseType.DOUBLE); + + private static MeasureType simpleAggType(String fn, BaseType baseType) + { + String sqlFn = StringUtils.toUpperCase(fn); + String nativeFn = baseType.nativeType.asTypeString() + org.apache.commons.lang3.StringUtils.capitalize(fn); + return new MeasureType( + sqlFn, + Collections.singletonList(baseType), + sqlFn, + null, + sqlFn, + baseType.nativeType, + nativeFn, + nativeFn + ); + } + + private static final List TYPE_LIST = + Arrays.asList( + COUNT_TYPE, + SUM_BIGINT_TYPE, + SUM_FLOAT_TYPE, + SUM_DOUBLE_TYPE, + MIN_BIGINT_TYPE, + MIN_FLOAT_TYPE, + MIN_DOUBLE_TYPE, + MAX_BIGINT_TYPE, + MAX_FLOAT_TYPE, + MAX_DOUBLE_TYPE + ); + public static final Map> TYPES; + + static { + Map> map = new HashMap<>(); + for (MeasureType fn : TYPE_LIST) { + List overloads = map.computeIfAbsent(fn.name, x -> new ArrayList<>()); + overloads.add(fn); + } + TYPES = ImmutableMap.>builder().putAll(map).build(); + } + + public static MeasureType parse(String typeStr) + { + Pattern p = Pattern.compile("(\\w+)(?:\\s*\\((.*)\\))?"); + Matcher m = p.matcher(StringUtils.toUpperCase(typeStr.trim())); + if (!m.matches()) { + throw new IAE(StringUtils.format( + "The type [%s] is not well-formed. It must be FN, FN(TYPE) or FN(TYPE,TYPE...)", + typeStr + )); + } + String fnName = m.group(1); + String[] args; + String argGroup = m.group(2); + argGroup = argGroup == null ? null : argGroup.trim(); + if (Strings.isNullOrEmpty(argGroup)) { + args = new String[] {}; + } else { + args = argGroup.trim().split("\\s*,\\s*"); + } + List candidates = TYPES.get(fnName); + if (candidates == null) { + throw new IAE(StringUtils.format( + "The metric type [%s] is not valid.", + fnName + )); + } + + top: + for (MeasureType type : candidates) { + if (type.argTypes.size() != args.length) { + continue; + } + for (int i = 0; i < args.length; i++) { + if (!type.argTypes.get(i).name().equalsIgnoreCase(args[i])) { + continue top; + } + } + return type; + } + + throw new IAE( + "[%s] is not a valid metric type. Valid forms are %s", + typeStr, + candidates.stream().map(t -> t.toString()).collect(Collectors.joining(", ")) + ); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java b/server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java new file mode 100644 index 000000000000..a3296443dfc4 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java @@ -0,0 +1,289 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.table; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import org.apache.druid.catalog.model.ColumnSpec; +import org.apache.druid.catalog.model.Columns; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableDefn; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.model.table.ExternalTableDefn.FormattedExternalTableDefn; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Informal table spec builder for tests. Takes the tedium out of building + * up property maps, column lists, and the various objects that define a + * table. Not useful outside of tests since Druid code should not create + * table objects except generically from specs provided by the user. + */ +public class TableBuilder +{ + private TableId id; + private TableDefn defn; + private String tableType; + private Map properties = new HashMap<>(); + private List columns = new ArrayList<>(); + + public static TableBuilder detailTable(String name, String granularity) + { + return new TableBuilder() + .datasource(name) + .type(DatasourceDefn.DETAIL_DATASOURCE_TYPE) + .segmentGranularity(granularity); + } + + public static TableBuilder rollupTable(String name, String granularity) + { + return new TableBuilder() + .datasource(name) + .type(DatasourceDefn.ROLLUP_DATASOURCE_TYPE) + .segmentGranularity(granularity); + } + + public static TableBuilder externalTable(String type, String name) + { + return new TableBuilder() + .external(name) + .type(type); + } + + public static TableBuilder updateFor(TableMetadata table) + { + return new TableBuilder() + .id(table.id()) + .type(table.spec().type()); + } + + public static TableBuilder copyOf(TableMetadata table) + { + return copyOf(table.id(), table.spec()); + } + + public static TableBuilder copyOf(TableId newId, TableSpec from) + { + return new TableBuilder() + .id(newId) + .type(from.type()) + .properties(new HashMap<>(from.properties())) + .columns(new ArrayList<>(from.columns())); + } + + public static TableBuilder of(TableDefn defn) + { + TableBuilder builder = new TableBuilder(); + builder.defn = defn; + builder.tableType = defn.typeValue(); + return builder; + } + + public TableBuilder copy() + { + TableBuilder builder = new TableBuilder(); + builder.defn = defn; + builder.tableType = tableType; + builder.id = id; + builder.properties.putAll(properties); + builder.columns.addAll(columns); + return builder; + } + + public TableBuilder id(TableId id) + { + this.id = id; + return this; + } + + public TableBuilder datasource(String name) + { + this.id = TableId.datasource(name); + return this; + } + + public TableBuilder external(String name) + { + this.id = TableId.of(TableId.EXTERNAL_SCHEMA, name); + return this; + } + + public TableBuilder path(String schema, String name) + { + this.id = TableId.of(schema, name); + return this; + } + + public TableBuilder type(String tableType) + { + this.tableType = tableType; + return this; + } + + public TableBuilder properties(Map properties) + { + this.properties = properties; + return this; + } + + public Map properties() + { + return properties; + } + + public TableBuilder property(String key, Object value) + { + this.properties.put(key, value); + return this; + } + + public TableBuilder description(String description) + { + return property(TableDefn.DESCRIPTION_PROPERTY, description); + } + + public TableBuilder rollupGranularity(String rollupGranularty) + { + return property(DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY, rollupGranularty); + } + + public TableBuilder segmentGranularity(String segmentGranularity) + { + return property(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, segmentGranularity); + } + + public TableBuilder clusterColumns(ClusterKeySpec...clusterKeys) + { + return property(DatasourceDefn.CLUSTER_KEYS_PROPERTY, Arrays.asList(clusterKeys)); + } + + public TableBuilder columns(List columns) + { + this.columns = columns; + return this; + } + + public List columns() + { + return columns; + } + + public TableBuilder column(ColumnSpec column) + { + if (Strings.isNullOrEmpty(column.name())) { + throw new IAE("Column name is required"); + } + columns.add(column); + return this; + } + + public TableBuilder timeColumn() + { + return column(Columns.TIME_COLUMN, Columns.TIMESTAMP); + } + + public TableBuilder column(String name, String sqlType) + { + Preconditions.checkNotNull(tableType); + String colType; + if (isInputTable(tableType)) { + colType = ExternalTableDefn.EXTERNAL_COLUMN_TYPE; + } else if (DatasourceDefn.DETAIL_DATASOURCE_TYPE.equals(tableType)) { + colType = DatasourceDefn.DETAIL_COLUMN_TYPE; + } else if (DatasourceDefn.ROLLUP_DATASOURCE_TYPE.equals(tableType)) { + colType = DatasourceDefn.DIMENSION_TYPE; + } else { + throw new ISE("Unknown table type: %s", tableType); + } + return column(colType, name, sqlType); + } + + public static boolean isInputTable(String tableType) + { + switch (tableType) { + case InlineTableDefn.TABLE_TYPE: + case HttpTableDefn.TABLE_TYPE: + case LocalTableDefn.TABLE_TYPE: + return true; + default: + return false; + } + } + + public TableBuilder column(String colType, String name, String sqlType) + { + return column(new ColumnSpec(colType, name, sqlType, null)); + } + + public TableBuilder measure(String name, String sqlType) + { + return column(DatasourceDefn.MEASURE_TYPE, name, sqlType); + } + + public TableBuilder hiddenColumns(List hiddenColumns) + { + return property(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, hiddenColumns); + } + + public TableBuilder hiddenColumns(String...hiddenColumns) + { + return hiddenColumns(Arrays.asList(hiddenColumns)); + } + + public TableBuilder format(String format) + { + return property(FormattedExternalTableDefn.FORMAT_PROPERTY, format); + } + + public TableBuilder data(List data) + { + return property(InlineTableDefn.DATA_PROPERTY, data); + } + + public TableBuilder data(String...data) + { + return data(Arrays.asList(data)); + } + + public TableSpec buildSpec() + { + return new TableSpec(tableType, properties, columns); + } + + public TableMetadata build() + { + return TableMetadata.newTable(id, buildSpec()); + } + + public ResolvedTable buildResolved(ObjectMapper mapper) + { + Preconditions.checkNotNull(defn); + return new ResolvedTable(defn, buildSpec(), mapper); + } +} diff --git a/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyMetadataStorage.java b/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyMetadataStorage.java index 725c531626ca..c79e62c75b5b 100644 --- a/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyMetadataStorage.java +++ b/server/src/main/java/org/apache/druid/metadata/storage/derby/DerbyMetadataStorage.java @@ -42,7 +42,6 @@ public DerbyMetadataStorage(MetadataStorageConnectorConfig config) catch (Exception e) { throw new RuntimeException(e); } - } @Override @@ -55,6 +54,26 @@ public void start() catch (Exception e) { throw new RuntimeException(e); } + + // It takes a while for the Derby server to start in another + // thread. Ping to ensure it is ready. Saves ugly failure/retry + // loops elsewhere in startup. Those loops look alarming in the + // log file. + while (true) { + try { + server.ping(); + break; + } + catch (Exception e) { + log.info("Derby server not yet ready, still trying..."); + try { + Thread.sleep(100); + } + catch (InterruptedException e1) { + // Ignore + } + } + } } @Override diff --git a/server/src/main/java/org/apache/druid/server/security/Access.java b/server/src/main/java/org/apache/druid/server/security/Access.java index 543ce1b0d734..1f5f5f5e0269 100644 --- a/server/src/main/java/org/apache/druid/server/security/Access.java +++ b/server/src/main/java/org/apache/druid/server/security/Access.java @@ -27,6 +27,7 @@ public class Access static final String DEFAULT_ERROR_MESSAGE = "Unauthorized"; public static final Access OK = new Access(true); + public static final Access DENIED = new Access(false); private final boolean allowed; private final String message; diff --git a/server/src/test/java/org/apache/druid/catalog/CatalogTest.java b/server/src/test/java/org/apache/druid/catalog/CatalogTest.java new file mode 100644 index 000000000000..498a82c85906 --- /dev/null +++ b/server/src/test/java/org/apache/druid/catalog/CatalogTest.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog; + +/** + * JUnit category for Catalog tests. + */ +public class CatalogTest +{ +} diff --git a/server/src/test/java/org/apache/druid/catalog/model/MeasureTypesTest.java b/server/src/test/java/org/apache/druid/catalog/model/MeasureTypesTest.java new file mode 100644 index 000000000000..762c1a6e3dda --- /dev/null +++ b/server/src/test/java/org/apache/druid/catalog/model/MeasureTypesTest.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import org.apache.druid.catalog.CatalogTest; +import org.apache.druid.catalog.model.table.MeasureTypes; +import org.apache.druid.catalog.model.table.MeasureTypes.MeasureType; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.Arrays; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThrows; + +@Category(CatalogTest.class) +public class MeasureTypesTest +{ + @Test + public void testInvalid() + { + assertThrows(IAE.class, () -> MeasureTypes.parse("")); + assertThrows(IAE.class, () -> MeasureTypes.parse("FOO")); + assertThrows(IAE.class, () -> MeasureTypes.parse("FOO(")); + assertThrows(IAE.class, () -> MeasureTypes.parse("FOO)")); + assertThrows(IAE.class, () -> MeasureTypes.parse("MIN(,,)")); + assertThrows(IAE.class, () -> MeasureTypes.parse("MIN(VARCHAR")); + } + + @Test + public void testCount() + { + MeasureType type = MeasureTypes.parse("count"); + assertSame(MeasureTypes.COUNT_TYPE, type); + assertEquals(0, type.argTypes.size()); + + type = MeasureTypes.parse("COUNT()"); + assertSame(MeasureTypes.COUNT_TYPE, type); + assertEquals(0, type.argTypes.size()); + + type = MeasureTypes.parse(" COUNT( ) "); + assertSame(MeasureTypes.COUNT_TYPE, type); + assertEquals(0, type.argTypes.size()); + } + + @Test + public void testSingleArg() + { + testOneArg("SUM"); + testOneArg("MIN"); + testOneArg("MAX"); + + // Invalid + + assertThrows(IAE.class, () -> MeasureTypes.parse("SUM(VARCHAR)")); + assertThrows(IAE.class, () -> MeasureTypes.parse("SUM()")); + assertThrows(IAE.class, () -> MeasureTypes.parse("SUM(BIGINT, BIGINT)")); + + // Parsing variations + MeasureType typeRef = MeasureTypes.parse(" Min ( BiGiNt ) "); + assertSame(MeasureTypes.MIN_BIGINT_TYPE, typeRef); + } + + private void testOneArg(String fn) + { + List types = MeasureTypes.TYPES.get(fn); + for (String name : Arrays.asList(fn, StringUtils.toLowerCase(fn))) { + for (MeasureType measureType : types) { + String argType = measureType.argTypes.get(0).name(); + for (String argName : Arrays.asList(argType, StringUtils.toLowerCase(argType))) { + MeasureType typeRef = MeasureTypes.parse(StringUtils.format("%s(%s)", name, argName)); + assertSame(measureType, typeRef); + } + } + } + } +} diff --git a/server/src/test/java/org/apache/druid/catalog/model/PropertyDefnTest.java b/server/src/test/java/org/apache/druid/catalog/model/PropertyDefnTest.java new file mode 100644 index 000000000000..794175517c44 --- /dev/null +++ b/server/src/test/java/org/apache/druid/catalog/model/PropertyDefnTest.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.catalog.CatalogTest; +import org.apache.druid.catalog.model.Properties.BooleanPropertyDefn; +import org.apache.druid.catalog.model.Properties.IntPropertyDefn; +import org.apache.druid.catalog.model.Properties.ListPropertyDefn; +import org.apache.druid.catalog.model.Properties.StringListPropertyDefn; +import org.apache.druid.catalog.model.Properties.StringPropertyDefn; +import org.apache.druid.catalog.model.table.ClusterKeySpec; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +@Category(CatalogTest.class) +public class PropertyDefnTest +{ + private final ObjectMapper mapper = new ObjectMapper(); + + @Test + public void testString() + { + StringPropertyDefn prop = new StringPropertyDefn("prop"); + assertEquals("prop", prop.name()); + assertEquals("String", prop.typeName()); + + assertNull(prop.decode(null, mapper)); + assertEquals("value", prop.decode("value", mapper)); + prop.validate("value", mapper); + + // Jackson is permissive in its conversions + assertEquals("10", prop.decode(10, mapper)); + prop.validate(10, mapper); + + // But, it does have its limits. + assertThrows(Exception.class, () -> prop.decode(Arrays.asList("a", "b"), mapper)); + assertThrows(Exception.class, () -> prop.validate(Arrays.asList("a", "b"), mapper)); + } + + @Test + public void testBoolean() + { + BooleanPropertyDefn prop = new BooleanPropertyDefn("prop"); + assertEquals("prop", prop.name()); + assertEquals("Boolean", prop.typeName()); + + assertNull(prop.decode(null, mapper)); + assertTrue(prop.decode("true", mapper)); + assertTrue(prop.decode(true, mapper)); + assertFalse(prop.decode("false", mapper)); + assertFalse(prop.decode(false, mapper)); + assertFalse(prop.decode(0, mapper)); + assertTrue(prop.decode(10, mapper)); + } + + @Test + public void testInt() + { + IntPropertyDefn prop = new IntPropertyDefn("prop"); + assertEquals("prop", prop.name()); + assertEquals("Integer", prop.typeName()); + + assertNull(prop.decode(null, mapper)); + assertEquals((Integer) 0, prop.decode(0, mapper)); + assertEquals((Integer) 0, prop.decode("0", mapper)); + assertEquals((Integer) 10, prop.decode(10, mapper)); + assertEquals((Integer) 10, prop.decode("10", mapper)); + assertThrows(Exception.class, () -> prop.decode("foo", mapper)); + } + + @Test + public void testStringList() + { + StringListPropertyDefn prop = new StringListPropertyDefn("prop"); + assertEquals("prop", prop.name()); + assertEquals("string list", prop.typeName()); + + assertNull(prop.decode(null, mapper)); + prop.validate(null, mapper); + List value = Arrays.asList("a", "b"); + assertEquals(value, prop.decode(value, mapper)); + prop.validate(value, mapper); + assertThrows(Exception.class, () -> prop.decode("foo", mapper)); + assertThrows(Exception.class, () -> prop.validate("foo", mapper)); + } + + @Test + public void testClusterKeyList() + { + ListPropertyDefn prop = new ListPropertyDefn( + "prop", + "cluster key list", + new TypeReference>() { } + ); + assertEquals("prop", prop.name()); + assertEquals("cluster key list", prop.typeName()); + + assertNull(prop.decode(null, mapper)); + List> value = Arrays.asList( + ImmutableMap.of("column", "a"), + ImmutableMap.of("column", "b", "desc", true) + ); + List expected = Arrays.asList( + new ClusterKeySpec("a", false), + new ClusterKeySpec("b", true) + ); + assertEquals(expected, prop.decode(value, mapper)); + assertThrows(Exception.class, () -> prop.decode("foo", mapper)); + } +} diff --git a/server/src/test/java/org/apache/druid/catalog/model/TableMetadataTest.java b/server/src/test/java/org/apache/druid/catalog/model/TableMetadataTest.java new file mode 100644 index 000000000000..af79bd314d7a --- /dev/null +++ b/server/src/test/java/org/apache/druid/catalog/model/TableMetadataTest.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.catalog.CatalogTest; +import org.apache.druid.catalog.model.TableMetadata.TableState; +import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.java.util.common.IAE; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertThrows; + +@Category(CatalogTest.class) +public class TableMetadataTest +{ + private ObjectMapper mapper = new ObjectMapper(); + + @Test + public void testId() + { + TableId id1 = new TableId("schema", "table"); + assertEquals(id1, id1); + assertEquals("schema", id1.schema()); + assertEquals("table", id1.name()); + assertEquals("\"schema\".\"table\"", id1.sqlName()); + assertEquals(id1.sqlName(), id1.toString()); + + TableId id2 = TableId.datasource("ds"); + assertEquals(TableId.DRUID_SCHEMA, id2.schema()); + assertEquals("ds", id2.name()); + + byte[] bytes = CatalogUtils.toBytes(mapper, id1); + TableId id3 = CatalogUtils.fromBytes(mapper, bytes, TableId.class); + assertEquals(id1, id3); + } + + @Test + public void testIdEquals() + { + EqualsVerifier.forClass(TableId.class) + .usingGetClass() + .verify(); + } + + @Test + public void testTableMetadata() + { + Map props = ImmutableMap.of( + DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D" + ); + TableSpec spec = new TableSpec(DatasourceDefn.DETAIL_DATASOURCE_TYPE, props, null); + { + TableMetadata table = new TableMetadata( + TableId.datasource("foo"), + 10, + 20, + TableState.ACTIVE, + spec + ); + table.validate(); + assertEquals(TableId.DRUID_SCHEMA, table.id().schema()); + assertEquals("foo", table.id().name()); + assertEquals(10, table.creationTime()); + assertEquals(20, table.updateTime()); + assertEquals(TableState.ACTIVE, table.state()); + assertNotNull(table.spec()); + + byte[] bytes = CatalogUtils.toBytes(mapper, table); + TableMetadata table2 = CatalogUtils.fromBytes(mapper, bytes, TableMetadata.class); + assertEquals(table, table2); + } + + { + TableMetadata table = TableMetadata.newTable( + TableId.of(null, "foo"), + spec + ); + assertThrows(IAE.class, () -> table.validate()); + } + + { + TableMetadata table = TableMetadata.newTable( + TableId.of(TableId.DRUID_SCHEMA, null), + spec + ); + assertThrows(IAE.class, () -> table.validate()); + } + } + @Test + public void testConversions() + { + Map props = ImmutableMap.of( + DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D" + ); + TableSpec spec = new TableSpec(DatasourceDefn.DETAIL_DATASOURCE_TYPE, props, null); + TableMetadata table = TableMetadata.newTable( + TableId.datasource("ds"), + spec + ); + assertEquals(TableId.datasource("ds"), table.id()); + assertEquals(TableState.ACTIVE, table.state()); + assertEquals(0, table.updateTime()); + assertSame(spec, table.spec()); + + TableMetadata table2 = TableMetadata.newTable( + TableId.datasource("ds"), + spec + ); + assertEquals(table, table2); + + TableMetadata table3 = table2.fromInsert(10); + assertEquals(10, table3.creationTime()); + assertEquals(10, table3.updateTime()); + + table3 = table3.asUpdate(20); + assertEquals(10, table3.creationTime()); + assertEquals(20, table3.updateTime()); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(TableMetadata.class) + .usingGetClass() + .verify(); + } +} diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/DatasourceTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/DatasourceTableTest.java new file mode 100644 index 000000000000..9dfeef3da2e5 --- /dev/null +++ b/server/src/test/java/org/apache/druid/catalog/model/table/DatasourceTableTest.java @@ -0,0 +1,672 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.table; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.catalog.CatalogTest; +import org.apache.druid.catalog.model.ColumnSpec; +import org.apache.druid.catalog.model.Columns; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableDefn; +import org.apache.druid.catalog.model.TableDefnRegistry; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.model.table.DatasourceDefn.DetailDatasourceDefn; +import org.apache.druid.catalog.model.table.DatasourceDefn.RollupDatasourceDefn; +import org.apache.druid.java.util.common.IAE; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +/** + * Test of validation and serialization of the catalog table definitions. + */ +@Category(CatalogTest.class) +public class DatasourceTableTest +{ + private static final String SUM_BIGINT = "SUM(BIGINT)"; + + private final ObjectMapper mapper = new ObjectMapper(); + private final TableDefnRegistry registry = new TableDefnRegistry(mapper); + + @Test + public void testMinimalSpec() + { + // Minimum possible definition + Map props = ImmutableMap.of( + DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D" + ); + { + TableSpec spec = new TableSpec(DatasourceDefn.DETAIL_DATASOURCE_TYPE, props, null); + ResolvedTable table = registry.resolve(spec); + assertNotNull(table); + assertTrue(table.defn() instanceof DetailDatasourceDefn); + table.validate(); + } + + { + TableSpec spec = new TableSpec(DatasourceDefn.ROLLUP_DATASOURCE_TYPE, props, null); + ResolvedTable table = registry.resolve(spec); + assertNotNull(table); + assertTrue(table.defn() instanceof RollupDatasourceDefn); + table.validate(); + } + } + + private void expectValidationFails(final ResolvedTable table) + { + assertThrows(IAE.class, () -> table.validate()); + } + + private void expectValidationFails(final TableSpec spec) + { + ResolvedTable table = registry.resolve(spec); + expectValidationFails(table); + } + + private void expectValidationSucceeds(final TableSpec spec) + { + ResolvedTable table = registry.resolve(spec); + table.validate(); + } + + @Test + public void testEmptySpec() + { + { + TableSpec spec = new TableSpec(null, ImmutableMap.of(), null); + assertThrows(IAE.class, () -> registry.resolve(spec)); + } + + { + TableSpec spec = new TableSpec(DatasourceDefn.DETAIL_DATASOURCE_TYPE, ImmutableMap.of(), null); + ResolvedTable table = registry.resolve(spec); + expectValidationFails(table); + } + + { + TableSpec spec = new TableSpec(DatasourceDefn.ROLLUP_DATASOURCE_TYPE, ImmutableMap.of(), null); + expectValidationFails(spec); + } + } + + @Test + public void testAllProperties() + { + Map props = ImmutableMap.builder() + .put(TableDefn.DESCRIPTION_PROPERTY, "My table") + .put(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D") + .put(DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY, "PT1M") + .put(DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000) + .put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("foo", "bar")) + .build(); + + { + TableSpec spec = new TableSpec(DatasourceDefn.DETAIL_DATASOURCE_TYPE, props, null); + expectValidationSucceeds(spec); + + // Check serialization + byte[] bytes = spec.toBytes(mapper); + assertEquals(spec, TableSpec.fromBytes(mapper, bytes)); + } + + { + TableSpec spec = new TableSpec(DatasourceDefn.ROLLUP_DATASOURCE_TYPE, props, null); + expectValidationSucceeds(spec); + + // Check serialization + byte[] bytes = spec.toBytes(mapper); + assertEquals(spec, TableSpec.fromBytes(mapper, bytes)); + } + } + + @Test + public void testWrongTypes() + { + { + TableSpec spec = new TableSpec("bogus", ImmutableMap.of(), null); + assertThrows(IAE.class, () -> registry.resolve(spec)); + } + + // Segment granularity + { + TableSpec spec = TableBuilder.detailTable("foo", "bogus").buildSpec(); + expectValidationFails(spec); + } + + { + TableSpec spec = TableBuilder.rollupTable("foo", "bogus").buildSpec(); + expectValidationFails(spec); + } + + // Rollup granularity + { + TableSpec spec = TableBuilder.rollupTable("foo", "P1D") + .rollupGranularity("bogus") + .buildSpec(); + expectValidationFails(spec); + } + + { + TableSpec spec = TableBuilder.rollupTable("foo", "P1D") + .property(DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY, 10) + .buildSpec(); + expectValidationFails(spec); + } + + // Target segment rows + { + TableSpec spec = TableBuilder.detailTable("foo", "P1D") + .property(DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, "bogus") + .buildSpec(); + expectValidationFails(spec); + } + + // Hidden columns + { + TableSpec spec = TableBuilder.detailTable("foo", "P1D") + .property(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, "bogus") + .buildSpec(); + expectValidationFails(spec); + } + { + TableSpec spec = TableBuilder.detailTable("foo", "P1D") + .hiddenColumns("a", Columns.TIME_COLUMN) + .buildSpec(); + expectValidationFails(spec); + } + } + + @Test + public void testExtendedProperties() + { + TableSpec spec = TableBuilder.detailTable("foo", "P1D") + .property("foo", 10) + .property("bar", "mumble") + .buildSpec(); + expectValidationSucceeds(spec); + } + + @Test + public void testColumnSpec() + { + // Type is required + { + ColumnSpec spec = new ColumnSpec(null, null, null, null); + assertThrows(IAE.class, () -> spec.validate()); + } + + // Name is required + { + ColumnSpec spec = new ColumnSpec(DatasourceDefn.DETAIL_COLUMN_TYPE, null, null, null); + assertThrows(IAE.class, () -> spec.validate()); + } + { + ColumnSpec spec = new ColumnSpec(DatasourceDefn.DETAIL_COLUMN_TYPE, "foo", null, null); + spec.validate(); + } + + // Type is optional + { + ColumnSpec spec = new ColumnSpec(DatasourceDefn.DETAIL_COLUMN_TYPE, "foo", "VARCHAR", null); + spec.validate(); + } + } + + @Test + public void testDetailTableColumns() + { + TableBuilder builder = TableBuilder.detailTable("foo", "P1D"); + + // OK to have no columns + { + TableSpec spec = builder.copy() + .buildSpec(); + expectValidationSucceeds(spec); + } + + // OK to have no column type + { + TableSpec spec = builder.copy() + .column("foo", null) + .buildSpec(); + expectValidationSucceeds(spec); + } + + // Time column can have no type + { + TableSpec spec = builder.copy() + .column(Columns.TIME_COLUMN, null) + .buildSpec(); + expectValidationSucceeds(spec); + } + + // Time column can only have TIMESTAMP type + { + TableSpec spec = builder.copy() + .timeColumn() + .buildSpec(); + expectValidationSucceeds(spec); + } + { + TableSpec spec = builder.copy() + .column(Columns.TIME_COLUMN, Columns.VARCHAR) + .buildSpec(); + expectValidationFails(spec); + } + + // Can have a legal scalar type + { + TableSpec spec = builder.copy() + .column("foo", Columns.VARCHAR) + .buildSpec(); + expectValidationSucceeds(spec); + } + + // Reject an unknown SQL type + { + TableSpec spec = builder.copy() + .column("foo", "BOGUS") + .buildSpec(); + expectValidationFails(spec); + } + + // Cannot use a measure type + { + TableSpec spec = builder.copy() + .column("foo", SUM_BIGINT) + .buildSpec(); + expectValidationFails(spec); + } + + // Cannot use a measure + { + TableSpec spec = builder.copy() + .measure("foo", SUM_BIGINT) + .buildSpec(); + expectValidationFails(spec); + } + + // Cannot use a dimension for a detail table + { + TableSpec spec = builder.copy() + .column(new ColumnSpec(DatasourceDefn.DIMENSION_TYPE, "foo", Columns.VARCHAR, null)) + .buildSpec(); + expectValidationFails(spec); + } + + // Reject duplicate columns + { + TableSpec spec = builder.copy() + .column("foo", Columns.VARCHAR) + .column("bar", Columns.BIGINT) + .buildSpec(); + expectValidationSucceeds(spec); + } + { + TableSpec spec = builder.copy() + .column("foo", Columns.VARCHAR) + .column("foo", Columns.BIGINT) + .buildSpec(); + expectValidationFails(spec); + } + } + + @Test + public void testRollupTableColumns() + { + TableBuilder builder = TableBuilder.rollupTable("foo", "P1D") + .rollupGranularity("PT1M"); + + // OK to have no columns + { + TableSpec spec = builder.buildSpec(); + expectValidationSucceeds(spec); + } + + // OK for a dimension to have no type + { + TableSpec spec = builder.copy() + .column("foo", null) + .buildSpec(); + expectValidationSucceeds(spec); + } + + // Dimensions must have a scalar type, if the type is non-null + { + TableSpec spec = builder.copy() + .column("foo", Columns.VARCHAR) + .buildSpec(); + expectValidationSucceeds(spec); + } + { + TableSpec spec = builder.copy() + .column("foo", "BOGUS") + .buildSpec(); + expectValidationFails(spec); + } + { + TableSpec spec = builder.copy() + .column("foo", SUM_BIGINT) + .buildSpec(); + expectValidationFails(spec); + } + + // Time column can be a dimension and can only have TIMESTAMP type + { + TableSpec spec = builder.copy() + .timeColumn() + .buildSpec(); + expectValidationSucceeds(spec); + } + { + TableSpec spec = builder.copy() + .column(Columns.TIME_COLUMN, Columns.VARCHAR) + .buildSpec(); + expectValidationFails(spec); + } + { + TableSpec spec = builder.copy() + .column(Columns.TIME_COLUMN, SUM_BIGINT) + .buildSpec(); + expectValidationFails(spec); + } + + // Measures must have an aggregate type + { + TableSpec spec = builder.copy() + .measure("foo", null) + .buildSpec(); + expectValidationFails(spec); + } + { + TableSpec spec = builder.copy() + .measure("foo", Columns.VARCHAR) + .buildSpec(); + expectValidationFails(spec); + } + { + TableSpec spec = builder.copy() + .measure("foo", SUM_BIGINT) + .buildSpec(); + expectValidationSucceeds(spec); + } + + // Cannot use a detail column + { + TableSpec spec = builder.copy() + .column(new ColumnSpec(DatasourceDefn.DETAIL_COLUMN_TYPE, "foo", null, null)) + .buildSpec(); + expectValidationFails(spec); + } + + // Reject duplicate columns + { + TableSpec spec = builder.copy() + .column("foo", Columns.VARCHAR) + .measure("bar", SUM_BIGINT) + .buildSpec(); + expectValidationSucceeds(spec); + } + { + TableSpec spec = builder.copy() + .column("foo", Columns.VARCHAR) + .measure("foo", SUM_BIGINT) + .buildSpec(); + expectValidationFails(spec); + } + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(ColumnSpec.class) + .usingGetClass() + .verify(); + EqualsVerifier.forClass(TableSpec.class) + .usingGetClass() + .verify(); + } + + private TableSpec exampleSpec() + { + Map colProps = ImmutableMap.builder() + .put("colProp1", "value 1") + .put("colProp2", "value 2") + .build(); + TableSpec spec = TableBuilder.rollupTable("foo", "PT1H") + .description("My table") + .rollupGranularity("PT1M") + .property(DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000) + .hiddenColumns("foo", "bar") + .property("tag1", "some value") + .property("tag2", "second value") + .column(new ColumnSpec(DatasourceDefn.DIMENSION_TYPE, "a", null, colProps)) + .column("b", Columns.VARCHAR) + .buildSpec(); + + // Sanity check + expectValidationSucceeds(spec); + return spec; + } + + @Test + public void testSerialization() + { + TableSpec spec = exampleSpec(); + + // Round-trip + TableSpec spec2 = TableSpec.fromBytes(mapper, spec.toBytes(mapper)); + assertEquals(spec, spec2); + + // Sanity check of toString, which uses JSON + assertNotNull(spec.toString()); + } + + private TableSpec mergeTables(TableSpec spec, TableSpec update) + { + ResolvedTable table = registry.resolve(spec); + assertNotNull(table); + return table.merge(update).spec(); + } + + @Test + public void testMergeEmpty() + { + TableSpec spec = exampleSpec(); + TableSpec update = new TableSpec(null, null, null); + + TableSpec merged = mergeTables(spec, update); + assertEquals(spec, merged); + } + + private void assertMergeFails(TableSpec spec, TableSpec update) + { + assertThrows(IAE.class, () -> mergeTables(spec, update)); + } + + @Test + public void testMergeTableType() + { + TableSpec spec = exampleSpec(); + + // Null type test is above. + // Wrong type + TableSpec update = new TableSpec("bogus", null, null); + assertMergeFails(spec, update); + + // Same type + update = new TableSpec(spec.type(), null, null); + TableSpec merged = mergeTables(spec, update); + assertEquals(spec, merged); + } + + @Test + public void testMergeProperties() + { + TableSpec spec = exampleSpec(); + + // Use a regular map, not an immutable one, because immutable maps, + // in their infinite wisdom, don't allow null values. But, we need + // such values to indicate which properties to remove. + Map updatedProps = new HashMap<>(); + // Update a property + updatedProps.put(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D"); + // Remove a property + updatedProps.put("tag1", null); + // Add a property + updatedProps.put("tag3", "third value"); + + TableSpec update = new TableSpec(null, updatedProps, null); + TableSpec merged = mergeTables(spec, update); + expectValidationSucceeds(merged); + + // We know that an empty map will leave the spec unchanged + // due to testMergeEmpty. Here we verify those that we + // changed. + assertNotEquals(spec, merged); + assertEquals( + updatedProps.get(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY), + merged.properties().get(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY) + ); + assertFalse(merged.properties().containsKey("tag1")); + assertEquals( + updatedProps.get("tag3"), + merged.properties().get("tag3") + ); + } + + @Test + public void testMergeHiddenCols() + { + TableSpec spec = exampleSpec(); + + // Remove all hidden columns + Map updatedProps = new HashMap<>(); + updatedProps.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, null); + TableSpec update = new TableSpec(null, updatedProps, null); + TableSpec merged = mergeTables(spec, update); + expectValidationSucceeds(merged); + assertFalse( + merged.properties().containsKey(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + + // Wrong type + updatedProps = ImmutableMap.of( + DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, "mumble" + ); + update = new TableSpec(null, updatedProps, null); + assertMergeFails(spec, update); + + // Merge + updatedProps = ImmutableMap.of( + DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.singletonList("mumble") + ); + update = new TableSpec(null, updatedProps, null); + merged = mergeTables(spec, update); + expectValidationSucceeds(merged); + + assertEquals( + Arrays.asList("foo", "bar", "mumble"), + merged.properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + } + + @Test + public void testMergeColsWithEmptyList() + { + Map props = ImmutableMap.of( + DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D" + ); + TableSpec spec = new TableSpec(DatasourceDefn.DETAIL_DATASOURCE_TYPE, props, null); + + List colUpdates = Collections.singletonList( + new ColumnSpec( + DatasourceDefn.DETAIL_COLUMN_TYPE, + "a", + Columns.BIGINT, + null + ) + ); + TableSpec update = new TableSpec(null, null, colUpdates); + TableSpec merged = mergeTables(spec, update); + List columns = merged.columns(); + assertEquals(1, columns.size()); + assertEquals("a", columns.get(0).name()); + assertEquals(Columns.BIGINT, columns.get(0).sqlType()); + } + + @Test + public void testMergeCols() + { + TableSpec spec = exampleSpec(); + + Map updatedProps = new HashMap<>(); + // Update a property + updatedProps.put("colProp1", "new value"); + // Remove a property + updatedProps.put("colProp2", null); + // Add a property + updatedProps.put("tag3", "third value"); + + List colUpdates = Arrays.asList( + new ColumnSpec( + DatasourceDefn.DIMENSION_TYPE, + "a", + Columns.BIGINT, + updatedProps + ), + new ColumnSpec( + DatasourceDefn.DIMENSION_TYPE, + "c", + Columns.VARCHAR, + null + ) + ); + TableSpec update = new TableSpec(null, null, colUpdates); + TableSpec merged = mergeTables(spec, update); + + assertNotEquals(spec, merged); + List columns = merged.columns(); + assertEquals(3, columns.size()); + assertEquals("a", columns.get(0).name()); + assertEquals(Columns.BIGINT, columns.get(0).sqlType()); + Map colProps = columns.get(0).properties(); + assertEquals(2, colProps.size()); + assertEquals("new value", colProps.get("colProp1")); + assertEquals("third value", colProps.get("tag3")); + + assertEquals("c", columns.get(2).name()); + assertEquals(Columns.VARCHAR, columns.get(2).sqlType()); + } +} diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java new file mode 100644 index 000000000000..c90dac5c8a47 --- /dev/null +++ b/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.table; + +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.catalog.CatalogTest; +import org.apache.druid.catalog.model.Columns; +import org.apache.druid.catalog.model.Parameterized; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableDefnRegistry; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.HttpInputSource; +import org.apache.druid.data.input.impl.HttpInputSourceConfig; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.metadata.DefaultPasswordProvider; +import org.apache.druid.metadata.EnvironmentVariablePasswordProvider; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThrows; + +@Category(CatalogTest.class) +public class HttpInputTableTest +{ + private final ObjectMapper mapper = new ObjectMapper(); + private final HttpTableDefn tableDefn = new HttpTableDefn(); + private final TableBuilder baseBuilder = TableBuilder.of(tableDefn) + .description("http input") + .format(InputFormats.CSV_FORMAT_TYPE) + .column("x", Columns.VARCHAR) + .column("y", Columns.BIGINT); + + public HttpInputTableTest() + { + mapper.setInjectableValues(new InjectableValues.Std().addValue( + HttpInputSourceConfig.class, + new HttpInputSourceConfig(HttpInputSourceConfig.DEFAULT_ALLOWED_PROTOCOLS) + )); + } + + @Test + public void testHappyPath() + { + ResolvedTable table = baseBuilder.copy() + .property(HttpTableDefn.USER_PROPERTY, "bob") + .property(HttpTableDefn.PASSWORD_PROPERTY, "secret") + .property(HttpTableDefn.URIS_PROPERTY, Collections.singletonList("http://foo.com/my.csv")) + .buildResolved(mapper); + + // Check validation + table.validate(); + + // Check registry + TableDefnRegistry registry = new TableDefnRegistry(mapper); + assertNotNull(registry.resolve(table.spec())); + + // Check serialization + byte[] bytes = table.spec().toBytes(mapper); + assertEquals(table.spec(), TableSpec.fromBytes(mapper, bytes)); + + // Convert to an external spec + ExternalSpec externSpec = tableDefn.convertToExtern(table); + + HttpInputSource sourceSpec = (HttpInputSource) externSpec.inputSource(); + assertEquals("bob", sourceSpec.getHttpAuthenticationUsername()); + assertEquals("secret", ((DefaultPasswordProvider) sourceSpec.getHttpAuthenticationPasswordProvider()).getPassword()); + assertEquals("http://foo.com/my.csv", sourceSpec.getUris().get(0).toString()); + + // Just a sanity check: details of CSV conversion are tested elsewhere. + CsvInputFormat csvFormat = (CsvInputFormat) externSpec.inputFormat(); + assertEquals(Arrays.asList("x", "y"), csvFormat.getColumns()); + + RowSignature sig = externSpec.signature(); + assertEquals(Arrays.asList("x", "y"), sig.getColumnNames()); + assertEquals(ColumnType.STRING, sig.getColumnType(0).get()); + assertEquals(ColumnType.LONG, sig.getColumnType(1).get()); + } + + @Test + public void testEnvPassword() + { + ResolvedTable table = baseBuilder.copy() + .property(HttpTableDefn.USER_PROPERTY, "bob") + .property(HttpTableDefn.PASSWORD_ENV_VAR_PROPERTY, "SECRET") + .property(HttpTableDefn.URIS_PROPERTY, Collections.singletonList("http://foo.com/my.csv")) + .buildResolved(mapper); + + // Check validation + table.validate(); + + // Convert to an external spec + ExternalSpec externSpec = tableDefn.convertToExtern(table); + + HttpInputSource sourceSpec = (HttpInputSource) externSpec.inputSource(); + assertEquals("bob", sourceSpec.getHttpAuthenticationUsername()); + assertEquals("SECRET", ((EnvironmentVariablePasswordProvider) sourceSpec.getHttpAuthenticationPasswordProvider()).getVariable()); + } + + @Test + public void testParameters() + { + ResolvedTable table = baseBuilder.copy() + .property(HttpTableDefn.USER_PROPERTY, "bob") + .property(HttpTableDefn.PASSWORD_ENV_VAR_PROPERTY, "SECRET") + .property(HttpTableDefn.URI_TEMPLATE_PROPERTY, "http://foo.com/{}") + .buildResolved(mapper); + + // Check validation + table.validate(); + + // Parameters + Parameterized parameterizedTable = tableDefn; + assertEquals(1, parameterizedTable.parameters().size()); + assertNotNull(parameterizedTable.parameter(HttpTableDefn.URIS_PARAMETER)); + + // Apply parameters + Map params = ImmutableMap.of( + HttpTableDefn.URIS_PARAMETER, "foo.csv,bar.csv" + ); + + // Convert to an external spec + ExternalSpec externSpec = parameterizedTable.applyParameters(table, params); + + HttpInputSource sourceSpec = (HttpInputSource) externSpec.inputSource(); + assertEquals("bob", sourceSpec.getHttpAuthenticationUsername()); + assertEquals("SECRET", ((EnvironmentVariablePasswordProvider) sourceSpec.getHttpAuthenticationPasswordProvider()).getVariable()); + assertEquals( + HttpTableDefn.convertUriList(Arrays.asList("http://foo.com/foo.csv", "http://foo.com/bar.csv")), + sourceSpec.getUris() + ); + } + + @Test + public void testNoTemplate() + { + ResolvedTable table = baseBuilder.copy() + .property(HttpTableDefn.URIS_PROPERTY, Collections.singletonList("http://foo.com/my.csv")) + .buildResolved(mapper); + + // Check validation + table.validate(); + + // Apply parameters + Map params = ImmutableMap.of( + HttpTableDefn.URIS_PARAMETER, "foo.csv,bar.csv" + ); + + // Convert to an external spec + assertThrows(IAE.class, () -> tableDefn.applyParameters(table, params)); + } + + @Test + public void testNoParameters() + { + ResolvedTable table = baseBuilder.copy() + .property(HttpTableDefn.URI_TEMPLATE_PROPERTY, "http://foo.com/{}") + .buildResolved(mapper); + + Map params = ImmutableMap.of(); + assertThrows(IAE.class, () -> tableDefn.applyParameters(table, params)); + } + + @Test + public void testInvalidParameters() + { + // The URI parser is forgiving about items in the path, so + // screw up the head, where URI is particular. + ResolvedTable table = baseBuilder.copy() + .property(HttpTableDefn.URI_TEMPLATE_PROPERTY, "//foo.com/{}") + .buildResolved(mapper); + + Map params = ImmutableMap.of( + HttpTableDefn.URIS_PARAMETER, "foo.csv" + ); + assertThrows(IAE.class, () -> tableDefn.applyParameters(table, params)); + } + + @Test + public void testInvalidURI() + { + // The URI parser is forgiving about items in the path, so + // screw up the head, where URI is particular. + ResolvedTable table = baseBuilder.copy() + .property(HttpTableDefn.URIS_PROPERTY, Collections.singletonList("//foo.com")) + .buildResolved(mapper); + + assertThrows(IAE.class, () -> table.validate()); + } +} diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/InlineTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/InlineTableTest.java new file mode 100644 index 000000000000..ede48da9f710 --- /dev/null +++ b/server/src/test/java/org/apache/druid/catalog/model/table/InlineTableTest.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.table; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.catalog.CatalogTest; +import org.apache.druid.catalog.model.Columns; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableDefnRegistry; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.InlineInputSource; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.Arrays; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThrows; + +@Category(CatalogTest.class) +public class InlineTableTest +{ + private final ObjectMapper mapper = new ObjectMapper(); + private final InlineTableDefn tableDefn = new InlineTableDefn(); + private final TableBuilder baseBuilder = TableBuilder.of(tableDefn) + .description("inline input") + .format(InputFormats.CSV_FORMAT_TYPE) + .column("x", Columns.VARCHAR) + .column("y", Columns.BIGINT); + + @Test + public void testEmptyData() + { + ResolvedTable table = baseBuilder.buildResolved(mapper); + + // Check validation + assertThrows(IAE.class, () -> table.validate()); + } + + @Test + public void testValidData() + { + ResolvedTable table = baseBuilder.copy() + .data("a,b", "c,d") + .buildResolved(mapper); + + // Check validation + table.validate(); + + // Check registry + TableDefnRegistry registry = new TableDefnRegistry(mapper); + assertNotNull(registry.resolve(table.spec())); + + // Check serialization + byte[] bytes = table.spec().toBytes(mapper); + assertEquals(table.spec(), TableSpec.fromBytes(mapper, bytes)); + + // Convert to an external spec + ExternalSpec externSpec = tableDefn.convertToExtern(table); + + InlineInputSource inlineSpec = (InlineInputSource) externSpec.inputSource(); + assertEquals("a,b\nc,d\n", inlineSpec.getData()); + + // Just a sanity check: details of CSV conversion are tested elsewhere. + CsvInputFormat csvFormat = (CsvInputFormat) externSpec.inputFormat(); + assertEquals(Arrays.asList("x", "y"), csvFormat.getColumns()); + + RowSignature sig = externSpec.signature(); + assertEquals(Arrays.asList("x", "y"), sig.getColumnNames()); + assertEquals(ColumnType.STRING, sig.getColumnType(0).get()); + assertEquals(ColumnType.LONG, sig.getColumnType(1).get()); + } +} diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/InputFormatTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/InputFormatTest.java new file mode 100644 index 000000000000..7336afc633e8 --- /dev/null +++ b/server/src/test/java/org/apache/druid/catalog/model/table/InputFormatTest.java @@ -0,0 +1,284 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.table; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.catalog.CatalogTest; +import org.apache.druid.catalog.model.ColumnSpec; +import org.apache.druid.catalog.model.Columns; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.model.table.InputFormats.GenericFormatDefn; +import org.apache.druid.catalog.model.table.InputFormats.InputFormatDefn; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DelimitedInputFormat; +import org.apache.druid.data.input.impl.JsonInputFormat; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; + +@Category(CatalogTest.class) +public class InputFormatTest +{ + private final ObjectMapper mapper = new ObjectMapper(); + + @Test + public void testCsvFormat() + { + InputFormatDefn converter = InputFormats.CSV_FORMAT_DEFN; + List cols = Arrays.asList( + new ColumnSpec("type", "x", Columns.VARCHAR, null), + new ColumnSpec("type", "y", Columns.BIGINT, null) + ); + + // Basic case + { + Map args = ImmutableMap.of( + "listDelimiter", "|", "skipRows", 1 + ); + TableSpec spec = new TableSpec("type", args, cols); + ResolvedTable table = new ResolvedTable(null, spec, mapper); + + CsvInputFormat expectedFormat = new CsvInputFormat( + Arrays.asList("x", "y"), + "|", + false, + false, + 1 + ); + InputFormat inputFormat = converter.convert(table); + assertEquals(expectedFormat, inputFormat); + } + + // Minimal case. (However, though skipRows is required, JSON will handle + // a null value and set the value to 0.) + { + Map args = ImmutableMap.of(); + TableSpec spec = new TableSpec("type", args, cols); + ResolvedTable table = new ResolvedTable(null, spec, mapper); + + CsvInputFormat expectedFormat = new CsvInputFormat( + Arrays.asList("x", "y"), + null, + false, + false, + 0 + ); + InputFormat inputFormat = converter.convert(table); + assertEquals(expectedFormat, inputFormat); + } + + // Invalid format + { + Map args = ImmutableMap.of( + "skipRows", "bogus" + ); + TableSpec spec = new TableSpec("type", args, cols); + ResolvedTable table = new ResolvedTable(null, spec, mapper); + + assertThrows(Exception.class, () -> converter.convert(table)); + } + + // No columns + { + Map args = ImmutableMap.of( + "skipRows", 1 + ); + TableSpec spec = new TableSpec("type", args, null); + ResolvedTable table = new ResolvedTable(null, spec, mapper); + + assertThrows(Exception.class, () -> converter.convert(table)); + } + } + + @Test + public void testDelimitedTextFormat() + { + InputFormatDefn converter = InputFormats.DELIMITED_FORMAT_DEFN; + List cols = Arrays.asList( + new ColumnSpec("type", "x", Columns.VARCHAR, null), + new ColumnSpec("type", "y", Columns.BIGINT, null) + ); + + Map args = ImmutableMap.of( + "delimiter", ",", "listDelimiter", "|", "skipRows", 1 + ); + TableSpec spec = new TableSpec("type", args, cols); + ResolvedTable table = new ResolvedTable(null, spec, mapper); + + DelimitedInputFormat expectedFormat = new DelimitedInputFormat( + Arrays.asList("x", "y"), + "|", + ",", + false, + false, + 1 + ); + InputFormat inputFormat = converter.convert(table); + assertEquals(expectedFormat, inputFormat); + } + + @Test + public void testJsonFormat() + { + InputFormatDefn converter = InputFormats.JSON_FORMAT_DEFN; + List cols = Arrays.asList( + new ColumnSpec("type", "x", Columns.VARCHAR, null), + new ColumnSpec("type", "y", Columns.BIGINT, null) + ); + + // The one supported property at present. + { + Map args = ImmutableMap.of( + "keepNulls", true + ); + TableSpec spec = new TableSpec("type", args, cols); + ResolvedTable table = new ResolvedTable(null, spec, mapper); + + InputFormat inputFormat = converter.convert(table); + assertEquals(new JsonInputFormat(null, null, true, null, null), inputFormat); + } + + // Empty + { + TableSpec spec = new TableSpec("type", null, cols); + ResolvedTable table = new ResolvedTable(null, spec, mapper); + + InputFormat inputFormat = converter.convert(table); + assertEquals(new JsonInputFormat(null, null, null, null, null), inputFormat); + } + } + + /** + * Test the generic format which allows a literal input spec. The + * drawback is that the user must repeat the columns. + */ + @Test + public void testGenericFormat() + { + InputFormatDefn converter = InputFormats.GENERIC_FORMAT_DEFN; + List cols = Arrays.asList( + new ColumnSpec("type", "x", Columns.VARCHAR, null), + new ColumnSpec("type", "y", Columns.BIGINT, null) + ); + + // No type + { + Map args = ImmutableMap.of( + "skipRows", 1 + ); + TableSpec spec = new TableSpec("type", args, null); + ResolvedTable table = new ResolvedTable(null, spec, mapper); + + assertThrows(Exception.class, () -> converter.convert(table)); + } + + // CSV + { + Map args = ImmutableMap.of( + GenericFormatDefn.INPUT_FORMAT_SPEC_PROPERTY, + ImmutableMap.of( + "type", CsvInputFormat.TYPE_KEY, + "listDelimiter", "|", + "skipHeaderRows", 1, + "findColumnsFromHeader", false, + "columns", Arrays.asList("x", "y") + ) + ); + TableSpec spec = new TableSpec("type", args, cols); + ResolvedTable table = new ResolvedTable(null, spec, mapper); + + CsvInputFormat expectedFormat = new CsvInputFormat( + Arrays.asList("x", "y"), + "|", + false, + false, + 1 + ); + InputFormat inputFormat = converter.convert(table); + assertEquals(expectedFormat, inputFormat); + } + + // No columns: when using generic, the columns must be in the + // JSON spec. + { + Map args = ImmutableMap.of( + "type", CsvInputFormat.TYPE_KEY, + "skipRows", 1 + ); + TableSpec spec = new TableSpec("type", args, cols); + ResolvedTable table = new ResolvedTable(null, spec, mapper); + + assertThrows(Exception.class, () -> converter.convert(table)); + } + + // Delimited text + { + Map args = ImmutableMap.of( + GenericFormatDefn.INPUT_FORMAT_SPEC_PROPERTY, + ImmutableMap.builder() + .put("type", DelimitedInputFormat.TYPE_KEY) + .put("delimiter", ",") + .put("listDelimiter", "|") + .put("skipHeaderRows", 1) + .put("findColumnsFromHeader", false) + .put("columns", Arrays.asList("x", "y")) + .build() + ); + TableSpec spec = new TableSpec("type", args, cols); + ResolvedTable table = new ResolvedTable(null, spec, mapper); + + DelimitedInputFormat expectedFormat = new DelimitedInputFormat( + Arrays.asList("x", "y"), + "|", + ",", + false, + false, + 1 + ); + InputFormat inputFormat = converter.convert(table); + assertEquals(expectedFormat, inputFormat); + } + + // JSON + { + Map args = ImmutableMap.of( + GenericFormatDefn.INPUT_FORMAT_SPEC_PROPERTY, + ImmutableMap.of( + "type", JsonInputFormat.TYPE_KEY, + "keepNullColumns", true + ) + ); + TableSpec spec = new TableSpec("type", args, cols); + ResolvedTable table = new ResolvedTable(null, spec, mapper); + + InputFormat inputFormat = converter.convert(table); + assertEquals(new JsonInputFormat(null, null, true, null, null), inputFormat); + } + } +} diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java new file mode 100644 index 000000000000..cc17443e77b6 --- /dev/null +++ b/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.table; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.catalog.CatalogTest; +import org.apache.druid.catalog.model.Columns; +import org.apache.druid.catalog.model.Parameterized; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableDefnRegistry; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.io.File; +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +@Category(CatalogTest.class) +public class LocalTableTest +{ + private final ObjectMapper mapper = new ObjectMapper(); + private final LocalTableDefn tableDefn = new LocalTableDefn(); + private final TableBuilder baseBuilder = TableBuilder.of(tableDefn) + .description("local file input") + .format(InputFormats.CSV_FORMAT_TYPE) + .column("x", Columns.VARCHAR) + .column("y", Columns.BIGINT); + + @Test + public void testFullyDefined() + { + ResolvedTable table = baseBuilder.copy() + .property(LocalTableDefn.BASE_DIR_PROPERTY, "/tmp") + .property(LocalTableDefn.FILE_FILTER_PROPERTY, "*.csv") + .property(LocalTableDefn.FILES_PROPERTY, Collections.singletonList("my.csv")) + .buildResolved(mapper); + + // Check validation + table.validate(); + + // Check registry + TableDefnRegistry registry = new TableDefnRegistry(mapper); + assertNotNull(registry.resolve(table.spec())); + + // Check serialization + byte[] bytes = table.spec().toBytes(mapper); + assertEquals(table.spec(), TableSpec.fromBytes(mapper, bytes)); + + // Convert to an external spec + ExternalSpec externSpec = tableDefn.convertToExtern(table); + + LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource(); + assertEquals("/tmp", sourceSpec.getBaseDir().toString()); + assertEquals("*.csv", sourceSpec.getFilter()); + assertEquals("my.csv", sourceSpec.getFiles().get(0).toString()); + + // Just a sanity check: details of CSV conversion are tested elsewhere. + CsvInputFormat csvFormat = (CsvInputFormat) externSpec.inputFormat(); + assertEquals(Arrays.asList("x", "y"), csvFormat.getColumns()); + + RowSignature sig = externSpec.signature(); + assertEquals(Arrays.asList("x", "y"), sig.getColumnNames()); + assertEquals(ColumnType.STRING, sig.getColumnType(0).get()); + assertEquals(ColumnType.LONG, sig.getColumnType(1).get()); + } + + @Test + public void testNoFilter() + { + ResolvedTable table = baseBuilder.copy() + .property(LocalTableDefn.BASE_DIR_PROPERTY, "/tmp") + .property(LocalTableDefn.FILES_PROPERTY, Collections.singletonList("my.csv")) + .buildResolved(mapper); + + // Check validation + table.validate(); + + // Convert to an external spec + ExternalSpec externSpec = tableDefn.convertToExtern(table); + + LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource(); + assertEquals("/tmp", sourceSpec.getBaseDir().toString()); + assertEquals("*", sourceSpec.getFilter()); + assertEquals("my.csv", sourceSpec.getFiles().get(0).toString()); + } + + @Test + public void testNoFiles() + { + ResolvedTable table = baseBuilder.copy() + .property(LocalTableDefn.BASE_DIR_PROPERTY, "/tmp") + .property(LocalTableDefn.FILE_FILTER_PROPERTY, "*.csv") + .buildResolved(mapper); + + // Check validation + table.validate(); + + // Convert to an external spec + ExternalSpec externSpec = tableDefn.convertToExtern(table); + + LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource(); + assertEquals("/tmp", sourceSpec.getBaseDir().toString()); + assertEquals("*.csv", sourceSpec.getFilter()); + assertTrue(sourceSpec.getFiles().isEmpty()); + } + + @Test + public void testNoFilesOrFlter() + { + ResolvedTable table = baseBuilder.copy() + .property(LocalTableDefn.BASE_DIR_PROPERTY, "/tmp") + .buildResolved(mapper); + + // Check validation + assertThrows(IAE.class, () -> table.validate()); + } + + @Test + public void testNoProperties() + { + ResolvedTable table = baseBuilder + .buildResolved(mapper); + + // Check validation: is legal for storage, but needs + // paramters to be valid at runtime. + table.validate(); + } + + @Test + public void testFilesParameter() + { + ResolvedTable table = baseBuilder.copy() + .property(LocalTableDefn.BASE_DIR_PROPERTY, "/tmp") + .buildResolved(mapper); + + Parameterized parameterizedTable = tableDefn; + assertEquals(2, parameterizedTable.parameters().size()); + assertNotNull(parameterizedTable.parameter(LocalTableDefn.FILE_FILTER_PROPERTY)); + assertNotNull(parameterizedTable.parameter(LocalTableDefn.FILES_PROPERTY)); + + + // Apply files parameter + Map params = ImmutableMap.of( + LocalTableDefn.FILES_PROPERTY, "foo.csv,bar.csv" + ); + + // Convert to an external spec + ExternalSpec externSpec = parameterizedTable.applyParameters(table, params); + + LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource(); + assertEquals("/tmp", sourceSpec.getBaseDir().toString()); + assertEquals("*", sourceSpec.getFilter()); + assertEquals( + Arrays.asList(new File("foo.csv"), new File("bar.csv")), + sourceSpec.getFiles() + ); + } + + @Test + public void testFilterParameter() + { + ResolvedTable table = baseBuilder.copy() + .property(LocalTableDefn.BASE_DIR_PROPERTY, "/tmp") + .buildResolved(mapper); + + // Apply files parameter + Map params = ImmutableMap.of( + LocalTableDefn.FILE_FILTER_PROPERTY, "Oct*.csv" + ); + + // Convert to an external spec + ExternalSpec externSpec = tableDefn.applyParameters(table, params); + + LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource(); + assertEquals("/tmp", sourceSpec.getBaseDir().toString()); + assertEquals("Oct*.csv", sourceSpec.getFilter()); + assertTrue(sourceSpec.getFiles().isEmpty()); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/Aggregation.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/Aggregation.java index f28ae185ba30..61cc58e12c03 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/Aggregation.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/Aggregation.java @@ -84,7 +84,6 @@ private Aggregation( } } - public static Aggregation create(final AggregatorFactory aggregatorFactory) { return new Aggregation( @@ -146,7 +145,6 @@ public Aggregation filter( return this; } - if (postAggregator != null) { // Verify that this Aggregation contains all input to its postAggregator. // If not, this "filter" call won't work right. diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCacheTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCacheTest.java index 1ba95b8bef0d..1787dae77470 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCacheTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCacheTest.java @@ -91,7 +91,7 @@ public class SegmentMetadataCacheTest extends SegmentMetadataCacheCommon { // Timeout to allow (rapid) debugging, while not blocking tests with errors. - private static final int WAIT_TIMEOUT_SECS = 60; + private static final int WAIT_TIMEOUT_SECS = 6; private SpecificSegmentsQuerySegmentWalker walker; private TestServerInventoryView serverView; From 398d6f530652c073a01a2d3e2798bc8c71268d17 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Tue, 4 Oct 2022 08:24:29 -0700 Subject: [PATCH 02/22] Build fixes --- .../java/org/apache/druid/catalog/http/CatalogResource.java | 2 +- .../java/org/apache/druid/catalog/storage/HideColumns.java | 6 ++++++ .../java/org/apache/druid/catalog/storage/MoveColumn.java | 6 ++++++ integration-tests-ex/cases/pom.xml | 4 ++++ .../java/org/apache/druid/catalog/model/Properties.java | 2 +- 5 files changed, 18 insertions(+), 2 deletions(-) diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java index b1491c52e35e..6f06cee15ae7 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java @@ -100,7 +100,7 @@ private enum PostAction * @param name The name of the table definition to modify. The user must * have write access to the table. * @param spec The new table definition. - * @param action What to do if the table already exists. + * @param actionParam What to do if the table already exists. * {@code ifNew} is the same as the SQL IF NOT EXISTS clause. If {@code new}, * then an error is raised if the table exists. If {@code ifNew}, then * the action silently does nothing if the table exists. Primarily for diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java index 5d1e1db39ccc..dccadad57b30 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java @@ -93,4 +93,10 @@ public boolean equals(Object o) return Objects.equals(this.hide, other.hide) && Objects.equals(this.unhide, other.unhide); } + + @Override + public int hashCode() + { + return Objects.hash(hide, unhide); + } } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MoveColumn.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MoveColumn.java index c91d68ae86a2..7f33dca742d5 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MoveColumn.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MoveColumn.java @@ -113,4 +113,10 @@ public boolean equals(Object o) && this.where == other.where && Objects.equals(this.anchor, other.anchor); } + + @Override + public int hashCode() + { + return Objects.hash(column, where, anchor); + } } diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml index 540b662c53f7..20859307aa86 100644 --- a/integration-tests-ex/cases/pom.xml +++ b/integration-tests-ex/cases/pom.xml @@ -218,6 +218,10 @@ JUnitParams test + + javax.ws.rs + jsr311-api + diff --git a/server/src/main/java/org/apache/druid/catalog/model/Properties.java b/server/src/main/java/org/apache/druid/catalog/model/Properties.java index eaf98317e5da..e99475ddfb28 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/Properties.java +++ b/server/src/main/java/org/apache/druid/catalog/model/Properties.java @@ -44,7 +44,7 @@ */ public interface Properties { - public interface PropertyDefn + interface PropertyDefn { String name(); String typeName(); From cbd46dc832cb9f9eb31b27e5e9794eac72ed17e6 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Tue, 4 Oct 2022 08:33:33 -0700 Subject: [PATCH 03/22] Fix LGTM errors --- .../org/apache/druid/catalog/http/CatalogResource.java | 7 ++++--- .../druid/catalog/model/table/ExternalTableDefn.java | 2 +- .../org/apache/druid/catalog/model/table/MeasureTypes.java | 5 ++++- 3 files changed, 9 insertions(+), 5 deletions(-) diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java index 6f06cee15ae7..9903476a4117 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java @@ -189,13 +189,14 @@ private Response authorizeTable(TableId tableId, TableSpec spec, final HttpServl } // The given table spec has to be valid for the given schema. - if (spec != null && !schema.accepts(spec.type())) { + if (!schema.accepts(spec.type())) { return Actions.badRequest( Actions.INVALID, StringUtils.format( "Cannot create tables of type %s in schema %s", - spec == null ? "null" : spec.getClass().getSimpleName(), - tableId.schema()) + spec.getClass().getSimpleName(), + tableId.schema() + ) ); } diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java index 4d65e91046b1..6aae9cc45ac4 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java @@ -200,7 +200,7 @@ public ExternalTableDefn( Map params = new HashMap<>(); for (ParameterDefn param : parameters) { if (params.put(param.name(), param) != null) { - throw new ISE("Duplicate parameter: ", param.name()); + throw new ISE("Duplicate parameter: %s", param.name()); } } this.parameterMap = ImmutableMap.copyOf(params); diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/MeasureTypes.java b/server/src/main/java/org/apache/druid/catalog/model/table/MeasureTypes.java index e40749ec431e..57ff20c36eea 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/MeasureTypes.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/MeasureTypes.java @@ -181,7 +181,10 @@ public static MeasureType parse(String typeStr) if (Strings.isNullOrEmpty(argGroup)) { args = new String[] {}; } else { - args = argGroup.trim().split("\\s*,\\s*"); + args = argGroup.split(","); + for (int i = 0; i < args.length; i++) { + args[i] = args[i].trim(); + } } List candidates = TYPES.get(fnName); if (candidates == null) { From 28a2d240821fcde8b2f69abf2f7b8ad77cda5a03 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Thu, 6 Oct 2022 09:09:56 -0700 Subject: [PATCH 04/22] Bug fix --- .../druid/catalog/http/CatalogResource.java | 32 ++++++++++++------- 1 file changed, 20 insertions(+), 12 deletions(-) diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java index 9903476a4117..78f9c89a4e53 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java @@ -188,18 +188,6 @@ private Response authorizeTable(TableId tableId, TableSpec spec, final HttpServl return Actions.badRequest(Actions.INVALID, "Table name cannot start or end with spaces"); } - // The given table spec has to be valid for the given schema. - if (!schema.accepts(spec.type())) { - return Actions.badRequest( - Actions.INVALID, - StringUtils.format( - "Cannot create tables of type %s in schema %s", - spec.getClass().getSimpleName(), - tableId.schema() - ) - ); - } - // The user has to have permission to modify the table. try { catalog.authorizer().authorizeTable(schema, tableId.name(), Action.WRITE, req); @@ -208,6 +196,26 @@ private Response authorizeTable(TableId tableId, TableSpec spec, final HttpServl return Actions.forbidden(e); } + // Validate the spec, if provided. + if (spec != null) { + + // The given table spec has to be valid for the given schema. + if (Strings.isNullOrEmpty(spec.type())) { + return Actions.badRequest(Actions.INVALID, "Table type is required"); + } + + if (!schema.accepts(spec.type())) { + return Actions.badRequest( + Actions.INVALID, + StringUtils.format( + "Cannot create tables of type %s in schema %s", + spec.getClass().getSimpleName(), + tableId.schema() + ) + ); + } + } + // Everything checks out, let the request proceed. return null; } From b3e5e0014bdbf1495b712707dacdb18c6b2e8f0a Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Mon, 17 Oct 2022 21:59:38 -0700 Subject: [PATCH 05/22] Revisions from review comments Addressed direct review comments Removed rollup support per offline discussion --- .../util/common/jackson/JacksonUtils.java | 51 +++- .../druid/catalog/http/CatalogResource.java | 32 ++- .../druid/catalog/storage/MoveColumn.java | 16 +- .../catalog/storage/TableManagerTest.java | 34 ++- .../catalog/sync/CatalogMetadataTest.java | 33 +-- .../http/catalog/CatalogResourceTest.java | 30 +-- .../server/http/catalog/CommandTest.java | 11 +- .../testsEx/catalog/ITCatalogRestTest.java | 14 +- .../druid/segment/column/RowSignature.java | 1 - server/pom.xml | 4 +- .../druid/catalog/model/CatalogUtils.java | 143 +---------- .../druid/catalog/model/ColumnDefn.java | 2 +- .../druid/catalog/model/ColumnSpec.java | 7 +- .../druid/catalog/model/ObjectDefn.java | 20 +- ...ameterized.java => ParameterizedDefn.java} | 2 +- .../druid/catalog/model/Properties.java | 11 +- .../catalog/model/SchemaRegistryImpl.java | 4 +- .../apache/druid/catalog/model/TableDefn.java | 4 +- .../catalog/model/TableDefnRegistry.java | 3 +- .../druid/catalog/model/TableMetadata.java | 7 +- .../apache/druid/catalog/model/TableSpec.java | 7 +- .../catalog/model/facade/ColumnFacade.java | 23 -- .../model/facade/DatasourceFacade.java | 35 +-- ...leFacade.java => ExternalTableFacade.java} | 4 +- .../model/table/AbstractDatasourceDefn.java | 144 +++++++++++ .../catalog/model/table/DatasourceDefn.java | 242 ++---------------- .../model/table/ExternalTableDefn.java | 25 +- .../catalog/model/table/HttpTableDefn.java | 4 +- .../catalog/model/table/InputFormats.java | 10 +- .../catalog/model/table/LocalTableDefn.java | 4 +- .../catalog/model/table/MeasureTypes.java | 216 ---------------- .../catalog/model/table/TableBuilder.java | 34 +-- .../druid/catalog/model/MeasureTypesTest.java | 98 ------- .../catalog/model/TableMetadataTest.java | 18 +- .../model/table/DatasourceTableTest.java | 221 +++------------- .../model/table/HttpInputTableTest.java | 4 +- .../catalog/model/table/LocalTableTest.java | 4 +- 37 files changed, 451 insertions(+), 1071 deletions(-) rename server/src/main/java/org/apache/druid/catalog/model/{Parameterized.java => ParameterizedDefn.java} (97%) rename server/src/main/java/org/apache/druid/catalog/model/facade/{InputTableFacade.java => ExternalTableFacade.java} (94%) create mode 100644 server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.java delete mode 100644 server/src/main/java/org/apache/druid/catalog/model/table/MeasureTypes.java delete mode 100644 server/src/test/java/org/apache/druid/catalog/model/MeasureTypesTest.java diff --git a/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java b/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java index b269544482de..2f5cd5d9fb61 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java +++ b/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java @@ -20,11 +20,13 @@ package org.apache.druid.java.util.common.jackson; import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.JsonSerializer; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializerProvider; +import org.apache.druid.java.util.common.ISE; import javax.annotation.Nullable; import java.io.IOException; @@ -45,6 +47,10 @@ public final class JacksonUtils { }; + private JacksonUtils() + { + } + /** * Silences Jackson's {@link IOException}. */ @@ -89,7 +95,50 @@ public static void writeObjectUsingSerializerProvider( } } - private JacksonUtils() + /** + * Convert the given object to an array of bytes. Use when the object is + * known serializable so that the Jackson exception can be suppressed. + */ + public static byte[] toBytes(ObjectMapper jsonMapper, Object obj) { + try { + return jsonMapper.writeValueAsBytes(obj); + } + catch (JsonProcessingException e) { + throw new ISE("Failed to serialize " + obj.getClass().getSimpleName()); + } + } + + /** + * Deserialize an object from an array of bytes. Use when the object is + * known deserializable so that the Jackson exception can be suppressed. + */ + public static T fromBytes(ObjectMapper jsonMapper, byte[] bytes, Class clazz) + { + try { + return jsonMapper.readValue(bytes, clazz); + } + catch (IOException e) { + throw new ISE(e, "Failed to deserialize a " + clazz.getSimpleName()); + } + } + + /** + * Quick & easy implementation of {@code toString()} for objects which are + * primarily representations of JSON objects. Use only for cases where the + * {@code toString()} is for debugging: the cost of creating an object mapper + * every time is undesirable for production code. Also, assumes that the + * type can serialized using the default mapper: doesn't work for types that + * require custom Jackson extensions. + */ + public static String toString(Object obj) + { + ObjectMapper jsonMapper = new ObjectMapper(); + try { + return jsonMapper.writeValueAsString(obj); + } + catch (JsonProcessingException e) { + throw new ISE("Failed to serialize TableDefn"); + } } } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java index 78f9c89a4e53..817f613f3f3e 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java @@ -21,13 +21,13 @@ import com.google.common.base.Strings; import org.apache.curator.shaded.com.google.common.collect.Lists; -import org.apache.druid.catalog.model.CatalogUtils; +import org.apache.druid.catalog.model.ColumnSpec; import org.apache.druid.catalog.model.SchemaRegistry.SchemaSpec; import org.apache.druid.catalog.model.TableDefnRegistry; import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; -import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; import org.apache.druid.catalog.storage.Actions; import org.apache.druid.catalog.storage.CatalogStorage; import org.apache.druid.catalog.storage.HideColumns; @@ -60,8 +60,11 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import java.util.ArrayList; import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.function.Function; /** @@ -412,13 +415,13 @@ public Response hideColumns( null, req, (spec) -> { - if (!DatasourceDefn.isDatasource(spec.type())) { + if (!AbstractDatasourceDefn.isDatasource(spec.type())) { throw new ISE("hideColumns is supported only for data source specs"); } @SuppressWarnings("unchecked") - List hiddenProps = (List) spec.properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY); + List hiddenProps = (List) spec.properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); return spec.withProperty( - DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, + AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, command.perform(hiddenProps) ); } @@ -444,10 +447,27 @@ public Response dropColumns( TableId.of(dbSchema, name), null, req, - (spec) -> spec.withColumns(CatalogUtils.dropColumns(spec.columns(), columns)) + (spec) -> spec.withColumns(dropColumns(spec.columns(), columns)) ); } + private static List dropColumns( + final List columns, + final List toDrop) + { + if (toDrop == null || toDrop.isEmpty()) { + return columns; + } + Set drop = new HashSet(toDrop); + List revised = new ArrayList<>(); + for (T col : columns) { + if (!drop.contains(col.name())) { + revised.add(col); + } + } + return revised; + } + /** * Retrieves the definition of the given table. *

diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MoveColumn.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MoveColumn.java index 7f33dca742d5..cfe889b977f1 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MoveColumn.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MoveColumn.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.catalog.model.CatalogUtils; import org.apache.druid.catalog.model.ColumnSpec; import org.apache.druid.java.util.common.ISE; @@ -67,13 +66,13 @@ public MoveColumn( public List perform(List columns) { List revised = new ArrayList<>(columns); - final int colPosn = CatalogUtils.findColumn(columns, column); + final int colPosn = findColumn(columns, column); if (colPosn == -1) { throw new ISE("Column [%s] is not defined", column); } int anchorPosn; if (where == Position.BEFORE || where == Position.AFTER) { - anchorPosn = CatalogUtils.findColumn(columns, anchor); + anchorPosn = findColumn(columns, anchor); if (anchorPosn == -1) { throw new ISE("Anchor [%s] is not defined", column); } @@ -102,6 +101,17 @@ public List perform(List columns) return revised; } + private static int findColumn(List columns, String colName) + { + for (int i = 0; i < columns.size(); i++) { + if (columns.get(i).name().equals(colName)) { + return i; + } + } + return -1; + } + + @Override public boolean equals(Object o) { diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java index 132c23b43017..0f51fe5179ef 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java @@ -25,6 +25,7 @@ import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; import org.apache.druid.catalog.model.table.DatasourceDefn; import org.apache.druid.catalog.storage.sql.CatalogManager; import org.apache.druid.catalog.storage.sql.CatalogManager.DuplicateKeyException; @@ -85,11 +86,10 @@ public void tearDown() public void testCreate() throws DuplicateKeyException { Map props = ImmutableMap.of( - DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", - DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY, "PT1M", - DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000 + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", + AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000 ); - TableSpec spec = new TableSpec(DatasourceDefn.ROLLUP_DATASOURCE_TYPE, props, null); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); TableMetadata table = TableMetadata.newTable(TableId.datasource("table1"), spec); // Table does not exist, read returns nothing. @@ -111,19 +111,17 @@ public void testCreate() throws DuplicateKeyException public void testUpdate() throws DuplicateKeyException, OutOfDateException, NotFoundException { Map props = ImmutableMap.of( - DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", - DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY, "PT1M", - DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000 + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", + AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000 ); - TableSpec spec = new TableSpec(DatasourceDefn.ROLLUP_DATASOURCE_TYPE, props, null); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); TableMetadata table = TableMetadata.newTable(TableId.datasource("table1"), spec); long version = manager.create(table); // Change the definition props = ImmutableMap.of( - DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", - DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY, "PT1H", - DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 2_000_000 + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", + AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 2_000_000 ); TableSpec spec2 = spec.withProperties(props); TableMetadata table2 = table.withSpec(spec2); @@ -152,11 +150,10 @@ public void testUpdate() throws DuplicateKeyException, OutOfDateException, NotFo public void testDelete() throws DuplicateKeyException { Map props = ImmutableMap.of( - DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", - DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY, "PT1M", - DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000 + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", + AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000 ); - TableSpec spec = new TableSpec(DatasourceDefn.ROLLUP_DATASOURCE_TYPE, props, null); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); TableMetadata table = TableMetadata.newTable(TableId.datasource("table1"), spec); assertFalse(manager.delete(table.id())); @@ -172,11 +169,10 @@ public void testList() throws DuplicateKeyException assertTrue(list.isEmpty()); Map props = ImmutableMap.of( - DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H", - DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY, "PT1M", - DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000 + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H", + AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000 ); - TableSpec spec = new TableSpec(DatasourceDefn.ROLLUP_DATASOURCE_TYPE, props, null); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); // Create tables in inverse order TableMetadata table2 = TableMetadata.newTable(TableId.datasource("table2"), spec); diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java index 899279c40b8b..1505a11051e3 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java @@ -28,6 +28,7 @@ import org.apache.druid.catalog.model.TableSpec; import org.apache.druid.catalog.model.facade.DatasourceFacade; import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.catalog.model.table.DatasourceDefn.DatasourceColumnDefn; import org.apache.druid.catalog.model.table.ExternalTableDefn; import org.apache.druid.catalog.model.table.InlineTableDefn; import org.apache.druid.catalog.model.table.InputFormats; @@ -47,7 +48,6 @@ import java.util.List; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThrows; @@ -184,18 +184,17 @@ private void doTestRemote(boolean useSmile) throws DuplicateKeyException, OutOfD */ private void populateCatalog() throws DuplicateKeyException { - TableMetadata table1 = TableBuilder.detailTable("table1", "P1D") + TableMetadata table1 = TableBuilder.datasource("table1", "P1D") .timeColumn() .column("a", Columns.VARCHAR) .build(); storage.validate(table1); storage.tables().create(table1); - TableMetadata table2 = TableBuilder.rollupTable("table2", "P1D") - .rollupGranularity("PT1H") + TableMetadata table2 = TableBuilder.datasource("table2", "P1D") .timeColumn() .column("dim", Columns.VARCHAR) - .measure("measure", "SUM(BIGINT)") + .column("measure", "BIGINT") .build(); storage.validate(table2); storage.tables().create(table2); @@ -218,21 +217,18 @@ private void verifyInitial(MetadataCatalog catalog) assertTrue(table.updateTime() > 0); TableSpec dsSpec = table.spec(); - assertEquals(DatasourceDefn.DETAIL_DATASOURCE_TYPE, dsSpec.type()); + assertEquals(DatasourceDefn.TABLE_TYPE, dsSpec.type()); List cols = dsSpec.columns(); assertEquals(2, cols.size()); assertEquals(Columns.TIME_COLUMN, cols.get(0).name()); assertEquals(Columns.TIMESTAMP, cols.get(0).sqlType()); - assertEquals(DatasourceDefn.DETAIL_COLUMN_TYPE, cols.get(0).type()); + assertEquals(DatasourceColumnDefn.COLUMN_TYPE, cols.get(0).type()); assertEquals("a", cols.get(1).name()); assertEquals(Columns.VARCHAR, cols.get(1).sqlType()); - assertEquals(DatasourceDefn.DETAIL_COLUMN_TYPE, cols.get(0).type()); + assertEquals(DatasourceColumnDefn.COLUMN_TYPE, cols.get(0).type()); DatasourceFacade ds = new DatasourceFacade(catalog.resolveTable(id)); assertEquals("P1D", ds.segmentGranularityString()); - assertTrue(ds.isDetail()); - assertFalse(ds.isRollup()); - assertNull(ds.rollupGranularity()); } { TableId id = TableId.datasource("table2"); @@ -241,7 +237,7 @@ private void verifyInitial(MetadataCatalog catalog) assertTrue(table.updateTime() > 0); TableSpec dsSpec = table.spec(); - assertEquals(DatasourceDefn.ROLLUP_DATASOURCE_TYPE, table.spec().type()); + assertEquals(DatasourceDefn.TABLE_TYPE, table.spec().type()); List cols = dsSpec.columns(); assertEquals(3, cols.size()); assertEquals("__time", cols.get(0).name()); @@ -249,16 +245,13 @@ private void verifyInitial(MetadataCatalog catalog) assertEquals(Columns.TIMESTAMP, cols.get(0).sqlType()); assertEquals("dim", cols.get(1).name()); assertEquals(Columns.VARCHAR, cols.get(1).sqlType()); - assertEquals(DatasourceDefn.DIMENSION_TYPE, cols.get(1).type()); + assertEquals(DatasourceColumnDefn.COLUMN_TYPE, cols.get(1).type()); assertEquals("measure", cols.get(2).name()); - assertEquals("SUM(BIGINT)", cols.get(2).sqlType()); - assertEquals(DatasourceDefn.MEASURE_TYPE, cols.get(2).type()); + assertEquals("BIGINT", cols.get(2).sqlType()); + assertEquals(DatasourceColumnDefn.COLUMN_TYPE, cols.get(2).type()); DatasourceFacade ds = new DatasourceFacade(catalog.resolveTable(id)); assertEquals("P1D", ds.segmentGranularityString()); - assertFalse(ds.isDetail()); - assertTrue(ds.isRollup()); - assertEquals("P1D", ds.segmentGranularityString()); } assertNull(catalog.getTable(TableId.datasource("table3"))); @@ -304,7 +297,7 @@ private void alterCatalog() throws DuplicateKeyException, OutOfDateException, No storage.tables().update(table1.withSpec(update1), table1.updateTime()); // Create a table 3 - TableMetadata table3 = TableBuilder.detailTable("table3", "P1D") + TableMetadata table3 = TableBuilder.datasource("table3", "P1D") .timeColumn() .column("x", "FLOAT") .build(); @@ -324,7 +317,7 @@ private void verifyAltered(MetadataCatalog catalog) assertEquals("a", cols.get(1).name()); assertEquals("b", cols.get(2).name()); assertEquals(Columns.DOUBLE, cols.get(2).sqlType()); - assertEquals(DatasourceDefn.DETAIL_COLUMN_TYPE, cols.get(2).type()); + assertEquals(DatasourceColumnDefn.COLUMN_TYPE, cols.get(2).type()); } { TableId id = TableId.datasource("table3"); diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java index 9d61b911bec6..101135e835e3 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java @@ -25,7 +25,7 @@ import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; -import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; import org.apache.druid.catalog.model.table.InlineTableDefn; import org.apache.druid.catalog.model.table.InputFormats; import org.apache.druid.catalog.model.table.TableBuilder; @@ -87,7 +87,7 @@ private static long getVersion(Response resp) public void testCreate() { final String tableName = "create"; - TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D").buildSpec(); + TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D").buildSpec(); // Blank schema name: infer the schema. Response resp = resource.postTable("", tableName, dsSpec, null, 0, postBy(CatalogTests.SUPER_USER)); @@ -155,7 +155,7 @@ public void testCreate() public void testUpdate() { final String tableName = "update"; - TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D").buildSpec(); + TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D").buildSpec(); // Does not exist Response resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "replace", 0, postBy(CatalogTests.SUPER_USER)); @@ -190,7 +190,7 @@ public void testUpdate() public void testForce() { final String tableName = "force"; - TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D").buildSpec(); + TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D").buildSpec(); // Create the table Response resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "force", 0, postBy(CatalogTests.WRITER_USER)); @@ -207,7 +207,7 @@ public void testForce() public void testRead() { final String tableName = "read"; - TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D").buildSpec(); + TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D").buildSpec(); // Missing schema name Response resp = resource.getTable("", tableName, getBy(CatalogTests.SUPER_USER)); @@ -291,7 +291,7 @@ public void testList() // Create a table final String tableName = "list"; - TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D").buildSpec(); + TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D").buildSpec(); resp = resource.postTable(TableId.DRUID_SCHEMA, "list", dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); @@ -361,7 +361,7 @@ public void testDelete() assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); // Create the table - TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D").buildSpec(); + TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D").buildSpec(); resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); @@ -385,7 +385,7 @@ public void testLifecycle() { // Operations for one table - create String table1Name = "lifecycle1"; - TableSpec dsSpec = TableBuilder.detailTable(table1Name, "P1D").buildSpec(); + TableSpec dsSpec = TableBuilder.datasource(table1Name, "P1D").buildSpec(); Response resp = resource.postTable(TableId.DRUID_SCHEMA, table1Name, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); long version = getVersion(resp); @@ -413,7 +413,7 @@ public void testLifecycle() assertEquals(id1.name(), tables.get(0)); // update - TableSpec table2Spec = TableBuilder.detailTable(table1Name, "PT1H").buildSpec(); + TableSpec table2Spec = TableBuilder.datasource(table1Name, "PT1H").buildSpec(); resp = resource.postTable(TableId.DRUID_SCHEMA, table1Name, table2Spec, "replace", version, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); assertTrue(getVersion(resp) > version); @@ -470,7 +470,7 @@ public void testLifecycle() public void testMoveColumn() { String tableName = "move"; - TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D") + TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D") .column("a", "VARCHAR") .column("b", "BIGINT") .column("c", "FLOAT") @@ -519,7 +519,7 @@ public void testMoveColumn() public void testHideColumns() { String tableName = "hide"; - TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D") + TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D") .buildSpec(); Response resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); @@ -540,7 +540,7 @@ public void testHideColumns() resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, postBy(CatalogTests.READER_USER)); TableMetadata read = (TableMetadata) resp.getEntity(); - assertNull(read.spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY)); + assertNull(read.spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY)); // Hide cmd = new HideColumns(Arrays.asList("a", "b"), null); @@ -551,7 +551,7 @@ public void testHideColumns() read = (TableMetadata) resp.getEntity(); assertEquals( Arrays.asList("a", "b"), - read.spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + read.spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) ); assertTrue(read.updateTime() > version); @@ -564,7 +564,7 @@ public void testHideColumns() read = (TableMetadata) resp.getEntity(); assertEquals( Arrays.asList("b", "c"), - read.spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + read.spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) ); assertTrue(read.updateTime() > version); @@ -575,7 +575,7 @@ public void testHideColumns() public void testDropColumns() { String tableName = "drop"; - TableSpec dsSpec = TableBuilder.detailTable(tableName, "P1D") + TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D") .column("a", "VARCHAR") .column("b", "BIGINT") .column("c", "FLOAT") diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java index 8640515dc60f..0d03a46e0098 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java @@ -26,6 +26,7 @@ import org.apache.druid.catalog.model.table.TableBuilder; import org.apache.druid.catalog.storage.HideColumns; import org.apache.druid.catalog.storage.MoveColumn; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.junit.Test; import java.util.Arrays; @@ -42,7 +43,7 @@ public class CommandTest @Test public void testMoveColumn() { - TableSpec dsSpec = TableBuilder.detailTable("foo", "P1D") + TableSpec dsSpec = TableBuilder.datasource("foo", "P1D") .column("a", "VARCHAR") .column("b", "BIGINT") .column("c", "FLOAT") @@ -96,8 +97,8 @@ public void testMoveColumn() CatalogUtils.columnNames(revised) ); - byte[] bytes = CatalogUtils.toBytes(mapper, cmd); - MoveColumn cmd2 = CatalogUtils.fromBytes(mapper, bytes, MoveColumn.class); + byte[] bytes = JacksonUtils.toBytes(mapper, cmd); + MoveColumn cmd2 = JacksonUtils.fromBytes(mapper, bytes, MoveColumn.class); assertEquals(cmd, cmd2); } @@ -149,8 +150,8 @@ public void testHideColumns() revised = cmd.perform(Arrays.asList("a", "b", "c")); assertEquals(Arrays.asList("a", "b", "d", "e"), revised); - byte[] bytes = CatalogUtils.toBytes(mapper, cmd); - HideColumns cmd2 = CatalogUtils.fromBytes(mapper, bytes, HideColumns.class); + byte[] bytes = JacksonUtils.toBytes(mapper, cmd); + HideColumns cmd2 = JacksonUtils.fromBytes(mapper, bytes, HideColumns.class); assertEquals(cmd, cmd2); } } diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java index 298ea62acae0..44f19acbac00 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java @@ -24,7 +24,7 @@ import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; -import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; import org.apache.druid.catalog.model.table.TableBuilder; import org.apache.druid.catalog.storage.HideColumns; import org.apache.druid.catalog.storage.MoveColumn; @@ -82,7 +82,7 @@ public void testErrors() { final TableMetadata table = new TableBuilder() .id(TableId.of(TableId.SYSTEM_SCHEMA, "foo")) - .property(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D") + .property(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D") .build(); assertThrows( Exception.class, @@ -92,7 +92,7 @@ public void testErrors() // Malformed table name { - final TableMetadata table = TableBuilder.detailTable(" foo ", "P1D") + final TableMetadata table = TableBuilder.datasource(" foo ", "P1D") .build(); assertThrows( Exception.class, @@ -111,7 +111,7 @@ public void testLifecycle() CatalogClient client = new CatalogClient(clusterClient); // Create a datasource - TableMetadata table = TableBuilder.detailTable("example", "P1D") + TableMetadata table = TableBuilder.datasource("example", "P1D") .column("a", "VARCHAR") .column("b", "BIGINT") .column("c", "FLOAT") @@ -123,7 +123,7 @@ public void testLifecycle() // Update the datasource TableSpec dsSpec2 = TableBuilder.copyOf(table) - .property(DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 3_000_000) + .property(AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 3_000_000) .column("d", "DOUBLE") .buildSpec(); @@ -156,7 +156,7 @@ public void testLifecycle() read = client.readTable(table.id()); assertEquals( Arrays.asList("e", "f"), - read.spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + read.spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) ); // Unhide @@ -168,7 +168,7 @@ public void testLifecycle() read = client.readTable(table.id()); assertEquals( Collections.singletonList("f"), - read.spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + read.spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) ); // List schemas diff --git a/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java b/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java index 93b7f06f81fd..a0e4dd029534 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java +++ b/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java @@ -32,7 +32,6 @@ import org.apache.druid.segment.ColumnInspector; import javax.annotation.Nullable; - import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; diff --git a/server/pom.xml b/server/pom.xml index 77644b7f8aa4..83f45873b802 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -314,8 +314,8 @@ jackson-module-guice - org.apache.commons - commons-lang3 + org.apache.commons + commons-lang3 diff --git a/server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java b/server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java index 3d558ddb9ba8..ce9cc91ad2a5 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java +++ b/server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java @@ -19,75 +19,26 @@ package org.apache.druid.catalog.model; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Strings; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.PeriodGranularity; import org.joda.time.Period; -import java.io.IOException; -import java.util.ArrayList; +import javax.annotation.Nullable; + import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; +import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.Set; +import java.util.Objects; import java.util.stream.Collectors; +import java.util.stream.Stream; public class CatalogUtils { - // Amazing that a parser doesn't already exist... - private static final Map GRANULARITIES = new HashMap<>(); - - static { - GRANULARITIES.put("millisecond", Granularities.SECOND); - GRANULARITIES.put("second", Granularities.SECOND); - GRANULARITIES.put("minute", Granularities.MINUTE); - GRANULARITIES.put("5 minute", Granularities.FIVE_MINUTE); - GRANULARITIES.put("5 minutes", Granularities.FIVE_MINUTE); - GRANULARITIES.put("five_minute", Granularities.FIVE_MINUTE); - GRANULARITIES.put("10 minute", Granularities.TEN_MINUTE); - GRANULARITIES.put("10 minutes", Granularities.TEN_MINUTE); - GRANULARITIES.put("ten_minute", Granularities.TEN_MINUTE); - GRANULARITIES.put("15 minute", Granularities.FIFTEEN_MINUTE); - GRANULARITIES.put("15 minutes", Granularities.FIFTEEN_MINUTE); - GRANULARITIES.put("fifteen_minute", Granularities.FIFTEEN_MINUTE); - GRANULARITIES.put("30 minute", Granularities.THIRTY_MINUTE); - GRANULARITIES.put("30 minutes", Granularities.THIRTY_MINUTE); - GRANULARITIES.put("thirty_minute", Granularities.THIRTY_MINUTE); - GRANULARITIES.put("hour", Granularities.HOUR); - GRANULARITIES.put("6 hour", Granularities.SIX_HOUR); - GRANULARITIES.put("6 hours", Granularities.SIX_HOUR); - GRANULARITIES.put("six_hour", Granularities.SIX_HOUR); - GRANULARITIES.put("day", Granularities.DAY); - GRANULARITIES.put("week", Granularities.WEEK); - GRANULARITIES.put("month", Granularities.MONTH); - GRANULARITIES.put("quarter", Granularities.QUARTER); - GRANULARITIES.put("year", Granularities.YEAR); - GRANULARITIES.put("all", Granularities.ALL); - } - - public static Granularity toGranularity(String value) - { - return GRANULARITIES.get(StringUtils.toLowerCase(value)); - } - - public static int findColumn(List columns, String colName) - { - for (int i = 0; i < columns.size(); i++) { - if (columns.get(i).name().equals(colName)) { - return i; - } - } - return -1; - } - public static List columnNames(List columns) { return columns @@ -96,23 +47,6 @@ public static List columnNames(List columns) .collect(Collectors.toList()); } - public static List dropColumns( - final List columns, - final List toDrop) - { - if (toDrop == null || toDrop.isEmpty()) { - return columns; - } - Set drop = new HashSet(toDrop); - List revised = new ArrayList<>(); - for (T col : columns) { - if (!drop.contains(col.name())) { - revised.add(col); - } - } - return revised; - } - /** * Convert a catalog granularity string to the Druid form. Catalog granularities * are either the usual descriptive strings (in any case), or an ISO period. @@ -124,11 +58,6 @@ public static Granularity asDruidGranularity(String value) if (Strings.isNullOrEmpty(value)) { return Granularities.ALL; } - Granularity gran = toGranularity(value); - if (gran != null) { - return gran; - } - try { return new PeriodGranularity(new Period(value), null, null); } @@ -180,63 +109,15 @@ public static String stringListToLines(List lines) return String.join("\n", lines) + "\n"; } - public static Set setOf(String...items) - { - if (items.length == 0) { - return null; - } - return new HashSet<>(Arrays.asList(items)); - } - - public static byte[] toBytes(ObjectMapper jsonMapper, Object obj) - { - try { - return jsonMapper.writeValueAsBytes(obj); - } - catch (JsonProcessingException e) { - throw new ISE("Failed to serialize " + obj.getClass().getSimpleName()); - } - } - - public static T fromBytes(ObjectMapper jsonMapper, byte[] bytes, Class clazz) - { - try { - return jsonMapper.readValue(bytes, clazz); - } - catch (IOException e) { - throw new ISE(e, "Failed to deserialize a " + clazz.getSimpleName()); - } - } - - public static String toString(Object obj) - { - ObjectMapper jsonMapper = new ObjectMapper(); - try { - return jsonMapper.writeValueAsString(obj); - } - catch (JsonProcessingException e) { - throw new ISE("Failed to serialize TableDefn"); - } - } - public static List concatLists( - final List base, - final List additions + @Nullable final List base, + @Nullable final List additions ) { - if (base == null && additions != null) { - return additions; - } - if (base != null && additions == null) { - return base; - } - List extended = new ArrayList<>(); - if (base != null) { - extended.addAll(base); - } - if (additions != null) { - extended.addAll(additions); - } - return extended; + return Stream + .of(base, additions) + .filter(Objects::nonNull) + .flatMap(Collection::stream) + .collect(Collectors.toList()); } } diff --git a/server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.java b/server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.java index 797abf5f83b7..bb8d11b82088 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.java @@ -69,7 +69,7 @@ public void validate(ObjectMapper jsonMapper) public ColumnDefn( final String name, final String typeValue, - final List fields + final List> fields ) { super(name, typeValue, fields); diff --git a/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java b/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java index f56e6c871c3b..fe816ad8681c 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java +++ b/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java @@ -27,6 +27,7 @@ import com.google.common.base.Strings; import org.apache.druid.guice.annotations.UnstableApi; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import javax.annotation.Nullable; @@ -98,18 +99,18 @@ public void validate() public byte[] toBytes(ObjectMapper jsonMapper) { - return CatalogUtils.toBytes(jsonMapper, this); + return JacksonUtils.toBytes(jsonMapper, this); } public static ColumnSpec fromBytes(ObjectMapper jsonMapper, byte[] bytes) { - return CatalogUtils.fromBytes(jsonMapper, bytes, ColumnSpec.class); + return JacksonUtils.fromBytes(jsonMapper, bytes, ColumnSpec.class); } @Override public String toString() { - return CatalogUtils.toString(this); + return JacksonUtils.toString(this); } @Override diff --git a/server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java b/server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java index b1fb5ed5834c..a4fe72aa2f33 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java @@ -39,12 +39,12 @@ public class ObjectDefn { private final String name; private final String typeValue; - private final Map properties; + private final Map> properties; public ObjectDefn( final String name, final String typeValue, - final List fields + final List> fields ) { this.name = name; @@ -52,11 +52,11 @@ public ObjectDefn( this.properties = toPropertyMap(fields); } - protected static Map toPropertyMap(final List props) + protected static Map> toPropertyMap(final List> props) { - ImmutableMap.Builder builder = ImmutableMap.builder(); + ImmutableMap.Builder> builder = ImmutableMap.builder(); if (props != null) { - for (PropertyDefn prop : props) { + for (PropertyDefn prop : props) { builder.put(prop.name(), prop); } } @@ -78,19 +78,19 @@ public String typeValue() return typeValue; } - public Map properties() + public Map> properties() { return properties; } - public PropertyDefn property(String key) + public PropertyDefn property(String key) { return properties.get(key); } /** * Merge the properties for an object using a set of updates in a map. If the - * update value is null, then remove the property in the revised set. If the + * update value is {@code null}, then remove the property in the revised set. If the * property is known, use the column definition to merge the values. Else, the * update replaces any existing value. *

@@ -113,7 +113,7 @@ protected Map mergeProperties( if (entry.getValue() == null) { merged.remove(entry.getKey()); } else { - PropertyDefn propDefn = property(entry.getKey()); + PropertyDefn propDefn = property(entry.getKey()); Object value = entry.getValue(); if (propDefn != null) { value = propDefn.merge(merged.get(entry.getKey()), entry.getValue()); @@ -131,7 +131,7 @@ protected Map mergeProperties( */ public void validate(Map spec, ObjectMapper jsonMapper) { - for (PropertyDefn propDefn : properties.values()) { + for (PropertyDefn propDefn : properties.values()) { propDefn.validate(spec.get(propDefn.name()), jsonMapper); } } diff --git a/server/src/main/java/org/apache/druid/catalog/model/Parameterized.java b/server/src/main/java/org/apache/druid/catalog/model/ParameterizedDefn.java similarity index 97% rename from server/src/main/java/org/apache/druid/catalog/model/Parameterized.java rename to server/src/main/java/org/apache/druid/catalog/model/ParameterizedDefn.java index 74531d9cced3..c111e76daea3 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/Parameterized.java +++ b/server/src/main/java/org/apache/druid/catalog/model/ParameterizedDefn.java @@ -24,7 +24,7 @@ import java.util.List; import java.util.Map; -public interface Parameterized +public interface ParameterizedDefn { interface ParameterDefn { diff --git a/server/src/main/java/org/apache/druid/catalog/model/Properties.java b/server/src/main/java/org/apache/druid/catalog/model/Properties.java index e99475ddfb28..52be17af9bd5 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/Properties.java +++ b/server/src/main/java/org/apache/druid/catalog/model/Properties.java @@ -44,15 +44,16 @@ */ public interface Properties { - interface PropertyDefn + interface PropertyDefn { String name(); String typeName(); void validate(Object value, ObjectMapper jsonMapper); Object merge(Object existing, Object update); + T decode(Object value, ObjectMapper jsonMapper); } - abstract class BasePropertyDefn implements PropertyDefn + abstract class BasePropertyDefn implements PropertyDefn { protected final String name; @@ -83,7 +84,7 @@ public String toString() } } - class SimplePropertyDefn extends BasePropertyDefn + class SimplePropertyDefn extends BasePropertyDefn { public final Class valueClass; @@ -108,6 +109,7 @@ public String typeName() * SQL parameters. As a side effect, verifies that the value is of * the correct type. */ + @Override public T decode(Object value, ObjectMapper jsonMapper) { if (value == null) { @@ -137,7 +139,7 @@ public void validate(Object value, ObjectMapper jsonMapper) } } - class TypeRefPropertyDefn extends BasePropertyDefn + class TypeRefPropertyDefn extends BasePropertyDefn { public final String typeName; public final TypeReference valueType; @@ -159,6 +161,7 @@ public String typeName() return typeName; } + @Override public T decode(Object value, ObjectMapper jsonMapper) { if (value == null) { diff --git a/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java index 3ca91d6fbc8a..a984502b316c 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java +++ b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java @@ -19,7 +19,7 @@ package org.apache.druid.catalog.model; -import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; import org.apache.druid.catalog.model.table.ExternalTableDefn; import org.apache.druid.server.security.ResourceType; @@ -92,7 +92,7 @@ public SchemaRegistryImpl() register(new SchemaDefnImpl( TableId.DRUID_SCHEMA, ResourceType.DATASOURCE, - DatasourceDefn.tableTypes() + AbstractDatasourceDefn.tableTypes() )); register(new SchemaDefnImpl( TableId.LOOKUP_SCHEMA, diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java index 3ec5de4a65de..0341f581337f 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java @@ -38,7 +38,7 @@ * properties and a schema. Subclasses define specific table types * such as datasources or input tables. Some tables may be parameterized * to allow the table to appear in a SQL table function by implementing - * the {@link Parameterized} interface. + * the {@link ParameterizedDefn} interface. */ public class TableDefn extends ObjectDefn { @@ -52,7 +52,7 @@ public class TableDefn extends ObjectDefn public TableDefn( final String name, final String typeValue, - final List properties, + final List> properties, final List columnDefns ) { diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java b/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java index bcf1d088b4a2..3a29911e2228 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java @@ -33,8 +33,7 @@ public class TableDefnRegistry { private static final TableDefn[] TABLE_DEFNS = { - new DatasourceDefn.DetailDatasourceDefn(), - new DatasourceDefn.RollupDatasourceDefn(), + new DatasourceDefn(), new InlineTableDefn(), new HttpTableDefn(), new LocalTableDefn() diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java index 5fbc69c26330..f920ef6055be 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java @@ -25,6 +25,7 @@ import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import java.util.Objects; @@ -196,18 +197,18 @@ public void validate() public byte[] toBytes(ObjectMapper jsonMapper) { - return CatalogUtils.toBytes(jsonMapper, this); + return JacksonUtils.toBytes(jsonMapper, this); } public static TableMetadata fromBytes(ObjectMapper jsonMapper, byte[] bytes) { - return CatalogUtils.fromBytes(jsonMapper, bytes, TableMetadata.class); + return JacksonUtils.fromBytes(jsonMapper, bytes, TableMetadata.class); } @Override public String toString() { - return CatalogUtils.toString(this); + return JacksonUtils.toString(this); } @Override diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java b/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java index be93046a616a..a0893f7da7d4 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java @@ -26,6 +26,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Strings; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import java.util.Collections; import java.util.HashMap; @@ -113,18 +114,18 @@ public void validate() public byte[] toBytes(ObjectMapper jsonMapper) { - return CatalogUtils.toBytes(jsonMapper, this); + return JacksonUtils.toBytes(jsonMapper, this); } public static TableSpec fromBytes(ObjectMapper jsonMapper, byte[] bytes) { - return CatalogUtils.fromBytes(jsonMapper, bytes, TableSpec.class); + return JacksonUtils.fromBytes(jsonMapper, bytes, TableSpec.class); } @Override public String toString() { - return CatalogUtils.toString(this); + return JacksonUtils.toString(this); } @Override diff --git a/server/src/main/java/org/apache/druid/catalog/model/facade/ColumnFacade.java b/server/src/main/java/org/apache/druid/catalog/model/facade/ColumnFacade.java index 783c5a7e41d6..13d0384509f1 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/facade/ColumnFacade.java +++ b/server/src/main/java/org/apache/druid/catalog/model/facade/ColumnFacade.java @@ -21,10 +21,6 @@ import org.apache.druid.catalog.model.ColumnDefn.ResolvedColumn; import org.apache.druid.catalog.model.Columns; -import org.apache.druid.catalog.model.table.DatasourceDefn; -import org.apache.druid.catalog.model.table.MeasureTypes; -import org.apache.druid.catalog.model.table.MeasureTypes.MeasureType; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.segment.column.ColumnType; public class ColumnFacade @@ -44,25 +40,6 @@ public ColumnType druidType() } return super.druidType(); } - - public boolean isMeasure() - { - return DatasourceDefn.MEASURE_TYPE.equals(column.spec().type()); - } - - public MeasureType measureType() - { - String sqlType = column.spec().sqlType(); - if (sqlType == null) { - return null; - } - try { - return MeasureTypes.parse(sqlType); - } - catch (ISE e) { - return null; - } - } } protected final ResolvedColumn column; diff --git a/server/src/main/java/org/apache/druid/catalog/model/facade/DatasourceFacade.java b/server/src/main/java/org/apache/druid/catalog/model/facade/DatasourceFacade.java index 5fa77342e589..bed1245da203 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/facade/DatasourceFacade.java +++ b/server/src/main/java/org/apache/druid/catalog/model/facade/DatasourceFacade.java @@ -21,18 +21,17 @@ import org.apache.druid.catalog.model.CatalogUtils; import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; import org.apache.druid.catalog.model.table.ClusterKeySpec; -import org.apache.druid.catalog.model.table.DatasourceDefn; import org.apache.druid.java.util.common.granularity.Granularity; import java.util.Collections; import java.util.List; /** - * Convenience wrapper on top of a resolved table (a table spec - * and its corresponding definition.) To be used by consumers - * of catalog objects that work with specific datasource properties - * rather than layers that work with specs generically. + * Convenience wrapper on top of a resolved table (a table spec and its corresponding + * definition.) To be used by consumers of catalog objects that work with specific + * datasource properties rather than layers that work with specs generically. */ public class DatasourceFacade extends TableFacade { @@ -41,48 +40,32 @@ public DatasourceFacade(ResolvedTable resolved) super(resolved); } - public boolean isRollup() - { - return DatasourceDefn.ROLLUP_DATASOURCE_TYPE.equals(spec().type()); - } - - public boolean isDetail() - { - return !isRollup(); - } - public String segmentGranularityString() { - return stringProperty(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY); + return stringProperty(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY); } public Granularity segmentGranularity() { - String value = stringProperty(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY); + String value = stringProperty(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY); return value == null ? null : CatalogUtils.asDruidGranularity(value); } public Integer targetSegmentRows() { - return intProperty(DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY); + return intProperty(AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY); } @SuppressWarnings("unchecked") public List clusterKeys() { - return (List) property(DatasourceDefn.CLUSTER_KEYS_PROPERTY); + return (List) property(AbstractDatasourceDefn.CLUSTER_KEYS_PROPERTY); } @SuppressWarnings("unchecked") public List hiddenColumns() { - Object value = property(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY); + Object value = property(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); return value == null ? Collections.emptyList() : (List) value; } - - public Granularity rollupGranularity() - { - String value = stringProperty(DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY); - return value == null ? null : CatalogUtils.asDruidGranularity(value); - } } diff --git a/server/src/main/java/org/apache/druid/catalog/model/facade/InputTableFacade.java b/server/src/main/java/org/apache/druid/catalog/model/facade/ExternalTableFacade.java similarity index 94% rename from server/src/main/java/org/apache/druid/catalog/model/facade/InputTableFacade.java rename to server/src/main/java/org/apache/druid/catalog/model/facade/ExternalTableFacade.java index c7416caf1145..f2571150305e 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/facade/InputTableFacade.java +++ b/server/src/main/java/org/apache/druid/catalog/model/facade/ExternalTableFacade.java @@ -28,9 +28,9 @@ import java.util.List; -public class InputTableFacade extends TableFacade +public class ExternalTableFacade extends TableFacade { - public InputTableFacade(ResolvedTable resolved) + public ExternalTableFacade(ResolvedTable resolved) { super(resolved); } diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.java new file mode 100644 index 000000000000..bf5d4f1a6aef --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.catalog.model.table; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.catalog.model.CatalogUtils; +import org.apache.druid.catalog.model.ColumnDefn; +import org.apache.druid.catalog.model.Columns; +import org.apache.druid.catalog.model.Properties; +import org.apache.druid.catalog.model.Properties.GranularityPropertyDefn; +import org.apache.druid.catalog.model.Properties.PropertyDefn; +import org.apache.druid.catalog.model.Properties.StringListPropertyDefn; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableDefn; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; + +import java.util.Arrays; +import java.util.List; +import java.util.Set; + +public class AbstractDatasourceDefn extends TableDefn +{ + /** + * Segment grain at ingestion and initial compaction. Aging rules + * may override the value as segments age. If not provided here, + * then it must be provided at ingestion time. + */ + public static final String SEGMENT_GRANULARITY_PROPERTY = "segmentGranularity"; + + /** + * The target segment size at ingestion and initial compaction. + * If unset, then the system setting is used. + */ + public static final String TARGET_SEGMENT_ROWS_PROPERTY = "targetSegmentRows"; + public static final String CLUSTER_KEYS_PROPERTY = "clusterKeys"; + public static final String HIDDEN_COLUMNS_PROPERTY = "hiddenColumns"; + + public static class SegmentGranularityFieldDefn extends GranularityPropertyDefn + { + public SegmentGranularityFieldDefn() + { + super(SEGMENT_GRANULARITY_PROPERTY); + } + + @Override + public void validate(Object value, ObjectMapper jsonMapper) + { + String gran = decode(value, jsonMapper); + if (Strings.isNullOrEmpty(gran)) { + throw new IAE("Segment granularity is required."); + } + validateGranularity(gran); + } + } + + public static class HiddenColumnsDefn extends StringListPropertyDefn + { + public HiddenColumnsDefn() + { + super(HIDDEN_COLUMNS_PROPERTY); + } + + @Override + public void validate(Object value, ObjectMapper jsonMapper) + { + if (value == null) { + return; + } + List hiddenColumns = decode(value, jsonMapper); + for (String col : hiddenColumns) { + if (Columns.TIME_COLUMN.equals(col)) { + throw new IAE( + StringUtils.format("Cannot hide column %s", col) + ); + } + } + } + } + + public AbstractDatasourceDefn( + final String name, + final String typeValue, + final List> properties, + final List columnDefns + ) + { + super( + name, + typeValue, + CatalogUtils.concatLists( + Arrays.asList( + new SegmentGranularityFieldDefn(), + new Properties.IntPropertyDefn(TARGET_SEGMENT_ROWS_PROPERTY), + new Properties.ListPropertyDefn( + CLUSTER_KEYS_PROPERTY, + "cluster keys", + new TypeReference>() { } + ), + new HiddenColumnsDefn() + ), + properties + ), + columnDefns + ); + } + + public static boolean isDatasource(String tableType) + { + return DatasourceDefn.TABLE_TYPE.equals(tableType); + } + + public static boolean isDatasource(ResolvedTable table) + { + return table.defn() instanceof AbstractDatasourceDefn; + } + + public static Set tableTypes() + { + return ImmutableSet.of( + DatasourceDefn.TABLE_TYPE + ); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/DatasourceDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/DatasourceDefn.java index 0d8c4d7a95a7..de09ece33f06 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/DatasourceDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/DatasourceDefn.java @@ -19,135 +19,31 @@ package org.apache.druid.catalog.model.table; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Strings; -import org.apache.druid.catalog.model.CatalogUtils; import org.apache.druid.catalog.model.ColumnDefn; import org.apache.druid.catalog.model.ColumnSpec; -import org.apache.druid.catalog.model.Columns; -import org.apache.druid.catalog.model.Properties; -import org.apache.druid.catalog.model.Properties.GranularityPropertyDefn; -import org.apache.druid.catalog.model.Properties.PropertyDefn; -import org.apache.druid.catalog.model.Properties.StringListPropertyDefn; -import org.apache.druid.catalog.model.ResolvedTable; -import org.apache.druid.catalog.model.TableDefn; -import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.StringUtils; -import java.util.Arrays; import java.util.Collections; -import java.util.List; -import java.util.Set; -public class DatasourceDefn extends TableDefn +/** + * Definition of a Druid datasource. The datasource may use rollup, but rollup + * is not represented in the catalog: it is just a way that applications store + * data into a table. + */ +public class DatasourceDefn extends AbstractDatasourceDefn { /** - * Segment grain at ingestion and initial compaction. Aging rules - * may override the value as segments age. If not provided here, - * then it must be provided at ingestion time. - */ - public static final String SEGMENT_GRANULARITY_PROPERTY = "segmentGranularity"; - - /** - * The target segment size at ingestion and initial compaction. - * If unset, then the system setting is used. + * Definition of a column in a datasource. */ - public static final String TARGET_SEGMENT_ROWS_PROPERTY = "targetSegmentRows"; - public static final String CLUSTER_KEYS_PROPERTY = "clusterKeys"; - public static final String HIDDEN_COLUMNS_PROPERTY = "hiddenColumns"; - - /** - * Ingestion and auto-compaction rollup granularity. If null, then no - * rollup is enabled. Same as {@code queryGranularity} in and ingest spec, - * but renamed since this granularity affects rollup, not queries. Can be - * overridden at ingestion time. The grain may change as segments evolve: - * this is the grain only for ingest. - */ - public static final String ROLLUP_GRANULARITY_PROPERTY = "rollupGranularity"; - - public static final String DETAIL_DATASOURCE_TYPE = "detail"; - public static final String ROLLUP_DATASOURCE_TYPE = "rollup"; - - public static final String DETAIL_COLUMN_TYPE = "detail"; - public static final String DIMENSION_TYPE = "dimension"; - public static final String MEASURE_TYPE = "measure"; - public static final String INPUT_COLUMN_TYPE = "input"; - - public static class SegmentGranularityFieldDefn extends GranularityPropertyDefn - { - public SegmentGranularityFieldDefn() - { - super(SEGMENT_GRANULARITY_PROPERTY); - } - - @Override - public void validate(Object value, ObjectMapper jsonMapper) - { - String gran = decode(value, jsonMapper); - if (Strings.isNullOrEmpty(gran)) { - throw new IAE("Segment granularity is required."); - } - validateGranularity(gran); - } - } - - public static class HiddenColumnsDefn extends StringListPropertyDefn + public static class DatasourceColumnDefn extends ColumnDefn { - public HiddenColumnsDefn() - { - super(HIDDEN_COLUMNS_PROPERTY); - } - - @Override - public void validate(Object value, ObjectMapper jsonMapper) - { - if (value == null) { - return; - } - List hiddenColumns = decode(value, jsonMapper); - for (String col : hiddenColumns) { - if (Columns.TIME_COLUMN.equals(col)) { - throw new IAE( - StringUtils.format("Cannot hide column %s", col) - ); - } - } - } - } + public static final String COLUMN_TYPE = "column"; - /** - * Definition of a column in a detail (non-rollup) datasource. - */ - public static class DetailColumnDefn extends ColumnDefn - { - public DetailColumnDefn() + public DatasourceColumnDefn() { super( "Column", - DETAIL_COLUMN_TYPE, - null - ); - } - - @Override - public void validate(ColumnSpec spec, ObjectMapper jsonMapper) - { - super.validate(spec, jsonMapper); - validateScalarColumn(spec); - } - } - - /** - * Definition of a dimension in a rollup datasource. - */ - public static class DimensionDefn extends ColumnDefn - { - public DimensionDefn() - { - super( - "Dimension", - DIMENSION_TYPE, + COLUMN_TYPE, null ); } @@ -160,119 +56,15 @@ public void validate(ColumnSpec spec, ObjectMapper jsonMapper) } } - /** - * Definition of a measure (metric) column. - * Types are expressed as compound types: "AGG_FN(ARG_TYPE,...)" - * where "AGG_FN" is one of the supported aggregate functions, - * and "ARG_TYPE" is zero or more argument types. - */ - public static class MeasureDefn extends ColumnDefn - { - public MeasureDefn() - { - super( - "Measure", - MEASURE_TYPE, - null - ); - } - - @Override - public void validate(ColumnSpec spec, ObjectMapper jsonMapper) - { - super.validate(spec, jsonMapper); - if (spec.sqlType() == null) { - throw new IAE("A type is required for measure column " + spec.name()); - } - if (Columns.isTimeColumn(spec.name())) { - throw new IAE(StringUtils.format( - "%s column cannot be a measure", - Columns.TIME_COLUMN - )); - } - MeasureTypes.parse(spec.sqlType()); - } - } - - public static class DetailDatasourceDefn extends DatasourceDefn - { - public DetailDatasourceDefn() - { - super( - "Detail datasource", - DETAIL_DATASOURCE_TYPE, - null, - Collections.singletonList(new DetailColumnDefn()) - ); - } - } - - public static class RollupDatasourceDefn extends DatasourceDefn - { - public RollupDatasourceDefn() - { - super( - "Rollup datasource", - ROLLUP_DATASOURCE_TYPE, - Collections.singletonList( - new Properties.GranularityPropertyDefn(ROLLUP_GRANULARITY_PROPERTY) - ), - Arrays.asList( - new DimensionDefn(), - new MeasureDefn() - ) - ); - } - } + public static final String TABLE_TYPE = "datasource"; - public DatasourceDefn( - final String name, - final String typeValue, - final List properties, - final List columnDefns - ) + public DatasourceDefn() { super( - name, - typeValue, - CatalogUtils.concatLists( - Arrays.asList( - new SegmentGranularityFieldDefn(), - new Properties.IntPropertyDefn(TARGET_SEGMENT_ROWS_PROPERTY), - new Properties.ListPropertyDefn( - CLUSTER_KEYS_PROPERTY, - "cluster keys", - new TypeReference>() { } - ), - new HiddenColumnsDefn() - ), - properties - ), - columnDefns - ); - } - - public static boolean isDatasource(String tableType) - { - return DETAIL_DATASOURCE_TYPE.equals(tableType) - || ROLLUP_DATASOURCE_TYPE.equals(tableType); - } - - public static boolean isDatasource(ResolvedTable table) - { - return table.defn() instanceof DatasourceDefn; - } - - public static boolean isMeasure(ColumnSpec col) - { - return DatasourceDefn.MEASURE_TYPE.equals(col.type()); - } - - public static Set tableTypes() - { - return CatalogUtils.setOf( - DatasourceDefn.DETAIL_DATASOURCE_TYPE, - DatasourceDefn.ROLLUP_DATASOURCE_TYPE + "Datasource", + TABLE_TYPE, + null, + Collections.singletonList(new DatasourceDefn.DatasourceColumnDefn()) ); } } diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java index 6aae9cc45ac4..9f388cbe1f5f 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java @@ -21,12 +21,13 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import org.apache.druid.catalog.model.CatalogUtils; import org.apache.druid.catalog.model.ColumnDefn; import org.apache.druid.catalog.model.ColumnSpec; import org.apache.druid.catalog.model.Columns; -import org.apache.druid.catalog.model.Parameterized; -import org.apache.druid.catalog.model.Parameterized.ParameterDefn; +import org.apache.druid.catalog.model.ParameterizedDefn; +import org.apache.druid.catalog.model.ParameterizedDefn.ParameterDefn; import org.apache.druid.catalog.model.Properties.PropertyDefn; import org.apache.druid.catalog.model.ResolvedTable; import org.apache.druid.catalog.model.TableDefn; @@ -50,7 +51,7 @@ * properties, as do all table definitions. *

* The external table implements the mechanism for parameterized tables, - * but does not implement the {@link Parameterized} interface itself. + * but does not implement the {@link ParameterizedDefn} interface itself. * Tables which are parameterized implement that interface to expose * methods defined here. */ @@ -67,7 +68,7 @@ public abstract static class FormattedExternalTableDefn extends ExternalTableDef public FormattedExternalTableDefn( final String name, final String typeValue, - final List properties, + final List> properties, final List columnDefns, final List formats, final List parameters @@ -92,16 +93,16 @@ public FormattedExternalTableDefn( * in the order defined by the format. Allow same-named properties across * formats, as long as the types are the same. */ - private static List addFormatProperties( - final List properties, + private static List> addFormatProperties( + final List> properties, final List formats ) { - List toAdd = new ArrayList<>(); - Map formatProps = new HashMap<>(); + List> toAdd = new ArrayList<>(); + Map> formatProps = new HashMap<>(); for (InputFormatDefn format : formats) { - for (PropertyDefn prop : format.properties()) { - PropertyDefn existing = formatProps.putIfAbsent(prop.name(), prop); + for (PropertyDefn prop : format.properties()) { + PropertyDefn existing = formatProps.putIfAbsent(prop.name(), prop); if (existing == null) { toAdd.add(prop); } else if (existing.getClass() != prop.getClass()) { @@ -186,7 +187,7 @@ public void validate(ColumnSpec spec, ObjectMapper jsonMapper) public ExternalTableDefn( final String name, final String typeValue, - final List fields, + final List> fields, final List columnDefns, final List parameters ) @@ -263,7 +264,7 @@ public static boolean isInputTable(ResolvedTable table) public static Set tableTypes() { // Known input tables. Get this from a registry later. - return CatalogUtils.setOf( + return ImmutableSet.of( InlineTableDefn.TABLE_TYPE, HttpTableDefn.TABLE_TYPE, LocalTableDefn.TABLE_TYPE diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.java index 6b4f5f1cbd34..9210a77eae50 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.java @@ -22,7 +22,7 @@ import com.google.common.base.Strings; import com.google.common.collect.ImmutableMap; import org.apache.druid.catalog.model.CatalogUtils; -import org.apache.druid.catalog.model.Parameterized; +import org.apache.druid.catalog.model.ParameterizedDefn; import org.apache.druid.catalog.model.Properties.StringListPropertyDefn; import org.apache.druid.catalog.model.Properties.StringPropertyDefn; import org.apache.druid.catalog.model.ResolvedTable; @@ -57,7 +57,7 @@ * form where the user provides the partial URLs to use for a particular * query. */ -public class HttpTableDefn extends FormattedExternalTableDefn implements Parameterized +public class HttpTableDefn extends FormattedExternalTableDefn implements ParameterizedDefn { public static final String TABLE_TYPE = HttpInputSource.TYPE_KEY; public static final String URI_TEMPLATE_PROPERTY = "template"; diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/InputFormats.java b/server/src/main/java/org/apache/druid/catalog/model/table/InputFormats.java index 658f1139c794..da93a272758c 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/InputFormats.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/InputFormats.java @@ -51,7 +51,7 @@ public interface InputFormatDefn { String name(); String typeTag(); - List properties(); + List> properties(); void validate(ResolvedTable table); InputFormat convert(ResolvedTable table); } @@ -60,12 +60,12 @@ public abstract static class BaseFormatDefn implements InputFormatDefn { private final String name; private final String typeTag; - private final List properties; + private final List> properties; public BaseFormatDefn( final String name, final String typeTag, - final List properties + final List> properties ) { this.name = name; @@ -86,7 +86,7 @@ public String typeTag() } @Override - public List properties() + public List> properties() { return properties; } @@ -119,7 +119,7 @@ public abstract static class FlatTextFormatDefn extends BaseFormatDefn public FlatTextFormatDefn( final String name, final String typeTag, - final List properties + final List> properties ) { super( diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/LocalTableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/LocalTableDefn.java index 3877c977d81d..51be5e71f6d3 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/LocalTableDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/LocalTableDefn.java @@ -20,7 +20,7 @@ package org.apache.druid.catalog.model.table; import org.apache.druid.catalog.model.CatalogUtils; -import org.apache.druid.catalog.model.Parameterized; +import org.apache.druid.catalog.model.ParameterizedDefn; import org.apache.druid.catalog.model.Properties.StringListPropertyDefn; import org.apache.druid.catalog.model.Properties.StringPropertyDefn; import org.apache.druid.catalog.model.ResolvedTable; @@ -35,7 +35,7 @@ import java.util.List; import java.util.Map; -public class LocalTableDefn extends FormattedExternalTableDefn implements Parameterized +public class LocalTableDefn extends FormattedExternalTableDefn implements ParameterizedDefn { public static final String TABLE_TYPE = LocalInputSource.TYPE_KEY; diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/MeasureTypes.java b/server/src/main/java/org/apache/druid/catalog/model/table/MeasureTypes.java deleted file mode 100644 index 57ff20c36eea..000000000000 --- a/server/src/main/java/org/apache/druid/catalog/model/table/MeasureTypes.java +++ /dev/null @@ -1,216 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.catalog.model.table; - -import com.google.common.base.Strings; -import com.google.common.collect.ImmutableMap; -import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.segment.column.ColumnType; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import java.util.stream.Collectors; - -public class MeasureTypes -{ - public static final String OPTIONAL = "--"; - - public enum BaseType - { - VARCHAR(ColumnType.STRING), - BIGINT(ColumnType.LONG), - FLOAT(ColumnType.FLOAT), - DOUBLE(ColumnType.DOUBLE); - - public final ColumnType nativeType; - - BaseType(ColumnType nativeType) - { - this.nativeType = nativeType; - } - } - - public static class MeasureType - { - public final String name; - public final List argTypes; - public final String sqlSeedFn; - public final String sqlReducerFn; - public final String nativeType; - public final String nativeAggFn; - public final ColumnType storageType; - public final String nativeReducerFn; - - public MeasureType( - final String name, - final List argTypes, - final String sqlSeedFn, - final String sqlReducerFn, - final String nativeType, - final ColumnType storageType, - final String nativeAggFn, - final String nativeReducerFn - ) - { - this.name = name; - this.argTypes = argTypes == null ? Collections.emptyList() : argTypes; - this.sqlSeedFn = sqlSeedFn; - this.sqlReducerFn = sqlReducerFn; - this.nativeType = nativeType; - this.storageType = storageType; - this.nativeAggFn = nativeAggFn; - this.nativeReducerFn = nativeReducerFn; - } - - @Override - public String toString() - { - StringBuilder buf = new StringBuilder() - .append(name) - .append("("); - for (int i = 0; i < argTypes.size(); i++) { - if (i > 0) { - buf.append(", "); - } - buf.append(argTypes.get(i).name()); - } - return buf.append(")").toString(); - } - } - - // See: https://druid.apache.org/docs/latest/querying/aggregations.html - public static final MeasureType COUNT_TYPE = new MeasureType( - "COUNT", - null, - null, - "SUM", - "longSum", - ColumnType.LONG, - "count", - "longSum" - ); - - public static final MeasureType SUM_BIGINT_TYPE = simpleAggType("sum", BaseType.BIGINT); - public static final MeasureType SUM_FLOAT_TYPE = simpleAggType("sum", BaseType.FLOAT); - public static final MeasureType SUM_DOUBLE_TYPE = simpleAggType("sum", BaseType.DOUBLE); - public static final MeasureType MIN_BIGINT_TYPE = simpleAggType("min", BaseType.BIGINT); - public static final MeasureType MIN_FLOAT_TYPE = simpleAggType("min", BaseType.FLOAT); - public static final MeasureType MIN_DOUBLE_TYPE = simpleAggType("min", BaseType.DOUBLE); - public static final MeasureType MAX_BIGINT_TYPE = simpleAggType("max", BaseType.BIGINT); - public static final MeasureType MAX_FLOAT_TYPE = simpleAggType("max", BaseType.FLOAT); - public static final MeasureType MAX_DOUBLE_TYPE = simpleAggType("max", BaseType.DOUBLE); - - private static MeasureType simpleAggType(String fn, BaseType baseType) - { - String sqlFn = StringUtils.toUpperCase(fn); - String nativeFn = baseType.nativeType.asTypeString() + org.apache.commons.lang3.StringUtils.capitalize(fn); - return new MeasureType( - sqlFn, - Collections.singletonList(baseType), - sqlFn, - null, - sqlFn, - baseType.nativeType, - nativeFn, - nativeFn - ); - } - - private static final List TYPE_LIST = - Arrays.asList( - COUNT_TYPE, - SUM_BIGINT_TYPE, - SUM_FLOAT_TYPE, - SUM_DOUBLE_TYPE, - MIN_BIGINT_TYPE, - MIN_FLOAT_TYPE, - MIN_DOUBLE_TYPE, - MAX_BIGINT_TYPE, - MAX_FLOAT_TYPE, - MAX_DOUBLE_TYPE - ); - public static final Map> TYPES; - - static { - Map> map = new HashMap<>(); - for (MeasureType fn : TYPE_LIST) { - List overloads = map.computeIfAbsent(fn.name, x -> new ArrayList<>()); - overloads.add(fn); - } - TYPES = ImmutableMap.>builder().putAll(map).build(); - } - - public static MeasureType parse(String typeStr) - { - Pattern p = Pattern.compile("(\\w+)(?:\\s*\\((.*)\\))?"); - Matcher m = p.matcher(StringUtils.toUpperCase(typeStr.trim())); - if (!m.matches()) { - throw new IAE(StringUtils.format( - "The type [%s] is not well-formed. It must be FN, FN(TYPE) or FN(TYPE,TYPE...)", - typeStr - )); - } - String fnName = m.group(1); - String[] args; - String argGroup = m.group(2); - argGroup = argGroup == null ? null : argGroup.trim(); - if (Strings.isNullOrEmpty(argGroup)) { - args = new String[] {}; - } else { - args = argGroup.split(","); - for (int i = 0; i < args.length; i++) { - args[i] = args[i].trim(); - } - } - List candidates = TYPES.get(fnName); - if (candidates == null) { - throw new IAE(StringUtils.format( - "The metric type [%s] is not valid.", - fnName - )); - } - - top: - for (MeasureType type : candidates) { - if (type.argTypes.size() != args.length) { - continue; - } - for (int i = 0; i < args.length; i++) { - if (!type.argTypes.get(i).name().equalsIgnoreCase(args[i])) { - continue top; - } - } - return type; - } - - throw new IAE( - "[%s] is not a valid metric type. Valid forms are %s", - typeStr, - candidates.stream().map(t -> t.toString()).collect(Collectors.joining(", ")) - ); - } -} diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java b/server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java index a3296443dfc4..63e00de50d63 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java @@ -53,19 +53,11 @@ public class TableBuilder private Map properties = new HashMap<>(); private List columns = new ArrayList<>(); - public static TableBuilder detailTable(String name, String granularity) + public static TableBuilder datasource(String name, String granularity) { return new TableBuilder() .datasource(name) - .type(DatasourceDefn.DETAIL_DATASOURCE_TYPE) - .segmentGranularity(granularity); - } - - public static TableBuilder rollupTable(String name, String granularity) - { - return new TableBuilder() - .datasource(name) - .type(DatasourceDefn.ROLLUP_DATASOURCE_TYPE) + .type(DatasourceDefn.TABLE_TYPE) .segmentGranularity(granularity); } @@ -168,19 +160,14 @@ public TableBuilder description(String description) return property(TableDefn.DESCRIPTION_PROPERTY, description); } - public TableBuilder rollupGranularity(String rollupGranularty) - { - return property(DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY, rollupGranularty); - } - public TableBuilder segmentGranularity(String segmentGranularity) { - return property(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, segmentGranularity); + return property(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, segmentGranularity); } public TableBuilder clusterColumns(ClusterKeySpec...clusterKeys) { - return property(DatasourceDefn.CLUSTER_KEYS_PROPERTY, Arrays.asList(clusterKeys)); + return property(AbstractDatasourceDefn.CLUSTER_KEYS_PROPERTY, Arrays.asList(clusterKeys)); } public TableBuilder columns(List columns) @@ -214,10 +201,8 @@ public TableBuilder column(String name, String sqlType) String colType; if (isInputTable(tableType)) { colType = ExternalTableDefn.EXTERNAL_COLUMN_TYPE; - } else if (DatasourceDefn.DETAIL_DATASOURCE_TYPE.equals(tableType)) { - colType = DatasourceDefn.DETAIL_COLUMN_TYPE; - } else if (DatasourceDefn.ROLLUP_DATASOURCE_TYPE.equals(tableType)) { - colType = DatasourceDefn.DIMENSION_TYPE; + } else if (DatasourceDefn.TABLE_TYPE.equals(tableType)) { + colType = DatasourceDefn.DatasourceColumnDefn.COLUMN_TYPE; } else { throw new ISE("Unknown table type: %s", tableType); } @@ -241,14 +226,9 @@ public TableBuilder column(String colType, String name, String sqlType) return column(new ColumnSpec(colType, name, sqlType, null)); } - public TableBuilder measure(String name, String sqlType) - { - return column(DatasourceDefn.MEASURE_TYPE, name, sqlType); - } - public TableBuilder hiddenColumns(List hiddenColumns) { - return property(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, hiddenColumns); + return property(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, hiddenColumns); } public TableBuilder hiddenColumns(String...hiddenColumns) diff --git a/server/src/test/java/org/apache/druid/catalog/model/MeasureTypesTest.java b/server/src/test/java/org/apache/druid/catalog/model/MeasureTypesTest.java deleted file mode 100644 index 762c1a6e3dda..000000000000 --- a/server/src/test/java/org/apache/druid/catalog/model/MeasureTypesTest.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.catalog.model; - -import org.apache.druid.catalog.CatalogTest; -import org.apache.druid.catalog.model.table.MeasureTypes; -import org.apache.druid.catalog.model.table.MeasureTypes.MeasureType; -import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.StringUtils; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -import java.util.Arrays; -import java.util.List; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertThrows; - -@Category(CatalogTest.class) -public class MeasureTypesTest -{ - @Test - public void testInvalid() - { - assertThrows(IAE.class, () -> MeasureTypes.parse("")); - assertThrows(IAE.class, () -> MeasureTypes.parse("FOO")); - assertThrows(IAE.class, () -> MeasureTypes.parse("FOO(")); - assertThrows(IAE.class, () -> MeasureTypes.parse("FOO)")); - assertThrows(IAE.class, () -> MeasureTypes.parse("MIN(,,)")); - assertThrows(IAE.class, () -> MeasureTypes.parse("MIN(VARCHAR")); - } - - @Test - public void testCount() - { - MeasureType type = MeasureTypes.parse("count"); - assertSame(MeasureTypes.COUNT_TYPE, type); - assertEquals(0, type.argTypes.size()); - - type = MeasureTypes.parse("COUNT()"); - assertSame(MeasureTypes.COUNT_TYPE, type); - assertEquals(0, type.argTypes.size()); - - type = MeasureTypes.parse(" COUNT( ) "); - assertSame(MeasureTypes.COUNT_TYPE, type); - assertEquals(0, type.argTypes.size()); - } - - @Test - public void testSingleArg() - { - testOneArg("SUM"); - testOneArg("MIN"); - testOneArg("MAX"); - - // Invalid - - assertThrows(IAE.class, () -> MeasureTypes.parse("SUM(VARCHAR)")); - assertThrows(IAE.class, () -> MeasureTypes.parse("SUM()")); - assertThrows(IAE.class, () -> MeasureTypes.parse("SUM(BIGINT, BIGINT)")); - - // Parsing variations - MeasureType typeRef = MeasureTypes.parse(" Min ( BiGiNt ) "); - assertSame(MeasureTypes.MIN_BIGINT_TYPE, typeRef); - } - - private void testOneArg(String fn) - { - List types = MeasureTypes.TYPES.get(fn); - for (String name : Arrays.asList(fn, StringUtils.toLowerCase(fn))) { - for (MeasureType measureType : types) { - String argType = measureType.argTypes.get(0).name(); - for (String argName : Arrays.asList(argType, StringUtils.toLowerCase(argType))) { - MeasureType typeRef = MeasureTypes.parse(StringUtils.format("%s(%s)", name, argName)); - assertSame(measureType, typeRef); - } - } - } - } -} diff --git a/server/src/test/java/org/apache/druid/catalog/model/TableMetadataTest.java b/server/src/test/java/org/apache/druid/catalog/model/TableMetadataTest.java index af79bd314d7a..329692096193 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/TableMetadataTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/TableMetadataTest.java @@ -24,8 +24,10 @@ import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.catalog.CatalogTest; import org.apache.druid.catalog.model.TableMetadata.TableState; +import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; import org.apache.druid.catalog.model.table.DatasourceDefn; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -55,8 +57,8 @@ public void testId() assertEquals(TableId.DRUID_SCHEMA, id2.schema()); assertEquals("ds", id2.name()); - byte[] bytes = CatalogUtils.toBytes(mapper, id1); - TableId id3 = CatalogUtils.fromBytes(mapper, bytes, TableId.class); + byte[] bytes = JacksonUtils.toBytes(mapper, id1); + TableId id3 = JacksonUtils.fromBytes(mapper, bytes, TableId.class); assertEquals(id1, id3); } @@ -72,9 +74,9 @@ public void testIdEquals() public void testTableMetadata() { Map props = ImmutableMap.of( - DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D" + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D" ); - TableSpec spec = new TableSpec(DatasourceDefn.DETAIL_DATASOURCE_TYPE, props, null); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); { TableMetadata table = new TableMetadata( TableId.datasource("foo"), @@ -91,8 +93,8 @@ public void testTableMetadata() assertEquals(TableState.ACTIVE, table.state()); assertNotNull(table.spec()); - byte[] bytes = CatalogUtils.toBytes(mapper, table); - TableMetadata table2 = CatalogUtils.fromBytes(mapper, bytes, TableMetadata.class); + byte[] bytes = JacksonUtils.toBytes(mapper, table); + TableMetadata table2 = JacksonUtils.fromBytes(mapper, bytes, TableMetadata.class); assertEquals(table, table2); } @@ -116,9 +118,9 @@ public void testTableMetadata() public void testConversions() { Map props = ImmutableMap.of( - DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D" + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D" ); - TableSpec spec = new TableSpec(DatasourceDefn.DETAIL_DATASOURCE_TYPE, props, null); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); TableMetadata table = TableMetadata.newTable( TableId.datasource("ds"), spec diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/DatasourceTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/DatasourceTableTest.java index 9dfeef3da2e5..9a76eee4f09c 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/DatasourceTableTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/DatasourceTableTest.java @@ -29,8 +29,7 @@ import org.apache.druid.catalog.model.TableDefn; import org.apache.druid.catalog.model.TableDefnRegistry; import org.apache.druid.catalog.model.TableSpec; -import org.apache.druid.catalog.model.table.DatasourceDefn.DetailDatasourceDefn; -import org.apache.druid.catalog.model.table.DatasourceDefn.RollupDatasourceDefn; +import org.apache.druid.catalog.model.table.DatasourceDefn.DatasourceColumnDefn; import org.apache.druid.java.util.common.IAE; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -64,21 +63,21 @@ public void testMinimalSpec() { // Minimum possible definition Map props = ImmutableMap.of( - DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D" + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D" ); { - TableSpec spec = new TableSpec(DatasourceDefn.DETAIL_DATASOURCE_TYPE, props, null); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); ResolvedTable table = registry.resolve(spec); assertNotNull(table); - assertTrue(table.defn() instanceof DetailDatasourceDefn); + assertTrue(table.defn() instanceof DatasourceDefn); table.validate(); } { - TableSpec spec = new TableSpec(DatasourceDefn.ROLLUP_DATASOURCE_TYPE, props, null); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); ResolvedTable table = registry.resolve(spec); assertNotNull(table); - assertTrue(table.defn() instanceof RollupDatasourceDefn); + assertTrue(table.defn() instanceof DatasourceDefn); table.validate(); } } @@ -109,13 +108,13 @@ public void testEmptySpec() } { - TableSpec spec = new TableSpec(DatasourceDefn.DETAIL_DATASOURCE_TYPE, ImmutableMap.of(), null); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, ImmutableMap.of(), null); ResolvedTable table = registry.resolve(spec); expectValidationFails(table); } { - TableSpec spec = new TableSpec(DatasourceDefn.ROLLUP_DATASOURCE_TYPE, ImmutableMap.of(), null); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, ImmutableMap.of(), null); expectValidationFails(spec); } } @@ -125,14 +124,13 @@ public void testAllProperties() { Map props = ImmutableMap.builder() .put(TableDefn.DESCRIPTION_PROPERTY, "My table") - .put(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D") - .put(DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY, "PT1M") - .put(DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000) - .put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("foo", "bar")) + .put(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D") + .put(AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000) + .put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("foo", "bar")) .build(); { - TableSpec spec = new TableSpec(DatasourceDefn.DETAIL_DATASOURCE_TYPE, props, null); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); expectValidationSucceeds(spec); // Check serialization @@ -141,7 +139,7 @@ public void testAllProperties() } { - TableSpec spec = new TableSpec(DatasourceDefn.ROLLUP_DATASOURCE_TYPE, props, null); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); expectValidationSucceeds(spec); // Check serialization @@ -160,47 +158,32 @@ public void testWrongTypes() // Segment granularity { - TableSpec spec = TableBuilder.detailTable("foo", "bogus").buildSpec(); + TableSpec spec = TableBuilder.datasource("foo", "bogus").buildSpec(); expectValidationFails(spec); } { - TableSpec spec = TableBuilder.rollupTable("foo", "bogus").buildSpec(); - expectValidationFails(spec); - } - - // Rollup granularity - { - TableSpec spec = TableBuilder.rollupTable("foo", "P1D") - .rollupGranularity("bogus") - .buildSpec(); - expectValidationFails(spec); - } - - { - TableSpec spec = TableBuilder.rollupTable("foo", "P1D") - .property(DatasourceDefn.ROLLUP_GRANULARITY_PROPERTY, 10) - .buildSpec(); + TableSpec spec = TableBuilder.datasource("foo", "bogus").buildSpec(); expectValidationFails(spec); } // Target segment rows { - TableSpec spec = TableBuilder.detailTable("foo", "P1D") - .property(DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, "bogus") + TableSpec spec = TableBuilder.datasource("foo", "P1D") + .property(AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, "bogus") .buildSpec(); expectValidationFails(spec); } // Hidden columns { - TableSpec spec = TableBuilder.detailTable("foo", "P1D") - .property(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, "bogus") + TableSpec spec = TableBuilder.datasource("foo", "P1D") + .property(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, "bogus") .buildSpec(); expectValidationFails(spec); } { - TableSpec spec = TableBuilder.detailTable("foo", "P1D") + TableSpec spec = TableBuilder.datasource("foo", "P1D") .hiddenColumns("a", Columns.TIME_COLUMN) .buildSpec(); expectValidationFails(spec); @@ -210,7 +193,7 @@ public void testWrongTypes() @Test public void testExtendedProperties() { - TableSpec spec = TableBuilder.detailTable("foo", "P1D") + TableSpec spec = TableBuilder.datasource("foo", "P1D") .property("foo", 10) .property("bar", "mumble") .buildSpec(); @@ -228,17 +211,17 @@ public void testColumnSpec() // Name is required { - ColumnSpec spec = new ColumnSpec(DatasourceDefn.DETAIL_COLUMN_TYPE, null, null, null); + ColumnSpec spec = new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, null, null, null); assertThrows(IAE.class, () -> spec.validate()); } { - ColumnSpec spec = new ColumnSpec(DatasourceDefn.DETAIL_COLUMN_TYPE, "foo", null, null); + ColumnSpec spec = new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "foo", null, null); spec.validate(); } // Type is optional { - ColumnSpec spec = new ColumnSpec(DatasourceDefn.DETAIL_COLUMN_TYPE, "foo", "VARCHAR", null); + ColumnSpec spec = new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "foo", "VARCHAR", null); spec.validate(); } } @@ -246,7 +229,7 @@ public void testColumnSpec() @Test public void testDetailTableColumns() { - TableBuilder builder = TableBuilder.detailTable("foo", "P1D"); + TableBuilder builder = TableBuilder.datasource("foo", "P1D"); // OK to have no columns { @@ -309,22 +292,6 @@ public void testDetailTableColumns() expectValidationFails(spec); } - // Cannot use a measure - { - TableSpec spec = builder.copy() - .measure("foo", SUM_BIGINT) - .buildSpec(); - expectValidationFails(spec); - } - - // Cannot use a dimension for a detail table - { - TableSpec spec = builder.copy() - .column(new ColumnSpec(DatasourceDefn.DIMENSION_TYPE, "foo", Columns.VARCHAR, null)) - .buildSpec(); - expectValidationFails(spec); - } - // Reject duplicate columns { TableSpec spec = builder.copy() @@ -342,111 +309,6 @@ public void testDetailTableColumns() } } - @Test - public void testRollupTableColumns() - { - TableBuilder builder = TableBuilder.rollupTable("foo", "P1D") - .rollupGranularity("PT1M"); - - // OK to have no columns - { - TableSpec spec = builder.buildSpec(); - expectValidationSucceeds(spec); - } - - // OK for a dimension to have no type - { - TableSpec spec = builder.copy() - .column("foo", null) - .buildSpec(); - expectValidationSucceeds(spec); - } - - // Dimensions must have a scalar type, if the type is non-null - { - TableSpec spec = builder.copy() - .column("foo", Columns.VARCHAR) - .buildSpec(); - expectValidationSucceeds(spec); - } - { - TableSpec spec = builder.copy() - .column("foo", "BOGUS") - .buildSpec(); - expectValidationFails(spec); - } - { - TableSpec spec = builder.copy() - .column("foo", SUM_BIGINT) - .buildSpec(); - expectValidationFails(spec); - } - - // Time column can be a dimension and can only have TIMESTAMP type - { - TableSpec spec = builder.copy() - .timeColumn() - .buildSpec(); - expectValidationSucceeds(spec); - } - { - TableSpec spec = builder.copy() - .column(Columns.TIME_COLUMN, Columns.VARCHAR) - .buildSpec(); - expectValidationFails(spec); - } - { - TableSpec spec = builder.copy() - .column(Columns.TIME_COLUMN, SUM_BIGINT) - .buildSpec(); - expectValidationFails(spec); - } - - // Measures must have an aggregate type - { - TableSpec spec = builder.copy() - .measure("foo", null) - .buildSpec(); - expectValidationFails(spec); - } - { - TableSpec spec = builder.copy() - .measure("foo", Columns.VARCHAR) - .buildSpec(); - expectValidationFails(spec); - } - { - TableSpec spec = builder.copy() - .measure("foo", SUM_BIGINT) - .buildSpec(); - expectValidationSucceeds(spec); - } - - // Cannot use a detail column - { - TableSpec spec = builder.copy() - .column(new ColumnSpec(DatasourceDefn.DETAIL_COLUMN_TYPE, "foo", null, null)) - .buildSpec(); - expectValidationFails(spec); - } - - // Reject duplicate columns - { - TableSpec spec = builder.copy() - .column("foo", Columns.VARCHAR) - .measure("bar", SUM_BIGINT) - .buildSpec(); - expectValidationSucceeds(spec); - } - { - TableSpec spec = builder.copy() - .column("foo", Columns.VARCHAR) - .measure("foo", SUM_BIGINT) - .buildSpec(); - expectValidationFails(spec); - } - } - @Test public void testEquals() { @@ -464,14 +326,13 @@ private TableSpec exampleSpec() .put("colProp1", "value 1") .put("colProp2", "value 2") .build(); - TableSpec spec = TableBuilder.rollupTable("foo", "PT1H") + TableSpec spec = TableBuilder.datasource("foo", "PT1H") .description("My table") - .rollupGranularity("PT1M") - .property(DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000) + .property(AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000) .hiddenColumns("foo", "bar") .property("tag1", "some value") .property("tag2", "second value") - .column(new ColumnSpec(DatasourceDefn.DIMENSION_TYPE, "a", null, colProps)) + .column(new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "a", null, colProps)) .column("b", Columns.VARCHAR) .buildSpec(); @@ -541,7 +402,7 @@ public void testMergeProperties() // such values to indicate which properties to remove. Map updatedProps = new HashMap<>(); // Update a property - updatedProps.put(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D"); + updatedProps.put(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D"); // Remove a property updatedProps.put("tag1", null); // Add a property @@ -556,8 +417,8 @@ public void testMergeProperties() // changed. assertNotEquals(spec, merged); assertEquals( - updatedProps.get(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY), - merged.properties().get(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY) + updatedProps.get(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY), + merged.properties().get(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY) ); assertFalse(merged.properties().containsKey("tag1")); assertEquals( @@ -573,24 +434,24 @@ public void testMergeHiddenCols() // Remove all hidden columns Map updatedProps = new HashMap<>(); - updatedProps.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, null); + updatedProps.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, null); TableSpec update = new TableSpec(null, updatedProps, null); TableSpec merged = mergeTables(spec, update); expectValidationSucceeds(merged); assertFalse( - merged.properties().containsKey(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + merged.properties().containsKey(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) ); // Wrong type updatedProps = ImmutableMap.of( - DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, "mumble" + AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, "mumble" ); update = new TableSpec(null, updatedProps, null); assertMergeFails(spec, update); // Merge updatedProps = ImmutableMap.of( - DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.singletonList("mumble") + AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.singletonList("mumble") ); update = new TableSpec(null, updatedProps, null); merged = mergeTables(spec, update); @@ -598,7 +459,7 @@ public void testMergeHiddenCols() assertEquals( Arrays.asList("foo", "bar", "mumble"), - merged.properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + merged.properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) ); } @@ -606,13 +467,13 @@ public void testMergeHiddenCols() public void testMergeColsWithEmptyList() { Map props = ImmutableMap.of( - DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D" + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D" ); - TableSpec spec = new TableSpec(DatasourceDefn.DETAIL_DATASOURCE_TYPE, props, null); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); List colUpdates = Collections.singletonList( new ColumnSpec( - DatasourceDefn.DETAIL_COLUMN_TYPE, + DatasourceColumnDefn.COLUMN_TYPE, "a", Columns.BIGINT, null @@ -641,13 +502,13 @@ public void testMergeCols() List colUpdates = Arrays.asList( new ColumnSpec( - DatasourceDefn.DIMENSION_TYPE, + DatasourceColumnDefn.COLUMN_TYPE, "a", Columns.BIGINT, updatedProps ), new ColumnSpec( - DatasourceDefn.DIMENSION_TYPE, + DatasourceColumnDefn.COLUMN_TYPE, "c", Columns.VARCHAR, null diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java index c90dac5c8a47..ee60e0cf3eb3 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java @@ -24,7 +24,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.catalog.CatalogTest; import org.apache.druid.catalog.model.Columns; -import org.apache.druid.catalog.model.Parameterized; +import org.apache.druid.catalog.model.ParameterizedDefn; import org.apache.druid.catalog.model.ResolvedTable; import org.apache.druid.catalog.model.TableDefnRegistry; import org.apache.druid.catalog.model.TableSpec; @@ -137,7 +137,7 @@ public void testParameters() table.validate(); // Parameters - Parameterized parameterizedTable = tableDefn; + ParameterizedDefn parameterizedTable = tableDefn; assertEquals(1, parameterizedTable.parameters().size()); assertNotNull(parameterizedTable.parameter(HttpTableDefn.URIS_PARAMETER)); diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java index cc17443e77b6..9b7258822c1e 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java @@ -23,7 +23,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.catalog.CatalogTest; import org.apache.druid.catalog.model.Columns; -import org.apache.druid.catalog.model.Parameterized; +import org.apache.druid.catalog.model.ParameterizedDefn; import org.apache.druid.catalog.model.ResolvedTable; import org.apache.druid.catalog.model.TableDefnRegistry; import org.apache.druid.catalog.model.TableSpec; @@ -163,7 +163,7 @@ public void testFilesParameter() .property(LocalTableDefn.BASE_DIR_PROPERTY, "/tmp") .buildResolved(mapper); - Parameterized parameterizedTable = tableDefn; + ParameterizedDefn parameterizedTable = tableDefn; assertEquals(2, parameterizedTable.parameters().size()); assertNotNull(parameterizedTable.parameter(LocalTableDefn.FILE_FILTER_PROPERTY)); assertNotNull(parameterizedTable.parameter(LocalTableDefn.FILES_PROPERTY)); From b1288eb18fe9e6293777706fb6f8779366af2148 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Tue, 18 Oct 2022 10:50:25 -0700 Subject: [PATCH 06/22] IT fix --- integration-tests-ex/cases/pom.xml | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/integration-tests-ex/cases/pom.xml b/integration-tests-ex/cases/pom.xml index 20859307aa86..5456d4b81b01 100644 --- a/integration-tests-ex/cases/pom.xml +++ b/integration-tests-ex/cases/pom.xml @@ -309,6 +309,15 @@ MultiStageQuery + + IT-Catalog + + false + + + Catalog + + docker-tests From 9f1d252a177f1d3908646c773c749893a34d0000 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Mon, 24 Oct 2022 17:11:54 -0700 Subject: [PATCH 07/22] Revisions from review comments --- .../apache/druid/data/input/InputFormat.java | 2 +- .../apache/druid/data/input/InputSource.java | 2 +- .../data/input/impl/CombiningInputSource.java | 2 + .../data/input/impl/RegexInputFormat.java | 2 + .../util/common/jackson/JacksonUtils.java | 19 ------ extensions-core/druid-catalog/pom.xml | 64 ++++--------------- .../catalog/http/CatalogListenerResource.java | 10 +++ .../druid/catalog/storage/HideColumns.java | 17 +++++ .../druid/catalog/model/CatalogUtils.java | 24 +++++++ .../druid/catalog/model/ColumnDefn.java | 2 +- .../druid/catalog/model/ColumnSpec.java | 2 +- .../{Properties.java => ModelProperties.java} | 2 +- .../druid/catalog/model/ObjectDefn.java | 2 +- .../catalog/model/ParameterizedDefn.java | 4 +- .../apache/druid/catalog/model/TableDefn.java | 4 +- .../druid/catalog/model/TableMetadata.java | 2 +- .../apache/druid/catalog/model/TableSpec.java | 2 +- .../model/table/AbstractDatasourceDefn.java | 12 ++-- .../catalog/model/table/ClusterKeySpec.java | 5 -- .../model/table/ExternalTableDefn.java | 8 +-- ...ternalSpec.java => ExternalTableSpec.java} | 4 +- .../catalog/model/table/HttpTableDefn.java | 4 +- .../catalog/model/table/InlineTableDefn.java | 2 +- .../catalog/model/table/InputFormats.java | 10 +-- .../catalog/model/table/LocalTableDefn.java | 4 +- .../druid/catalog/model/PropertyDefnTest.java | 10 +-- .../model/table/HttpInputTableTest.java | 6 +- .../catalog/model/table/InlineTableTest.java | 2 +- .../catalog/model/table/LocalTableTest.java | 10 +-- 29 files changed, 114 insertions(+), 125 deletions(-) rename server/src/main/java/org/apache/druid/catalog/model/{Properties.java => ModelProperties.java} (99%) rename server/src/main/java/org/apache/druid/catalog/model/table/{ExternalSpec.java => ExternalTableSpec.java} (96%) diff --git a/core/src/main/java/org/apache/druid/data/input/InputFormat.java b/core/src/main/java/org/apache/druid/data/input/InputFormat.java index 0ee6276675e9..4957bfb1e975 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/InputFormat.java @@ -45,7 +45,7 @@ @JsonSubTypes(value = { @Type(name = CsvInputFormat.TYPE_KEY, value = CsvInputFormat.class), @Type(name = JsonInputFormat.TYPE_KEY, value = JsonInputFormat.class), - @Type(name = "regex", value = RegexInputFormat.class), + @Type(name = RegexInputFormat.TYPE_KEY, value = RegexInputFormat.class), @Type(name = DelimitedInputFormat.TYPE_KEY, value = DelimitedInputFormat.class) }) public interface InputFormat diff --git a/core/src/main/java/org/apache/druid/data/input/InputSource.java b/core/src/main/java/org/apache/druid/data/input/InputSource.java index b7a50a8dc634..793cad2fedcc 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/InputSource.java @@ -53,7 +53,7 @@ @Type(name = LocalInputSource.TYPE_KEY, value = LocalInputSource.class), @Type(name = HttpInputSource.TYPE_KEY, value = HttpInputSource.class), @Type(name = InlineInputSource.TYPE_KEY, value = InlineInputSource.class), - @Type(name = "combining", value = CombiningInputSource.class) + @Type(name = CombiningInputSource.TYPE_KEY, value = CombiningInputSource.class) }) public interface InputSource { diff --git a/core/src/main/java/org/apache/druid/data/input/impl/CombiningInputSource.java b/core/src/main/java/org/apache/druid/data/input/impl/CombiningInputSource.java index 0b8201e312cf..05899021a6ea 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/CombiningInputSource.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/CombiningInputSource.java @@ -45,6 +45,8 @@ public class CombiningInputSource extends AbstractInputSource implements SplittableInputSource { + public static final String TYPE_KEY = "combining"; + private final List delegates; @JsonCreator diff --git a/core/src/main/java/org/apache/druid/data/input/impl/RegexInputFormat.java b/core/src/main/java/org/apache/druid/data/input/impl/RegexInputFormat.java index edd0edc93263..6c32de5f167d 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/RegexInputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/RegexInputFormat.java @@ -37,6 +37,8 @@ public class RegexInputFormat implements InputFormat { + public static final String TYPE_KEY = "regex"; + private final String pattern; private final String listDelimiter; private final List columns; diff --git a/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java b/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java index 2f5cd5d9fb61..2098eb5059eb 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java +++ b/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java @@ -122,23 +122,4 @@ public static T fromBytes(ObjectMapper jsonMapper, byte[] bytes, Class cl throw new ISE(e, "Failed to deserialize a " + clazz.getSimpleName()); } } - - /** - * Quick & easy implementation of {@code toString()} for objects which are - * primarily representations of JSON objects. Use only for cases where the - * {@code toString()} is for debugging: the cost of creating an object mapper - * every time is undesirable for production code. Also, assumes that the - * type can serialized using the default mapper: doesn't work for types that - * require custom Jackson extensions. - */ - public static String toString(Object obj) - { - ObjectMapper jsonMapper = new ObjectMapper(); - try { - return jsonMapper.writeValueAsString(obj); - } - catch (JsonProcessingException e) { - throw new ISE("Failed to serialize TableDefn"); - } - } } diff --git a/extensions-core/druid-catalog/pom.xml b/extensions-core/druid-catalog/pom.xml index 72d427c474ad..0678f95ff047 100644 --- a/extensions-core/druid-catalog/pom.xml +++ b/extensions-core/druid-catalog/pom.xml @@ -137,17 +137,17 @@ calcite-core provided - - javax.inject - javax.inject - 1 - provided - - - org.apache.curator - curator-client - provided - + + javax.inject + javax.inject + 1 + provided + + + org.apache.curator + curator-client + provided + com.fasterxml.jackson.dataformat jackson-dataformat-smile @@ -283,46 +283,4 @@ - - - - - - - - surefire-java9 - - (1.9,) - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - - - @{jacocoArgLine} - -Xmx1500m - -XX:MaxDirectMemorySize=512m - -Duser.language=en - -Duser.GroupByQueryRunnerTest.javacountry=US - -Dfile.encoding=UTF-8 - -Duser.timezone=UTC - -Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager - -Daws.region=us-east-1 - -Ddruid.indexing.doubleStorage=double - --add-exports java.base/jdk.internal.ref=ALL-UNNAMED - - false - - true - - - - - - - diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogListenerResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogListenerResource.java index bdbeda600cb2..cb3ac42ef44f 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogListenerResource.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogListenerResource.java @@ -48,6 +48,12 @@ import java.io.IOException; import java.io.InputStream; +/** + * Resource on the Broker to listen to catalog update events from the + * Coordinator. Since this is an internal API, it supports the efficient + * Smile encoding as well as the JSON encoding. (JSON is more convenient + * for debugging as it can be easily created or interpreted.) + */ @Path(CatalogListenerResource.BASE_URL) public class CatalogListenerResource { @@ -72,6 +78,10 @@ public CatalogListenerResource( this.jsonMapper = jsonMapper; } + /** + * Event sent from the Coordinator to indicate that a table has changed + * (or been deleted.) + */ @POST @Path(SYNC_URL) @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java index dccadad57b30..1cb403866119 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java @@ -32,6 +32,14 @@ import java.util.Objects; import java.util.Set; +/** + * Represents a request sent from a client to update just the list of hidden + * columns for a datasource table. Allows sending "delta encoded" changes: just + * the entries to add or remove. Exists as a separate operation since the + * generic merge can't handle removing items from a list. + * + * @see {@link org.apache.druid.catalog.http.CatalogResource#hideColumns(String, String, HideColumns, javax.servlet.http.HttpServletRequest)} + */ public class HideColumns { @JsonProperty @@ -56,6 +64,15 @@ public boolean isEmpty() && (unhide == null || unhide.isEmpty()); } + /** + * Given the existing list of hidden columns, perform the update action to add the + * requested new columns (if they don't yet exist) and remove the requested columns + * (if they do exist). If someone is silly enough to include the same column in + * both lists, the remove action takes precedence. + * + * @param hiddenColumns exiting hidden columns list + * @return revised hidden columns list after applying the requested changes + */ public List perform(List hiddenColumns) { if (hiddenColumns == null) { diff --git a/server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java b/server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java index ce9cc91ad2a5..9301a931afbf 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java +++ b/server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java @@ -19,8 +19,11 @@ package org.apache.druid.catalog.model; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Strings; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; @@ -109,6 +112,27 @@ public static String stringListToLines(List lines) return String.join("\n", lines) + "\n"; } + /** + * Catalog-specific 1uick & easy implementation of {@code toString()} for objects + * which are primarily representations of JSON objects. Use only for cases where the + * {@code toString()} is for debugging: the cost of creating an object mapper + * every time is undesirable for production code. Also, assumes that the + * type can serialized using the default mapper: doesn't work for types that + * require custom Jackson extensions. The catalog, however, has a simple type + * hierarchy, which is not extended via extensions, and so the default object mapper is + * fine. + */ + public static String toString(Object obj) + { + ObjectMapper jsonMapper = new ObjectMapper(); + try { + return jsonMapper.writeValueAsString(obj); + } + catch (JsonProcessingException e) { + throw new ISE("Failed to serialize TableDefn"); + } + } + public static List concatLists( @Nullable final List base, @Nullable final List additions diff --git a/server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.java b/server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.java index bb8d11b82088..a7f4b9ceab30 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.java @@ -20,7 +20,7 @@ package org.apache.druid.catalog.model; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.catalog.model.Properties.PropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.PropertyDefn; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; diff --git a/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java b/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java index fe816ad8681c..90ff4e7542cc 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java +++ b/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java @@ -110,7 +110,7 @@ public static ColumnSpec fromBytes(ObjectMapper jsonMapper, byte[] bytes) @Override public String toString() { - return JacksonUtils.toString(this); + return CatalogUtils.toString(this); } @Override diff --git a/server/src/main/java/org/apache/druid/catalog/model/Properties.java b/server/src/main/java/org/apache/druid/catalog/model/ModelProperties.java similarity index 99% rename from server/src/main/java/org/apache/druid/catalog/model/Properties.java rename to server/src/main/java/org/apache/druid/catalog/model/ModelProperties.java index 52be17af9bd5..c6c786534830 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/Properties.java +++ b/server/src/main/java/org/apache/druid/catalog/model/ModelProperties.java @@ -42,7 +42,7 @@ * via the validation, as is needed when the type is actually a map * which represents a Java object, or when the value is a list. */ -public interface Properties +public interface ModelProperties { interface PropertyDefn { diff --git a/server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java b/server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java index a4fe72aa2f33..1058a3bcde99 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java @@ -21,7 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; -import org.apache.druid.catalog.model.Properties.PropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.PropertyDefn; import java.util.HashMap; import java.util.List; diff --git a/server/src/main/java/org/apache/druid/catalog/model/ParameterizedDefn.java b/server/src/main/java/org/apache/druid/catalog/model/ParameterizedDefn.java index c111e76daea3..e6841007848a 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/ParameterizedDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/ParameterizedDefn.java @@ -19,7 +19,7 @@ package org.apache.druid.catalog.model; -import org.apache.druid.catalog.model.table.ExternalSpec; +import org.apache.druid.catalog.model.table.ExternalTableSpec; import java.util.List; import java.util.Map; @@ -58,5 +58,5 @@ public Class valueClass() List parameters(); ParameterDefn parameter(String name); - ExternalSpec applyParameters(ResolvedTable table, Map parameters); + ExternalTableSpec applyParameters(ResolvedTable table, Map parameters); } diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java index 0341f581337f..1ab390deda8b 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Strings; import com.google.common.collect.ImmutableMap; -import org.apache.druid.catalog.model.Properties.PropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.PropertyDefn; import org.apache.druid.java.util.common.IAE; import java.util.ArrayList; @@ -61,7 +61,7 @@ public TableDefn( typeValue, CatalogUtils.concatLists( Collections.singletonList( - new Properties.StringPropertyDefn(DESCRIPTION_PROPERTY) + new ModelProperties.StringPropertyDefn(DESCRIPTION_PROPERTY) ), properties ) diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java index f920ef6055be..f9ce13f22178 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java @@ -208,7 +208,7 @@ public static TableMetadata fromBytes(ObjectMapper jsonMapper, byte[] bytes) @Override public String toString() { - return JacksonUtils.toString(this); + return CatalogUtils.toString(this); } @Override diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java b/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java index a0893f7da7d4..4b1587f474b9 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java @@ -125,7 +125,7 @@ public static TableSpec fromBytes(ObjectMapper jsonMapper, byte[] bytes) @Override public String toString() { - return JacksonUtils.toString(this); + return CatalogUtils.toString(this); } @Override diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.java index bf5d4f1a6aef..4619de9048c9 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.java @@ -26,10 +26,10 @@ import org.apache.druid.catalog.model.CatalogUtils; import org.apache.druid.catalog.model.ColumnDefn; import org.apache.druid.catalog.model.Columns; -import org.apache.druid.catalog.model.Properties; -import org.apache.druid.catalog.model.Properties.GranularityPropertyDefn; -import org.apache.druid.catalog.model.Properties.PropertyDefn; -import org.apache.druid.catalog.model.Properties.StringListPropertyDefn; +import org.apache.druid.catalog.model.ModelProperties; +import org.apache.druid.catalog.model.ModelProperties.GranularityPropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.PropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.StringListPropertyDefn; import org.apache.druid.catalog.model.ResolvedTable; import org.apache.druid.catalog.model.TableDefn; import org.apache.druid.java.util.common.IAE; @@ -111,8 +111,8 @@ public AbstractDatasourceDefn( CatalogUtils.concatLists( Arrays.asList( new SegmentGranularityFieldDefn(), - new Properties.IntPropertyDefn(TARGET_SEGMENT_ROWS_PROPERTY), - new Properties.ListPropertyDefn( + new ModelProperties.IntPropertyDefn(TARGET_SEGMENT_ROWS_PROPERTY), + new ModelProperties.ListPropertyDefn( CLUSTER_KEYS_PROPERTY, "cluster keys", new TypeReference>() { } diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/ClusterKeySpec.java b/server/src/main/java/org/apache/druid/catalog/model/table/ClusterKeySpec.java index 2e8eb2181880..7f2429da5434 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/ClusterKeySpec.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/ClusterKeySpec.java @@ -41,11 +41,6 @@ public ClusterKeySpec( this.desc = desc != null && desc == true; } - public ClusterKeySpec(String expr) - { - this(expr, false); - } - @JsonProperty("column") public String expr() { diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java index 9f388cbe1f5f..8eb004311421 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java @@ -26,9 +26,9 @@ import org.apache.druid.catalog.model.ColumnDefn; import org.apache.druid.catalog.model.ColumnSpec; import org.apache.druid.catalog.model.Columns; +import org.apache.druid.catalog.model.ModelProperties.PropertyDefn; import org.apache.druid.catalog.model.ParameterizedDefn; import org.apache.druid.catalog.model.ParameterizedDefn.ParameterDefn; -import org.apache.druid.catalog.model.Properties.PropertyDefn; import org.apache.druid.catalog.model.ResolvedTable; import org.apache.druid.catalog.model.TableDefn; import org.apache.druid.catalog.model.table.InputFormats.InputFormatDefn; @@ -220,9 +220,9 @@ public ParameterDefn parameter(String key) public abstract ResolvedTable mergeParameters(ResolvedTable table, Map values); - public ExternalSpec convertToExtern(ResolvedTable table) + public ExternalTableSpec convertToExtern(ResolvedTable table) { - return new ExternalSpec( + return new ExternalTableSpec( convertSource(table), convertFormat(table), Columns.convertSignature(table.spec()) @@ -250,7 +250,7 @@ protected InputSource convertObject( } } - public ExternalSpec applyParameters(ResolvedTable table, Map parameters) + public ExternalTableSpec applyParameters(ResolvedTable table, Map parameters) { ResolvedTable revised = mergeParameters(table, parameters); return convertToExtern(revised); diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/ExternalSpec.java b/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableSpec.java similarity index 96% rename from server/src/main/java/org/apache/druid/catalog/model/table/ExternalSpec.java rename to server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableSpec.java index 7e0921110216..588e5e6b03a2 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/ExternalSpec.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableSpec.java @@ -28,13 +28,13 @@ * pass along the three components needed for an external table * in MSQ ingest. */ -public class ExternalSpec +public class ExternalTableSpec { protected final InputSource inputSource; protected final InputFormat inputFormat; protected final RowSignature signature; - public ExternalSpec( + public ExternalTableSpec( final InputSource inputSource, final InputFormat inputFormat, final RowSignature signature) diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.java index 9210a77eae50..08e107be1f9b 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.java @@ -22,9 +22,9 @@ import com.google.common.base.Strings; import com.google.common.collect.ImmutableMap; import org.apache.druid.catalog.model.CatalogUtils; +import org.apache.druid.catalog.model.ModelProperties.StringListPropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.StringPropertyDefn; import org.apache.druid.catalog.model.ParameterizedDefn; -import org.apache.druid.catalog.model.Properties.StringListPropertyDefn; -import org.apache.druid.catalog.model.Properties.StringPropertyDefn; import org.apache.druid.catalog.model.ResolvedTable; import org.apache.druid.catalog.model.table.ExternalTableDefn.FormattedExternalTableDefn; import org.apache.druid.data.input.InputSource; diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/InlineTableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/InlineTableDefn.java index 703d1ae4a314..273aad2696d9 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/InlineTableDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/InlineTableDefn.java @@ -20,7 +20,7 @@ package org.apache.druid.catalog.model.table; import org.apache.druid.catalog.model.CatalogUtils; -import org.apache.druid.catalog.model.Properties.StringListPropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.StringListPropertyDefn; import org.apache.druid.catalog.model.ResolvedTable; import org.apache.druid.catalog.model.table.ExternalTableDefn.FormattedExternalTableDefn; import org.apache.druid.data.input.InputSource; diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/InputFormats.java b/server/src/main/java/org/apache/druid/catalog/model/table/InputFormats.java index da93a272758c..9d1235aac862 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/InputFormats.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/InputFormats.java @@ -21,11 +21,11 @@ import org.apache.curator.shaded.com.google.common.collect.ImmutableList; import org.apache.druid.catalog.model.CatalogUtils; -import org.apache.druid.catalog.model.Properties.BooleanPropertyDefn; -import org.apache.druid.catalog.model.Properties.IntPropertyDefn; -import org.apache.druid.catalog.model.Properties.PropertyDefn; -import org.apache.druid.catalog.model.Properties.SimplePropertyDefn; -import org.apache.druid.catalog.model.Properties.StringPropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.BooleanPropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.IntPropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.PropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.SimplePropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.StringPropertyDefn; import org.apache.druid.catalog.model.ResolvedTable; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.CsvInputFormat; diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/LocalTableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/LocalTableDefn.java index 51be5e71f6d3..d6ef86959786 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/LocalTableDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/LocalTableDefn.java @@ -20,9 +20,9 @@ package org.apache.druid.catalog.model.table; import org.apache.druid.catalog.model.CatalogUtils; +import org.apache.druid.catalog.model.ModelProperties.StringListPropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.StringPropertyDefn; import org.apache.druid.catalog.model.ParameterizedDefn; -import org.apache.druid.catalog.model.Properties.StringListPropertyDefn; -import org.apache.druid.catalog.model.Properties.StringPropertyDefn; import org.apache.druid.catalog.model.ResolvedTable; import org.apache.druid.catalog.model.table.ExternalTableDefn.FormattedExternalTableDefn; import org.apache.druid.data.input.InputSource; diff --git a/server/src/test/java/org/apache/druid/catalog/model/PropertyDefnTest.java b/server/src/test/java/org/apache/druid/catalog/model/PropertyDefnTest.java index 794175517c44..99714f0411ef 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/PropertyDefnTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/PropertyDefnTest.java @@ -23,11 +23,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import org.apache.druid.catalog.CatalogTest; -import org.apache.druid.catalog.model.Properties.BooleanPropertyDefn; -import org.apache.druid.catalog.model.Properties.IntPropertyDefn; -import org.apache.druid.catalog.model.Properties.ListPropertyDefn; -import org.apache.druid.catalog.model.Properties.StringListPropertyDefn; -import org.apache.druid.catalog.model.Properties.StringPropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.BooleanPropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.IntPropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.ListPropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.StringListPropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.StringPropertyDefn; import org.apache.druid.catalog.model.table.ClusterKeySpec; import org.junit.Test; import org.junit.experimental.categories.Category; diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java index ee60e0cf3eb3..ab36adfad885 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java @@ -87,7 +87,7 @@ public void testHappyPath() assertEquals(table.spec(), TableSpec.fromBytes(mapper, bytes)); // Convert to an external spec - ExternalSpec externSpec = tableDefn.convertToExtern(table); + ExternalTableSpec externSpec = tableDefn.convertToExtern(table); HttpInputSource sourceSpec = (HttpInputSource) externSpec.inputSource(); assertEquals("bob", sourceSpec.getHttpAuthenticationUsername()); @@ -117,7 +117,7 @@ public void testEnvPassword() table.validate(); // Convert to an external spec - ExternalSpec externSpec = tableDefn.convertToExtern(table); + ExternalTableSpec externSpec = tableDefn.convertToExtern(table); HttpInputSource sourceSpec = (HttpInputSource) externSpec.inputSource(); assertEquals("bob", sourceSpec.getHttpAuthenticationUsername()); @@ -147,7 +147,7 @@ public void testParameters() ); // Convert to an external spec - ExternalSpec externSpec = parameterizedTable.applyParameters(table, params); + ExternalTableSpec externSpec = parameterizedTable.applyParameters(table, params); HttpInputSource sourceSpec = (HttpInputSource) externSpec.inputSource(); assertEquals("bob", sourceSpec.getHttpAuthenticationUsername()); diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/InlineTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/InlineTableTest.java index ede48da9f710..f4c81e5a14d1 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/InlineTableTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/InlineTableTest.java @@ -78,7 +78,7 @@ public void testValidData() assertEquals(table.spec(), TableSpec.fromBytes(mapper, bytes)); // Convert to an external spec - ExternalSpec externSpec = tableDefn.convertToExtern(table); + ExternalTableSpec externSpec = tableDefn.convertToExtern(table); InlineInputSource inlineSpec = (InlineInputSource) externSpec.inputSource(); assertEquals("a,b\nc,d\n", inlineSpec.getData()); diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java index 9b7258822c1e..cec2c8cba4c3 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java @@ -77,7 +77,7 @@ public void testFullyDefined() assertEquals(table.spec(), TableSpec.fromBytes(mapper, bytes)); // Convert to an external spec - ExternalSpec externSpec = tableDefn.convertToExtern(table); + ExternalTableSpec externSpec = tableDefn.convertToExtern(table); LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource(); assertEquals("/tmp", sourceSpec.getBaseDir().toString()); @@ -106,7 +106,7 @@ public void testNoFilter() table.validate(); // Convert to an external spec - ExternalSpec externSpec = tableDefn.convertToExtern(table); + ExternalTableSpec externSpec = tableDefn.convertToExtern(table); LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource(); assertEquals("/tmp", sourceSpec.getBaseDir().toString()); @@ -126,7 +126,7 @@ public void testNoFiles() table.validate(); // Convert to an external spec - ExternalSpec externSpec = tableDefn.convertToExtern(table); + ExternalTableSpec externSpec = tableDefn.convertToExtern(table); LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource(); assertEquals("/tmp", sourceSpec.getBaseDir().toString()); @@ -175,7 +175,7 @@ public void testFilesParameter() ); // Convert to an external spec - ExternalSpec externSpec = parameterizedTable.applyParameters(table, params); + ExternalTableSpec externSpec = parameterizedTable.applyParameters(table, params); LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource(); assertEquals("/tmp", sourceSpec.getBaseDir().toString()); @@ -199,7 +199,7 @@ public void testFilterParameter() ); // Convert to an external spec - ExternalSpec externSpec = tableDefn.applyParameters(table, params); + ExternalTableSpec externSpec = tableDefn.applyParameters(table, params); LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource(); assertEquals("/tmp", sourceSpec.getBaseDir().toString()); From a397298176ab82f3801274d110039312f2e6526e Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Wed, 26 Oct 2022 13:11:50 -0700 Subject: [PATCH 08/22] Revisions from review comments --- .../guice/CatalogCoordinatorModule.java | 6 +- .../catalog/http/CatalogListenerResource.java | 48 ++----- .../druid/catalog/http/CatalogResource.java | 131 +++++++++++------- .../apache/druid/catalog/storage/Actions.java | 20 --- .../catalog/storage/CatalogAuthorizer.java | 100 ------------- .../druid/catalog/storage/CatalogStorage.java | 26 ++-- .../druid/catalog/storage/HideColumns.java | 9 ++ ...rImpl.java => MetadataStorageManager.java} | 45 +++--- .../catalog/storage/MetastoreManager.java | 60 -------- .../catalog/storage/sql/CatalogManager.java | 47 ++++++- .../storage/sql/SQLCatalogManager.java | 19 ++- .../druid/catalog/storage/CatalogTests.java | 11 +- .../catalog/storage/TableManagerTest.java | 3 +- .../catalog/sync/CatalogMetadataTest.java | 2 +- .../druid/catalog/sync/MockCatalogSync.java | 8 +- .../http/catalog/CatalogResourceTest.java | 32 ++--- 16 files changed, 212 insertions(+), 355 deletions(-) delete mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogAuthorizer.java rename extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/{MetastoreManagerImpl.java => MetadataStorageManager.java} (71%) delete mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetastoreManager.java diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/guice/CatalogCoordinatorModule.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/guice/CatalogCoordinatorModule.java index 3a04d678cf3f..649f3515dcea 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/guice/CatalogCoordinatorModule.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/guice/CatalogCoordinatorModule.java @@ -25,8 +25,7 @@ import org.apache.druid.catalog.model.SchemaRegistry; import org.apache.druid.catalog.model.SchemaRegistryImpl; import org.apache.druid.catalog.storage.CatalogStorage; -import org.apache.druid.catalog.storage.MetastoreManager; -import org.apache.druid.catalog.storage.MetastoreManagerImpl; +import org.apache.druid.catalog.storage.MetadataStorageManager; import org.apache.druid.catalog.storage.sql.CatalogManager; import org.apache.druid.catalog.storage.sql.SQLCatalogManager; import org.apache.druid.catalog.sync.CatalogUpdateNotifier; @@ -64,8 +63,7 @@ public void configure(Binder binder) .bind(CatalogStorage.class) .in(LazySingleton.class); binder - .bind(MetastoreManager.class) - .to(MetastoreManagerImpl.class) + .bind(MetadataStorageManager.class) .in(LazySingleton.class); // At present, the set of schemas is fixed. diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogListenerResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogListenerResource.java index cb3ac42ef44f..b9e291a628b4 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogListenerResource.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogListenerResource.java @@ -21,20 +21,16 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import com.sun.jersey.spi.container.ResourceFilters; import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; import org.apache.druid.catalog.sync.CatalogUpdateNotifier; import org.apache.druid.catalog.sync.MetadataCatalog.CatalogListener; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Smile; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.server.security.Access; -import org.apache.druid.server.security.Action; -import org.apache.druid.server.security.AuthorizationUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.http.security.ConfigResourceFilter; import org.apache.druid.server.security.AuthorizerMapper; -import org.apache.druid.server.security.Resource; -import org.apache.druid.server.security.ResourceAction; -import org.apache.druid.server.security.ResourceType; import javax.inject.Inject; import javax.servlet.http.HttpServletRequest; @@ -59,6 +55,8 @@ public class CatalogListenerResource { public static final String BASE_URL = "/druid/broker/v1/catalog"; public static final String SYNC_URL = "/sync"; + private static final Logger log = new Logger(CatalogListenerResource.class); + private final CatalogListener listener; private final AuthorizerMapper authorizerMapper; @@ -70,7 +68,8 @@ public CatalogListenerResource( final CatalogListener listener, @Smile final ObjectMapper smileMapper, @Json final ObjectMapper jsonMapper, - final AuthorizerMapper authorizerMapper) + final AuthorizerMapper authorizerMapper + ) { this.listener = listener; this.authorizerMapper = authorizerMapper; @@ -85,14 +84,12 @@ public CatalogListenerResource( @POST @Path(SYNC_URL) @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + @ResourceFilters(ConfigResourceFilter.class) public Response syncTable( final InputStream inputStream, - @Context final HttpServletRequest req) + @Context final HttpServletRequest req + ) { - Response resp = checkAuth(req); - if (resp != null) { - return resp; - } final String reqContentType = req.getContentType(); final boolean isSmile = SmileMediaTypes.APPLICATION_JACKSON_SMILE.equals(reqContentType); final ObjectMapper mapper = isSmile ? smileMapper : jsonMapper; @@ -101,7 +98,8 @@ public Response syncTable( tableSpec = mapper.readValue(inputStream, TableMetadata.class); } catch (IOException e) { - return Response.serverError().entity(e.getMessage()).build(); + log.error(e, "Bad catalog sync request received!"); + return Response.status(Response.Status.BAD_REQUEST).entity(e.getMessage()).build(); } TableSpec spec = tableSpec.spec(); if (CatalogUpdateNotifier.TOMBSTONE_TABLE_TYPE.equals(spec.type())) { @@ -111,26 +109,4 @@ public Response syncTable( } return Response.status(Response.Status.ACCEPTED).build(); } - - private Response checkAuth(final HttpServletRequest request) - { - final ResourceAction resourceAction = new ResourceAction( - new Resource("CONFIG", ResourceType.CONFIG), - Action.WRITE - ); - - final Access authResult = AuthorizationUtils.authorizeResourceAction( - request, - resourceAction, - authorizerMapper - ); - - if (authResult.isAllowed()) { - return null; - } - return Response.status(Response.Status.FORBIDDEN) - .type(MediaType.TEXT_PLAIN) - .entity(StringUtils.format("Access-Check-Result: %s", authResult.toString())) - .build(); - } } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java index 817f613f3f3e..80aa5c97059d 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java @@ -40,9 +40,13 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthorizationUtils; +import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ForbiddenException; +import org.apache.druid.server.security.Resource; +import org.apache.druid.server.security.ResourceAction; import org.apache.druid.server.security.ResourceType; import javax.inject.Inject; @@ -80,17 +84,22 @@ public class CatalogResource public static final String ROOT_PATH = "/druid/coordinator/v1/catalog"; private final CatalogStorage catalog; + private final AuthorizerMapper authorizerMapper; @Inject - public CatalogResource(final CatalogStorage catalog) + public CatalogResource( + final CatalogStorage catalog, + final AuthorizerMapper authorizerMapper + ) { this.catalog = catalog; + this.authorizerMapper = authorizerMapper; } private enum PostAction { NEW, - IFNEW, + IF_NEW, REPLACE, FORCE; } @@ -128,8 +137,10 @@ public Response postTable( if (actionParam == null) { action = PostAction.NEW; } else { - action = PostAction.valueOf(StringUtils.toUpperCase(actionParam)); - if (action == null) { + try { + action = PostAction.valueOf(StringUtils.toUpperCase(actionParam)); + } + catch (IllegalArgumentException e) { return Actions.badRequest( Actions.INVALID, StringUtils.format( @@ -140,7 +151,7 @@ public Response postTable( } } TableId tableId = TableId.of(dbSchema, name); - Response response = authorizeTable(tableId, spec, req); + Response response = validateTable(tableId, spec, req); if (response != null) { return response; } @@ -155,7 +166,7 @@ public Response postTable( switch (action) { case NEW: return insertTableSpec(table, false); - case IFNEW: + case IF_NEW: return insertTableSpec(table, true); case REPLACE: return updateTableSpec(table, version); @@ -166,7 +177,7 @@ public Response postTable( } } - private Response authorizeTable(TableId tableId, TableSpec spec, final HttpServletRequest req) + private Response validateTable(TableId tableId, TableSpec spec, final HttpServletRequest req) { // Druid has a fixed set of schemas. Ensure the one provided is valid. Pair result = validateSchema(tableId.schema()); @@ -183,7 +194,7 @@ private Response authorizeTable(TableId tableId, TableSpec spec, final HttpServl ); } - // Table name can't be blank or have spaces + // Table name can't be blank or have leading/trailing spaces if (Strings.isNullOrEmpty(tableId.name())) { return Actions.badRequest(Actions.INVALID, "Table name is required"); } @@ -191,20 +202,19 @@ private Response authorizeTable(TableId tableId, TableSpec spec, final HttpServl return Actions.badRequest(Actions.INVALID, "Table name cannot start or end with spaces"); } - // The user has to have permission to modify the table. - try { - catalog.authorizer().authorizeTable(schema, tableId.name(), Action.WRITE, req); - } - catch (ForbiddenException e) { - return Actions.forbidden(e); - } + // The user has to have permission to modify the table. Throws an exception + // if not. + authorizeTable(schema, tableId.name(), Action.WRITE, req); // Validate the spec, if provided. if (spec != null) { // The given table spec has to be valid for the given schema. - if (Strings.isNullOrEmpty(spec.type())) { - return Actions.badRequest(Actions.INVALID, "Table type is required"); + try { + spec.validate(); + } + catch (IAE e) { + return Actions.badRequest(Actions.INVALID, e.getMessage()); } if (!schema.accepts(spec.type())) { @@ -335,7 +345,7 @@ private Response incrementalUpdate( Function action ) { - Response response = authorizeTable(tableId, newSpec, req); + Response response = validateTable(tableId, newSpec, req); if (response != null) { return response; } @@ -352,7 +362,7 @@ private Response incrementalUpdate( } /** - * Move a single column to the start end of the column list, or before or after + * Move a single column to the start or end of the column list, or before or after * another column. Both columns must exist. Returns the version of the table * after the update. *

@@ -499,12 +509,7 @@ public Response getTable( if (Strings.isNullOrEmpty(name)) { return Actions.badRequest(Actions.INVALID, "Table name is required"); } - try { - catalog.authorizer().authorizeTable(result.rhs, name, Action.READ, req); - } - catch (ForbiddenException e) { - return Actions.forbidden(e); - } + authorizeTable(result.rhs, name, Action.READ, req); try { TableId tableId = new TableId(dbSchema, name); TableMetadata table = catalog.tables().read(tableId); @@ -530,7 +535,7 @@ public Response listSchemas( ) { // No good resource to use: we really need finer-grain control. - catalog.authorizer().authorizeAccess(ResourceType.STATE, "schemas", Action.READ, req); + authorizeAccess(ResourceType.STATE, "schemas", Action.READ, req); return Response.ok().entity(catalog.schemaRegistry().names()).build(); } @@ -541,7 +546,7 @@ public Response listSchemas( @GET @Path("/list/tables/names") @Produces(MediaType.APPLICATION_JSON) - public Response listTables( + public Response listTableNames( @Context final HttpServletRequest req ) { @@ -556,9 +561,10 @@ public Response listTables( return null; } return Collections.singletonList( - catalog.authorizer().resourceAction(schema, tableId.name(), Action.READ)); + resourceAction(schema, tableId.name(), Action.READ)); }, - catalog.authorizer().mapper()); + authorizerMapper + ); return Response.ok().entity(Lists.newArrayList(filtered)).build(); } @@ -573,7 +579,7 @@ public Response listTables( @GET @Path("/schemas/{dbSchema}/names") @Produces(MediaType.APPLICATION_JSON) - public Response listTables( + public Response listTableNamesForSchema( @PathParam("dbSchema") String dbSchema, @Context final HttpServletRequest req ) @@ -589,8 +595,9 @@ public Response listTables( tables, name -> Collections.singletonList( - catalog.authorizer().resourceAction(schema, name, Action.READ)), - catalog.authorizer().mapper()); + resourceAction(schema, name, Action.READ)), + authorizerMapper + ); return Response.ok().entity(Lists.newArrayList(filtered)).build(); } @@ -601,7 +608,7 @@ public Response listTables( @GET @Path("/schemas/{dbSchema}/tables") @Produces(MediaType.APPLICATION_JSON) - public Response listTableDetails( + public Response listTableMetadataForSchema( @PathParam("dbSchema") String dbSchema, @Context final HttpServletRequest req ) @@ -618,9 +625,10 @@ public Response listTableDetails( table -> { TableId tableId = table.id(); return Collections.singletonList( - catalog.authorizer().resourceAction(schema, tableId.name(), Action.READ)); + resourceAction(schema, tableId.name(), Action.READ)); }, - catalog.authorizer().mapper()); + authorizerMapper + ); return Response.ok().entity(Lists.newArrayList(filtered)).build(); } @@ -648,6 +656,9 @@ public Response deleteTable( ) { TableId tableId = new TableId(dbSchema, name); + + // Validate the schema. Returns either an error response (lhs) or the + // validated schema (rhs). Pair result = validateSchema(tableId.schema()); if (result.lhs != null) { return result.lhs; @@ -661,12 +672,7 @@ public Response deleteTable( if (Strings.isNullOrEmpty(name)) { return Actions.badRequest(Actions.INVALID, "Table name is required"); } - try { - catalog.authorizer().authorizeTable(schema, tableId.name(), Action.WRITE, req); - } - catch (ForbiddenException e) { - return Actions.forbidden(e); - } + authorizeTable(schema, tableId.name(), Action.WRITE, req); try { if (!catalog.tables().delete(tableId) && !ifExists) { return Actions.notFound(tableId.sqlName()); @@ -699,7 +705,7 @@ public Response syncSchema( { // Same as the user-command for now. This endpoint reserves the right to change // over time as needed, while the user endpoint cannot easily change. - return listTableDetails(dbSchema, req); + return listTableMetadataForSchema(dbSchema, req); } public static final String TABLE_SYNC = "/tables/{dbSchema}/{name}/sync"; @@ -721,16 +727,6 @@ public Response syncTable( return getTable(dbSchema, name, req); } - @POST - @Path("/flush") - public Response flush( - @Context final HttpServletRequest req - ) - { - // Nothing to do yet. - return Actions.ok(); - } - private Pair validateSchema(String dbSchema) { if (Strings.isNullOrEmpty(dbSchema)) { @@ -744,4 +740,35 @@ private Pair validateSchema(String dbSchema) } return Pair.of(null, schema); } + + private static ResourceAction resourceAction(SchemaSpec schema, String name, Action action) + { + return new ResourceAction(new Resource(name, schema.securityResource()), action); + } + + private void authorizeTable(SchemaSpec schema, String name, Action action, HttpServletRequest request) + { + if (action == Action.WRITE && !schema.writable()) { + throw new ForbiddenException( + "Cannot create table definitions in schema: " + schema.name()); + } + authorize(schema.securityResource(), name, action, request); + } + + private void authorize(String resource, String key, Action action, HttpServletRequest request) + { + final Access authResult = authorizeAccess(resource, key, action, request); + if (!authResult.isAllowed()) { + throw new ForbiddenException(authResult.toString()); + } + } + + private Access authorizeAccess(String resource, String key, Action action, HttpServletRequest request) + { + return AuthorizationUtils.authorizeResourceAction( + request, + new ResourceAction(new Resource(key, resource), action), + authorizerMapper + ); + } } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/Actions.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/Actions.java index 08a561b7517e..7d51fedff6d1 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/Actions.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/Actions.java @@ -20,7 +20,6 @@ package org.apache.druid.catalog.storage; import com.google.common.collect.ImmutableMap; -import org.apache.druid.server.security.ForbiddenException; import javax.ws.rs.core.Response; @@ -74,25 +73,6 @@ public static Response ok() return Response.ok().build(); } - public static Response forbidden() - { - return forbidden("Unauthorized"); - } - - public static Response forbidden(ForbiddenException e) - { - return forbidden(e.getMessage()); - } - - public static Response forbidden(String msg) - { - // Like ForbiddenExceptionMapper, but in the standard error - // format. Used instead of throwing ForbiddenException - return Response.status(Response.Status.FORBIDDEN) - .entity(error(FORBIDDEN, msg)) - .build(); - } - public static Response okWithVersion(long version) { return Response diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogAuthorizer.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogAuthorizer.java deleted file mode 100644 index 558463c2bf02..000000000000 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogAuthorizer.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.catalog.storage; - -import org.apache.druid.catalog.model.SchemaRegistry.SchemaSpec; -import org.apache.druid.server.security.Access; -import org.apache.druid.server.security.Action; -import org.apache.druid.server.security.AuthorizationUtils; -import org.apache.druid.server.security.AuthorizerMapper; -import org.apache.druid.server.security.ForbiddenException; -import org.apache.druid.server.security.Resource; -import org.apache.druid.server.security.ResourceAction; - -import javax.inject.Inject; -import javax.servlet.http.HttpServletRequest; - -/** - * Encapsulates the details of catalog authorization. - */ -public class CatalogAuthorizer -{ - private final AuthorizerMapper authorizerMapper; - - @Inject - public CatalogAuthorizer( - AuthorizerMapper authorizerMapper) - { - this.authorizerMapper = authorizerMapper; - } - - public AuthorizerMapper mapper() - { - return authorizerMapper; - } - - public void authorizeTable(SchemaSpec schema, String name, Action action, HttpServletRequest request) - { - if (action == Action.WRITE && !schema.writable()) { - throw new ForbiddenException( - "Cannot create table definitions in schema: " + schema.name()); - } - authorize(schema.securityResource(), name, action, request); - } - - public void authorize(String resource, String key, Action action, HttpServletRequest request) - { - final Access authResult = authorizeAccess(resource, key, action, request); - if (!authResult.isAllowed()) { - throw new ForbiddenException(authResult.toString()); - } - } - - public boolean isAuthorized(String resource, String key, Action action, HttpServletRequest request) - { - final Access authResult = authorizeAccess(resource, key, action, request); - return authResult.isAllowed(); - } - - public Access authorizeAccess(String resource, String key, Action action, HttpServletRequest request) - { - return AuthorizationUtils.authorizeResourceAction( - request, - new ResourceAction(new Resource(key, resource), action), - authorizerMapper - ); - } - - public ResourceAction resourceAction(SchemaSpec schema, String name, Action action) - { - return new ResourceAction(new Resource(name, schema.securityResource()), action); - } - - public Action inferAction(HttpServletRequest request) - { - switch (request.getMethod()) { - case "GET": - case "HEAD": - return Action.READ; - default: - return Action.WRITE; - } - } -} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java index 061cd856a846..fdccf95782a4 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java @@ -32,15 +32,23 @@ import org.apache.druid.catalog.sync.MetadataCatalog.CatalogSource; import org.apache.druid.catalog.sync.MetadataCatalog.CatalogUpdateProvider; import org.apache.druid.guice.annotations.Json; -import org.apache.druid.server.security.AuthorizerMapper; +import javax.annotation.Nullable; import javax.inject.Inject; import java.util.List; /** * Facade over the three internal components used to manage the metadata - * catalog from the REST API. + * catalog from the REST API: + *

    + *
  • Schema registry: the hard-coded set of schemas which Druid supports. + * (User does not support user-defined schemas.)
  • + *
  • Table definition registry: the set of table types which Druid + * supports: datasources, external tables, etc.
  • + *
  • Catalog manager: database storage for the user's table specifications + * within the catalog.
  • + *
*/ public class CatalogStorage implements CatalogUpdateProvider, CatalogSource { @@ -75,24 +83,16 @@ public void deleted(TableId id) protected final SchemaRegistry schemaRegistry; protected final TableDefnRegistry tableRegistry; protected final CatalogManager catalogMgr; - protected final CatalogAuthorizer authorizer; @Inject public CatalogStorage( - CatalogManager catalogMgr, - AuthorizerMapper authorizerMapper, - @Json ObjectMapper jsonMapper + final CatalogManager catalogMgr, + @Json final ObjectMapper jsonMapper ) { this.schemaRegistry = new SchemaRegistryImpl(); this.tableRegistry = new TableDefnRegistry(jsonMapper); this.catalogMgr = catalogMgr; - this.authorizer = new CatalogAuthorizer(authorizerMapper); - } - - public CatalogAuthorizer authorizer() - { - return authorizer; } public CatalogManager tables() @@ -140,7 +140,7 @@ public TableDefnRegistry tableRegistry() } @Override - public ResolvedTable resolveTable(TableId id) + public @Nullable ResolvedTable resolveTable(TableId id) { TableMetadata table = table(id); return table == null ? null : tableRegistry.resolve(table.spec()); diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java index 1cb403866119..38633c14af08 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java @@ -42,8 +42,17 @@ */ public class HideColumns { + /** + * The list of columns to hide. That is, the list of columns to add to the + * table's hidden column list. + */ @JsonProperty public final List hide; + + /** + * The list of columns to no longer hide. That is, the list of columns to + * remove from the table's hidden column list. + */ @JsonProperty public final List unhide; diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetastoreManagerImpl.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetadataStorageManager.java similarity index 71% rename from extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetastoreManagerImpl.java rename to extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetadataStorageManager.java index ad1248a5ee52..b5c7173698ee 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetastoreManagerImpl.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetadataStorageManager.java @@ -29,7 +29,12 @@ import javax.inject.Inject; -public class MetastoreManagerImpl implements MetastoreManager +/** + * Represents the metastore manager database and its implementation. + * Abstracts away the various kick-knacks used to define the metastore. + * The metastore operations are defined via table-specific classes. + */ +public class MetadataStorageManager { private final ObjectMapper jsonMapper; private final MetadataStorageConnector connector; @@ -37,11 +42,11 @@ public class MetastoreManagerImpl implements MetastoreManager private final MetadataStorageTablesConfig tablesConfig; @Inject - public MetastoreManagerImpl( - @Json ObjectMapper jsonMapper, - MetadataStorageConnector connector, - Supplier configSupplier, - Supplier tablesConfigSupplier + public MetadataStorageManager( + @Json final ObjectMapper jsonMapper, + final MetadataStorageConnector connector, + final Supplier configSupplier, + final Supplier tablesConfigSupplier ) { this.jsonMapper = jsonMapper; @@ -50,43 +55,37 @@ public MetastoreManagerImpl( this.tablesConfig = tablesConfigSupplier.get(); } - @Override - public MetadataStorageConnector connector() - { - return connector; - } - - @Override public MetadataStorageConnectorConfig config() { return config; } - @Override public MetadataStorageTablesConfig tablesConfig() { return tablesConfig; } - @Override - public boolean createTables() - { - return config.isCreateTables(); - } - - @Override + /** + * Object mapper to use for serializing and deserializing + * JSON objects stored in the metastore DB. + */ public ObjectMapper jsonMapper() { return jsonMapper; } - @Override + /** + * Is the implementation SQL-based? + */ public boolean isSql() { return connector instanceof SQLMetadataConnector; } - @Override + /** + * If SQL based, return the SQL version of the metastore + * connector. Throws an exception if not SQL-based. + */ public SQLMetadataConnector sqlConnector() { return (SQLMetadataConnector) connector; diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetastoreManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetastoreManager.java deleted file mode 100644 index 15ae898176e6..000000000000 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetastoreManager.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.catalog.storage; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.metadata.MetadataStorageConnector; -import org.apache.druid.metadata.MetadataStorageConnectorConfig; -import org.apache.druid.metadata.MetadataStorageTablesConfig; -import org.apache.druid.metadata.SQLMetadataConnector; - -/** - * Represents the metastore manager database and its implementation. - * Abstracts away the various kick-knacks used to define the metastore. - * The metastore operations are defined via table-specific classes. - */ -public interface MetastoreManager -{ - MetadataStorageConnector connector(); - MetadataStorageConnectorConfig config(); - MetadataStorageTablesConfig tablesConfig(); - - /** - * Whether to create tables if they do not exist. - */ - boolean createTables(); - - /** - * Object mapper to use for serializing and deserializing - * JSON objects stored in the metastore DB. - */ - ObjectMapper jsonMapper(); - - /** - * Is the implementation SQL-based? - */ - boolean isSql(); - - /** - * If SQL based, return the SQL version of the metastore - * connector. Throws an exception if not SQL-based. - */ - SQLMetadataConnector sqlConnector(); -} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java index a137ae413610..856ee1926fe5 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java @@ -31,8 +31,7 @@ /** * Manages catalog data. Used in Coordinator, which will be in either * an leader or standby state. The Coordinator calls the {@link #start()} - * method when it becomes the leader, and calls {@link #stop()} when - * it loses leadership, or shuts down. + * method when it becomes the leader. * * Performs detailed CRUD operations on the catalog tables table. * Higher-level operations appear elsewhere. @@ -52,6 +51,10 @@ public OutOfDateException(String msg) } } + /** + * Thrown when a record does not exist in the database. Allows + * the caller to check for this specific case in a generic way. + */ class NotFoundException extends Exception { public NotFoundException(String msg) @@ -74,18 +77,40 @@ public DuplicateKeyException(String msg, Exception e) } } + /** + * Generic interface for changes to the catalog at the storage level. + * Implemented by the catalog sync mechanism to send update events + * to the Broker. Note that these events are about the catalog, + * not about the physical storage of tables (i.e. datasources.) + */ interface Listener { + /** + * A new catalog table entry was added. + */ void added(TableMetadata table); + + /** + * An existing catalog table entry was updated. + */ void updated(TableMetadata table); + + /** + * An existing catalog table entry was deleted. + */ void deleted(TableId id); } + /** + * Start the catalog manager within a Druid run. Called from lifecycle + * management and when a coordinator becomes the leader node. + */ void start(); - + /** + * Register a listener for catalog events. + */ void register(Listener listener); - void createTableDefnTable(); /** * Create a table entry. @@ -159,9 +184,19 @@ interface Listener */ boolean delete(TableId id); + /** + * Returns a list of the ids (schema, name) of all tables in the catalog. + */ List list(); + + /** + * Returns a list of the names of all tables within the given schema. + */ List list(String dbSchema); - List listDetails(String dbSchema); - void stop(); + /** + * Returns a list of the table metadata for all tables within the given + * schema. + */ + List listDetails(String dbSchema); } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java index a7888d2270f3..99f49e10f9b6 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java @@ -27,7 +27,7 @@ import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; -import org.apache.druid.catalog.storage.MetastoreManager; +import org.apache.druid.catalog.storage.MetadataStorageManager; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -115,7 +115,7 @@ public class SQLCatalogManager implements CatalogManager "DELETE FROM %s\n" + WHERE_TABLE_ID; - private final MetastoreManager metastoreManager; + private final MetadataStorageManager metastoreManager; private final SQLMetadataConnector connector; private final ObjectMapper jsonMapper; private final IDBI dbi; @@ -123,7 +123,7 @@ public class SQLCatalogManager implements CatalogManager private final Deque listeners = new ConcurrentLinkedDeque<>(); @Inject - public SQLCatalogManager(MetastoreManager metastoreManager) + public SQLCatalogManager(MetadataStorageManager metastoreManager) { if (!metastoreManager.isSql()) { throw new ISE("SQLCatalogManager only works with SQL based metadata store at this time"); @@ -142,11 +142,6 @@ public void start() createTableDefnTable(); } - @Override - public void stop() - { - } - // Mimics what MetadataStorageTablesConfig should do. public String getTableDefnTable() { @@ -159,10 +154,9 @@ public String getTableDefnTable() } // TODO: Move to SqlMetadataConnector - @Override public void createTableDefnTable() { - if (!metastoreManager.createTables()) { + if (!metastoreManager.config().isCreateTables()) { return; } connector.createTable( @@ -179,7 +173,10 @@ public void createTableDefnTable() + " PRIMARY KEY(schemaName, name)\n" + ")", tableName, - connector.getPayloadType()))); + connector.getPayloadType() + ) + ) + ); } @Override diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/CatalogTests.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/CatalogTests.java index fce8fa87f67e..d89d5f7a20ce 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/CatalogTests.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/CatalogTests.java @@ -46,7 +46,7 @@ public class CatalogTests public static final String WRITER_USER = "writer"; public static final String DENY_USER = "denyAll"; - protected static final AuthorizerMapper AUTH_MAPPER = new AuthorizerMapper( + public static final AuthorizerMapper AUTH_MAPPER = new AuthorizerMapper( ImmutableMap.of(TEST_AUTHORITY, new TestAuthorizer())); private static class TestAuthorizer implements Authorizer @@ -94,7 +94,7 @@ public static class DbFixture public DbFixture(DerbyConnectorRule derbyConnectorRule) { - MetastoreManager metastoreMgr = new MetastoreManagerImpl( + MetadataStorageManager metastoreMgr = new MetadataStorageManager( JSON_MAPPER, derbyConnectorRule.getConnector(), () -> derbyConnectorRule.getMetadataConnectorConfig(), @@ -102,17 +102,12 @@ public DbFixture(DerbyConnectorRule derbyConnectorRule) ); manager = new SQLCatalogManager(metastoreMgr); manager.start(); - storage = new CatalogStorage( - manager, - AUTH_MAPPER, - JSON_MAPPER - ); + storage = new CatalogStorage(manager, JSON_MAPPER); } public void tearDown() { if (manager != null) { - manager.stop(); manager = null; } } diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java index 0f51fe5179ef..0839be8124b2 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java @@ -63,7 +63,7 @@ public class TableManagerTest @Before public void setUp() { - MetastoreManager metastoreMgr = new MetastoreManagerImpl( + MetadataStorageManager metastoreMgr = new MetadataStorageManager( JSON_MAPPER, derbyConnectorRule.getConnector(), () -> derbyConnectorRule.getMetadataConnectorConfig(), @@ -77,7 +77,6 @@ public void setUp() public void tearDown() { if (manager != null) { - manager.stop(); manager = null; } } diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java index 1505a11051e3..9146c088c30a 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java @@ -160,7 +160,7 @@ public void testRemoteWithSmile() throws DuplicateKeyException, OutOfDateExcepti private void doTestRemote(boolean useSmile) throws DuplicateKeyException, OutOfDateException, NotFoundException { populateCatalog(); - MockCatalogSync sync = new MockCatalogSync(storage, jsonMapper, smileMapper, useSmile); + MockCatalogSync sync = new MockCatalogSync(storage, CatalogTests.AUTH_MAPPER, jsonMapper, smileMapper, useSmile); MetadataCatalog catalog = sync.catalog(); storage.register(sync); verifyInitial(catalog); diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/MockCatalogSync.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/MockCatalogSync.java index 45464085fd55..87a01d4081c2 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/MockCatalogSync.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/MockCatalogSync.java @@ -29,6 +29,7 @@ import org.apache.druid.catalog.storage.CatalogTests; import org.apache.druid.catalog.sync.MetadataCatalog.CatalogListener; import org.apache.druid.server.http.catalog.DummyRequest; +import org.apache.druid.server.security.AuthorizerMapper; import javax.ws.rs.core.MediaType; @@ -46,10 +47,11 @@ public class MockCatalogSync implements CatalogListener private final ObjectMapper jsonMapper; public MockCatalogSync( - CatalogStorage storage, + final CatalogStorage storage, + final AuthorizerMapper authorizerMapper, final ObjectMapper smileMapper, final ObjectMapper jsonMapper, - boolean useSmile + final boolean useSmile ) { this.catalog = new CachedMetadataCatalog(storage, storage.schemaRegistry(), jsonMapper); @@ -57,7 +59,7 @@ public MockCatalogSync( catalog, smileMapper, jsonMapper, - storage.authorizer().mapper() + authorizerMapper ); this.useSmile = useSmile; this.smileMapper = smileMapper; diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java index 101135e835e3..ea0fc1b482d9 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java @@ -67,7 +67,7 @@ public class CatalogResourceTest public void setUp() { dbFixture = new CatalogTests.DbFixture(derbyConnectorRule); - resource = new CatalogResource(dbFixture.storage); + resource = new CatalogResource(dbFixture.storage, CatalogTests.AUTH_MAPPER); } @After @@ -271,22 +271,22 @@ private List getDetailsList(Response resp) public void testList() { // No entries - Response resp = resource.listTables(getBy(CatalogTests.READER_USER)); + Response resp = resource.listTableNames(getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); List tableIds = getTableIdList(resp); assertTrue(tableIds.isEmpty()); - resp = resource.listTables(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + resp = resource.listTableNamesForSchema(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); List tables = getTableList(resp); assertTrue(tables.isEmpty()); // Missing schema - resp = resource.listTables(null, getBy(CatalogTests.READER_USER)); + resp = resource.listTableNamesForSchema(null, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); // Invalid schema - resp = resource.listTables("bogus", getBy(CatalogTests.READER_USER)); + resp = resource.listTableNamesForSchema("bogus", getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); // Create a table @@ -296,28 +296,28 @@ public void testList() assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); // No read access - resp = resource.listTables(getBy(CatalogTests.DENY_USER)); + resp = resource.listTableNames(getBy(CatalogTests.DENY_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); tableIds = getTableIdList(resp); assertTrue(tableIds.isEmpty()); - resp = resource.listTables(TableId.DRUID_SCHEMA, getBy(CatalogTests.DENY_USER)); + resp = resource.listTableNamesForSchema(TableId.DRUID_SCHEMA, getBy(CatalogTests.DENY_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); tables = getTableList(resp); assertTrue(tables.isEmpty()); // Read access - resp = resource.listTables(getBy(CatalogTests.READER_USER)); + resp = resource.listTableNames(getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); tableIds = getTableIdList(resp); assertEquals(1, tableIds.size()); - resp = resource.listTables(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + resp = resource.listTableNamesForSchema(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); tables = getTableList(resp); assertEquals(1, tables.size()); - resp = resource.listTables(TableId.SYSTEM_SCHEMA, getBy(CatalogTests.READER_USER)); + resp = resource.listTableNamesForSchema(TableId.SYSTEM_SCHEMA, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); tables = getTableList(resp); assertTrue(tables.isEmpty()); @@ -400,13 +400,13 @@ public void testLifecycle() assertEquals(dsSpec, read1.spec()); // list - resp = resource.listTables(getBy(CatalogTests.READER_USER)); + resp = resource.listTableNames(getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); List tableIds = getTableIdList(resp); assertEquals(1, tableIds.size()); assertEquals(id1, tableIds.get(0)); - resp = resource.listTables(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + resp = resource.listTableNamesForSchema(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); List tables = getTableList(resp); assertEquals(1, tables.size()); @@ -434,14 +434,14 @@ public void testLifecycle() TableId id2 = TableId.of(TableId.DRUID_SCHEMA, table2Name); // verify lists - resp = resource.listTables(getBy(CatalogTests.READER_USER)); + resp = resource.listTableNames(getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); tableIds = getTableIdList(resp); assertEquals(2, tableIds.size()); assertEquals(id1, tableIds.get(0)); assertEquals(id2, tableIds.get(1)); - resp = resource.listTables(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + resp = resource.listTableNamesForSchema(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); tables = getTableList(resp); assertEquals(2, tables.size()); @@ -452,7 +452,7 @@ public void testLifecycle() resp = resource.deleteTable(TableId.DRUID_SCHEMA, table1Name, false, deleteBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - resp = resource.listTables(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + resp = resource.listTableNamesForSchema(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); tables = getTableList(resp); assertEquals(1, tables.size()); @@ -460,7 +460,7 @@ public void testLifecycle() resp = resource.deleteTable(TableId.DRUID_SCHEMA, table2Name, false, deleteBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - resp = resource.listTables(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + resp = resource.listTableNamesForSchema(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); tables = getTableList(resp); assertEquals(0, tables.size()); From 48cd785e7afb39ef72dc2c62d58ec258b1eec74c Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Thu, 27 Oct 2022 17:55:25 -0700 Subject: [PATCH 09/22] Revisions Revise SQL layer code Generalize exceptions Start on revision of HTTP layer --- .../util/common/jackson/JacksonUtils.java | 31 +- .../druid/catalog/CatalogException.java | 76 +++ .../druid/catalog/http/CatalogResource.java | 41 +- .../druid/catalog/http/CatalogResource2.java | 303 +++++++++ .../druid/catalog/storage/CatalogStorage.java | 2 +- .../catalog/storage/sql/CatalogManager.java | 84 +-- .../storage/sql/SQLCatalogManager.java | 592 ++++++++++++------ .../catalog/storage/TableManagerTest.java | 60 +- .../catalog/sync/CatalogMetadataTest.java | 17 +- .../server/http/catalog/CommandTest.java | 8 - .../druid/catalog/model/ColumnSpec.java | 10 - .../druid/catalog/model/TableMetadata.java | 10 - .../apache/druid/catalog/model/TableSpec.java | 10 - .../catalog/model/TableMetadataTest.java | 8 - .../model/table/DatasourceTableTest.java | 21 - .../model/table/HttpInputTableTest.java | 4 - .../catalog/model/table/InlineTableTest.java | 4 - .../catalog/model/table/LocalTableTest.java | 4 - 18 files changed, 869 insertions(+), 416 deletions(-) create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/CatalogException.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource2.java diff --git a/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java b/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java index 2098eb5059eb..16528f5edb42 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java +++ b/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java @@ -20,15 +20,14 @@ package org.apache.druid.java.util.common.jackson; import com.fasterxml.jackson.core.JsonGenerator; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.JsonSerializer; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializerProvider; -import org.apache.druid.java.util.common.ISE; import javax.annotation.Nullable; + import java.io.IOException; import java.util.Map; @@ -94,32 +93,4 @@ public static void writeObjectUsingSerializerProvider( serializer.serialize(o, jsonGenerator, serializers); } } - - /** - * Convert the given object to an array of bytes. Use when the object is - * known serializable so that the Jackson exception can be suppressed. - */ - public static byte[] toBytes(ObjectMapper jsonMapper, Object obj) - { - try { - return jsonMapper.writeValueAsBytes(obj); - } - catch (JsonProcessingException e) { - throw new ISE("Failed to serialize " + obj.getClass().getSimpleName()); - } - } - - /** - * Deserialize an object from an array of bytes. Use when the object is - * known deserializable so that the Jackson exception can be suppressed. - */ - public static T fromBytes(ObjectMapper jsonMapper, byte[] bytes, Class clazz) - { - try { - return jsonMapper.readValue(bytes, clazz); - } - catch (IOException e) { - throw new ISE(e, "Failed to deserialize a " + clazz.getSimpleName()); - } - } } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/CatalogException.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/CatalogException.java new file mode 100644 index 000000000000..e4fe2350ba47 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/CatalogException.java @@ -0,0 +1,76 @@ +package org.apache.druid.catalog; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.StringUtils; + +import javax.ws.rs.core.Response; + +public class CatalogException extends Exception +{ + public static final String DUPLICATE_ERROR = "Already exists"; + public static final String FAILED_ERROR = "Failed"; + public static final String INVALID_ERROR = "Invalid"; + public static final String NOT_FOUND_ERROR = "Not found"; + + public static final String ERROR_KEY = "error"; + public static final String ERR_MSG_KEY = "errorMessage"; + + /** + * Thrown when a record does not exist in the database. Allows + * the caller to check for this specific case in a generic way. + */ + public static class NotFoundException extends CatalogException + { + public NotFoundException(String msg, Object...args) + { + super(NOT_FOUND_ERROR, Response.Status.NOT_FOUND, msg, args); + } + } + + /** + * Indicates an attempt to insert a duplicate key into a table. + * This could indicate a logic error, or a race condition. It is + * generally not retryable: it us unrealistic to expect the other + * thread to helpfully delete the record it just added. + */ + public static class DuplicateKeyException extends CatalogException + { + public DuplicateKeyException(String msg, Object...args) + { + super(DUPLICATE_ERROR, Response.Status.BAD_REQUEST, msg, args); + } + } + + private final String errorCode; + private final Response.Status responseCode; + + public CatalogException( + final String errorCode, + final Response.Status responseCode, + final String message, + final Object...args + ) + { + super(StringUtils.format(message, args)); + this.errorCode = errorCode; + this.responseCode = responseCode; + } + + public static CatalogException badRequest(String msg, Object...args) + { + return new CatalogException( + CatalogException.INVALID_ERROR, + Response.Status.BAD_REQUEST, + msg, + args + ); + } + + public Response toResponse() + { + return Response + .status(responseCode) + .entity(ImmutableMap.of(ERROR_KEY, errorCode, ERR_MSG_KEY, getMessage())) + .build(); + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java index 80aa5c97059d..734bf44aef2e 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java @@ -21,6 +21,8 @@ import com.google.common.base.Strings; import org.apache.curator.shaded.com.google.common.collect.Lists; +import org.apache.druid.catalog.CatalogException.DuplicateKeyException; +import org.apache.druid.catalog.CatalogException.NotFoundException; import org.apache.druid.catalog.model.ColumnSpec; import org.apache.druid.catalog.model.SchemaRegistry.SchemaSpec; import org.apache.druid.catalog.model.TableDefnRegistry; @@ -33,9 +35,6 @@ import org.apache.druid.catalog.storage.HideColumns; import org.apache.druid.catalog.storage.MoveColumn; import org.apache.druid.catalog.storage.MoveColumn.Position; -import org.apache.druid.catalog.storage.sql.CatalogManager.DuplicateKeyException; -import org.apache.druid.catalog.storage.sql.CatalogManager.NotFoundException; -import org.apache.druid.catalog.storage.sql.CatalogManager.OutOfDateException; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; @@ -264,9 +263,6 @@ private Response updateTableSpec(TableMetadata table, long version) return Actions.okWithVersion(newVersion); } catch (NotFoundException e) { - return Response.status(Response.Status.NOT_FOUND).build(); - } - catch (OutOfDateException e) { return Response .status(Response.Status.BAD_REQUEST) .entity( @@ -338,7 +334,30 @@ public Response updateTableDefn( ); } - private Response incrementalUpdate( + private Response incrementalPropertiesUpdate( + TableId tableId, + TableSpec newSpec, + @Context final HttpServletRequest req, + Function action + ) + { + Response response = validateTable(tableId, newSpec, req); + if (response != null) { + return response; + } + try { + long newVersion = catalog.tables().updateTableSpec(tableId, action); + return Actions.okWithVersion(newVersion); + } + catch (NotFoundException e) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + catch (Exception e) { + return Actions.exception(e); + } + } + + private Response incrementalColumnsUpdate( TableId tableId, TableSpec newSpec, @Context final HttpServletRequest req, @@ -350,7 +369,7 @@ private Response incrementalUpdate( return response; } try { - long newVersion = catalog.tables().updatePayload(tableId, action); + long newVersion = catalog.tables().updateTableSpec(tableId, action); return Actions.okWithVersion(newVersion); } catch (NotFoundException e) { @@ -550,7 +569,7 @@ public Response listTableNames( @Context final HttpServletRequest req ) { - List tables = catalog.tables().list(); + List tables = catalog.tables().allTablePaths(); Iterable filtered = AuthorizationUtils.filterAuthorizedResources( req, tables, @@ -589,7 +608,7 @@ public Response listTableNamesForSchema( return result.lhs; } SchemaSpec schema = result.rhs; - List tables = catalog.tables().list(dbSchema); + List tables = catalog.tables().tableNamesInSchema(dbSchema); Iterable filtered = AuthorizationUtils.filterAuthorizedResources( req, tables, @@ -618,7 +637,7 @@ public Response listTableMetadataForSchema( return result.lhs; } SchemaSpec schema = result.rhs; - List tables = catalog.tables().listDetails(schema.name()); + List tables = catalog.tables().tablesInSchema(schema.name()); Iterable filtered = AuthorizationUtils.filterAuthorizedResources( req, tables, diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource2.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource2.java new file mode 100644 index 000000000000..f69e2c130253 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource2.java @@ -0,0 +1,303 @@ +package org.apache.druid.catalog.http; + +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.catalog.CatalogException; +import org.apache.druid.catalog.CatalogException.DuplicateKeyException; +import org.apache.druid.catalog.model.SchemaRegistry.SchemaSpec; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.storage.Actions; +import org.apache.druid.catalog.storage.CatalogStorage; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.Action; +import org.apache.druid.server.security.AuthorizationUtils; +import org.apache.druid.server.security.AuthorizerMapper; +import org.apache.druid.server.security.ForbiddenException; +import org.apache.druid.server.security.Resource; +import org.apache.druid.server.security.ResourceAction; + +import javax.inject.Inject; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +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.QueryParam; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; + +/** + * REST endpoint for user and internal catalog actions. Catalog actions + * are divided by operation: configuration-as-code, edits, retrieval, + * etc. Operations occur at the global level (all schemas), the schema level, or the + * table level. + * + * @see {@link CatalogListenerResource} for the broker-side API. + */ +public class CatalogResource2 +{ + public static final String ROOT_PATH = "/druid/coordinator/v1/catalog"; + + private final CatalogStorage catalog; + private final AuthorizerMapper authorizerMapper; + + @Inject + public CatalogResource2( + final CatalogStorage catalog, + final AuthorizerMapper authorizerMapper + ) + { + this.catalog = catalog; + this.authorizerMapper = authorizerMapper; + } + + // --------------------------------------------------------------------- + // Configuration-as-code style methods + + /** + * Create or update a new table containing the given table specification. + * Supports three use cases: + *
    + *
  • "create if not exists": default use case with no options.
  • + *
  • "create or update": set {@code overwrite=true}.
  • + *
  • "update": set {@code version} to the expected current version. + * This form enforces optimistic locking.
  • + *
+ * + * @param dbSchema The name of the Druid schema, which must be writable + * and the user must have at least read access. + * @param name The name of the table definition to modify. The user must + * have write access to the table. + * @param spec The new table definition. + * @param version the expected version of an existing table. The version must + * match. If not (or if the table does not exist), returns an error. + * @param overwrite if {@code true}, then overwrites any existing table. + * If {@code false}, then the operation fails if the table already exists. + * @param req the HTTP request used for authorization. + */ + @POST + @Path("/resource/tables/{dbSchema}/{name}") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response postTable( + @PathParam("dbSchema") String dbSchema, + @PathParam("name") String name, + TableSpec spec, + @QueryParam("version") long version, + @QueryParam("overwrite") boolean overwrite, + @Context final HttpServletRequest req + ) + { + try { + final SchemaSpec schema = validateSchema(dbSchema, true); + validateTable(schema, name, spec, req); + final TableMetadata table = TableMetadata.newTable(TableId.of(dbSchema, name), spec); + try { + catalog.validate(table); + } + catch (IAE e) { + throw CatalogException.badRequest(e.getMessage()); + } + + long newVersion; + if (version != 0) { + // A version is provided. Update that version (only). + newVersion = catalog.tables().update(table, version); + } else { + try { + // No version. Create the table. + newVersion = catalog.tables().create(table); + } + catch (DuplicateKeyException e) { + // Table exists + if (overwrite) { + // User wants to overwrite, so do so. + newVersion = catalog.tables().replace(table); + } else { + throw e; + } + } + } + return okWithVersion(newVersion); + } + catch (CatalogException e) + { + return e.toResponse(); + } + } + + /** + * Retrieves a table spec. + *

+ * Returns a 404 (NOT FOUND) error if the table definition does not exist. + * Note that this check is only for the specification; the table (or + * datasource) itself may exist. Similarly, this call may return a specification + * even if there is no datasource of the same name (typically occurs when + * the definition is created before the datasource itself.) + * + * @param dbSchema The Druid schema. The user must have read access. + * @param name The name of the table within the schema. The user must have + * read access. + * @param req the HTTP request used for authorization. + * @return the definition for the table, if any. + */ + @GET + @Path("/resource/tables/{dbSchema}/{name}") + @Produces(MediaType.APPLICATION_JSON) + public Response getTable( + @PathParam("dbSchema") String dbSchema, + @PathParam("name") String name, + @Context final HttpServletRequest req + ) + { + try { + final SchemaSpec schema = validateSchema(dbSchema, false); + authorizeTable(schema, name, Action.READ, req); + final TableMetadata table = catalog.tables().read(new TableId(dbSchema, name)); + return Response.ok().entity(table.spec()).build(); + } + catch (CatalogException e) + { + return e.toResponse(); + } + } + + /** + * Deletes the table definition (but not the underlying table or datasource) + * for the given schema and table. + * + * @param dbSchema The name of the schema that holds the table. + * @param name The name of the table definition to delete. The user must have + * write access. + */ + @DELETE + @Path("/tables/{dbSchema}/{name}") + @Produces(MediaType.APPLICATION_JSON) + public Response deleteTable( + @PathParam("dbSchema") String dbSchema, + @PathParam("name") String name, + @Context final HttpServletRequest req + ) + { + try { + final SchemaSpec schema = validateSchema(dbSchema, true); + authorizeTable(schema, name, Action.WRITE, req); + catalog.tables().delete(new TableId(dbSchema, name)); + return ok(); + } + catch (CatalogException e) + { + return e.toResponse(); + } + } + + // --------------------------------------------------------------------- + // Helper methods + + private void validateTable(SchemaSpec schema, String name, TableSpec spec, final HttpServletRequest req) throws CatalogException + { + // Table name can't be blank or have leading/trailing spaces + if (Strings.isNullOrEmpty(name)) { + throw CatalogException.badRequest("Table name is required"); + } + if (!name.equals(name.trim())) { + throw CatalogException.badRequest("Table name cannot start or end with spaces"); + } + + // The user has to have permission to modify the table. + authorizeTable(schema, name, Action.WRITE, req); + + // Validate the spec, if provided. + if (spec != null) { + + // The given table spec has to be valid for the given schema. + try { + spec.validate(); + } + catch (IAE e) { + throw CatalogException.badRequest(e.getMessage()); + } + + if (!schema.accepts(spec.type())) { + throw CatalogException.badRequest( + "Cannot create tables of type %s in schema %s", + spec.type(), + schema.name() + ); + } + } + + // Everything checks out, let the request proceed. + } + + private SchemaSpec validateSchema(String dbSchema, boolean forWrite) throws CatalogException + { + if (Strings.isNullOrEmpty(dbSchema)) { + throw CatalogException.badRequest("Schema name is required"); + } + SchemaSpec schema = catalog.resolveSchema(dbSchema); + if (schema == null) { + throw CatalogException.badRequest("Unknown schema %s", dbSchema); + } + + if (forWrite && !schema.writable()) { + throw CatalogException.badRequest( + "Cannot modify schema %s", + dbSchema + ); + } + return schema; + } + + private void authorizeTable(SchemaSpec schema, String name, Action action, HttpServletRequest request) throws CatalogException + { + if (Strings.isNullOrEmpty(name)) { + throw CatalogException.badRequest("Table name is required"); + } + if (action == Action.WRITE && !schema.writable()) { + throw new ForbiddenException( + "Cannot create table definitions in schema: " + schema.name()); + } + authorize(schema.securityResource(), name, action, request); + } + + private void authorize(String resource, String key, Action action, HttpServletRequest request) + { + final Access authResult = authorizeAccess(resource, key, action, request); + if (!authResult.isAllowed()) { + throw new ForbiddenException(authResult.toString()); + } + } + + private Access authorizeAccess(String resource, String key, Action action, HttpServletRequest request) + { + return AuthorizationUtils.authorizeResourceAction( + request, + new ResourceAction(new Resource(key, resource), action), + authorizerMapper + ); + } + + private static Response okWithVersion(long version) + { + return Response + .ok() + .entity(ImmutableMap.of("version", version)) + .build(); + } + + private static Response ok() + { + return Response.ok().build(); + } + +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java index fdccf95782a4..aaff76d875c0 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java @@ -119,7 +119,7 @@ public void register(CatalogListener listener) @Override public List tablesForSchema(String dbSchema) { - return tables().listDetails(dbSchema); + return tables().tablesInSchema(dbSchema); } @Override diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java index 856ee1926fe5..53814ccb7cbf 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java @@ -19,13 +19,16 @@ package org.apache.druid.catalog.storage.sql; +import org.apache.druid.catalog.CatalogException.DuplicateKeyException; +import org.apache.druid.catalog.CatalogException.NotFoundException; +import org.apache.druid.catalog.model.ColumnSpec; import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; -import org.apache.druid.catalog.model.TableSpec; import javax.annotation.Nullable; import java.util.List; +import java.util.Map; import java.util.function.Function; /** @@ -38,45 +41,6 @@ */ public interface CatalogManager { - /** - * Thrown with an "optimistic lock" fails: the version of a - * catalog object being updated is not the same as that of - * the expected version. - */ - class OutOfDateException extends Exception - { - public OutOfDateException(String msg) - { - super(msg); - } - } - - /** - * Thrown when a record does not exist in the database. Allows - * the caller to check for this specific case in a generic way. - */ - class NotFoundException extends Exception - { - public NotFoundException(String msg) - { - super(msg); - } - } - - /** - * Indicates an attempt to insert a duplicate key into a table. - * This could indicate a logic error, or a race condition. It is - * generally not retryable: it us unrealistic to expect the other - * thread to helpfully delete the record it just added. - */ - class DuplicateKeyException extends Exception - { - public DuplicateKeyException(String msg, Exception e) - { - super(msg, e); - } - } - /** * Generic interface for changes to the catalog at the storage level. * Implemented by the catalog sync mechanism to send update events @@ -130,26 +94,34 @@ interface Listener /** * Update a table definition. *

- * If {@code oldVersion == 0}, overwrites any current content. - * This is a potential race conditions if this is a partial update - * because of the possibility of another user doing an update since the - * read. Fine when the goal is to replace the entire definition. - * Else, only does the update is at the given version. + * The table must be at the {@code oldVersion}. Use this for optimistic-locking + * style updates. + * + * @throws NotFoundException if either the table does not exist, the table is + * not in the active state, or the version does not match. If the exception + * is thrown, the application should re-request the data, which will reveal + * the actual state at that moment + */ + long update(TableMetadata table, long oldVersion) throws NotFoundException; + + /** + * Replace a table definition. *

- * Retryable only if the version is given, and an - * {@code OutOfDateException} is thrown. + * Use this when the desire to replace whatever exists with the new information, + * such as configuration-as-code style updates. */ - long update(TableMetadata table, long oldVersion) throws OutOfDateException, NotFoundException; + long replace(TableMetadata table) throws NotFoundException; /** - * Update the table spec incrementally using the transform provided. Performs the update + * Update the table properties incrementally using the transform provided. Performs the update * in a transaction to ensure the read and write are atomic. * * @param id the table to update - * @param transform the transform to apply to the table spec + * @param transform the transform to apply to the table properties * @return the update timestamp (version) of the updated record */ - long updatePayload(TableId id, Function transform) throws NotFoundException; + long updateProperties(TableId id, Function, Map> transform) throws NotFoundException; + long updateColumns(TableId id, Function, List> transform) throws NotFoundException; /** * Move the table to the deleting state. No version check: fine @@ -167,7 +139,7 @@ interface Listener * @return the table record, or {@code null} if the entry is not * found in the DB. */ - @Nullable TableMetadata read(TableId id); + TableMetadata read(TableId id) throws NotFoundException; /** * Delete the table record for the given ID. Essentially does a @@ -182,21 +154,21 @@ interface Listener * @return {@code true} if the table exists and was deleted, * {@code false} if the table did not exist. */ - boolean delete(TableId id); + void delete(TableId id) throws NotFoundException; /** * Returns a list of the ids (schema, name) of all tables in the catalog. */ - List list(); + List allTablePaths(); /** * Returns a list of the names of all tables within the given schema. */ - List list(String dbSchema); + List tableNamesInSchema(String dbSchema); /** * Returns a list of the table metadata for all tables within the given * schema. */ - List listDetails(String dbSchema); + List tablesInSchema(String dbSchema); } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java index 99f49e10f9b6..db735f942d9f 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java @@ -19,11 +19,16 @@ package org.apache.druid.catalog.storage.sql; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.inject.Inject; +import org.apache.druid.catalog.CatalogException.DuplicateKeyException; +import org.apache.druid.catalog.CatalogException.NotFoundException; +import org.apache.druid.catalog.model.ColumnSpec; import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; @@ -42,6 +47,9 @@ import org.skife.jdbi.v2.exceptions.UnableToExecuteStatementException; import org.skife.jdbi.v2.tweak.HandleCallback; +import javax.annotation.Nullable; + +import java.io.IOException; import java.util.Deque; import java.util.List; import java.util.Map; @@ -53,68 +61,6 @@ public class SQLCatalogManager implements CatalogManager { public static final String TABLES_TABLE = "tableDefs"; - private static final String INSERT_TABLE = - "INSERT INTO %s\n" + - " (schemaName, name, creationTime, updateTime, state, payload)\n" + - " VALUES(:schemaName, :name, :creationTime, :updateTime, :state, :payload)"; - - private static final String UPDATE_HEAD = - "UPDATE %s\n SET\n"; - - private static final String WHERE_TABLE_ID = - "WHERE schemaName = :schemaName\n" + - " AND name = :name\n"; - - private static final String SAFETY_CHECK = - " AND updateTime = :oldVersion"; - - private static final String UPDATE_DEFN_UNSAFE = - UPDATE_HEAD + - " payload = :payload,\n" + - " updateTime = :updateTime\n" + - WHERE_TABLE_ID; - - private static final String UPDATE_DEFN_SAFE = - UPDATE_DEFN_UNSAFE + - SAFETY_CHECK; - - private static final String UPDATE_STATE = - UPDATE_HEAD + - " state = :state,\n" + - " updateTime = :updateTime\n" + - WHERE_TABLE_ID; - - private static final String SELECT_TABLE = - "SELECT creationTime, updateTime, state, payload\n" + - "FROM %s\n" + - WHERE_TABLE_ID; - - private static final String SELECT_PAYLOAD = - "SELECT state, payload\n" + - "FROM %s\n" + - WHERE_TABLE_ID; - - private static final String SELECT_ALL_TABLES = - "SELECT schemaName, name\n" + - "FROM %s\n" + - "ORDER BY schemaName, name"; - - private static final String SELECT_TABLES_IN_SCHEMA = - "SELECT name\n" + - "FROM %s\n" + - "WHERE schemaName = :schemaName\n" + - "ORDER BY name"; - - private static final String SELECT_TABLE_DETAILS_IN_SCHEMA = - "SELECT name, creationTime, updateTime, state, payload\n" + - "FROM %s\n" + - "WHERE schemaName = :schemaName\n" + - "ORDER BY name"; - - private static final String DELETE_TABLE = - "DELETE FROM %s\n" + - WHERE_TABLE_ID; - private final MetadataStorageManager metastoreManager; private final SQLMetadataConnector connector; private final ObjectMapper jsonMapper; @@ -142,16 +88,18 @@ public void start() createTableDefnTable(); } - // Mimics what MetadataStorageTablesConfig should do. - public String getTableDefnTable() - { - final String base = metastoreManager.tablesConfig().getBase(); - if (Strings.isNullOrEmpty(base)) { - return TABLES_TABLE; - } else { - return StringUtils.format("%s_%s", base, TABLES_TABLE); - } - } + public static final String CREATE_TABLE = + "CREATE TABLE %s (\n" + + " schemaName VARCHAR(255) NOT NULL,\n" + + " name VARCHAR(255) NOT NULL,\n" + + " creationTime BIGINT NOT NULL,\n" + + " updateTime BIGINT NOT NULL,\n" + + " state CHAR(1) NOT NULL,\n" + + " tableType VARCHAR(20) NOT NULL,\n" + + " properties %s,\n" + + " columns %s,\n" + + " PRIMARY KEY(schemaName, name)\n" + + ")"; // TODO: Move to SqlMetadataConnector public void createTableDefnTable() @@ -163,22 +111,22 @@ public void createTableDefnTable() tableName, ImmutableList.of( StringUtils.format( - "CREATE TABLE %s (\n" - + " schemaName VARCHAR(255) NOT NULL,\n" - + " name VARCHAR(255) NOT NULL,\n" - + " creationTime BIGINT NOT NULL,\n" - + " updateTime BIGINT NOT NULL,\n" - + " state CHAR(1) NOT NULL,\n" - + " payload %s,\n" - + " PRIMARY KEY(schemaName, name)\n" - + ")", + CREATE_TABLE, tableName, + connector.getPayloadType(), connector.getPayloadType() ) ) ); } + private static final String INSERT_TABLE = + "INSERT INTO %s\n" + + " (schemaName, name, creationTime, updateTime, state,\n" + + " tableType, properties, columns)\n" + + " VALUES(:schemaName, :name, :creationTime, :updateTime, :state,\n" + + " :tableType, :properties, :columns)"; + @Override public long create(TableMetadata table) throws DuplicateKeyException { @@ -189,24 +137,27 @@ public long create(TableMetadata table) throws DuplicateKeyException @Override public Long withHandle(Handle handle) throws DuplicateKeyException { - long updateTime = System.currentTimeMillis(); - Update stmt = handle.createStatement( - StringUtils.format(INSERT_TABLE, tableName) - ) + final TableSpec spec = table.spec(); + final long updateTime = System.currentTimeMillis(); + final Update stmt = handle + .createStatement(statement(INSERT_TABLE)) .bind("schemaName", table.id().schema()) .bind("name", table.id().name()) .bind("creationTime", updateTime) .bind("updateTime", updateTime) .bind("state", TableMetadata.TableState.ACTIVE.code()) - .bind("payload", table.spec().toBytes(jsonMapper)); + .bind("tableType", spec.type()) + .bind("properties", toBytes(jsonMapper, spec.properties())) + .bind("columns", toBytes(jsonMapper, spec.columns())); try { stmt.execute(); } catch (UnableToExecuteStatementException e) { if (DbUtils.isDuplicateRecordException(e)) { throw new DuplicateKeyException( - "Tried to insert a duplicate table: " + table.sqlName(), - e); + "Tried to insert a duplicate table: %s", + table.sqlName() + ); } else { throw e; } @@ -225,73 +176,87 @@ public Long withHandle(Handle handle) throws DuplicateKeyException } } + private static final String SELECT_TABLE = + "SELECT creationTime, updateTime, state, tableType, properties, columns\n" + + "FROM %s\n" + + "WHERE schemaName = :schemaName\n" + + " AND name = :name\n"; + @Override - public TableMetadata read(TableId id) + public TableMetadata read(TableId id) throws NotFoundException { - return dbi.withHandle( - new HandleCallback() - { - @Override - public TableMetadata withHandle(Handle handle) + try { + return dbi.withHandle( + new HandleCallback() { - Query> query = handle.createQuery( - StringUtils.format(SELECT_TABLE, tableName) - ) - .setFetchSize(connector.getStreamingFetchSize()) - .bind("schemaName", id.schema()) - .bind("name", id.name()); - final ResultIterator resultIterator = - query.map((index, r, ctx) -> - new TableMetadata( - id, - r.getLong(1), - r.getLong(2), - TableMetadata.TableState.fromCode(r.getString(3)), - TableSpec.fromBytes(jsonMapper, r.getBytes(4)) - )) - .iterator(); - if (resultIterator.hasNext()) { - return resultIterator.next(); + @Override + public TableMetadata withHandle(Handle handle) throws NotFoundException + { + final Query> query = handle + .createQuery(statement(SELECT_TABLE)) + .setFetchSize(connector.getStreamingFetchSize()) + .bind("schemaName", id.schema()) + .bind("name", id.name()); + final ResultIterator resultIterator = + query.map((index, r, ctx) -> + new TableMetadata( + id, + r.getLong(1), + r.getLong(2), + TableMetadata.TableState.fromCode(r.getString(3)), + tableSpecFromBytes(jsonMapper, r.getString(4), r.getBytes(5), r.getBytes(6)) + )) + .iterator(); + if (resultIterator.hasNext()) { + return resultIterator.next(); + } + throw tableNotFound(id); } - return null; } - } - ); - } - - @Override - public long update(TableMetadata table, long oldVersion) throws OutOfDateException, NotFoundException - { - if (oldVersion == 0) { - return updateUnsafe(table.id(), table.spec()); - } else { - return updateSafe(table.id(), table.spec(), oldVersion); + ); + } + catch (CallbackFailedException e) { + if (e.getCause() instanceof NotFoundException) { + throw (NotFoundException) e.getCause(); + } + throw e; } } - private long updateSafe(TableId id, TableSpec defn, long oldVersion) throws OutOfDateException + private static final String REPLACE_SPEC_STMT = + "UPDATE %s\n SET\n" + + " tableType = :tableType,\n" + + " properties = :properties,\n" + + " columns = :columns,\n" + + " updateTime = :updateTime\n" + + "WHERE schemaName = :schemaName\n" + + " AND name = :name\n" + + " AND state = 'A'"; + + @Override + public long replace(TableMetadata table) throws NotFoundException { try { return dbi.withHandle( new HandleCallback() { @Override - public Long withHandle(Handle handle) throws OutOfDateException + public Long withHandle(Handle handle) throws NotFoundException { - long updateTime = System.currentTimeMillis(); - int updateCount = handle.createStatement( - StringUtils.format(UPDATE_DEFN_SAFE, tableName)) + final TableId id = table.id(); + final TableSpec spec = table.spec(); + final long updateTime = System.currentTimeMillis(); + final int updateCount = handle + .createStatement(statement(REPLACE_SPEC_STMT)) .bind("schemaName", id.schema()) .bind("name", id.name()) - .bind("payload", defn.toBytes(jsonMapper)) + .bind("tableType", spec.type()) + .bind("properties", toBytes(jsonMapper, spec.properties())) + .bind("columns", toBytes(jsonMapper, spec.columns())) .bind("updateTime", updateTime) - .bind("oldVersion", oldVersion) .execute(); if (updateCount == 0) { - throw new OutOfDateException( - StringUtils.format( - "Table %s: not found or update version does not match DB version", - id.sqlName())); + throw tableNotFound(id); } sendUpdate(id); return updateTime; @@ -300,14 +265,19 @@ public Long withHandle(Handle handle) throws OutOfDateException ); } catch (CallbackFailedException e) { - if (e.getCause() instanceof OutOfDateException) { - throw (OutOfDateException) e.getCause(); + if (e.getCause() instanceof NotFoundException) { + throw (NotFoundException) e.getCause(); } throw e; } } - private long updateUnsafe(TableId id, TableSpec defn) throws NotFoundException + private static final String UPDATE_SPEC_STMT = + REPLACE_SPEC_STMT + + " AND updateTime = :oldVersion"; + + @Override + public long update(TableMetadata table, long oldVersion) throws NotFoundException { try { return dbi.withHandle( @@ -316,17 +286,23 @@ private long updateUnsafe(TableId id, TableSpec defn) throws NotFoundException @Override public Long withHandle(Handle handle) throws NotFoundException { - long updateTime = System.currentTimeMillis(); - int updateCount = handle.createStatement( - StringUtils.format(UPDATE_DEFN_UNSAFE, tableName)) + final TableId id = table.id(); + final TableSpec spec = table.spec(); + final long updateTime = System.currentTimeMillis(); + final int updateCount = handle + .createStatement(statement(UPDATE_SPEC_STMT)) .bind("schemaName", id.schema()) .bind("name", id.name()) - .bind("payload", defn.toBytes(jsonMapper)) + .bind("tableType", spec.type()) + .bind("properties", toBytes(jsonMapper, spec.properties())) + .bind("columns", toBytes(jsonMapper, spec.columns())) .bind("updateTime", updateTime) + .bind("oldVersion", oldVersion) .execute(); if (updateCount == 0) { throw new NotFoundException( - StringUtils.format("Table %s: not found", id.sqlName()) + "Table %s: not found, is being deleted or update version does not match DB version", + id.sqlName() ); } sendUpdate(id); @@ -343,8 +319,24 @@ public Long withHandle(Handle handle) throws NotFoundException } } + private static final String SELECT_TABLE_PROPERTIES_STMT = + "SELECT properties\n" + + "FROM %s\n" + + "WHERE schemaName = :schemaName\n" + + " AND name = :name\n" + + " AND state = 'A'"; + + private static final String UPDATE_TABLE_PROPERTIES_STMT = + "UPDATE %s\n SET\n" + + " properties = :properties\n" + + "WHERE schemaName = :schemaName\n" + + " AND name = :name\n"; + @Override - public long updatePayload(TableId id, Function transform) throws NotFoundException + public long updateProperties( + final TableId id, + final Function, Map> transform + ) throws NotFoundException { try { return dbi.withHandle( @@ -355,42 +347,106 @@ public Long withHandle(Handle handle) throws NotFoundException { handle.begin(); try { - Query> query = handle.createQuery( - StringUtils.format(SELECT_PAYLOAD, tableName) - ) + final Query> query = handle + .createQuery(statement(SELECT_TABLE_PROPERTIES_STMT)) .setFetchSize(connector.getStreamingFetchSize()) .bind("schemaName", id.schema()) .bind("name", id.name()); - final ResultIterator resultIterator = - query.map((index, r, ctx) -> - new TableMetadata( - id, - 0, - 0, - TableMetadata.TableState.fromCode(r.getString(1)), - TableSpec.fromBytes(jsonMapper, r.getBytes(2)) - )) - .iterator(); - TableMetadata table; + final ResultIterator> resultIterator = query + .map((index, r, ctx) -> propertiesFromBytes(jsonMapper, r.getBytes(1))) + .iterator(); + final Map properites; if (resultIterator.hasNext()) { - table = resultIterator.next(); + properites = resultIterator.next(); } else { handle.rollback(); - throw new NotFoundException( - StringUtils.format("Table %s: not found", id.sqlName()) - ); + throw tableNotFound(id); } - if (table.state() != TableMetadata.TableState.ACTIVE) { - throw new ISE("Table is in state [%s] and cannot be updated", table.state()); + final Map revised = transform.apply(properites); + final long updateTime = System.currentTimeMillis(); + final int updateCount = handle + .createStatement(statement(UPDATE_TABLE_PROPERTIES_STMT)) + .bind("schemaName", id.schema()) + .bind("name", id.name()) + .bind("properties", toBytes(jsonMapper, revised)) + .bind("updateTime", updateTime) + .execute(); + if (updateCount == 0) { + // Should never occur because we're holding a lock. + throw new ISE("Table %s: not found", id.sqlName()); } - TableSpec revised = transform.apply(table.spec()); - long updateTime = System.currentTimeMillis(); - int updateCount = handle.createStatement( - StringUtils.format(UPDATE_DEFN_UNSAFE, tableName)) + handle.commit(); + sendUpdate(id); + return updateTime; + } + catch (RuntimeException e) { + handle.rollback(); + throw e; + } + } + } + ); + } + catch (CallbackFailedException e) { + if (e.getCause() instanceof NotFoundException) { + throw (NotFoundException) e.getCause(); + } + throw e; + } + } + + private static final String SELECT_COLUMNS_STMT = + "SELECT columns\n" + + "FROM %s\n" + + "WHERE schemaName = :schemaName\n" + + " AND name = :name\n" + + " AND state = 'A'"; + + private static final String UPDATE_COLUMNS_STMT = + "UPDATE %s\n SET\n" + + " columns = :columns\n" + + "WHERE schemaName = :schemaName\n" + + " AND name = :name\n"; + + @Override + public long updateColumns( + final TableId id, + final Function, List> transform + ) throws NotFoundException + { + try { + return dbi.withHandle( + new HandleCallback() + { + @Override + public Long withHandle(Handle handle) throws NotFoundException + { + handle.begin(); + try { + final Query> query = handle + .createQuery(statement(SELECT_COLUMNS_STMT)) + .setFetchSize(connector.getStreamingFetchSize()) + .bind("schemaName", id.schema()) + .bind("name", id.name()); + + final ResultIterator> resultIterator = query + .map((index, r, ctx) -> columnsFromBytes(jsonMapper, r.getBytes(1))) + .iterator(); + final List columns; + if (resultIterator.hasNext()) { + columns = resultIterator.next(); + } else { + handle.rollback(); + throw tableNotFound(id); + } + final List revised = transform.apply(columns); + final long updateTime = System.currentTimeMillis(); + final int updateCount = handle + .createStatement(statement(UPDATE_COLUMNS_STMT)) .bind("schemaName", id.schema()) .bind("name", id.name()) - .bind("payload", revised.toBytes(jsonMapper)) + .bind("properties", toBytes(jsonMapper, revised)) .bind("updateTime", updateTime) .execute(); if (updateCount == 0) { @@ -417,6 +473,13 @@ public Long withHandle(Handle handle) throws NotFoundException } } + private static final String UPDATE_STATE = + "UPDATE %s\n SET\n" + + " state = :state,\n" + + " updateTime = :updateTime\n" + + "WHERE schemaName = :schemaName\n" + + " AND name = :name\n"; + @Override public long markDeleting(TableId id) { @@ -427,8 +490,8 @@ public long markDeleting(TableId id) public Long withHandle(Handle handle) { long updateTime = System.currentTimeMillis(); - int updateCount = handle.createStatement( - StringUtils.format(UPDATE_STATE, tableName)) + int updateCount = handle + .createStatement(statement(UPDATE_STATE)) .bind("schemaName", id.schema()) .bind("name", id.name()) .bind("updateTime", updateTime) @@ -441,29 +504,51 @@ public Long withHandle(Handle handle) ); } + private static final String DELETE_TABLE = + "DELETE FROM %s\n" + + "WHERE schemaName = :schemaName\n" + + " AND name = :name\n"; + @Override - public boolean delete(TableId id) + public void delete(TableId id) throws NotFoundException { - return dbi.withHandle( - new HandleCallback() - { - @Override - public Boolean withHandle(Handle handle) + try { + dbi.withHandle( + new HandleCallback() { - int updateCount = handle.createStatement( - StringUtils.format(DELETE_TABLE, tableName)) - .bind("schemaName", id.schema()) - .bind("name", id.name()) - .execute(); - sendDeletion(id); - return updateCount > 0; + @Override + public Void withHandle(Handle handle) throws NotFoundException + { + int updateCount = handle + .createStatement(statement(DELETE_TABLE)) + .bind("schemaName", id.schema()) + .bind("name", id.name()) + .execute(); + if (updateCount == 0) { + throw tableNotFound(id); + } else { + sendDeletion(id); + return null; + } + } } - } - ); + ); + } + catch (CallbackFailedException e) { + if (e.getCause() instanceof NotFoundException) { + throw (NotFoundException) e.getCause(); + } + throw e; + } } + private static final String SELECT_ALL_TABLE_PATHS = + "SELECT schemaName, name\n" + + "FROM %s\n" + + "ORDER BY schemaName, name"; + @Override - public List list() + public List allTablePaths() { return dbi.withHandle( new HandleCallback>() @@ -471,9 +556,8 @@ public List list() @Override public List withHandle(Handle handle) { - Query> query = handle.createQuery( - StringUtils.format(SELECT_ALL_TABLES, tableName) - ) + Query> query = handle + .createQuery(statement(SELECT_ALL_TABLE_PATHS)) .setFetchSize(connector.getStreamingFetchSize()); final ResultIterator resultIterator = query.map((index, r, ctx) -> @@ -485,8 +569,14 @@ public List withHandle(Handle handle) ); } + private static final String SELECT_TABLE_NAMES_IN_SCHEMA = + "SELECT name\n" + + "FROM %s\n" + + "WHERE schemaName = :schemaName\n" + + "ORDER BY name"; + @Override - public List list(String dbSchema) + public List tableNamesInSchema(String dbSchema) { return dbi.withHandle( new HandleCallback>() @@ -494,9 +584,8 @@ public List list(String dbSchema) @Override public List withHandle(Handle handle) { - Query> query = handle.createQuery( - StringUtils.format(SELECT_TABLES_IN_SCHEMA, tableName) - ) + Query> query = handle + .createQuery(statement(SELECT_TABLE_NAMES_IN_SCHEMA)) .bind("schemaName", dbSchema) .setFetchSize(connector.getStreamingFetchSize()); final ResultIterator resultIterator = @@ -509,8 +598,14 @@ public List withHandle(Handle handle) ); } + private static final String SELECT_TABLES_IN_SCHEMA = + "SELECT name, creationTime, updateTime, state, tableType, properties, columns\n" + + "FROM %s\n" + + "WHERE schemaName = :schemaName\n" + + "ORDER BY name"; + @Override - public List listDetails(String dbSchema) + public List tablesInSchema(String dbSchema) { return dbi.withHandle( new HandleCallback>() @@ -518,19 +613,20 @@ public List listDetails(String dbSchema) @Override public List withHandle(Handle handle) { - Query> query = handle.createQuery( - StringUtils.format(SELECT_TABLE_DETAILS_IN_SCHEMA, tableName) - ) + Query> query = handle + .createQuery(statement(SELECT_TABLES_IN_SCHEMA)) .bind("schemaName", dbSchema) .setFetchSize(connector.getStreamingFetchSize()); final ResultIterator resultIterator = query.map((index, r, ctx) -> - new TableMetadata( - TableId.of(dbSchema, r.getString(1)), - r.getLong(2), - r.getLong(3), - TableMetadata.TableState.fromCode(r.getString(4)), - TableSpec.fromBytes(jsonMapper, r.getBytes(5)))) + new TableMetadata( + TableId.of(dbSchema, r.getString(1)), + r.getLong(2), + r.getLong(3), + TableMetadata.TableState.fromCode(r.getString(4)), + tableSpecFromBytes(jsonMapper, r.getString(5), r.getBytes(6), r.getBytes(7)) + ) + ) .iterator(); return Lists.newArrayList(resultIterator); } @@ -572,4 +668,92 @@ protected synchronized void sendDeletion(TableId id) listener.deleted(id); } } + + // Mimics what MetadataStorageTablesConfig should do. + public String getTableDefnTable() + { + final String base = metastoreManager.tablesConfig().getBase(); + if (Strings.isNullOrEmpty(base)) { + return TABLES_TABLE; + } else { + return StringUtils.format("%s_%s", base, TABLES_TABLE); + } + } + + private String statement(String baseStmt) + { + return StringUtils.format(baseStmt, tableName); + } + + private NotFoundException tableNotFound(TableId id) + { + return new NotFoundException( + "Table %s: not found", + id.sqlName() + ); + } + + /** + * Convert the given object to an array of bytes. Use when the object is + * known serializable so that the Jackson exception can be suppressed. + */ + private static byte[] toBytes(ObjectMapper jsonMapper, Object obj) + { + try { + return jsonMapper.writeValueAsBytes(obj); + } + catch (JsonProcessingException e) { + throw new ISE("Failed to serialize " + obj.getClass().getSimpleName()); + } + } + + /** + * Deserialize an object from an array of bytes. Use when the object is + * known deserializable so that the Jackson exception can be suppressed. + */ + private static T fromBytes(ObjectMapper jsonMapper, byte[] bytes, TypeReference typeRef) + { + try { + return jsonMapper.readValue(bytes, typeRef); + } + catch (IOException e) { + throw new ISE(e, "Failed to deserialize a DB object"); + } + } + + private static TableSpec tableSpecFromBytes( + final ObjectMapper jsonMapper, + final String type, + final byte[] properties, + final byte[] columns + ) + { + return new TableSpec( + type, + propertiesFromBytes(jsonMapper, properties), + columnsFromBytes(jsonMapper, columns) + ); + } + + private static final TypeReference> PROPERTIES_TYPE_REF = + new TypeReference>() { }; + + private static Map propertiesFromBytes( + final ObjectMapper jsonMapper, + final byte[] properties + ) + { + return fromBytes(jsonMapper, properties, PROPERTIES_TYPE_REF); + } + + private static final TypeReference> COLUMNS_TYPE_REF = + new TypeReference>() { }; + + private static List columnsFromBytes( + final ObjectMapper jsonMapper, + final byte[] properties + ) + { + return fromBytes(jsonMapper, properties, COLUMNS_TYPE_REF); + } } diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java index 0839be8124b2..5d6ece6a817d 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java @@ -21,6 +21,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; +import org.apache.druid.catalog.CatalogException.DuplicateKeyException; +import org.apache.druid.catalog.CatalogException.NotFoundException; import org.apache.druid.catalog.CatalogTest; import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; @@ -28,9 +30,6 @@ import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; import org.apache.druid.catalog.model.table.DatasourceDefn; import org.apache.druid.catalog.storage.sql.CatalogManager; -import org.apache.druid.catalog.storage.sql.CatalogManager.DuplicateKeyException; -import org.apache.druid.catalog.storage.sql.CatalogManager.NotFoundException; -import org.apache.druid.catalog.storage.sql.CatalogManager.OutOfDateException; import org.apache.druid.catalog.storage.sql.SQLCatalogManager; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.metadata.TestDerbyConnector; @@ -107,7 +106,7 @@ public void testCreate() throws DuplicateKeyException } @Test - public void testUpdate() throws DuplicateKeyException, OutOfDateException, NotFoundException + public void testUpdate() throws DuplicateKeyException, NotFoundException { Map props = ImmutableMap.of( AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", @@ -115,7 +114,7 @@ public void testUpdate() throws DuplicateKeyException, OutOfDateException, NotFo ); TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); TableMetadata table = TableMetadata.newTable(TableId.datasource("table1"), spec); - long version = manager.create(table); + final long version1 = manager.create(table); // Change the definition props = ImmutableMap.of( @@ -124,25 +123,34 @@ public void testUpdate() throws DuplicateKeyException, OutOfDateException, NotFo ); TableSpec spec2 = spec.withProperties(props); TableMetadata table2 = table.withSpec(spec2); - assertThrows(OutOfDateException.class, () -> manager.update(table2, 3)); + assertThrows(NotFoundException.class, () -> manager.update(table2, 3)); + + assertEquals(version1, manager.read(table.id()).updateTime()); + final long version2 = manager.update(table2, version1); + TableMetadata read = manager.read(table.id()); + assertEquals(spec2, read.spec()); + assertEquals(version2, read.updateTime()); - assertEquals(version, manager.read(table.id()).updateTime()); - long newVersion = manager.update(table2, version); - TableMetadata table3 = manager.read(table.id()); - assertEquals(spec2, table3.spec()); - assertEquals(newVersion, table3.updateTime()); + // Replace + TableMetadata table3 = table.withSpec(spec2); + final long version3 = manager.replace(table3); + assertTrue(version3 > version2); + read = manager.read(table.id()); + assertEquals(spec2, read.spec()); + assertEquals(version3, read.updateTime()); // Changing the state requires no version check - assertEquals(TableMetadata.TableState.ACTIVE, table3.state()); - newVersion = manager.markDeleting(table.id()); - TableMetadata table4 = manager.read(table.id()); - assertEquals(TableMetadata.TableState.DELETING, table4.state()); - assertEquals(newVersion, table4.updateTime()); - - // Update: no version check) - TableMetadata table5 = table.withSpec(spec2); - long newerVersion = manager.update(table5, 0); - assertTrue(newerVersion > newVersion); + assertEquals(TableMetadata.TableState.ACTIVE, read.state()); + long version4 = manager.markDeleting(table.id()); + read = manager.read(table.id()); + assertEquals(TableMetadata.TableState.DELETING, read.state()); + assertEquals(version4, read.updateTime()); + + // Can't update when deleting + assertThrows(NotFoundException.class, () -> manager.update(table3, version4)); + + // Can't replace when deleting + assertThrows(NotFoundException.class, () -> manager.replace(table3)); } @Test @@ -164,7 +172,7 @@ public void testDelete() throws DuplicateKeyException @Test public void testList() throws DuplicateKeyException { - List list = manager.list(); + List list = manager.allTablePaths(); assertTrue(list.isEmpty()); Map props = ImmutableMap.of( @@ -181,7 +189,7 @@ public void testList() throws DuplicateKeyException version = manager.create(table1); table1 = table1.fromInsert(version); - list = manager.list(); + list = manager.allTablePaths(); assertEquals(2, list.size()); TableId id = list.get(0); assertEquals(TableId.DRUID_SCHEMA, id.schema()); @@ -190,13 +198,13 @@ public void testList() throws DuplicateKeyException assertEquals(TableId.DRUID_SCHEMA, id.schema()); assertEquals("table2", id.name()); - List names = manager.list(TableId.DRUID_SCHEMA); + List names = manager.tableNamesInSchema(TableId.DRUID_SCHEMA); assertEquals(2, names.size()); - names = manager.list(TableId.SYSTEM_SCHEMA); + names = manager.tableNamesInSchema(TableId.SYSTEM_SCHEMA); assertEquals(0, names.size()); - List details = manager.listDetails(TableId.DRUID_SCHEMA); + List details = manager.tablesInSchema(TableId.DRUID_SCHEMA); assertEquals(Arrays.asList(table1, table2), details); } } diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java index 9146c088c30a..bae8cff5a475 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java @@ -21,6 +21,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.smile.SmileFactory; +import org.apache.druid.catalog.CatalogException.DuplicateKeyException; +import org.apache.druid.catalog.CatalogException.NotFoundException; import org.apache.druid.catalog.model.ColumnSpec; import org.apache.druid.catalog.model.Columns; import org.apache.druid.catalog.model.TableId; @@ -35,9 +37,6 @@ import org.apache.druid.catalog.model.table.TableBuilder; import org.apache.druid.catalog.storage.CatalogStorage; import org.apache.druid.catalog.storage.CatalogTests; -import org.apache.druid.catalog.storage.sql.CatalogManager.DuplicateKeyException; -import org.apache.druid.catalog.storage.sql.CatalogManager.NotFoundException; -import org.apache.druid.catalog.storage.sql.CatalogManager.OutOfDateException; import org.apache.druid.java.util.common.IAE; import org.apache.druid.metadata.TestDerbyConnector; import org.junit.After; @@ -115,7 +114,7 @@ public void testInputValidation() } @Test - public void testDirect() throws DuplicateKeyException, OutOfDateException, NotFoundException + public void testDirect() throws DuplicateKeyException, NotFoundException { populateCatalog(); MetadataCatalog catalog = new LocalMetadataCatalog(storage, storage.schemaRegistry()); @@ -125,7 +124,7 @@ public void testDirect() throws DuplicateKeyException, OutOfDateException, NotFo } @Test - public void testCached() throws DuplicateKeyException, OutOfDateException, NotFoundException + public void testCached() throws DuplicateKeyException, NotFoundException { populateCatalog(); CachedMetadataCatalog catalog = new CachedMetadataCatalog(storage, storage.schemaRegistry(), jsonMapper); @@ -146,18 +145,18 @@ public void testCached() throws DuplicateKeyException, OutOfDateException, NotFo } @Test - public void testRemoteWithJson() throws DuplicateKeyException, OutOfDateException, NotFoundException + public void testRemoteWithJson() throws DuplicateKeyException, NotFoundException { doTestRemote(false); } @Test - public void testRemoteWithSmile() throws DuplicateKeyException, OutOfDateException, NotFoundException + public void testRemoteWithSmile() throws DuplicateKeyException, NotFoundException { doTestRemote(true); } - private void doTestRemote(boolean useSmile) throws DuplicateKeyException, OutOfDateException, NotFoundException + private void doTestRemote(boolean useSmile) throws DuplicateKeyException, NotFoundException { populateCatalog(); MockCatalogSync sync = new MockCatalogSync(storage, CatalogTests.AUTH_MAPPER, jsonMapper, smileMapper, useSmile); @@ -284,7 +283,7 @@ private void verifyInitial(MetadataCatalog catalog) assertEquals("table3", tables.get(0).id().name()); } - private void alterCatalog() throws DuplicateKeyException, OutOfDateException, NotFoundException + private void alterCatalog() throws DuplicateKeyException, NotFoundException { // Add a column to table 1 TableId id1 = TableId.datasource("table1"); diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java index 0d03a46e0098..4bdb75af9874 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java @@ -96,10 +96,6 @@ public void testMoveColumn() Arrays.asList("b", "a", "c"), CatalogUtils.columnNames(revised) ); - - byte[] bytes = JacksonUtils.toBytes(mapper, cmd); - MoveColumn cmd2 = JacksonUtils.fromBytes(mapper, bytes, MoveColumn.class); - assertEquals(cmd, cmd2); } @Test @@ -149,9 +145,5 @@ public void testHideColumns() cmd = new HideColumns(Arrays.asList("b", "d", "e"), Arrays.asList("c", "d")); revised = cmd.perform(Arrays.asList("a", "b", "c")); assertEquals(Arrays.asList("a", "b", "d", "e"), revised); - - byte[] bytes = JacksonUtils.toBytes(mapper, cmd); - HideColumns cmd2 = JacksonUtils.fromBytes(mapper, bytes, HideColumns.class); - assertEquals(cmd, cmd2); } } diff --git a/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java b/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java index 90ff4e7542cc..e8b6b24442e9 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java +++ b/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java @@ -97,16 +97,6 @@ public void validate() } } - public byte[] toBytes(ObjectMapper jsonMapper) - { - return JacksonUtils.toBytes(jsonMapper, this); - } - - public static ColumnSpec fromBytes(ObjectMapper jsonMapper, byte[] bytes) - { - return JacksonUtils.fromBytes(jsonMapper, bytes, ColumnSpec.class); - } - @Override public String toString() { diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java index f9ce13f22178..af8d54b364c3 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java @@ -195,16 +195,6 @@ public void validate() } } - public byte[] toBytes(ObjectMapper jsonMapper) - { - return JacksonUtils.toBytes(jsonMapper, this); - } - - public static TableMetadata fromBytes(ObjectMapper jsonMapper, byte[] bytes) - { - return JacksonUtils.fromBytes(jsonMapper, bytes, TableMetadata.class); - } - @Override public String toString() { diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java b/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java index 4b1587f474b9..6d89bf1c3c23 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java @@ -112,16 +112,6 @@ public void validate() } } - public byte[] toBytes(ObjectMapper jsonMapper) - { - return JacksonUtils.toBytes(jsonMapper, this); - } - - public static TableSpec fromBytes(ObjectMapper jsonMapper, byte[] bytes) - { - return JacksonUtils.fromBytes(jsonMapper, bytes, TableSpec.class); - } - @Override public String toString() { diff --git a/server/src/test/java/org/apache/druid/catalog/model/TableMetadataTest.java b/server/src/test/java/org/apache/druid/catalog/model/TableMetadataTest.java index 329692096193..92d33aa4a071 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/TableMetadataTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/TableMetadataTest.java @@ -56,10 +56,6 @@ public void testId() TableId id2 = TableId.datasource("ds"); assertEquals(TableId.DRUID_SCHEMA, id2.schema()); assertEquals("ds", id2.name()); - - byte[] bytes = JacksonUtils.toBytes(mapper, id1); - TableId id3 = JacksonUtils.fromBytes(mapper, bytes, TableId.class); - assertEquals(id1, id3); } @Test @@ -92,10 +88,6 @@ public void testTableMetadata() assertEquals(20, table.updateTime()); assertEquals(TableState.ACTIVE, table.state()); assertNotNull(table.spec()); - - byte[] bytes = JacksonUtils.toBytes(mapper, table); - TableMetadata table2 = JacksonUtils.fromBytes(mapper, bytes, TableMetadata.class); - assertEquals(table, table2); } { diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/DatasourceTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/DatasourceTableTest.java index 9a76eee4f09c..b544ab4ca877 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/DatasourceTableTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/DatasourceTableTest.java @@ -132,19 +132,11 @@ public void testAllProperties() { TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); expectValidationSucceeds(spec); - - // Check serialization - byte[] bytes = spec.toBytes(mapper); - assertEquals(spec, TableSpec.fromBytes(mapper, bytes)); } { TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); expectValidationSucceeds(spec); - - // Check serialization - byte[] bytes = spec.toBytes(mapper); - assertEquals(spec, TableSpec.fromBytes(mapper, bytes)); } } @@ -341,19 +333,6 @@ private TableSpec exampleSpec() return spec; } - @Test - public void testSerialization() - { - TableSpec spec = exampleSpec(); - - // Round-trip - TableSpec spec2 = TableSpec.fromBytes(mapper, spec.toBytes(mapper)); - assertEquals(spec, spec2); - - // Sanity check of toString, which uses JSON - assertNotNull(spec.toString()); - } - private TableSpec mergeTables(TableSpec spec, TableSpec update) { ResolvedTable table = registry.resolve(spec); diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java index ab36adfad885..1fcfd551142b 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java @@ -82,10 +82,6 @@ public void testHappyPath() TableDefnRegistry registry = new TableDefnRegistry(mapper); assertNotNull(registry.resolve(table.spec())); - // Check serialization - byte[] bytes = table.spec().toBytes(mapper); - assertEquals(table.spec(), TableSpec.fromBytes(mapper, bytes)); - // Convert to an external spec ExternalTableSpec externSpec = tableDefn.convertToExtern(table); diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/InlineTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/InlineTableTest.java index f4c81e5a14d1..f3037e16e1b7 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/InlineTableTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/InlineTableTest.java @@ -73,10 +73,6 @@ public void testValidData() TableDefnRegistry registry = new TableDefnRegistry(mapper); assertNotNull(registry.resolve(table.spec())); - // Check serialization - byte[] bytes = table.spec().toBytes(mapper); - assertEquals(table.spec(), TableSpec.fromBytes(mapper, bytes)); - // Convert to an external spec ExternalTableSpec externSpec = tableDefn.convertToExtern(table); diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java index cec2c8cba4c3..5f247429bfac 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java @@ -72,10 +72,6 @@ public void testFullyDefined() TableDefnRegistry registry = new TableDefnRegistry(mapper); assertNotNull(registry.resolve(table.spec())); - // Check serialization - byte[] bytes = table.spec().toBytes(mapper); - assertEquals(table.spec(), TableSpec.fromBytes(mapper, bytes)); - // Convert to an external spec ExternalTableSpec externSpec = tableDefn.convertToExtern(table); From 9cfd0e09220144bac0ccebdf668f822ff3f424a4 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Fri, 28 Oct 2022 09:02:00 -0700 Subject: [PATCH 10/22] Revised HTTP endpoint Revised updated notifications Implemented the "edit" API --- .../druid/catalog/CatalogException.java | 1 + .../druid/catalog/http/CatalogResource.java | 4 +- .../druid/catalog/http/CatalogResource2.java | 83 +++-- .../catalog/{storage => http}/MoveColumn.java | 57 +--- .../druid/catalog/http/TableEditRequest.java | 84 +++++ .../druid/catalog/http/TableEditor.java | 293 ++++++++++++++++++ .../apache/druid/catalog/storage/Actions.java | 84 ----- .../druid/catalog/storage/HideColumns.java | 128 -------- .../catalog/storage/sql/CatalogManager.java | 28 +- .../storage/sql/SQLCatalogManager.java | 132 +++++--- .../catalog/storage/sql/UpdateEvent.java | 58 ++++ .../http/catalog/CatalogResourceTest.java | 4 +- .../server/http/catalog/CommandTest.java | 4 +- .../testsEx/catalog/ITCatalogRestTest.java | 4 +- .../druid/testsEx/cluster/CatalogClient.java | 4 +- .../druid/catalog/model/ObjectDefn.java | 2 +- .../apache/druid/catalog/model/TableDefn.java | 2 +- .../catalog/model/TableDefnRegistry.java | 5 + .../druid/catalog/model/TableMetadata.java | 31 +- 19 files changed, 625 insertions(+), 383 deletions(-) rename extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/{storage => http}/MoveColumn.java (58%) create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditRequest.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java delete mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/Actions.java delete mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/UpdateEvent.java diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/CatalogException.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/CatalogException.java index e4fe2350ba47..253b00d53c35 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/CatalogException.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/CatalogException.java @@ -11,6 +11,7 @@ public class CatalogException extends Exception public static final String FAILED_ERROR = "Failed"; public static final String INVALID_ERROR = "Invalid"; public static final String NOT_FOUND_ERROR = "Not found"; + public static final String BAD_STATE = "Invalid table spec"; public static final String ERROR_KEY = "error"; public static final String ERR_MSG_KEY = "errorMessage"; diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java index 734bf44aef2e..a9eaac71cf55 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java @@ -23,6 +23,7 @@ import org.apache.curator.shaded.com.google.common.collect.Lists; import org.apache.druid.catalog.CatalogException.DuplicateKeyException; import org.apache.druid.catalog.CatalogException.NotFoundException; +import org.apache.druid.catalog.http.MoveColumn.Position; import org.apache.druid.catalog.model.ColumnSpec; import org.apache.druid.catalog.model.SchemaRegistry.SchemaSpec; import org.apache.druid.catalog.model.TableDefnRegistry; @@ -32,9 +33,6 @@ import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; import org.apache.druid.catalog.storage.Actions; import org.apache.druid.catalog.storage.CatalogStorage; -import org.apache.druid.catalog.storage.HideColumns; -import org.apache.druid.catalog.storage.MoveColumn; -import org.apache.druid.catalog.storage.MoveColumn.Position; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource2.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource2.java index f69e2c130253..b48dee39ed2c 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource2.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource2.java @@ -8,11 +8,9 @@ import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; -import org.apache.druid.catalog.storage.Actions; import org.apache.druid.catalog.storage.CatalogStorage; +import org.apache.druid.common.utils.IdUtils; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthorizationUtils; @@ -99,7 +97,9 @@ public Response postTable( { try { final SchemaSpec schema = validateSchema(dbSchema, true); - validateTable(schema, name, spec, req); + validateTableName(name); + authorizeTable(schema, name, Action.WRITE, req); + validateTableSpec(schema, name, spec); final TableMetadata table = TableMetadata.newTable(TableId.of(dbSchema, name), spec); try { catalog.validate(table); @@ -201,42 +201,65 @@ public Response deleteTable( } // --------------------------------------------------------------------- - // Helper methods + // Modify a table within the catalog - private void validateTable(SchemaSpec schema, String name, TableSpec spec, final HttpServletRequest req) throws CatalogException + @POST + @Path("/edit/tables/{dbSchema}/{name}") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response editTable( + @PathParam("dbSchema") String dbSchema, + @PathParam("name") String name, + TableEditRequest editRequest, + @Context final HttpServletRequest req + ) { - // Table name can't be blank or have leading/trailing spaces - if (Strings.isNullOrEmpty(name)) { - throw CatalogException.badRequest("Table name is required"); + try { + final SchemaSpec schema = validateSchema(dbSchema, true); + authorizeTable(schema, name, Action.WRITE, req); + final long newVersion = new TableEditor(catalog, TableId.of(dbSchema, name), editRequest).go(); + return okWithVersion(newVersion); } - if (!name.equals(name.trim())) { - throw CatalogException.badRequest("Table name cannot start or end with spaces"); + catch (CatalogException e) + { + return e.toResponse(); } + } - // The user has to have permission to modify the table. - authorizeTable(schema, name, Action.WRITE, req); - // Validate the spec, if provided. - if (spec != null) { + // --------------------------------------------------------------------- + // Helper methods - // The given table spec has to be valid for the given schema. - try { - spec.validate(); - } - catch (IAE e) { - throw CatalogException.badRequest(e.getMessage()); - } + private void validateTableName(String name) throws CatalogException + { + try { + IdUtils.validateId("table", name); + } + catch (Exception e) { + throw CatalogException.badRequest(e.getMessage()); + } + if (!name.equals(name.trim())) { + throw CatalogException.badRequest("Table name cannot start or end with spaces"); + } + } - if (!schema.accepts(spec.type())) { - throw CatalogException.badRequest( - "Cannot create tables of type %s in schema %s", - spec.type(), - schema.name() - ); - } + private void validateTableSpec(SchemaSpec schema, String name, TableSpec spec) throws CatalogException + { + // The given table spec has to be valid for the given schema. + try { + spec.validate(); + } + catch (IAE e) { + throw CatalogException.badRequest(e.getMessage()); } - // Everything checks out, let the request proceed. + if (!schema.accepts(spec.type())) { + throw CatalogException.badRequest( + "Cannot create tables of type %s in schema %s", + spec.type(), + schema.name() + ); + } } private SchemaSpec validateSchema(String dbSchema, boolean forWrite) throws CatalogException diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MoveColumn.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/MoveColumn.java similarity index 58% rename from extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MoveColumn.java rename to extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/MoveColumn.java index cfe889b977f1..941ce4862f4b 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MoveColumn.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/MoveColumn.java @@ -17,23 +17,19 @@ * under the License. */ -package org.apache.druid.catalog.storage; +package org.apache.druid.catalog.http; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.catalog.model.ColumnSpec; -import org.apache.druid.java.util.common.ISE; import javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.List; import java.util.Objects; /** * JSON payload for the reorder column API. */ -public class MoveColumn +public class MoveColumn extends TableEditRequest { public enum Position { @@ -63,55 +59,6 @@ public MoveColumn( this.anchor = anchor; } - public List perform(List columns) - { - List revised = new ArrayList<>(columns); - final int colPosn = findColumn(columns, column); - if (colPosn == -1) { - throw new ISE("Column [%s] is not defined", column); - } - int anchorPosn; - if (where == Position.BEFORE || where == Position.AFTER) { - anchorPosn = findColumn(columns, anchor); - if (anchorPosn == -1) { - throw new ISE("Anchor [%s] is not defined", column); - } - if (anchorPosn > colPosn) { - anchorPosn--; - } - } else { - anchorPosn = -1; - } - - ColumnSpec col = revised.remove(colPosn); - switch (where) { - case FIRST: - revised.add(0, col); - break; - case LAST: - revised.add(col); - break; - case BEFORE: - revised.add(anchorPosn, col); - break; - case AFTER: - revised.add(anchorPosn + 1, col); - break; - } - return revised; - } - - private static int findColumn(List columns, String colName) - { - for (int i = 0; i < columns.size(); i++) { - if (columns.get(i).name().equals(colName)) { - return i; - } - } - return -1; - } - - @Override public boolean equals(Object o) { diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditRequest.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditRequest.java new file mode 100644 index 000000000000..9947a0b2fb0f --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditRequest.java @@ -0,0 +1,84 @@ +package org.apache.druid.catalog.http; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonSubTypes.Type; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.catalog.model.ColumnSpec; +import org.apache.druid.data.input.InputSource; + +import java.util.List; +import java.util.Map; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = InputSource.TYPE_PROPERTY) +@JsonSubTypes(value = { + @Type(name = "hideColumns", value = TableEditRequest.HideColumns.class), + @Type(name = "unhideColumns", value = TableEditRequest.UnhideColumns.class), + @Type(name = "dropColumns", value = TableEditRequest.DropColumns.class), + @Type(name = "updateProperties", value = TableEditRequest.UpdateProperties.class), + @Type(name = "updateColumns", value = TableEditRequest.UpdateColumns.class), + @Type(name = "moveColumn", value = MoveColumn.class), +}) +public class TableEditRequest +{ + public static class HideColumns extends TableEditRequest + { + @JsonProperty("columns") + public final List columns; + + @JsonCreator + public HideColumns( @JsonProperty("columns") List columns) + { + this.columns = columns; + } + } + + public static class UnhideColumns extends TableEditRequest + { + @JsonProperty("columns") + public final List columns; + + @JsonCreator + public UnhideColumns( @JsonProperty("columns") List columns) + { + this.columns = columns; + } + } + + public static class DropColumns extends TableEditRequest + { + @JsonProperty("columns") + public final List columns; + + @JsonCreator + public DropColumns( @JsonProperty("columns") List columns) + { + this.columns = columns; + } + } + + public static class UpdateProperties extends TableEditRequest + { + @JsonProperty("properties") + public final Map properties; + + @JsonCreator + public UpdateProperties( @JsonProperty("properties") Map properties) + { + this.properties = properties; + } + } + + public static class UpdateColumns extends TableEditRequest + { + @JsonProperty("columns") + public final List columns; + + @JsonCreator + public UpdateColumns( @JsonProperty("columns") List columns) + { + this.columns = columns; + } + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java new file mode 100644 index 000000000000..dc6bf495ac7f --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java @@ -0,0 +1,293 @@ +package org.apache.druid.catalog.http; + +import com.google.common.base.Strings; +import org.apache.druid.catalog.CatalogException; +import org.apache.druid.catalog.http.MoveColumn.Position; +import org.apache.druid.catalog.http.TableEditRequest.DropColumns; +import org.apache.druid.catalog.http.TableEditRequest.HideColumns; +import org.apache.druid.catalog.http.TableEditRequest.UnhideColumns; +import org.apache.druid.catalog.http.TableEditRequest.UpdateColumns; +import org.apache.druid.catalog.http.TableEditRequest.UpdateProperties; +import org.apache.druid.catalog.model.ColumnSpec; +import org.apache.druid.catalog.model.TableDefn; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; +import org.apache.druid.catalog.storage.CatalogStorage; +import org.apache.druid.utils.CollectionUtils; + +import javax.ws.rs.core.Response; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class TableEditor +{ + private final CatalogStorage catalog; + private final TableId id; + private final TableEditRequest editRequest; + + public TableEditor(CatalogStorage catalog, TableId id, TableEditRequest editRequest) + { + this.catalog = catalog; + this.id = id; + this.editRequest = editRequest; + } + + public long go() throws CatalogException + { + if (editRequest instanceof HideColumns) { + return hideColumns(((HideColumns) editRequest).columns); + } else if (editRequest instanceof UnhideColumns) { + return unHideColumns(((UnhideColumns) editRequest).columns); + } else if (editRequest instanceof DropColumns) { + return dropColumns(((DropColumns) editRequest).columns); + } else if (editRequest instanceof UpdateProperties) { + return updateProperties(((UpdateProperties) editRequest).properties); + } else if (editRequest instanceof UpdateColumns) { + return updateColumns(((UpdateColumns) editRequest).columns); + } else if (editRequest instanceof MoveColumn) { + return moveColumn(((MoveColumn) editRequest)); + } else { + // More of a server error: if we can deserialize the request, + // we should know how to perform that request. + throw CatalogException.badRequest( + "Unknown edit request: %s", + editRequest.getClass().getSimpleName() + ); + } + } + + private long hideColumns(List columns) throws CatalogException + { + return catalog.tables().updateProperties( + id, + table -> applyHiddenColumns(table, columns) + ); + } + + /** + * Given the existing set of properties, which may contain a list of hidden columns, + * perform the update action to add the requested new columns (if they don't yet exist). + * + * @return revised properties with the revised hidden columns list after applying + * the requested changes + */ + private TableSpec applyHiddenColumns(TableMetadata table, List columns) throws CatalogException + { + if (!AbstractDatasourceDefn.isDatasource(table.spec().type())) { + throw CatalogException.badRequest("hideColumns is supported only for data source specs"); + } + TableSpec spec = table.spec(); + if (columns.isEmpty()) { + return null; + } + Map props = spec.properties(); + @SuppressWarnings("unchecked") + List hiddenColumns = (List) props.get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); + if (hiddenColumns == null) { + hiddenColumns = Collections.emptyList(); + } + Set existing = new HashSet<>(hiddenColumns); + List revised = new ArrayList<>(hiddenColumns); + for (String col : columns) { + if (existing.contains(col)) { + revised.add(col); + } + } + return spec; + } + + private long unHideColumns(List columns) throws CatalogException + { + return catalog.tables().updateProperties( + id, + table -> applyUnhideColumns(table, columns) + ); + } + + /** + * Given the existing set of properties, which may contain a list of hidden columns, + * perform the update action remove the requested columns (if they exist). + * + * @return revised properties with the revised hidden columns list after applying + * the requested changes + */ + private TableMetadata applyUnhideColumns(TableMetadata table, List columns) throws CatalogException + { + if (!AbstractDatasourceDefn.isDatasource(type)) { + throw CatalogException.badRequest("hideColumns is supported only for data source specs"); + } + @SuppressWarnings("unchecked") + List hiddenColumns = (List) props.get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); + if (CollectionUtils.isNullOrEmpty(hiddenColumns) || columns.isEmpty()) { + return props; + } + Set removals = new HashSet<>(columns); + List revised = new ArrayList<>(); + for (String col : hiddenColumns) { + if (!removals.contains(col)) { + revised.add(col); + } + } + if (revised.isEmpty()) { + props.remove(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); + } else { + props.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, revised); + } + return props; + } + + + private long dropColumns(List columnsToDrop) throws CatalogException + { + return catalog.tables().updateColumns( + id, + (type, cols) -> { + return applyDropColumns(type, cols, columnsToDrop); + } + ); + } + + private List applyDropColumns( + final String tableType, + final List columns, + final List toDrop + ) + { + if (toDrop.isEmpty() || columns.isEmpty()) { + return columns; + } + Set drop = new HashSet(toDrop); + List revised = new ArrayList<>(); + for (ColumnSpec col : columns) { + if (!drop.contains(col.name())) { + revised.add(col); + } + } + return revised; + } + + + private long updateProperties(Map updates) throws CatalogException + { + return catalog.tables().updateProperties( + id, + (type, props) -> { + return applyUpdateProperties(type, props, updates); + } + ); + } + + private Map applyUpdateProperties(String type, Map props, Map updates) throws CatalogException + { + TableDefn defn = catalog.tableRegistry().defnFor(type); + if (defn == null) { + throw new CatalogException( + CatalogException.BAD_STATE, + Response.Status.INTERNAL_SERVER_ERROR, + "Table %s has an invalid type [%s]", + id.sqlName(), + type + ); + } + return defn.mergeProperties(props, updates); + } + + private long updateColumns(List updates) throws CatalogException + { + return catalog.tables().updateColumns( + id, + (type, cols) -> { + return applyUpdateColumns(type, cols, updates); + } + ); + } + + private List applyUpdateColumns(String type, List cols, List updates) throws CatalogException + { + TableDefn defn = catalog.tableRegistry().defnFor(type); + if (defn == null) { + throw new CatalogException( + CatalogException.BAD_STATE, + Response.Status.INTERNAL_SERVER_ERROR, + "Table %s has an invalid type [%s]", + id.sqlName(), + type + ); + } + return defn.mergeColumns(cols, updates); + } + + private long moveColumn(MoveColumn moveColumn) throws CatalogException + { + if (Strings.isNullOrEmpty(moveColumn.column)) { + throw CatalogException.badRequest("A column name is required"); + } + if (moveColumn.where == null) { + throw CatalogException.badRequest("A target location is required"); + } + if ((moveColumn.where == Position.BEFORE || moveColumn.where == Position.AFTER) && Strings.isNullOrEmpty(moveColumn.anchor)) { + throw CatalogException.badRequest("A anchor column is required for BEFORE or AFTER"); + } + return catalog.tables().updateColumns( + id, + (type, cols) -> { + return applyMoveColumn(cols, moveColumn); + } + ); + } + + public List applyMoveColumn(List columns, MoveColumn moveColumn) throws CatalogException + { + List revised = new ArrayList<>(columns); + final int colPosn = findColumn(columns, moveColumn.column); + if (colPosn == -1) { + throw CatalogException.badRequest("Column [%s] is not defined", moveColumn.column); + } + int anchorPosn; + if (moveColumn.where == Position.BEFORE || moveColumn.where == Position.AFTER) { + anchorPosn = findColumn(columns, moveColumn.anchor); + if (anchorPosn == -1) { + throw CatalogException.badRequest("Anchor [%s] is not defined", moveColumn.column); + } + if (anchorPosn > colPosn) { + anchorPosn--; + } + } else { + anchorPosn = -1; + } + + ColumnSpec col = revised.remove(colPosn); + switch (moveColumn.where) { + case FIRST: + revised.add(0, col); + break; + case LAST: + revised.add(col); + break; + case BEFORE: + revised.add(anchorPosn, col); + break; + case AFTER: + revised.add(anchorPosn + 1, col); + break; + } + return revised; + } + + private static int findColumn(List columns, String colName) + { + for (int i = 0; i < columns.size(); i++) { + if (columns.get(i).name().equals(colName)) { + return i; + } + } + return -1; + } +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/Actions.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/Actions.java deleted file mode 100644 index 7d51fedff6d1..000000000000 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/Actions.java +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.catalog.storage; - -import com.google.common.collect.ImmutableMap; - -import javax.ws.rs.core.Response; - -import java.util.Map; - -/** - * Helper functions for the catalog REST API actions. - */ -public class Actions -{ - public static final String DUPLICATE_ERROR = "Already exists"; - public static final String FAILED_ERROR = "Failed"; - public static final String INVALID = "Invalid"; - public static final String FORBIDDEN = "Forbidden"; - public static final String NOT_FOUND = "Not found"; - - public static final String ERROR_KEY = "error"; - public static final String ERR_MSG_KEY = "errorMessage"; - - public static Map error(String code, String msg) - { - return ImmutableMap.of(ERROR_KEY, code, ERR_MSG_KEY, msg); - } - - public static Response exception(Exception e) - { - return Response - .serverError() - .entity(error(FAILED_ERROR, e.getMessage())) - .build(); - } - - public static Response badRequest(String code, String msg) - { - return Response - .status(Response.Status.BAD_REQUEST) - .entity(error(code, msg)) - .build(); - } - - public static Response notFound(String msg) - { - return Response - .status(Response.Status.NOT_FOUND) - .entity(error(NOT_FOUND, msg)) - .build(); - } - - public static Response ok() - { - return Response.ok().build(); - } - - public static Response okWithVersion(long version) - { - return Response - .ok() - .entity(ImmutableMap.of("version", version)) - .build(); - - } -} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java deleted file mode 100644 index 38633c14af08..000000000000 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/HideColumns.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.catalog.storage; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; - -import javax.annotation.Nullable; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Objects; -import java.util.Set; - -/** - * Represents a request sent from a client to update just the list of hidden - * columns for a datasource table. Allows sending "delta encoded" changes: just - * the entries to add or remove. Exists as a separate operation since the - * generic merge can't handle removing items from a list. - * - * @see {@link org.apache.druid.catalog.http.CatalogResource#hideColumns(String, String, HideColumns, javax.servlet.http.HttpServletRequest)} - */ -public class HideColumns -{ - /** - * The list of columns to hide. That is, the list of columns to add to the - * table's hidden column list. - */ - @JsonProperty - public final List hide; - - /** - * The list of columns to no longer hide. That is, the list of columns to - * remove from the table's hidden column list. - */ - @JsonProperty - public final List unhide; - - @JsonCreator - public HideColumns( - @JsonProperty("hide") @Nullable final List hide, - @JsonProperty("unhide") @Nullable final List unhide - ) - { - this.hide = hide; - this.unhide = unhide; - } - - @JsonIgnore - public boolean isEmpty() - { - return (hide == null || hide.isEmpty()) - && (unhide == null || unhide.isEmpty()); - } - - /** - * Given the existing list of hidden columns, perform the update action to add the - * requested new columns (if they don't yet exist) and remove the requested columns - * (if they do exist). If someone is silly enough to include the same column in - * both lists, the remove action takes precedence. - * - * @param hiddenColumns exiting hidden columns list - * @return revised hidden columns list after applying the requested changes - */ - public List perform(List hiddenColumns) - { - if (hiddenColumns == null) { - hiddenColumns = Collections.emptyList(); - } - Set existing = new HashSet<>(hiddenColumns); - if (unhide != null) { - for (String col : unhide) { - existing.remove(col); - } - } - List revised = new ArrayList<>(); - for (String col : hiddenColumns) { - if (existing.contains(col)) { - revised.add(col); - } - } - if (hide != null) { - for (String col : hide) { - if (!existing.contains(col)) { - revised.add(col); - } - } - } - return revised.isEmpty() ? null : revised; - } - - @Override - public boolean equals(Object o) - { - if (o == null || o.getClass() != getClass()) { - return false; - } - HideColumns other = (HideColumns) o; - return Objects.equals(this.hide, other.hide) - && Objects.equals(this.unhide, other.unhide); - } - - @Override - public int hashCode() - { - return Objects.hash(hide, unhide); - } -} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java index 53814ccb7cbf..bee0148575bc 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java @@ -19,13 +19,13 @@ package org.apache.druid.catalog.storage.sql; +import org.apache.druid.catalog.CatalogException; import org.apache.druid.catalog.CatalogException.DuplicateKeyException; import org.apache.druid.catalog.CatalogException.NotFoundException; import org.apache.druid.catalog.model.ColumnSpec; import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; - -import javax.annotation.Nullable; +import org.apache.druid.catalog.model.TableSpec; import java.util.List; import java.util.Map; @@ -49,20 +49,12 @@ public interface CatalogManager */ interface Listener { - /** - * A new catalog table entry was added. - */ - void added(TableMetadata table); - - /** - * An existing catalog table entry was updated. - */ - void updated(TableMetadata table); - - /** - * An existing catalog table entry was deleted. - */ - void deleted(TableId id); + void delta(UpdateEvent event); + } + + public interface TableTransform + { + TableSpec apply(TableMetadata spec) throws CatalogException; } /** @@ -120,8 +112,8 @@ interface Listener * @param transform the transform to apply to the table properties * @return the update timestamp (version) of the updated record */ - long updateProperties(TableId id, Function, Map> transform) throws NotFoundException; - long updateColumns(TableId id, Function, List> transform) throws NotFoundException; + long updateProperties(TableId id, TableTransform transform) throws CatalogException; + long updateColumns(TableId id, TableTransform transform) throws CatalogException; /** * Move the table to the deleting state. No version check: fine diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java index db735f942d9f..c61d97216a5f 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.inject.Inject; +import org.apache.druid.catalog.CatalogException; import org.apache.druid.catalog.CatalogException.DuplicateKeyException; import org.apache.druid.catalog.CatalogException.NotFoundException; import org.apache.druid.catalog.model.ColumnSpec; @@ -33,6 +34,7 @@ import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; import org.apache.druid.catalog.storage.MetadataStorageManager; +import org.apache.druid.catalog.storage.sql.UpdateEvent.EventType; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -47,14 +49,11 @@ import org.skife.jdbi.v2.exceptions.UnableToExecuteStatementException; import org.skife.jdbi.v2.tweak.HandleCallback; -import javax.annotation.Nullable; - import java.io.IOException; import java.util.Deque; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentLinkedDeque; -import java.util.function.Function; @ManageLifecycle public class SQLCatalogManager implements CatalogManager @@ -237,11 +236,11 @@ public TableMetadata withHandle(Handle handle) throws NotFoundException public long replace(TableMetadata table) throws NotFoundException { try { - return dbi.withHandle( - new HandleCallback() + final TableMetadata revised = dbi.withHandle( + new HandleCallback() { @Override - public Long withHandle(Handle handle) throws NotFoundException + public TableMetadata withHandle(Handle handle) throws NotFoundException { final TableId id = table.id(); final TableSpec spec = table.spec(); @@ -258,11 +257,12 @@ public Long withHandle(Handle handle) throws NotFoundException if (updateCount == 0) { throw tableNotFound(id); } - sendUpdate(id); - return updateTime; + return table.fromInsert(updateTime); } } ); + sendUpdate(EventType.UPDATE, revised); + return table.updateTime(); } catch (CallbackFailedException e) { if (e.getCause() instanceof NotFoundException) { @@ -280,11 +280,11 @@ public Long withHandle(Handle handle) throws NotFoundException public long update(TableMetadata table, long oldVersion) throws NotFoundException { try { - return dbi.withHandle( - new HandleCallback() + final TableMetadata revised = dbi.withHandle( + new HandleCallback() { @Override - public Long withHandle(Handle handle) throws NotFoundException + public TableMetadata withHandle(Handle handle) throws NotFoundException { final TableId id = table.id(); final TableSpec spec = table.spec(); @@ -305,11 +305,12 @@ public Long withHandle(Handle handle) throws NotFoundException id.sqlName() ); } - sendUpdate(id); - return updateTime; + return table.asUpdate(updateTime); } } ); + sendUpdate(EventType.UPDATE, revised); + return table.updateTime(); } catch (CallbackFailedException e) { if (e.getCause() instanceof NotFoundException) { @@ -320,7 +321,7 @@ public Long withHandle(Handle handle) throws NotFoundException } private static final String SELECT_TABLE_PROPERTIES_STMT = - "SELECT properties\n" + + "SELECT tableType, properties\n" + "FROM %s\n" + "WHERE schemaName = :schemaName\n" + " AND name = :name\n" + @@ -335,15 +336,15 @@ public Long withHandle(Handle handle) throws NotFoundException @Override public long updateProperties( final TableId id, - final Function, Map> transform - ) throws NotFoundException + final TableTransform transform + ) throws CatalogException { try { - return dbi.withHandle( - new HandleCallback() + final TableMetadata result = dbi.withHandle( + new HandleCallback() { @Override - public Long withHandle(Handle handle) throws NotFoundException + public TableMetadata withHandle(Handle handle) throws CatalogException { handle.begin(); try { @@ -353,17 +354,28 @@ public Long withHandle(Handle handle) throws NotFoundException .bind("schemaName", id.schema()) .bind("name", id.name()); - final ResultIterator> resultIterator = query - .map((index, r, ctx) -> propertiesFromBytes(jsonMapper, r.getBytes(1))) + final ResultIterator resultIterator = query + .map((index, r, ctx) -> + tableSpecFromBytes( + jsonMapper, + r.getString(1), + r.getBytes(2), + null + ) + ) .iterator(); - final Map properites; + final TableSpec tableSpec; if (resultIterator.hasNext()) { - properites = resultIterator.next(); + tableSpec = resultIterator.next(); } else { handle.rollback(); throw tableNotFound(id); } - final Map revised = transform.apply(properites); + final TableSpec revised = transform.apply(TableMetadata.of(id, tableSpec)); + if (revised == null) { + handle.rollback(); + return null; + } final long updateTime = System.currentTimeMillis(); final int updateCount = handle .createStatement(statement(UPDATE_TABLE_PROPERTIES_STMT)) @@ -377,8 +389,7 @@ public Long withHandle(Handle handle) throws NotFoundException throw new ISE("Table %s: not found", id.sqlName()); } handle.commit(); - sendUpdate(id); - return updateTime; + return TableMetadata.forUpdate(id, updateTime, revised); } catch (RuntimeException e) { handle.rollback(); @@ -387,6 +398,11 @@ public Long withHandle(Handle handle) throws NotFoundException } } ); + if (result == null) { + return 0; + } + sendUpdate(EventType.PROPERTY_UPDATE, result); + return result.updateTime(); } catch (CallbackFailedException e) { if (e.getCause() instanceof NotFoundException) { @@ -397,7 +413,7 @@ public Long withHandle(Handle handle) throws NotFoundException } private static final String SELECT_COLUMNS_STMT = - "SELECT columns\n" + + "SELECT tableType, columns\n" + "FROM %s\n" + "WHERE schemaName = :schemaName\n" + " AND name = :name\n" + @@ -412,15 +428,15 @@ public Long withHandle(Handle handle) throws NotFoundException @Override public long updateColumns( final TableId id, - final Function, List> transform + final TableTransform transform ) throws NotFoundException { try { - return dbi.withHandle( - new HandleCallback() + final TableMetadata result = dbi.withHandle( + new HandleCallback() { @Override - public Long withHandle(Handle handle) throws NotFoundException + public TableMetadata withHandle(Handle handle) throws CatalogException { handle.begin(); try { @@ -430,17 +446,28 @@ public Long withHandle(Handle handle) throws NotFoundException .bind("schemaName", id.schema()) .bind("name", id.name()); - final ResultIterator> resultIterator = query - .map((index, r, ctx) -> columnsFromBytes(jsonMapper, r.getBytes(1))) + final ResultIterator resultIterator = query + .map((index, r, ctx) -> + tableSpecFromBytes( + jsonMapper, + r.getString(1), + null, + r.getBytes(2) + ) + ) .iterator(); - final List columns; + final TableSpec tableSpec; if (resultIterator.hasNext()) { - columns = resultIterator.next(); + tableSpec = resultIterator.next(); } else { handle.rollback(); throw tableNotFound(id); } - final List revised = transform.apply(columns); + final TableSpec revised = transform.apply(TableMetadata.of(id, tableSpec)); + if (revised == null) { + handle.rollback(); + return null; + } final long updateTime = System.currentTimeMillis(); final int updateCount = handle .createStatement(statement(UPDATE_COLUMNS_STMT)) @@ -454,8 +481,7 @@ public Long withHandle(Handle handle) throws NotFoundException throw new ISE("Table %s: not found", id.sqlName()); } handle.commit(); - sendUpdate(id); - return updateTime; + return TableMetadata.forUpdate(id, updateTime, revised); } catch (RuntimeException e) { handle.rollback(); @@ -464,6 +490,11 @@ public Long withHandle(Handle handle) throws NotFoundException } } ); + if (result == null) { + return 0; + } + sendUpdate(EventType.COLUMNS_UPDATE, result); + return result.updateTime(); } catch (CallbackFailedException e) { if (e.getCause() instanceof NotFoundException) { @@ -645,27 +676,26 @@ protected synchronized void sendAddition(TableMetadata table, long updateTime) if (listeners.isEmpty()) { return; } - TableMetadata newTable = table.fromInsert(updateTime); - for (Listener listener : listeners) { - listener.added(newTable); - } + sendEvent(new UpdateEvent(EventType.CREATE, table.fromInsert(updateTime))); } - protected synchronized void sendUpdate(TableId id) + protected synchronized void sendUpdate(EventType eventType, TableMetadata table) { if (listeners.isEmpty()) { return; } - TableMetadata updatedTable = read(id); - for (Listener listener : listeners) { - listener.updated(updatedTable); - } + sendEvent(new UpdateEvent(eventType, table)); + } + + protected void sendDeletion(TableId id) + { + sendEvent(new UpdateEvent(EventType.DELETE, id)); } - protected synchronized void sendDeletion(TableId id) + protected synchronized void sendEvent(UpdateEvent event) { for (Listener listener : listeners) { - listener.deleted(id); + listener.delta(event); } } @@ -730,8 +760,8 @@ private static TableSpec tableSpecFromBytes( { return new TableSpec( type, - propertiesFromBytes(jsonMapper, properties), - columnsFromBytes(jsonMapper, columns) + properties == null ? null : propertiesFromBytes(jsonMapper, properties), + columns == null ? null : columnsFromBytes(jsonMapper, columns) ); } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/UpdateEvent.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/UpdateEvent.java new file mode 100644 index 000000000000..da9379b15fdc --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/UpdateEvent.java @@ -0,0 +1,58 @@ +package org.apache.druid.catalog.storage.sql; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.model.TableSpec; + +public class UpdateEvent +{ + public enum EventType + { + CREATE, + UPDATE, + PROPERTY_UPDATE, + COLUMNS_UPDATE, + DELETE + } + + @JsonProperty("type") + public final EventType type; + @JsonProperty("id") + public final TableId id; + @JsonProperty("id") + public final long updateTime; + @JsonProperty("spec") + public final TableSpec spec; + + @JsonCreator + public UpdateEvent( + @JsonProperty("type") final EventType type, + @JsonProperty("id") final TableId id, + @JsonProperty("updateTime") final long updateTime, + @JsonProperty("spec") final TableSpec spec + ) + { + this.type = type; + this.id = id; + this.updateTime = updateTime; + this.spec = spec; + } + + public UpdateEvent( + final EventType type, + final TableMetadata table + ) + { + this(type, table.id(), table.updateTime(), table.spec()); + } + + public UpdateEvent( + final EventType type, + final TableId id + ) + { + this(type, id, 0, null); + } +} diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java index ea0fc1b482d9..a62fd94dfcaf 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java @@ -20,6 +20,8 @@ package org.apache.druid.server.http.catalog; import org.apache.druid.catalog.http.CatalogResource; +import org.apache.druid.catalog.http.HideColumns; +import org.apache.druid.catalog.http.MoveColumn; import org.apache.druid.catalog.model.CatalogUtils; import org.apache.druid.catalog.model.Columns; import org.apache.druid.catalog.model.TableId; @@ -30,8 +32,6 @@ import org.apache.druid.catalog.model.table.InputFormats; import org.apache.druid.catalog.model.table.TableBuilder; import org.apache.druid.catalog.storage.CatalogTests; -import org.apache.druid.catalog.storage.HideColumns; -import org.apache.druid.catalog.storage.MoveColumn; import org.apache.druid.metadata.TestDerbyConnector; import org.junit.After; import org.junit.Before; diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java index 4bdb75af9874..cf4054d5ff74 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java @@ -20,12 +20,12 @@ package org.apache.druid.server.http.catalog; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.catalog.http.HideColumns; +import org.apache.druid.catalog.http.MoveColumn; import org.apache.druid.catalog.model.CatalogUtils; import org.apache.druid.catalog.model.ColumnSpec; import org.apache.druid.catalog.model.TableSpec; import org.apache.druid.catalog.model.table.TableBuilder; -import org.apache.druid.catalog.storage.HideColumns; -import org.apache.druid.catalog.storage.MoveColumn; import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.junit.Test; diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java index 44f19acbac00..7c4527531e2a 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java @@ -20,14 +20,14 @@ package org.apache.druid.testsEx.catalog; import com.google.inject.Inject; +import org.apache.druid.catalog.http.HideColumns; +import org.apache.druid.catalog.http.MoveColumn; import org.apache.druid.catalog.model.CatalogUtils; import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; import org.apache.druid.catalog.model.table.TableBuilder; -import org.apache.druid.catalog.storage.HideColumns; -import org.apache.druid.catalog.storage.MoveColumn; import org.apache.druid.testsEx.categories.Catalog; import org.apache.druid.testsEx.cluster.CatalogClient; import org.apache.druid.testsEx.cluster.DruidClusterClient; diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/CatalogClient.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/CatalogClient.java index b4f64577dc05..4788f419beea 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/CatalogClient.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/CatalogClient.java @@ -23,11 +23,11 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import org.apache.druid.catalog.http.CatalogResource; +import org.apache.druid.catalog.http.HideColumns; +import org.apache.druid.catalog.http.MoveColumn; import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; -import org.apache.druid.catalog.storage.HideColumns; -import org.apache.druid.catalog.storage.MoveColumn; import org.apache.druid.java.util.common.StringUtils; import org.jboss.netty.handler.codec.http.HttpMethod; diff --git a/server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java b/server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java index 1058a3bcde99..895ea64c88c4 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java @@ -97,7 +97,7 @@ public PropertyDefn property(String key) * This method does not validate the properties, except as needed to do a * merge. A separate validation step is done on the final, merged object. */ - protected Map mergeProperties( + public Map mergeProperties( final Map source, final Map update ) diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java index 1ab390deda8b..6cf2e3f013af 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java @@ -139,7 +139,7 @@ public TableSpec merge(TableSpec spec, TableSpec update, ObjectMapper jsonMapper * the column does not exist, then the new column is appended to the existing * list. This merge operation cannot remove columns or change order. */ - private List mergeColumns(List columns, List update) + public List mergeColumns(List columns, List update) { if (update == null || update.isEmpty()) { return columns; diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java b/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java index 3a29911e2228..532df200eb8a 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java @@ -62,6 +62,11 @@ public TableDefnRegistry( this(TABLE_DEFNS, jsonMapper); } + public TableDefn defnFor(String type) + { + return defns.get(type); + } + public ResolvedTable resolve(TableSpec spec) { String type = spec.type(); diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java index af8d54b364c3..9d84eff2f1d3 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java @@ -20,12 +20,10 @@ package org.apache.druid.catalog.model; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Strings; import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.jackson.JacksonUtils; import java.util.Objects; @@ -89,7 +87,7 @@ public TableMetadata( public static TableMetadata newTable( TableId id, - TableSpec defn + TableSpec spec ) { return new TableMetadata( @@ -97,7 +95,7 @@ public static TableMetadata newTable( 0, 0, TableState.ACTIVE, - defn + spec ); } @@ -111,6 +109,31 @@ public static TableMetadata newSegmentTable( defn); } + public static TableMetadata of( + TableId id, + TableSpec spec + ) + { + return new TableMetadata( + id, + 0, + 0, + TableState.ACTIVE, + spec + ); + } + + public static TableMetadata forUpdate(TableId id, long updateTime, TableSpec spec) + { + return new TableMetadata( + id, + 0, + updateTime, + TableState.ACTIVE, + spec + ); + } + public TableMetadata fromInsert(long updateTime) { return new TableMetadata( From cd813c3891aab00e33223a78dc2c01094e5f189a Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Fri, 28 Oct 2022 13:04:39 -0700 Subject: [PATCH 11/22] Snapshot --- .../druid/catalog/storage/CatalogStorage.java | 43 ++--- .../catalog/storage/sql/CatalogManager.java | 17 +- .../storage/sql/SQLCatalogManager.java | 14 +- .../catalog/storage/sql/UpdateEvent.java | 58 ------- .../catalog/sync/CachedMetadataCatalog.java | 164 ++++++++++++++---- .../catalog/sync/CatalogUpdateListener.java | 12 ++ .../catalog/sync/CatalogUpdateNotifier.java | 31 ++-- .../druid/catalog/sync/MetadataCatalog.java | 14 +- .../druid/catalog/sync/UpdateEvent.java | 32 ++++ .../druid/catalog/model/ColumnSpec.java | 2 - .../druid/catalog/model/TableMetadata.java | 33 ++++ .../apache/druid/catalog/model/TableSpec.java | 2 - 12 files changed, 245 insertions(+), 177 deletions(-) delete mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/UpdateEvent.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateListener.java create mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/UpdateEvent.java diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java index aaff76d875c0..abc99d1f28cd 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java @@ -20,6 +20,7 @@ package org.apache.druid.catalog.storage; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.catalog.CatalogException.NotFoundException; import org.apache.druid.catalog.model.ResolvedTable; import org.apache.druid.catalog.model.SchemaRegistry; import org.apache.druid.catalog.model.SchemaRegistry.SchemaSpec; @@ -28,7 +29,7 @@ import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.storage.sql.CatalogManager; -import org.apache.druid.catalog.sync.MetadataCatalog.CatalogListener; +import org.apache.druid.catalog.sync.CatalogUpdateListener; import org.apache.druid.catalog.sync.MetadataCatalog.CatalogSource; import org.apache.druid.catalog.sync.MetadataCatalog.CatalogUpdateProvider; import org.apache.druid.guice.annotations.Json; @@ -52,34 +53,6 @@ */ public class CatalogStorage implements CatalogUpdateProvider, CatalogSource { - public static class ListenerAdapter implements CatalogManager.Listener - { - private final CatalogListener dest; - - public ListenerAdapter(CatalogListener dest) - { - this.dest = dest; - } - - @Override - public void added(TableMetadata table) - { - dest.updated(table); - } - - @Override - public void updated(TableMetadata table) - { - dest.updated(table); - } - - @Override - public void deleted(TableId id) - { - dest.deleted(id); - } - } - protected final SchemaRegistry schemaRegistry; protected final TableDefnRegistry tableRegistry; protected final CatalogManager catalogMgr; @@ -111,9 +84,9 @@ public SchemaSpec resolveSchema(String dbSchema) } @Override - public void register(CatalogListener listener) + public void register(CatalogUpdateListener listener) { - tables().register(new ListenerAdapter(listener)); + tables().register(listener); } @Override @@ -123,9 +96,13 @@ public List tablesForSchema(String dbSchema) } @Override - public TableMetadata table(TableId id) + public @Nullable TableMetadata table(TableId id) { - return tables().read(id); + try { + return tables().read(id); + } catch (NotFoundException e) { + return null; + } } public void validate(TableMetadata table) diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java index bee0148575bc..a723e199e454 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java @@ -22,14 +22,12 @@ import org.apache.druid.catalog.CatalogException; import org.apache.druid.catalog.CatalogException.DuplicateKeyException; import org.apache.druid.catalog.CatalogException.NotFoundException; -import org.apache.druid.catalog.model.ColumnSpec; import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.sync.CatalogUpdateListener; import java.util.List; -import java.util.Map; -import java.util.function.Function; /** * Manages catalog data. Used in Coordinator, which will be in either @@ -41,17 +39,6 @@ */ public interface CatalogManager { - /** - * Generic interface for changes to the catalog at the storage level. - * Implemented by the catalog sync mechanism to send update events - * to the Broker. Note that these events are about the catalog, - * not about the physical storage of tables (i.e. datasources.) - */ - interface Listener - { - void delta(UpdateEvent event); - } - public interface TableTransform { TableSpec apply(TableMetadata spec) throws CatalogException; @@ -66,7 +53,7 @@ public interface TableTransform /** * Register a listener for catalog events. */ - void register(Listener listener); + void register(CatalogUpdateListener listener); /** * Create a table entry. diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java index c61d97216a5f..c807503cb06d 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java @@ -34,7 +34,9 @@ import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; import org.apache.druid.catalog.storage.MetadataStorageManager; -import org.apache.druid.catalog.storage.sql.UpdateEvent.EventType; +import org.apache.druid.catalog.sync.CatalogUpdateListener; +import org.apache.druid.catalog.sync.UpdateEvent; +import org.apache.druid.catalog.sync.UpdateEvent.EventType; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -65,7 +67,7 @@ public class SQLCatalogManager implements CatalogManager private final ObjectMapper jsonMapper; private final IDBI dbi; private final String tableName; - private final Deque listeners = new ConcurrentLinkedDeque<>(); + private final Deque listeners = new ConcurrentLinkedDeque<>(); @Inject public SQLCatalogManager(MetadataStorageManager metastoreManager) @@ -666,7 +668,7 @@ public List withHandle(Handle handle) } @Override - public synchronized void register(Listener listener) + public synchronized void register(CatalogUpdateListener listener) { listeners.add(listener); } @@ -689,13 +691,13 @@ protected synchronized void sendUpdate(EventType eventType, TableMetadata table) protected void sendDeletion(TableId id) { - sendEvent(new UpdateEvent(EventType.DELETE, id)); + sendEvent(new UpdateEvent(EventType.DELETE, TableMetadata.empty(id))); } protected synchronized void sendEvent(UpdateEvent event) { - for (Listener listener : listeners) { - listener.delta(event); + for (CatalogUpdateListener listener : listeners) { + listener.updated(event); } } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/UpdateEvent.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/UpdateEvent.java deleted file mode 100644 index da9379b15fdc..000000000000 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/UpdateEvent.java +++ /dev/null @@ -1,58 +0,0 @@ -package org.apache.druid.catalog.storage.sql; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.catalog.model.TableId; -import org.apache.druid.catalog.model.TableMetadata; -import org.apache.druid.catalog.model.TableSpec; - -public class UpdateEvent -{ - public enum EventType - { - CREATE, - UPDATE, - PROPERTY_UPDATE, - COLUMNS_UPDATE, - DELETE - } - - @JsonProperty("type") - public final EventType type; - @JsonProperty("id") - public final TableId id; - @JsonProperty("id") - public final long updateTime; - @JsonProperty("spec") - public final TableSpec spec; - - @JsonCreator - public UpdateEvent( - @JsonProperty("type") final EventType type, - @JsonProperty("id") final TableId id, - @JsonProperty("updateTime") final long updateTime, - @JsonProperty("spec") final TableSpec spec - ) - { - this.type = type; - this.id = id; - this.updateTime = updateTime; - this.spec = spec; - } - - public UpdateEvent( - final EventType type, - final TableMetadata table - ) - { - this(type, table.id(), table.updateTime(), table.spec()); - } - - public UpdateEvent( - final EventType type, - final TableId id - ) - { - this(type, id, 0, null); - } -} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java index 980e1bc5f309..a0afff343e2c 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java @@ -26,8 +26,8 @@ import org.apache.druid.catalog.model.TableDefnRegistry; import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; -import org.apache.druid.catalog.sync.MetadataCatalog.CatalogListener; import org.apache.druid.guice.annotations.Json; +import org.apache.druid.java.util.common.logger.Logger; import javax.inject.Inject; @@ -58,24 +58,26 @@ * populates the cache with updates. For a local cache, the DB layer * provides the updates. For a remote cache, the DB host pushes updates. */ -public class CachedMetadataCatalog implements MetadataCatalog, CatalogListener +public class CachedMetadataCatalog implements MetadataCatalog, CatalogUpdateListener { + private static final Logger LOG = new Logger(CachedMetadataCatalog.class); + public static final int NOT_FETCHED = -1; public static final int UNDEFINED = 0; + /** + * Cache entry. Normally wraps a catalog table entry. Can also wrap a null + * entry which says that we tried to resolve the table, but there is no such + * entry, and there is no need to check again. + */ private static class TableEntry { private final TableMetadata table; - protected TableEntry(SchemaSpec schema, TableMetadata table) + protected TableEntry(TableMetadata table) { this.table = table; } - - protected long version() - { - return table == null ? UNDEFINED : table.updateTime(); - } } private class SchemaEntry @@ -93,7 +95,7 @@ protected TableMetadata resolveTable(TableId tableId) { TableEntry entry = cache.computeIfAbsent( tableId.name(), - key -> new TableEntry(schema, base.table(tableId)) + key -> new TableEntry(base.table(tableId)) ); return entry.table; } @@ -106,7 +108,7 @@ public synchronized List tables() if (version == NOT_FETCHED) { List catalogTables = base.tablesForSchema(schema.name()); for (TableMetadata table : catalogTables) { - update(table); + cache.put(table.id().name(), new TableEntry(table)); } } List orderedTables = new ArrayList<>(); @@ -121,20 +123,129 @@ public synchronized List tables() return orderedTables; } - public synchronized void update(TableMetadata table) + public synchronized void update(UpdateEvent event) { - cache.compute( - table.id().name(), - (k, v) -> v == null || v.version() < table.updateTime() - ? new TableEntry(schema, table) - : v - ); + TableMetadata table = event.table; + final String name = table.id().name(); + switch (event.type) { + case CREATE: + cache.compute( + name, + (k, v) -> computeCreate(v, table) + ); + break; + case UPDATE: + cache.compute( + name, + (k, v) -> computeUpdate(v, table) + ); + break; + case DELETE: + cache.remove(name); + break; + case COLUMNS_UPDATE: + cache.compute( + name, + (k, v) -> computeColumnsUpdate(v, table) + ); + break; + case PROPERTY_UPDATE: + cache.compute( + name, + (k, v) -> computePropertiesUpdate(v, table) + ); + break; + default: + // Don't know what to do + return; + } version = Math.max(version, table.updateTime()); } - public void remove(String name) + protected TableEntry computeCreate(TableEntry entry, TableMetadata update) + { + if (entry != null && entry.table != null) { + LOG.warn("Received creation event for existing entry: %s", update.id().sqlName()); + return computeUpdate(entry, update); + } + return new TableEntry(update); + } + + private TableEntry computeUpdate(TableEntry entry, TableMetadata update) + { + if (!checkExists(entry, update)) { + return new TableEntry(update); + } + if (!checkVersion(entry, update)) { + return entry; + } + return new TableEntry(update) ; + } + + private boolean checkExists(TableEntry entry, TableMetadata update) + { + if (entry == null || entry.table == null) { + LOG.error("Reveived update for missing cache entry: %s", update.id().sqlName()); + // TODO: force resync + return false; + } + return true; + } + + private TableEntry computeColumnsUpdate(TableEntry entry, TableMetadata update) + { + if (!checkExists(entry, update)) { + return new TableEntry(null); + } + if (!checkResolved(entry, update, "columns")) { + return entry; + } + if (!checkVersion(entry, update)) { + return entry; + } + return new TableEntry(entry.table.withColumns(update)); + } + + private TableEntry computePropertiesUpdate(TableEntry entry, TableMetadata update) + { + if (!checkExists(entry, update)) { + return new TableEntry(null); + } + if (!checkResolved(entry, update, "properties")) { + return entry; + } + if (!checkVersion(entry, update)) { + return entry; + } + return new TableEntry(entry.table.withProperties(update)); + } + + private boolean checkResolved(TableEntry entry, TableMetadata update, String action) { - cache.remove(name); + if (entry.table == null) { + LOG.error("Received %s update for unresolved table: %s", + action, + update.id().sqlName() + ); + // TODO: force resync + return false; + } + return true; + } + + private boolean checkVersion(TableEntry entry, TableMetadata update) + { + if (entry.table.updateTime() > update.updateTime()) { + LOG.warn( + "Received out-of-order update for table: %s. Cache: %d, update:%d", + update.id().sqlName(), + entry.table.updateTime(), + update.updateTime() + ); + // TODO: force resync + return false; + } + return true; } public Set tableNames() @@ -188,20 +299,11 @@ public List tables(String schemaName) } @Override - public void updated(TableMetadata table) + public void updated(UpdateEvent event) { - SchemaEntry schemaEntry = entryFor(table.id().schema()); - if (schemaEntry != null) { - schemaEntry.update(table); - } - } - - @Override - public void deleted(TableId tableId) - { - SchemaEntry schemaEntry = entryFor(tableId.schema()); + SchemaEntry schemaEntry = entryFor(event.table.id().schema()); if (schemaEntry != null) { - schemaEntry.remove(tableId.name()); + schemaEntry.update(event); } } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateListener.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateListener.java new file mode 100644 index 000000000000..6db90fb0ed01 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateListener.java @@ -0,0 +1,12 @@ +package org.apache.druid.catalog.sync; + +/** + * Generic interface for changes to the catalog at the storage level. + * Implemented by the catalog sync mechanism to send update events + * to the Broker. Note that these events are about the catalog, + * not about the physical storage of tables (i.e. datasources.) + */ +public interface CatalogUpdateListener +{ + void updated(UpdateEvent event); +} \ No newline at end of file diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateNotifier.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateNotifier.java index 55b2e848df4b..7b39033036ab 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateNotifier.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateNotifier.java @@ -19,19 +19,17 @@ package org.apache.druid.catalog.sync; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.catalog.http.CatalogListenerResource; -import org.apache.druid.catalog.model.TableId; -import org.apache.druid.catalog.model.TableMetadata; -import org.apache.druid.catalog.model.TableSpec; import org.apache.druid.catalog.storage.CatalogStorage; -import org.apache.druid.catalog.sync.MetadataCatalog.CatalogListener; import org.apache.druid.catalog.sync.RestUpdateSender.RestSender; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.annotations.EscalatedClient; import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.http.client.HttpClient; @@ -53,18 +51,11 @@ * a wrapper class to handle deletes. */ @ManageLifecycle -public class CatalogUpdateNotifier implements CatalogListener +public class CatalogUpdateNotifier implements CatalogUpdateListener { private static final String CALLER_NAME = "Catalog Sync"; private static final long TIMEOUT_MS = 5000; - /** - * Internal table type used in updates to notify listeners that a table has - * been deleted. Avoids the need for a special "table deleted" message. - */ - public static final String TOMBSTONE_TABLE_TYPE = "tombstone"; - private static final TableSpec TABLE_TOMBSTONE = new TableSpec(TOMBSTONE_TABLE_TYPE, null, null); - private final CacheNotifier notifier; private final ObjectMapper smileMapper; @@ -107,15 +98,13 @@ public void stop() } @Override - public void updated(TableMetadata update) - { - notifier.send(update.toBytes(smileMapper)); - } - - @Override - public void deleted(TableId tableId) + public void updated(UpdateEvent event) { - TableMetadata spec = TableMetadata.newTable(tableId, TABLE_TOMBSTONE); - notifier.send(spec.toBytes(smileMapper)); + try { + notifier.send(smileMapper.writeValueAsBytes(event)); + } + catch (JsonProcessingException e) { + throw new ISE("Failed to serialize " + event.getClass().getSimpleName()); + } } } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/MetadataCatalog.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/MetadataCatalog.java index ed3861cce4e8..89e398e56df6 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/MetadataCatalog.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/MetadataCatalog.java @@ -23,6 +23,8 @@ import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; +import javax.annotation.Nullable; + import java.util.List; import java.util.Set; @@ -44,15 +46,9 @@ interface CatalogSource ResolvedTable resolveTable(TableId id); } - interface CatalogListener - { - void updated(TableMetadata update); - void deleted(TableId tableId); - } - interface CatalogUpdateProvider { - void register(CatalogListener listener); + void register(CatalogUpdateListener listener); } /** @@ -64,8 +60,8 @@ interface CatalogUpdateProvider * metadata. Views and input sources exist only if their * metadata exists. System tables never have metadata. */ - TableMetadata getTable(TableId tableId); - ResolvedTable resolveTable(TableId tableId); + @Nullable TableMetadata getTable(TableId tableId); + @Nullable ResolvedTable resolveTable(TableId tableId); /** * List of tables defined within the given schema. Does not filter the diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/UpdateEvent.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/UpdateEvent.java new file mode 100644 index 000000000000..946e549f27ca --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/UpdateEvent.java @@ -0,0 +1,32 @@ +package org.apache.druid.catalog.sync; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.catalog.model.TableMetadata; + +public class UpdateEvent +{ + public enum EventType + { + CREATE, + UPDATE, + PROPERTY_UPDATE, + COLUMNS_UPDATE, + DELETE + } + + @JsonProperty("type") + public final EventType type; + @JsonProperty("table") + public final TableMetadata table; + + @JsonCreator + public UpdateEvent( + @JsonProperty("type") final EventType type, + @JsonProperty("table") final TableMetadata table + ) + { + this.type = type; + this.table = table; + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java b/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java index e8b6b24442e9..119ee182140c 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java +++ b/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java @@ -23,11 +23,9 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonInclude.Include; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Strings; import org.apache.druid.guice.annotations.UnstableApi; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.jackson.JacksonUtils; import javax.annotation.Nullable; diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java index 9d84eff2f1d3..b2b6e6c962cd 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java @@ -134,6 +134,17 @@ public static TableMetadata forUpdate(TableId id, long updateTime, TableSpec spe ); } + public static TableMetadata empty(TableId id) + { + return new TableMetadata( + id, + 0, + 0, + null, + null + ); + } + public TableMetadata fromInsert(long updateTime) { return new TableMetadata( @@ -166,6 +177,28 @@ public TableMetadata withSpec(TableSpec spec) ); } + public TableMetadata withColumns(TableMetadata update) + { + return new TableMetadata( + id, + creationTime, + update.updateTime(), + state, + spec.withColumns(update.spec().columns()) + ); + } + + public TableMetadata withProperties(TableMetadata update) + { + return new TableMetadata( + id, + creationTime, + update.updateTime(), + state, + spec.withProperties(update.spec().properties()) + ); + } + @JsonProperty("id") public TableId id() { diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java b/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java index 6d89bf1c3c23..1785a136b1ac 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java @@ -23,10 +23,8 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonInclude.Include; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Strings; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.jackson.JacksonUtils; import java.util.Collections; import java.util.HashMap; From 17c996db59d3bd49d7cf14675f1b9a692db357fc Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Fri, 28 Oct 2022 16:46:22 -0700 Subject: [PATCH 12/22] Updated sync events --- .../util/common/jackson/JacksonUtils.java | 17 ++- .../catalog/http/CatalogListenerResource.java | 54 +------- .../druid/catalog/http/TableEditor.java | 115 +++++++++-------- .../storage/sql/SQLCatalogManager.java | 72 +++++------ .../catalog/sync/CatalogUpdateNotifier.java | 10 +- .../catalog/storage/TableManagerTest.java | 116 ++++++++++++++++-- .../catalog/sync/CatalogMetadataTest.java | 25 ++-- .../druid/catalog/sync/MockCatalogSync.java | 56 +-------- 8 files changed, 237 insertions(+), 228 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java b/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java index 16528f5edb42..d51c042e91d2 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java +++ b/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java @@ -20,14 +20,15 @@ package org.apache.druid.java.util.common.jackson; import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.JsonSerializer; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializerProvider; +import org.apache.druid.java.util.common.ISE; import javax.annotation.Nullable; - import java.io.IOException; import java.util.Map; @@ -93,4 +94,18 @@ public static void writeObjectUsingSerializerProvider( serializer.serialize(o, jsonGenerator, serializers); } } + + /** + * Convert the given object to an array of bytes. Use when the object is + * known serializable so that the Jackson exception can be suppressed. + */ + public static byte[] toBytes(ObjectMapper jsonMapper, Object obj) + { + try { + return jsonMapper.writeValueAsBytes(obj); + } + catch (JsonProcessingException e) { + throw new ISE("Failed to serialize " + obj.getClass().getSimpleName()); + } + } } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogListenerResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogListenerResource.java index b9e291a628b4..67b4d29fbef1 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogListenerResource.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogListenerResource.java @@ -19,31 +19,19 @@ package org.apache.druid.catalog.http; -import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; import com.sun.jersey.spi.container.ResourceFilters; -import org.apache.druid.catalog.model.TableMetadata; -import org.apache.druid.catalog.model.TableSpec; -import org.apache.druid.catalog.sync.CatalogUpdateNotifier; -import org.apache.druid.catalog.sync.MetadataCatalog.CatalogListener; -import org.apache.druid.guice.annotations.Json; -import org.apache.druid.guice.annotations.Smile; -import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.catalog.sync.CatalogUpdateListener; +import org.apache.druid.catalog.sync.UpdateEvent; import org.apache.druid.server.http.security.ConfigResourceFilter; -import org.apache.druid.server.security.AuthorizerMapper; import javax.inject.Inject; -import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; import javax.ws.rs.POST; import javax.ws.rs.Path; -import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import java.io.IOException; -import java.io.InputStream; - /** * Resource on the Broker to listen to catalog update events from the * Coordinator. Since this is an internal API, it supports the efficient @@ -55,26 +43,15 @@ public class CatalogListenerResource { public static final String BASE_URL = "/druid/broker/v1/catalog"; public static final String SYNC_URL = "/sync"; - private static final Logger log = new Logger(CatalogListenerResource.class); - - private final CatalogListener listener; - private final AuthorizerMapper authorizerMapper; - private final ObjectMapper smileMapper; - private final ObjectMapper jsonMapper; + private final CatalogUpdateListener listener; @Inject public CatalogListenerResource( - final CatalogListener listener, - @Smile final ObjectMapper smileMapper, - @Json final ObjectMapper jsonMapper, - final AuthorizerMapper authorizerMapper + final CatalogUpdateListener listener ) { this.listener = listener; - this.authorizerMapper = authorizerMapper; - this.smileMapper = smileMapper; - this.jsonMapper = jsonMapper; } /** @@ -85,28 +62,9 @@ public CatalogListenerResource( @Path(SYNC_URL) @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) @ResourceFilters(ConfigResourceFilter.class) - public Response syncTable( - final InputStream inputStream, - @Context final HttpServletRequest req - ) + public Response syncTable(final UpdateEvent event) { - final String reqContentType = req.getContentType(); - final boolean isSmile = SmileMediaTypes.APPLICATION_JACKSON_SMILE.equals(reqContentType); - final ObjectMapper mapper = isSmile ? smileMapper : jsonMapper; - TableMetadata tableSpec; - try { - tableSpec = mapper.readValue(inputStream, TableMetadata.class); - } - catch (IOException e) { - log.error(e, "Bad catalog sync request received!"); - return Response.status(Response.Status.BAD_REQUEST).entity(e.getMessage()).build(); - } - TableSpec spec = tableSpec.spec(); - if (CatalogUpdateNotifier.TOMBSTONE_TABLE_TYPE.equals(spec.type())) { - listener.deleted(tableSpec.id()); - } else { - listener.updated(tableSpec); - } + listener.updated(event); return Response.status(Response.Status.ACCEPTED).build(); } } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java index dc6bf495ac7f..27d69cce078a 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -118,59 +119,58 @@ private long unHideColumns(List columns) throws CatalogException * @return revised properties with the revised hidden columns list after applying * the requested changes */ - private TableMetadata applyUnhideColumns(TableMetadata table, List columns) throws CatalogException + private TableSpec applyUnhideColumns(TableMetadata table, List columns) throws CatalogException { - if (!AbstractDatasourceDefn.isDatasource(type)) { + final TableSpec existingSpec = table.spec(); + if (!AbstractDatasourceDefn.isDatasource(existingSpec.type())) { throw CatalogException.badRequest("hideColumns is supported only for data source specs"); } + + final Map props = existingSpec.properties(); + final Map revised = new HashMap<>(props); @SuppressWarnings("unchecked") - List hiddenColumns = (List) props.get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); + List hiddenColumns = (List) revised.get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); if (CollectionUtils.isNullOrEmpty(hiddenColumns) || columns.isEmpty()) { - return props; + return null; } Set removals = new HashSet<>(columns); - List revised = new ArrayList<>(); + List revisedHiddenCols = new ArrayList<>(); for (String col : hiddenColumns) { if (!removals.contains(col)) { - revised.add(col); + revisedHiddenCols.add(col); } } if (revised.isEmpty()) { - props.remove(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); + revised.remove(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); } else { - props.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, revised); + revised.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, revised); } - return props; + return existingSpec.withProperties(revised); } - private long dropColumns(List columnsToDrop) throws CatalogException { return catalog.tables().updateColumns( id, - (type, cols) -> { - return applyDropColumns(type, cols, columnsToDrop); - } + table -> applyDropColumns(table, columnsToDrop) ); } - private List applyDropColumns( - final String tableType, - final List columns, - final List toDrop - ) + private TableSpec applyDropColumns(final TableMetadata table, final List toDrop) { - if (toDrop.isEmpty() || columns.isEmpty()) { - return columns; + final TableSpec existingSpec = table.spec(); + List existingColumns = existingSpec.columns(); + if (toDrop.isEmpty() || existingColumns.isEmpty()) { + return existingSpec; } Set drop = new HashSet(toDrop); List revised = new ArrayList<>(); - for (ColumnSpec col : columns) { + for (ColumnSpec col : existingColumns) { if (!drop.contains(col.name())) { revised.add(col); } } - return revised; + return existingSpec.withColumns(revised); } @@ -178,50 +178,55 @@ private long updateProperties(Map updates) throws CatalogExcepti { return catalog.tables().updateProperties( id, - (type, props) -> { - return applyUpdateProperties(type, props, updates); - } + table -> applyUpdateProperties(table, updates) ); } - private Map applyUpdateProperties(String type, Map props, Map updates) throws CatalogException + private TableSpec applyUpdateProperties( + final TableMetadata table, + final Map updates + ) throws CatalogException { - TableDefn defn = catalog.tableRegistry().defnFor(type); + final TableSpec existingSpec = table.spec(); + final TableDefn defn = resolveDefn(existingSpec.type()); + return existingSpec.withProperties( + defn.mergeProperties(existingSpec.properties(), updates) + ); + } + + private TableDefn resolveDefn(String tableType) throws CatalogException + { + TableDefn defn = catalog.tableRegistry().defnFor(tableType); if (defn == null) { throw new CatalogException( CatalogException.BAD_STATE, Response.Status.INTERNAL_SERVER_ERROR, "Table %s has an invalid type [%s]", id.sqlName(), - type + tableType ); } - return defn.mergeProperties(props, updates); + return defn; } - private long updateColumns(List updates) throws CatalogException + private long updateColumns(final List updates) throws CatalogException { return catalog.tables().updateColumns( id, - (type, cols) -> { - return applyUpdateColumns(type, cols, updates); - } + table -> applyUpdateColumns(table, updates) ); } - private List applyUpdateColumns(String type, List cols, List updates) throws CatalogException + private TableSpec applyUpdateColumns( + final TableMetadata table, + final List updates + ) throws CatalogException { - TableDefn defn = catalog.tableRegistry().defnFor(type); - if (defn == null) { - throw new CatalogException( - CatalogException.BAD_STATE, - Response.Status.INTERNAL_SERVER_ERROR, - "Table %s has an invalid type [%s]", - id.sqlName(), - type - ); - } - return defn.mergeColumns(cols, updates); + final TableSpec existingSpec = table.spec(); + final TableDefn defn = resolveDefn(existingSpec.type()); + return existingSpec.withColumns( + defn.mergeColumns(existingSpec.columns(), updates) + ); } private long moveColumn(MoveColumn moveColumn) throws CatalogException @@ -237,22 +242,25 @@ private long moveColumn(MoveColumn moveColumn) throws CatalogException } return catalog.tables().updateColumns( id, - (type, cols) -> { - return applyMoveColumn(cols, moveColumn); - } + table -> applyMoveColumn(table, moveColumn) ); } - public List applyMoveColumn(List columns, MoveColumn moveColumn) throws CatalogException + private TableSpec applyMoveColumn( + final TableMetadata table, + final MoveColumn moveColumn + ) throws CatalogException { - List revised = new ArrayList<>(columns); - final int colPosn = findColumn(columns, moveColumn.column); + final TableSpec existingSpec = table.spec(); + final List existingCols = existingSpec.columns(); + final List revised = new ArrayList<>(existingCols); + final int colPosn = findColumn(existingCols, moveColumn.column); if (colPosn == -1) { throw CatalogException.badRequest("Column [%s] is not defined", moveColumn.column); } int anchorPosn; if (moveColumn.where == Position.BEFORE || moveColumn.where == Position.AFTER) { - anchorPosn = findColumn(columns, moveColumn.anchor); + anchorPosn = findColumn(existingCols, moveColumn.anchor); if (anchorPosn == -1) { throw CatalogException.badRequest("Anchor [%s] is not defined", moveColumn.column); } @@ -278,7 +286,8 @@ public List applyMoveColumn(List columns, MoveColumn mov revised.add(anchorPosn + 1, col); break; } - return revised; + + return existingSpec.withColumns(revised); } private static int findColumn(List columns, String colName) diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java index c807503cb06d..649cc570a4e8 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java @@ -19,7 +19,6 @@ package org.apache.druid.catalog.storage.sql; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Strings; @@ -40,6 +39,7 @@ import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.metadata.SQLMetadataConnector; import org.skife.jdbi.v2.Handle; @@ -90,17 +90,17 @@ public void start() } public static final String CREATE_TABLE = - "CREATE TABLE %s (\n" - + " schemaName VARCHAR(255) NOT NULL,\n" - + " name VARCHAR(255) NOT NULL,\n" - + " creationTime BIGINT NOT NULL,\n" - + " updateTime BIGINT NOT NULL,\n" - + " state CHAR(1) NOT NULL,\n" - + " tableType VARCHAR(20) NOT NULL,\n" - + " properties %s,\n" - + " columns %s,\n" - + " PRIMARY KEY(schemaName, name)\n" - + ")"; + "CREATE TABLE %s (\n" + + " schemaName VARCHAR(255) NOT NULL,\n" + + " name VARCHAR(255) NOT NULL,\n" + + " creationTime BIGINT NOT NULL,\n" + + " updateTime BIGINT NOT NULL,\n" + + " state CHAR(1) NOT NULL,\n" + + " tableType VARCHAR(20) NOT NULL,\n" + + " properties %s,\n" + + " columns %s,\n" + + " PRIMARY KEY(schemaName, name)\n" + + ")"; // TODO: Move to SqlMetadataConnector public void createTableDefnTable() @@ -148,8 +148,8 @@ public Long withHandle(Handle handle) throws DuplicateKeyException .bind("updateTime", updateTime) .bind("state", TableMetadata.TableState.ACTIVE.code()) .bind("tableType", spec.type()) - .bind("properties", toBytes(jsonMapper, spec.properties())) - .bind("columns", toBytes(jsonMapper, spec.columns())); + .bind("properties", JacksonUtils.toBytes(jsonMapper, spec.properties())) + .bind("columns", JacksonUtils.toBytes(jsonMapper, spec.columns())); try { stmt.execute(); } @@ -252,8 +252,8 @@ public TableMetadata withHandle(Handle handle) throws NotFoundException .bind("schemaName", id.schema()) .bind("name", id.name()) .bind("tableType", spec.type()) - .bind("properties", toBytes(jsonMapper, spec.properties())) - .bind("columns", toBytes(jsonMapper, spec.columns())) + .bind("properties", JacksonUtils.toBytes(jsonMapper, spec.properties())) + .bind("columns", JacksonUtils.toBytes(jsonMapper, spec.columns())) .bind("updateTime", updateTime) .execute(); if (updateCount == 0) { @@ -264,7 +264,7 @@ public TableMetadata withHandle(Handle handle) throws NotFoundException } ); sendUpdate(EventType.UPDATE, revised); - return table.updateTime(); + return revised.updateTime(); } catch (CallbackFailedException e) { if (e.getCause() instanceof NotFoundException) { @@ -296,8 +296,8 @@ public TableMetadata withHandle(Handle handle) throws NotFoundException .bind("schemaName", id.schema()) .bind("name", id.name()) .bind("tableType", spec.type()) - .bind("properties", toBytes(jsonMapper, spec.properties())) - .bind("columns", toBytes(jsonMapper, spec.columns())) + .bind("properties", JacksonUtils.toBytes(jsonMapper, spec.properties())) + .bind("columns", JacksonUtils.toBytes(jsonMapper, spec.columns())) .bind("updateTime", updateTime) .bind("oldVersion", oldVersion) .execute(); @@ -312,7 +312,7 @@ public TableMetadata withHandle(Handle handle) throws NotFoundException } ); sendUpdate(EventType.UPDATE, revised); - return table.updateTime(); + return revised.updateTime(); } catch (CallbackFailedException e) { if (e.getCause() instanceof NotFoundException) { @@ -322,16 +322,17 @@ public TableMetadata withHandle(Handle handle) throws NotFoundException } } - private static final String SELECT_TABLE_PROPERTIES_STMT = + private static final String SELECT_PROPERTIES_STMT = "SELECT tableType, properties\n" + "FROM %s\n" + "WHERE schemaName = :schemaName\n" + " AND name = :name\n" + " AND state = 'A'"; - private static final String UPDATE_TABLE_PROPERTIES_STMT = + private static final String UPDATE_PROPERTIES_STMT = "UPDATE %s\n SET\n" + - " properties = :properties\n" + + " properties = :properties,\n" + + " updateTime = :updateTime\n" + "WHERE schemaName = :schemaName\n" + " AND name = :name\n"; @@ -351,7 +352,7 @@ public TableMetadata withHandle(Handle handle) throws CatalogException handle.begin(); try { final Query> query = handle - .createQuery(statement(SELECT_TABLE_PROPERTIES_STMT)) + .createQuery(statement(SELECT_PROPERTIES_STMT)) .setFetchSize(connector.getStreamingFetchSize()) .bind("schemaName", id.schema()) .bind("name", id.name()); @@ -380,10 +381,10 @@ public TableMetadata withHandle(Handle handle) throws CatalogException } final long updateTime = System.currentTimeMillis(); final int updateCount = handle - .createStatement(statement(UPDATE_TABLE_PROPERTIES_STMT)) + .createStatement(statement(UPDATE_PROPERTIES_STMT)) .bind("schemaName", id.schema()) .bind("name", id.name()) - .bind("properties", toBytes(jsonMapper, revised)) + .bind("properties", JacksonUtils.toBytes(jsonMapper, revised.properties())) .bind("updateTime", updateTime) .execute(); if (updateCount == 0) { @@ -423,7 +424,8 @@ public TableMetadata withHandle(Handle handle) throws CatalogException private static final String UPDATE_COLUMNS_STMT = "UPDATE %s\n SET\n" + - " columns = :columns\n" + + " columns = :columns,\n" + + " updateTime = :updateTime\n" + "WHERE schemaName = :schemaName\n" + " AND name = :name\n"; @@ -475,7 +477,7 @@ public TableMetadata withHandle(Handle handle) throws CatalogException .createStatement(statement(UPDATE_COLUMNS_STMT)) .bind("schemaName", id.schema()) .bind("name", id.name()) - .bind("properties", toBytes(jsonMapper, revised)) + .bind("columns", JacksonUtils.toBytes(jsonMapper, revised.columns())) .bind("updateTime", updateTime) .execute(); if (updateCount == 0) { @@ -725,20 +727,6 @@ private NotFoundException tableNotFound(TableId id) ); } - /** - * Convert the given object to an array of bytes. Use when the object is - * known serializable so that the Jackson exception can be suppressed. - */ - private static byte[] toBytes(ObjectMapper jsonMapper, Object obj) - { - try { - return jsonMapper.writeValueAsBytes(obj); - } - catch (JsonProcessingException e) { - throw new ISE("Failed to serialize " + obj.getClass().getSimpleName()); - } - } - /** * Deserialize an object from an array of bytes. Use when the object is * known deserializable so that the Jackson exception can be suppressed. diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateNotifier.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateNotifier.java index 7b39033036ab..deeab56cd256 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateNotifier.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateNotifier.java @@ -19,7 +19,6 @@ package org.apache.druid.catalog.sync; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.catalog.http.CatalogListenerResource; import org.apache.druid.catalog.storage.CatalogStorage; @@ -29,7 +28,7 @@ import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.annotations.EscalatedClient; import org.apache.druid.guice.annotations.Smile; -import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.http.client.HttpClient; @@ -100,11 +99,6 @@ public void stop() @Override public void updated(UpdateEvent event) { - try { - notifier.send(smileMapper.writeValueAsBytes(event)); - } - catch (JsonProcessingException e) { - throw new ISE("Failed to serialize " + event.getClass().getSimpleName()); - } + notifier.send(JacksonUtils.toBytes(smileMapper, event)); } } diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java index 5d6ece6a817d..9706b3b06b70 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java @@ -21,14 +21,18 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; +import org.apache.druid.catalog.CatalogException; import org.apache.druid.catalog.CatalogException.DuplicateKeyException; import org.apache.druid.catalog.CatalogException.NotFoundException; import org.apache.druid.catalog.CatalogTest; +import org.apache.druid.catalog.model.ColumnSpec; +import org.apache.druid.catalog.model.Columns; import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.catalog.model.table.DatasourceDefn.DatasourceColumnDefn; import org.apache.druid.catalog.storage.sql.CatalogManager; import org.apache.druid.catalog.storage.sql.SQLCatalogManager; import org.apache.druid.jackson.DefaultObjectMapper; @@ -39,13 +43,13 @@ import org.junit.Test; import org.junit.experimental.categories.Category; +import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Map; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; @@ -81,7 +85,7 @@ public void tearDown() } @Test - public void testCreate() throws DuplicateKeyException + public void testCreate() throws DuplicateKeyException, NotFoundException { Map props = ImmutableMap.of( AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", @@ -90,8 +94,8 @@ public void testCreate() throws DuplicateKeyException TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); TableMetadata table = TableMetadata.newTable(TableId.datasource("table1"), spec); - // Table does not exist, read returns nothing. - assertNull(manager.read(table.id())); + // Table does not exist, read throws an exception. + assertThrows(NotFoundException.class, () -> manager.read(table.id())); // Create the table long version = manager.create(table); @@ -154,7 +158,101 @@ public void testUpdate() throws DuplicateKeyException, NotFoundException } @Test - public void testDelete() throws DuplicateKeyException + public void testUpdateProperties() throws CatalogException + { + Map props = ImmutableMap.of( + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", + AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000 + ); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); + TableMetadata table = TableMetadata.newTable(TableId.datasource("table1"), spec); + final long version1 = manager.create(table); + + // Transform properties by adding a new one + final long version2 = manager.updateProperties(table.id(), t -> { + TableSpec target = table.spec(); + Map updated = new HashMap<>(target.properties()); + updated.put("foo", "bar"); + return target.withProperties(updated); + }); + assertTrue(version2 > version1); + + TableMetadata read = manager.read(table.id()); + assertEquals(version2, read.updateTime()); + Map expected = new HashMap<>(props); + expected.put("foo", "bar"); + assertEquals(expected, read.spec().properties()); + + // Not found + assertThrows( + NotFoundException.class, + () -> manager.updateProperties(TableId.datasource("bogus"), t -> t.spec()) + ); + + // No update + final long version3 = manager.updateProperties(table.id(), t -> null); + assertEquals(0, version3); + + // Update fails if table is in the Deleting state + manager.markDeleting(table.id()); + assertThrows( + NotFoundException.class, + () -> manager.updateProperties(table.id(), t -> t.spec()) + ); + } + + @Test + public void testUpdateColumns() throws CatalogException + { + Map props = ImmutableMap.of( + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", + AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000 + ); + List cols = Arrays.asList( + new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "a", Columns.VARCHAR, null), + new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "b", Columns.BIGINT, null) + ); + ColumnSpec colC = new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "c", Columns.DOUBLE, null); + + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, cols); + TableMetadata table = TableMetadata.newTable(TableId.datasource("table1"), spec); + final long version1 = manager.create(table); + + // Transform columns by adding a new one + final long version2 = manager.updateColumns(table.id(), t -> { + TableSpec target = table.spec(); + List updated = new ArrayList<>(target.columns()); + updated.add(colC); + return target.withColumns(updated); + }); + assertTrue(version2 > version1); + + TableMetadata read = manager.read(table.id()); + assertEquals(version2, read.updateTime()); + List expected = new ArrayList<>(cols); + expected.add(colC); + assertEquals(expected, read.spec().columns()); + + // Not found + assertThrows( + NotFoundException.class, + () -> manager.updateColumns(TableId.datasource("bogus"), t -> t.spec()) + ); + + // No update + final long version3 = manager.updateColumns(table.id(), t -> null); + assertEquals(0, version3); + + // Update fails if table is in the Deleting state + manager.markDeleting(table.id()); + assertThrows( + NotFoundException.class, + () -> manager.updateColumns(table.id(), t -> t.spec()) + ); + } + + @Test + public void testDelete() throws DuplicateKeyException, NotFoundException { Map props = ImmutableMap.of( AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", @@ -163,10 +261,10 @@ public void testDelete() throws DuplicateKeyException TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); TableMetadata table = TableMetadata.newTable(TableId.datasource("table1"), spec); - assertFalse(manager.delete(table.id())); + assertThrows(NotFoundException.class, () -> manager.delete(table.id())); manager.create(table); - assertTrue(manager.delete(table.id())); - assertFalse(manager.delete(table.id())); + manager.delete(table.id()); + assertThrows(NotFoundException.class, () -> manager.delete(table.id())); } @Test diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java index bae8cff5a475..629bf4f1b4a6 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java @@ -20,7 +20,6 @@ package org.apache.druid.catalog.sync; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.dataformat.smile.SmileFactory; import org.apache.druid.catalog.CatalogException.DuplicateKeyException; import org.apache.druid.catalog.CatalogException.NotFoundException; import org.apache.druid.catalog.model.ColumnSpec; @@ -52,6 +51,11 @@ import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; +/** + * Test the catalog synchronization mechanisms: direct (reads from the DB), + * cached (holds a copy of the DB, based on update events) and remote + * (like cached, but receives events over HTTP.) + */ public class CatalogMetadataTest { @Rule @@ -60,7 +64,6 @@ public class CatalogMetadataTest private CatalogTests.DbFixture dbFixture; private CatalogStorage storage; private ObjectMapper jsonMapper; - private ObjectMapper smileMapper; @Before public void setUp() @@ -68,7 +71,6 @@ public void setUp() dbFixture = new CatalogTests.DbFixture(derbyConnectorRule); storage = dbFixture.storage; jsonMapper = new ObjectMapper(); - smileMapper = new ObjectMapper(new SmileFactory()); } @After @@ -136,7 +138,7 @@ public void testCached() throws DuplicateKeyException, NotFoundException // Also test the deletion case TableId table2 = TableId.datasource("table2"); storage.tables().delete(table2); - assertNull(storage.tables().read(table2)); + assertThrows(NotFoundException.class, () -> storage.tables().read(table2)); List tables = catalog.tables(TableId.DRUID_SCHEMA); assertEquals(2, tables.size()); @@ -146,20 +148,9 @@ public void testCached() throws DuplicateKeyException, NotFoundException @Test public void testRemoteWithJson() throws DuplicateKeyException, NotFoundException - { - doTestRemote(false); - } - - @Test - public void testRemoteWithSmile() throws DuplicateKeyException, NotFoundException - { - doTestRemote(true); - } - - private void doTestRemote(boolean useSmile) throws DuplicateKeyException, NotFoundException { populateCatalog(); - MockCatalogSync sync = new MockCatalogSync(storage, CatalogTests.AUTH_MAPPER, jsonMapper, smileMapper, useSmile); + MockCatalogSync sync = new MockCatalogSync(storage, jsonMapper); MetadataCatalog catalog = sync.catalog(); storage.register(sync); verifyInitial(catalog); @@ -169,7 +160,7 @@ private void doTestRemote(boolean useSmile) throws DuplicateKeyException, NotFou // Also test the deletion case TableId table2 = TableId.datasource("table2"); storage.tables().delete(table2); - assertNull(storage.tables().read(table2)); + assertThrows(NotFoundException.class, () -> storage.tables().read(table2)); List tables = catalog.tables(TableId.DRUID_SCHEMA); assertEquals(2, tables.size()); diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/MockCatalogSync.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/MockCatalogSync.java index 87a01d4081c2..7a5b7f03b273 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/MockCatalogSync.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/MockCatalogSync.java @@ -20,79 +20,35 @@ package org.apache.druid.catalog.sync; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; import org.apache.druid.catalog.http.CatalogListenerResource; -import org.apache.druid.catalog.model.TableId; -import org.apache.druid.catalog.model.TableMetadata; -import org.apache.druid.catalog.model.TableSpec; import org.apache.druid.catalog.storage.CatalogStorage; -import org.apache.druid.catalog.storage.CatalogTests; -import org.apache.druid.catalog.sync.MetadataCatalog.CatalogListener; -import org.apache.druid.server.http.catalog.DummyRequest; -import org.apache.druid.server.security.AuthorizerMapper; - -import javax.ws.rs.core.MediaType; - -import java.io.ByteArrayInputStream; /** * Simulates a network sync from catalog (Coordinator) to consumer (Broker). */ -public class MockCatalogSync implements CatalogListener +public class MockCatalogSync implements CatalogUpdateListener { private final CatalogListenerResource listenerResource; private final CachedMetadataCatalog catalog; - private final boolean useSmile; - private final ObjectMapper smileMapper; - private final ObjectMapper jsonMapper; public MockCatalogSync( final CatalogStorage storage, - final AuthorizerMapper authorizerMapper, - final ObjectMapper smileMapper, - final ObjectMapper jsonMapper, - final boolean useSmile + final ObjectMapper jsonMapper ) { this.catalog = new CachedMetadataCatalog(storage, storage.schemaRegistry(), jsonMapper); - this.listenerResource = new CatalogListenerResource( - catalog, - smileMapper, - jsonMapper, - authorizerMapper - ); - this.useSmile = useSmile; - this.smileMapper = smileMapper; - this.jsonMapper = jsonMapper; + this.listenerResource = new CatalogListenerResource(catalog); } @Override - public void updated(TableMetadata update) + public void updated(UpdateEvent update) { doSync(update); } - private void doSync(TableMetadata update) - { - byte[] encoded = update.toBytes(useSmile ? smileMapper : jsonMapper); - listenerResource.syncTable( - new ByteArrayInputStream(encoded), - new DummyRequest( - DummyRequest.POST, - CatalogTests.SUPER_USER, - useSmile ? SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON - ) - ); - } - - @Override - public void deleted(TableId tableId) + private void doSync(UpdateEvent event) { - TableMetadata spec = TableMetadata.newTable( - tableId, - new TableSpec(CatalogUpdateNotifier.TOMBSTONE_TABLE_TYPE, null, null) - ); - doSync(spec); + listenerResource.syncTable(event); } public MetadataCatalog catalog() From 39382ac2e0ee32a148c03f1b2e558f7cf493f8d7 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Fri, 28 Oct 2022 22:31:47 -0700 Subject: [PATCH 13/22] Revised edit commands Tests for revised sync events --- .../druid/catalog/http/CatalogResource.java | 726 ++++++------------ .../druid/catalog/http/CatalogResource2.java | 326 -------- .../druid/catalog/http/TableEditor.java | 33 +- .../storage/sql/SQLCatalogManager.java | 6 +- .../catalog/storage/TableManagerTest.java | 4 +- ...MetadataTest.java => CatalogSyncTest.java} | 53 +- .../http/catalog/CatalogResourceTest.java | 188 +++-- 7 files changed, 387 insertions(+), 949 deletions(-) delete mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource2.java rename extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/{CatalogMetadataTest.java => CatalogSyncTest.java} (86%) diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java index a9eaac71cf55..d226ec6cdfa1 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java @@ -1,42 +1,18 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - package org.apache.druid.catalog.http; import com.google.common.base.Strings; +import com.google.common.collect.ImmutableMap; import org.apache.curator.shaded.com.google.common.collect.Lists; +import org.apache.druid.catalog.CatalogException; import org.apache.druid.catalog.CatalogException.DuplicateKeyException; import org.apache.druid.catalog.CatalogException.NotFoundException; -import org.apache.druid.catalog.http.MoveColumn.Position; -import org.apache.druid.catalog.model.ColumnSpec; import org.apache.druid.catalog.model.SchemaRegistry.SchemaSpec; -import org.apache.druid.catalog.model.TableDefnRegistry; import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; -import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; -import org.apache.druid.catalog.storage.Actions; import org.apache.druid.catalog.storage.CatalogStorage; +import org.apache.druid.common.utils.IdUtils; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthorizationUtils; @@ -52,7 +28,6 @@ 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; @@ -61,19 +36,16 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import java.util.ArrayList; import java.util.Collections; -import java.util.HashSet; import java.util.List; -import java.util.Set; -import java.util.function.Function; /** * REST endpoint for user and internal catalog actions. Catalog actions - * occur at the global level (all schemas), the schema level, or the + * are divided by operation: configuration-as-code, edits, retrieval, + * etc. Operations occur at the global level (all schemas), the schema level, or the * table level. * - * @see {@link CatalogListenerResource} for the client-side API. + * @see {@link CatalogListenerResource} for the broker-side API. */ @Path(CatalogResource.ROOT_PATH) public class CatalogResource @@ -93,459 +65,183 @@ public CatalogResource( this.authorizerMapper = authorizerMapper; } - private enum PostAction - { - NEW, - IF_NEW, - REPLACE, - FORCE; - } + // --------------------------------------------------------------------- + // Configuration-as-code style methods /** - * Create a new table containing the given table specification. + * Create or update a new table containing the given table specification. + * Supports three use cases: + *

    + *
  • "create if not exists": default use case with no options.
  • + *
  • "create or update": set {@code overwrite=true}.
  • + *
  • "update": set {@code version} to the expected current version. + * This form enforces optimistic locking.
  • + *
* * @param dbSchema The name of the Druid schema, which must be writable * and the user must have at least read access. * @param name The name of the table definition to modify. The user must * have write access to the table. * @param spec The new table definition. - * @param actionParam What to do if the table already exists. - * {@code ifNew} is the same as the SQL IF NOT EXISTS clause. If {@code new}, - * then an error is raised if the table exists. If {@code ifNew}, then - * the action silently does nothing if the table exists. Primarily for - * use in scripts. The other two options are primarily for use in tests. + * @param version the expected version of an existing table. The version must + * match. If not (or if the table does not exist), returns an error. + * @param overwrite if {@code true}, then overwrites any existing table. + * If {@code false}, then the operation fails if the table already exists. * @param req the HTTP request used for authorization. - * @return the version number of the table - */ + */ @POST - @Path("/tables/{dbSchema}/{name}") + @Path("/schemas/{schema}/{name}") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) public Response postTable( - @PathParam("dbSchema") String dbSchema, + @PathParam("schema") String dbSchema, @PathParam("name") String name, TableSpec spec, - @QueryParam("action") String actionParam, @QueryParam("version") long version, + @QueryParam("overwrite") boolean overwrite, @Context final HttpServletRequest req ) { - final PostAction action; - if (actionParam == null) { - action = PostAction.NEW; - } else { - try { - action = PostAction.valueOf(StringUtils.toUpperCase(actionParam)); - } - catch (IllegalArgumentException e) { - return Actions.badRequest( - Actions.INVALID, - StringUtils.format( - "Not a valid action: [%s]. Valid actions are new, ifNew, replace, force", - actionParam - ) - ); - } - } - TableId tableId = TableId.of(dbSchema, name); - Response response = validateTable(tableId, spec, req); - if (response != null) { - return response; - } - TableMetadata table = TableMetadata.newTable(tableId, spec); try { - catalog.validate(table); - } - catch (IAE e) { - return Actions.badRequest(Actions.INVALID, e.getMessage()); - } - - switch (action) { - case NEW: - return insertTableSpec(table, false); - case IF_NEW: - return insertTableSpec(table, true); - case REPLACE: - return updateTableSpec(table, version); - case FORCE: - return addOrUpdateTableSpec(table); - default: - throw new ISE("Unknown action."); - } - } - - private Response validateTable(TableId tableId, TableSpec spec, final HttpServletRequest req) - { - // Druid has a fixed set of schemas. Ensure the one provided is valid. - Pair result = validateSchema(tableId.schema()); - if (result.lhs != null) { - return result.lhs; - } - SchemaSpec schema = result.rhs; - - // The schema has to be one that allows table definitions. - if (!schema.writable()) { - return Actions.badRequest( - Actions.INVALID, - StringUtils.format("Cannot modify schema %s", tableId.schema()) - ); - } - - // Table name can't be blank or have leading/trailing spaces - if (Strings.isNullOrEmpty(tableId.name())) { - return Actions.badRequest(Actions.INVALID, "Table name is required"); - } - if (!tableId.name().equals(tableId.name().trim())) { - return Actions.badRequest(Actions.INVALID, "Table name cannot start or end with spaces"); - } - - // The user has to have permission to modify the table. Throws an exception - // if not. - authorizeTable(schema, tableId.name(), Action.WRITE, req); - - // Validate the spec, if provided. - if (spec != null) { - - // The given table spec has to be valid for the given schema. + final SchemaSpec schema = validateSchema(dbSchema, true); + validateTableName(name); + authorizeTable(schema, name, Action.WRITE, req); + validateTableSpec(schema, name, spec); + final TableMetadata table = TableMetadata.newTable(TableId.of(dbSchema, name), spec); try { - spec.validate(); + catalog.validate(table); } catch (IAE e) { - return Actions.badRequest(Actions.INVALID, e.getMessage()); + throw CatalogException.badRequest(e.getMessage()); } - if (!schema.accepts(spec.type())) { - return Actions.badRequest( - Actions.INVALID, - StringUtils.format( - "Cannot create tables of type %s in schema %s", - spec.getClass().getSimpleName(), - tableId.schema() - ) - ); - } - } - - // Everything checks out, let the request proceed. - return null; - } - - private Response insertTableSpec(TableMetadata table, boolean ifNew) - { - try { - long createVersion = catalog.tables().create(table); - return Actions.okWithVersion(createVersion); - } - catch (DuplicateKeyException e) { - if (!ifNew) { - return Actions.badRequest( - Actions.DUPLICATE_ERROR, - StringUtils.format( - "A table of name %s already exists", - table.id().sqlName() - ) - ); + long newVersion; + if (version != 0) { + // A version is provided. Update that version (only). + newVersion = catalog.tables().update(table, version); } else { - return Actions.okWithVersion(0); + try { + // No version. Create the table. + newVersion = catalog.tables().create(table); + } + catch (DuplicateKeyException e) { + // Table exists + if (overwrite) { + // User wants to overwrite, so do so. + newVersion = catalog.tables().replace(table); + } else { + throw e; + } + } } + return okWithVersion(newVersion); } - catch (Exception e) { - return Actions.exception(e); - } - } - - private Response updateTableSpec(TableMetadata table, long version) - { - try { - long newVersion = catalog.tables().update(table, version); - return Actions.okWithVersion(newVersion); - } - catch (NotFoundException e) { - return Response - .status(Response.Status.BAD_REQUEST) - .entity( - Actions.error( - Actions.DUPLICATE_ERROR, - "The table entry not found or is older than the given version: reload and retry")) - .build(); - } - catch (Exception e) { - return Actions.exception(e); - } - } - - private Response addOrUpdateTableSpec(TableMetadata table) - { - try { - long newVersion = catalog.tables().create(table); - return Actions.okWithVersion(newVersion); - } - catch (DuplicateKeyException e) { - // Fall through - } - catch (Exception e) { - return Actions.exception(e); - } - try { - long newVersion = catalog.tables().update(table, 0); - return Actions.okWithVersion(newVersion); - } - catch (Exception e) { - return Actions.exception(e); + catch (CatalogException e) + { + return e.toResponse(); } } /** - * Update a table within the given schema. + * Retrieves the table metadata, including the spec. + *

+ * Returns a 404 (NOT FOUND) error if the table catalog entry does not exist. + * Note that this check is only for the specification; the table (or + * datasource) itself may exist. Similarly, this call may return a specification + * even if there is no datasource of the same name (typically occurs when + * the definition is created before the datasource itself.) * - * @param dbSchema The name of the Druid schema, which must be writable - * and the user must have at least read access. - * @param name The name of the table definition to modify. The user must - * have write access to the table. - * @param spec The new table definition. - * @param version An optional table version. If provided, the metadata DB - * entry for the table must be at this exact version or the update - * will fail. (Provides "optimistic locking.") If omitted (that is, - * if zero), then no update conflict change is done. + * @param dbSchema The Druid schema. The user must have read access. + * @param name The name of the table within the schema. The user must have + * read access. * @param req the HTTP request used for authorization. - * @return the new version number of the table + * @return the definition for the table, if any. */ - @PUT - @Path("/tables/{dbSchema}/{name}") - @Consumes(MediaType.APPLICATION_JSON) + @GET + @Path("/schemas/{schema}/{name}") @Produces(MediaType.APPLICATION_JSON) - public Response updateTableDefn( - @PathParam("dbSchema") String dbSchema, + public Response getTable( + @PathParam("schema") String dbSchema, @PathParam("name") String name, - TableSpec spec, - @QueryParam("version") long version, @Context final HttpServletRequest req ) { - - TableDefnRegistry tableRegistry = catalog.tableRegistry(); - return incrementalUpdate( - TableId.of(dbSchema, name), - spec, - req, - (existing) -> tableRegistry.resolve(existing).merge(spec).spec() - ); - } - - private Response incrementalPropertiesUpdate( - TableId tableId, - TableSpec newSpec, - @Context final HttpServletRequest req, - Function action - ) - { - Response response = validateTable(tableId, newSpec, req); - if (response != null) { - return response; - } - try { - long newVersion = catalog.tables().updateTableSpec(tableId, action); - return Actions.okWithVersion(newVersion); - } - catch (NotFoundException e) { - return Response.status(Response.Status.NOT_FOUND).build(); - } - catch (Exception e) { - return Actions.exception(e); - } - } - - private Response incrementalColumnsUpdate( - TableId tableId, - TableSpec newSpec, - @Context final HttpServletRequest req, - Function action - ) - { - Response response = validateTable(tableId, newSpec, req); - if (response != null) { - return response; - } try { - long newVersion = catalog.tables().updateTableSpec(tableId, action); - return Actions.okWithVersion(newVersion); - } - catch (NotFoundException e) { - return Response.status(Response.Status.NOT_FOUND).build(); + final SchemaSpec schema = validateSchema(dbSchema, false); + authorizeTable(schema, name, Action.READ, req); + final TableMetadata table = catalog.tables().read(new TableId(dbSchema, name)); + return Response.ok().entity(table).build(); } - catch (Exception e) { - return Actions.exception(e); + catch (CatalogException e) + { + return e.toResponse(); } } /** - * Move a single column to the start or end of the column list, or before or after - * another column. Both columns must exist. Returns the version of the table - * after the update. - *

- * The operation is done atomically so no optimistic locking is required. + * Deletes the table definition (but not the underlying table or datasource) + * for the given schema and table. * - * @param dbSchema - * @param name - * @param command - * @param req - * @return + * @param dbSchema The name of the schema that holds the table. + * @param name The name of the table definition to delete. The user must have + * write access. */ - @POST - @Path("/tables/{dbSchema}/{name}/moveColumn") - @Consumes(MediaType.APPLICATION_JSON) + @DELETE + @Path("/schemas/{schema}/{name}") @Produces(MediaType.APPLICATION_JSON) - public Response moveColumn( - @PathParam("dbSchema") final String dbSchema, - @PathParam("name") final String name, - final MoveColumn command, + public Response deleteTable( + @PathParam("schema") String dbSchema, + @PathParam("name") String name, @Context final HttpServletRequest req ) { - if (command == null) { - return Actions.badRequest(Actions.INVALID, "A MoveColumn object is required"); - } - if (Strings.isNullOrEmpty(command.column)) { - return Actions.badRequest(Actions.INVALID, "A column name is required"); - } - if (command.where == null) { - return Actions.badRequest(Actions.INVALID, "A target location is required"); + try { + final SchemaSpec schema = validateSchema(dbSchema, true); + authorizeTable(schema, name, Action.WRITE, req); + catalog.tables().delete(new TableId(dbSchema, name)); + return ok(); } - if ((command.where == Position.BEFORE || command.where == Position.AFTER) && Strings.isNullOrEmpty(command.anchor)) { - return Actions.badRequest(Actions.INVALID, "A anchor column is required for BEFORE or AFTER"); + catch (CatalogException e) + { + return e.toResponse(); } - return incrementalUpdate( - TableId.of(dbSchema, name), - null, - req, - (spec) -> spec.withColumns(command.perform(spec.columns())) - ); } - /** - * Hide or unhide columns. If both appear, hide takes precedence. Returns the - * new table version. - */ - @POST - @Path("/tables/{dbSchema}/{name}/hideColumns") - @Consumes(MediaType.APPLICATION_JSON) - @Produces(MediaType.APPLICATION_JSON) - public Response hideColumns( - @PathParam("dbSchema") final String dbSchema, - @PathParam("name") final String name, - final HideColumns command, - @Context final HttpServletRequest req - ) - { - return incrementalUpdate( - TableId.of(dbSchema, name), - null, - req, - (spec) -> { - if (!AbstractDatasourceDefn.isDatasource(spec.type())) { - throw new ISE("hideColumns is supported only for data source specs"); - } - @SuppressWarnings("unchecked") - List hiddenProps = (List) spec.properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); - return spec.withProperty( - AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, - command.perform(hiddenProps) - ); - } - ); - } + // --------------------------------------------------------------------- + // Modify a table within the catalog - /** - * Drop column metadata. Only removes metadata entries, has no effect on the - * physical segments. Returns the new table version. - */ @POST - @Path("/tables/{dbSchema}/{name}/dropColumns") + @Path("/schemas/{schema}/{name}/edit") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) - public Response dropColumns( - @PathParam("dbSchema") final String dbSchema, - @PathParam("name") final String name, - final List columns, - @Context final HttpServletRequest req - ) - { - return incrementalUpdate( - TableId.of(dbSchema, name), - null, - req, - (spec) -> spec.withColumns(dropColumns(spec.columns(), columns)) - ); - } - - private static List dropColumns( - final List columns, - final List toDrop) - { - if (toDrop == null || toDrop.isEmpty()) { - return columns; - } - Set drop = new HashSet(toDrop); - List revised = new ArrayList<>(); - for (T col : columns) { - if (!drop.contains(col.name())) { - revised.add(col); - } - } - return revised; - } - - /** - * Retrieves the definition of the given table. - *

- * Returns a 404 (NOT FOUND) error if the table definition does not exist. - * Note that this check is only for the definition; the table (or - * datasource) itself may exist. Similarly, this call may return a definition - * even if there is no datasource of the same name (typically occurs when - * the definition is created before the datasource itself.) - * - * @param dbSchema The Druid schema. The user must have read access. - * @param name The name of the table within the schema. The user must have - * read access. - * @param req the HTTP request used for authorization. - * @return the definition for the table, if any. - */ - @GET - @Path("/tables/{dbSchema}/{name}") - @Produces(MediaType.APPLICATION_JSON) - public Response getTable( - @PathParam("dbSchema") String dbSchema, + public Response editTable( + @PathParam("schema") String dbSchema, @PathParam("name") String name, + TableEditRequest editRequest, @Context final HttpServletRequest req ) { - Pair result = validateSchema(dbSchema); - if (result.lhs != null) { - return result.lhs; - } - if (Strings.isNullOrEmpty(name)) { - return Actions.badRequest(Actions.INVALID, "Table name is required"); - } - authorizeTable(result.rhs, name, Action.READ, req); try { - TableId tableId = new TableId(dbSchema, name); - TableMetadata table = catalog.tables().read(tableId); - if (table == null) { - return Response.status(Response.Status.NOT_FOUND).build(); - } - return Response.ok().entity(table).build(); + final SchemaSpec schema = validateSchema(dbSchema, true); + authorizeTable(schema, name, Action.WRITE, req); + final long newVersion = new TableEditor(catalog, TableId.of(dbSchema, name), editRequest).go(); + return okWithVersion(newVersion); } - catch (Exception e) { - return Actions.exception(e); + catch (CatalogException e) + { + return e.toResponse(); } } + // --------------------------------------------------------------------- + // Query names and table specs + /** * Retrieves the list of all Druid schema names. At present, Druid does * not impose security on schemas, only tables within schemas. */ @GET - @Path("/list/schemas/names") + @Path("/names/schemas") @Produces(MediaType.APPLICATION_JSON) public Response listSchemas( @Context final HttpServletRequest req @@ -556,12 +252,13 @@ public Response listSchemas( return Response.ok().entity(catalog.schemaRegistry().names()).build(); } + /** * Retrieves the list of all Druid table names for which the user has at * least read access. */ @GET - @Path("/list/tables/names") + @Path("/names/tables") @Produces(MediaType.APPLICATION_JSON) public Response listTableNames( @Context final HttpServletRequest req @@ -594,28 +291,30 @@ public Response listTableNames( * @param dbSchema The Druid schema to query. The user must have read access. */ @GET - @Path("/schemas/{dbSchema}/names") + @Path("/names/schemas/{schema}") @Produces(MediaType.APPLICATION_JSON) public Response listTableNamesForSchema( - @PathParam("dbSchema") String dbSchema, + @PathParam("schema") String dbSchema, @Context final HttpServletRequest req ) { - Pair result = validateSchema(dbSchema); - if (result.lhs != null) { - return result.lhs; + try { + SchemaSpec schema = validateSchema(dbSchema, false); + List tables = catalog.tables().tableNamesInSchema(dbSchema); + Iterable filtered = AuthorizationUtils.filterAuthorizedResources( + req, + tables, + name -> + Collections.singletonList( + resourceAction(schema, name, Action.READ)), + authorizerMapper + ); + return Response.ok().entity(Lists.newArrayList(filtered)).build(); + } + catch (CatalogException e) + { + return e.toResponse(); } - SchemaSpec schema = result.rhs; - List tables = catalog.tables().tableNamesInSchema(dbSchema); - Iterable filtered = AuthorizationUtils.filterAuthorizedResources( - req, - tables, - name -> - Collections.singletonList( - resourceAction(schema, name, Action.READ)), - authorizerMapper - ); - return Response.ok().entity(Lists.newArrayList(filtered)).build(); } /** @@ -623,85 +322,39 @@ public Response listTableNamesForSchema( * least read access. */ @GET - @Path("/schemas/{dbSchema}/tables") + @Path("/schemas/{dbSchema}") @Produces(MediaType.APPLICATION_JSON) public Response listTableMetadataForSchema( @PathParam("dbSchema") String dbSchema, @Context final HttpServletRequest req ) { - Pair result = validateSchema(dbSchema); - if (result.lhs != null) { - return result.lhs; - } - SchemaSpec schema = result.rhs; - List tables = catalog.tables().tablesInSchema(schema.name()); - Iterable filtered = AuthorizationUtils.filterAuthorizedResources( - req, - tables, - table -> { - TableId tableId = table.id(); - return Collections.singletonList( - resourceAction(schema, tableId.name(), Action.READ)); - }, - authorizerMapper - ); - - return Response.ok().entity(Lists.newArrayList(filtered)).build(); - } - - /** - * Deletes the table definition (but not the underlying table or datasource) - * for the given schema and table. - * - * @param dbSchema The name of the schema that holds the table. - * @param name The name of the table definition to delete. The user must have - * write access. - * @param ifExists Optional flag. If {@code false} (the default), 404 (NOT FOUND) - * error is returned if the table does not exist. If {@code true}, - * then acts like the SQL IF EXISTS clause and does not return an - * error if the table does not exist, - */ - @DELETE - @Path("/tables/{dbSchema}/{name}") - @Produces(MediaType.APPLICATION_JSON) - public Response deleteTable( - @PathParam("dbSchema") String dbSchema, - @PathParam("name") String name, - @QueryParam("ifExists") boolean ifExists, - @Context final HttpServletRequest req - ) - { - TableId tableId = new TableId(dbSchema, name); - - // Validate the schema. Returns either an error response (lhs) or the - // validated schema (rhs). - Pair result = validateSchema(tableId.schema()); - if (result.lhs != null) { - return result.lhs; - } - SchemaSpec schema = result.rhs; - if (!schema.writable()) { - return Actions.badRequest( - Actions.INVALID, - StringUtils.format("Cannot delete tables from schema %s", tableId.schema())); - } - if (Strings.isNullOrEmpty(name)) { - return Actions.badRequest(Actions.INVALID, "Table name is required"); - } - authorizeTable(schema, tableId.name(), Action.WRITE, req); try { - if (!catalog.tables().delete(tableId) && !ifExists) { - return Actions.notFound(tableId.sqlName()); - } + SchemaSpec schema = validateSchema(dbSchema, false); + List tables = catalog.tables().tablesInSchema(schema.name()); + Iterable filtered = AuthorizationUtils.filterAuthorizedResources( + req, + tables, + table -> { + TableId tableId = table.id(); + return Collections.singletonList( + resourceAction(schema, tableId.name(), Action.READ)); + }, + authorizerMapper + ); + + return Response.ok().entity(Lists.newArrayList(filtered)).build(); } - catch (Exception e) { - return Actions.exception(e); + catch (CatalogException e) + { + return e.toResponse(); } - return Actions.ok(); } - public static final String SCHEMA_SYNC = "/schemas/{dbSchema}/sync"; + // --------------------------------------------------------------------- + // Sync methods + + public static final String SCHEMA_SYNC = "/sync/schemas/{schema}"; /** * Synchronization request from the Broker for a database schema. Requests all @@ -716,7 +369,7 @@ public Response deleteTable( @Path(SCHEMA_SYNC) @Produces(MediaType.APPLICATION_JSON) public Response syncSchema( - @PathParam("dbSchema") String dbSchema, + @PathParam("schema") String dbSchema, @Context final HttpServletRequest req ) { @@ -725,7 +378,7 @@ public Response syncSchema( return listTableMetadataForSchema(dbSchema, req); } - public static final String TABLE_SYNC = "/tables/{dbSchema}/{name}/sync"; + public static final String TABLE_SYNC = "/sync/schemas/{schema}/{name}"; /** * Synchronization request from the Broker for information about a specific table @@ -736,7 +389,7 @@ public Response syncSchema( @Path(TABLE_SYNC) @Produces(MediaType.APPLICATION_JSON) public Response syncTable( - @PathParam("dbSchema") String dbSchema, + @PathParam("schema") String dbSchema, @PathParam("name") String name, @Context final HttpServletRequest req ) @@ -744,18 +397,58 @@ public Response syncTable( return getTable(dbSchema, name, req); } - private Pair validateSchema(String dbSchema) + // --------------------------------------------------------------------- + // Helper methods + + private void validateTableName(String name) throws CatalogException + { + try { + IdUtils.validateId("table", name); + } + catch (Exception e) { + throw CatalogException.badRequest(e.getMessage()); + } + if (!name.equals(name.trim())) { + throw CatalogException.badRequest("Table name cannot start or end with spaces"); + } + } + + private void validateTableSpec(SchemaSpec schema, String name, TableSpec spec) throws CatalogException + { + // The given table spec has to be valid for the given schema. + try { + spec.validate(); + } + catch (IAE e) { + throw CatalogException.badRequest(e.getMessage()); + } + + if (!schema.accepts(spec.type())) { + throw CatalogException.badRequest( + "Cannot create tables of type %s in schema %s", + spec.type(), + schema.name() + ); + } + } + + private SchemaSpec validateSchema(String dbSchema, boolean forWrite) throws CatalogException { if (Strings.isNullOrEmpty(dbSchema)) { - return Pair.of(Actions.badRequest(Actions.INVALID, "Schema name is required"), null); + throw CatalogException.badRequest("Schema name is required"); } SchemaSpec schema = catalog.resolveSchema(dbSchema); if (schema == null) { - return Pair.of(Actions.notFound( - StringUtils.format("Unknown schema %s", dbSchema)), - null); + throw new NotFoundException("Unknown schema %s", dbSchema); + } + + if (forWrite && !schema.writable()) { + throw CatalogException.badRequest( + "Cannot modify schema %s", + dbSchema + ); } - return Pair.of(null, schema); + return schema; } private static ResourceAction resourceAction(SchemaSpec schema, String name, Action action) @@ -763,8 +456,11 @@ private static ResourceAction resourceAction(SchemaSpec schema, String name, Act return new ResourceAction(new Resource(name, schema.securityResource()), action); } - private void authorizeTable(SchemaSpec schema, String name, Action action, HttpServletRequest request) + private void authorizeTable(SchemaSpec schema, String name, Action action, HttpServletRequest request) throws CatalogException { + if (Strings.isNullOrEmpty(name)) { + throw CatalogException.badRequest("Table name is required"); + } if (action == Action.WRITE && !schema.writable()) { throw new ForbiddenException( "Cannot create table definitions in schema: " + schema.name()); @@ -788,4 +484,18 @@ private Access authorizeAccess(String resource, String key, Action action, HttpS authorizerMapper ); } + + private static Response okWithVersion(long version) + { + return Response + .ok() + .entity(ImmutableMap.of("version", version)) + .build(); + } + + private static Response ok() + { + return Response.ok().build(); + } + } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource2.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource2.java deleted file mode 100644 index b48dee39ed2c..000000000000 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource2.java +++ /dev/null @@ -1,326 +0,0 @@ -package org.apache.druid.catalog.http; - -import com.google.common.base.Strings; -import com.google.common.collect.ImmutableMap; -import org.apache.druid.catalog.CatalogException; -import org.apache.druid.catalog.CatalogException.DuplicateKeyException; -import org.apache.druid.catalog.model.SchemaRegistry.SchemaSpec; -import org.apache.druid.catalog.model.TableId; -import org.apache.druid.catalog.model.TableMetadata; -import org.apache.druid.catalog.model.TableSpec; -import org.apache.druid.catalog.storage.CatalogStorage; -import org.apache.druid.common.utils.IdUtils; -import org.apache.druid.java.util.common.IAE; -import org.apache.druid.server.security.Access; -import org.apache.druid.server.security.Action; -import org.apache.druid.server.security.AuthorizationUtils; -import org.apache.druid.server.security.AuthorizerMapper; -import org.apache.druid.server.security.ForbiddenException; -import org.apache.druid.server.security.Resource; -import org.apache.druid.server.security.ResourceAction; - -import javax.inject.Inject; -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.Consumes; -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.QueryParam; -import javax.ws.rs.core.Context; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; - -/** - * REST endpoint for user and internal catalog actions. Catalog actions - * are divided by operation: configuration-as-code, edits, retrieval, - * etc. Operations occur at the global level (all schemas), the schema level, or the - * table level. - * - * @see {@link CatalogListenerResource} for the broker-side API. - */ -public class CatalogResource2 -{ - public static final String ROOT_PATH = "/druid/coordinator/v1/catalog"; - - private final CatalogStorage catalog; - private final AuthorizerMapper authorizerMapper; - - @Inject - public CatalogResource2( - final CatalogStorage catalog, - final AuthorizerMapper authorizerMapper - ) - { - this.catalog = catalog; - this.authorizerMapper = authorizerMapper; - } - - // --------------------------------------------------------------------- - // Configuration-as-code style methods - - /** - * Create or update a new table containing the given table specification. - * Supports three use cases: - *

    - *
  • "create if not exists": default use case with no options.
  • - *
  • "create or update": set {@code overwrite=true}.
  • - *
  • "update": set {@code version} to the expected current version. - * This form enforces optimistic locking.
  • - *
- * - * @param dbSchema The name of the Druid schema, which must be writable - * and the user must have at least read access. - * @param name The name of the table definition to modify. The user must - * have write access to the table. - * @param spec The new table definition. - * @param version the expected version of an existing table. The version must - * match. If not (or if the table does not exist), returns an error. - * @param overwrite if {@code true}, then overwrites any existing table. - * If {@code false}, then the operation fails if the table already exists. - * @param req the HTTP request used for authorization. - */ - @POST - @Path("/resource/tables/{dbSchema}/{name}") - @Consumes(MediaType.APPLICATION_JSON) - @Produces(MediaType.APPLICATION_JSON) - public Response postTable( - @PathParam("dbSchema") String dbSchema, - @PathParam("name") String name, - TableSpec spec, - @QueryParam("version") long version, - @QueryParam("overwrite") boolean overwrite, - @Context final HttpServletRequest req - ) - { - try { - final SchemaSpec schema = validateSchema(dbSchema, true); - validateTableName(name); - authorizeTable(schema, name, Action.WRITE, req); - validateTableSpec(schema, name, spec); - final TableMetadata table = TableMetadata.newTable(TableId.of(dbSchema, name), spec); - try { - catalog.validate(table); - } - catch (IAE e) { - throw CatalogException.badRequest(e.getMessage()); - } - - long newVersion; - if (version != 0) { - // A version is provided. Update that version (only). - newVersion = catalog.tables().update(table, version); - } else { - try { - // No version. Create the table. - newVersion = catalog.tables().create(table); - } - catch (DuplicateKeyException e) { - // Table exists - if (overwrite) { - // User wants to overwrite, so do so. - newVersion = catalog.tables().replace(table); - } else { - throw e; - } - } - } - return okWithVersion(newVersion); - } - catch (CatalogException e) - { - return e.toResponse(); - } - } - - /** - * Retrieves a table spec. - *

- * Returns a 404 (NOT FOUND) error if the table definition does not exist. - * Note that this check is only for the specification; the table (or - * datasource) itself may exist. Similarly, this call may return a specification - * even if there is no datasource of the same name (typically occurs when - * the definition is created before the datasource itself.) - * - * @param dbSchema The Druid schema. The user must have read access. - * @param name The name of the table within the schema. The user must have - * read access. - * @param req the HTTP request used for authorization. - * @return the definition for the table, if any. - */ - @GET - @Path("/resource/tables/{dbSchema}/{name}") - @Produces(MediaType.APPLICATION_JSON) - public Response getTable( - @PathParam("dbSchema") String dbSchema, - @PathParam("name") String name, - @Context final HttpServletRequest req - ) - { - try { - final SchemaSpec schema = validateSchema(dbSchema, false); - authorizeTable(schema, name, Action.READ, req); - final TableMetadata table = catalog.tables().read(new TableId(dbSchema, name)); - return Response.ok().entity(table.spec()).build(); - } - catch (CatalogException e) - { - return e.toResponse(); - } - } - - /** - * Deletes the table definition (but not the underlying table or datasource) - * for the given schema and table. - * - * @param dbSchema The name of the schema that holds the table. - * @param name The name of the table definition to delete. The user must have - * write access. - */ - @DELETE - @Path("/tables/{dbSchema}/{name}") - @Produces(MediaType.APPLICATION_JSON) - public Response deleteTable( - @PathParam("dbSchema") String dbSchema, - @PathParam("name") String name, - @Context final HttpServletRequest req - ) - { - try { - final SchemaSpec schema = validateSchema(dbSchema, true); - authorizeTable(schema, name, Action.WRITE, req); - catalog.tables().delete(new TableId(dbSchema, name)); - return ok(); - } - catch (CatalogException e) - { - return e.toResponse(); - } - } - - // --------------------------------------------------------------------- - // Modify a table within the catalog - - @POST - @Path("/edit/tables/{dbSchema}/{name}") - @Consumes(MediaType.APPLICATION_JSON) - @Produces(MediaType.APPLICATION_JSON) - public Response editTable( - @PathParam("dbSchema") String dbSchema, - @PathParam("name") String name, - TableEditRequest editRequest, - @Context final HttpServletRequest req - ) - { - try { - final SchemaSpec schema = validateSchema(dbSchema, true); - authorizeTable(schema, name, Action.WRITE, req); - final long newVersion = new TableEditor(catalog, TableId.of(dbSchema, name), editRequest).go(); - return okWithVersion(newVersion); - } - catch (CatalogException e) - { - return e.toResponse(); - } - } - - - // --------------------------------------------------------------------- - // Helper methods - - private void validateTableName(String name) throws CatalogException - { - try { - IdUtils.validateId("table", name); - } - catch (Exception e) { - throw CatalogException.badRequest(e.getMessage()); - } - if (!name.equals(name.trim())) { - throw CatalogException.badRequest("Table name cannot start or end with spaces"); - } - } - - private void validateTableSpec(SchemaSpec schema, String name, TableSpec spec) throws CatalogException - { - // The given table spec has to be valid for the given schema. - try { - spec.validate(); - } - catch (IAE e) { - throw CatalogException.badRequest(e.getMessage()); - } - - if (!schema.accepts(spec.type())) { - throw CatalogException.badRequest( - "Cannot create tables of type %s in schema %s", - spec.type(), - schema.name() - ); - } - } - - private SchemaSpec validateSchema(String dbSchema, boolean forWrite) throws CatalogException - { - if (Strings.isNullOrEmpty(dbSchema)) { - throw CatalogException.badRequest("Schema name is required"); - } - SchemaSpec schema = catalog.resolveSchema(dbSchema); - if (schema == null) { - throw CatalogException.badRequest("Unknown schema %s", dbSchema); - } - - if (forWrite && !schema.writable()) { - throw CatalogException.badRequest( - "Cannot modify schema %s", - dbSchema - ); - } - return schema; - } - - private void authorizeTable(SchemaSpec schema, String name, Action action, HttpServletRequest request) throws CatalogException - { - if (Strings.isNullOrEmpty(name)) { - throw CatalogException.badRequest("Table name is required"); - } - if (action == Action.WRITE && !schema.writable()) { - throw new ForbiddenException( - "Cannot create table definitions in schema: " + schema.name()); - } - authorize(schema.securityResource(), name, action, request); - } - - private void authorize(String resource, String key, Action action, HttpServletRequest request) - { - final Access authResult = authorizeAccess(resource, key, action, request); - if (!authResult.isAllowed()) { - throw new ForbiddenException(authResult.toString()); - } - } - - private Access authorizeAccess(String resource, String key, Action action, HttpServletRequest request) - { - return AuthorizationUtils.authorizeResourceAction( - request, - new ResourceAction(new Resource(key, resource), action), - authorizerMapper - ); - } - - private static Response okWithVersion(long version) - { - return Response - .ok() - .entity(ImmutableMap.of("version", version)) - .build(); - } - - private static Response ok() - { - return Response.ok().build(); - } - -} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java index 27d69cce078a..b22732a1cd63 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java @@ -66,6 +66,9 @@ public long go() throws CatalogException private long hideColumns(List columns) throws CatalogException { + if (columns == null) { + return 0; + } return catalog.tables().updateProperties( id, table -> applyHiddenColumns(table, columns) @@ -97,15 +100,20 @@ private TableSpec applyHiddenColumns(TableMetadata table, List columns) Set existing = new HashSet<>(hiddenColumns); List revised = new ArrayList<>(hiddenColumns); for (String col : columns) { - if (existing.contains(col)) { + if (!existing.contains(col)) { revised.add(col); } } - return spec; + Map revisedProps = new HashMap<>(props); + revisedProps.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, revised); + return spec.withProperties(revisedProps); } private long unHideColumns(List columns) throws CatalogException { + if (columns == null) { + return 0; + } return catalog.tables().updateProperties( id, table -> applyUnhideColumns(table, columns) @@ -127,9 +135,8 @@ private TableSpec applyUnhideColumns(TableMetadata table, List columns) } final Map props = existingSpec.properties(); - final Map revised = new HashMap<>(props); @SuppressWarnings("unchecked") - List hiddenColumns = (List) revised.get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); + List hiddenColumns = (List) props.get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); if (CollectionUtils.isNullOrEmpty(hiddenColumns) || columns.isEmpty()) { return null; } @@ -140,16 +147,20 @@ private TableSpec applyUnhideColumns(TableMetadata table, List columns) revisedHiddenCols.add(col); } } - if (revised.isEmpty()) { - revised.remove(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); + final Map revisedProps = new HashMap<>(props); + if (revisedHiddenCols.isEmpty()) { + revisedProps.remove(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); } else { - revised.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, revised); + revisedProps.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, revisedHiddenCols); } - return existingSpec.withProperties(revised); + return existingSpec.withProperties(revisedProps); } private long dropColumns(List columnsToDrop) throws CatalogException { + if (columnsToDrop == null) { + return 0; + } return catalog.tables().updateColumns( id, table -> applyDropColumns(table, columnsToDrop) @@ -176,6 +187,9 @@ private TableSpec applyDropColumns(final TableMetadata table, final List private long updateProperties(Map updates) throws CatalogException { + if (updates == null) { + return 0; + } return catalog.tables().updateProperties( id, table -> applyUpdateProperties(table, updates) @@ -211,6 +225,9 @@ private TableDefn resolveDefn(String tableType) throws CatalogException private long updateColumns(final List updates) throws CatalogException { + if (updates == null) { + return 0; + } return catalog.tables().updateColumns( id, table -> applyUpdateColumns(table, updates) diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java index 649cc570a4e8..0146d0341516 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java @@ -371,7 +371,6 @@ public TableMetadata withHandle(Handle handle) throws CatalogException if (resultIterator.hasNext()) { tableSpec = resultIterator.next(); } else { - handle.rollback(); throw tableNotFound(id); } final TableSpec revised = transform.apply(TableMetadata.of(id, tableSpec)); @@ -394,7 +393,7 @@ public TableMetadata withHandle(Handle handle) throws CatalogException handle.commit(); return TableMetadata.forUpdate(id, updateTime, revised); } - catch (RuntimeException e) { + catch (Exception e) { handle.rollback(); throw e; } @@ -464,7 +463,6 @@ public TableMetadata withHandle(Handle handle) throws CatalogException if (resultIterator.hasNext()) { tableSpec = resultIterator.next(); } else { - handle.rollback(); throw tableNotFound(id); } final TableSpec revised = transform.apply(TableMetadata.of(id, tableSpec)); @@ -487,7 +485,7 @@ public TableMetadata withHandle(Handle handle) throws CatalogException handle.commit(); return TableMetadata.forUpdate(id, updateTime, revised); } - catch (RuntimeException e) { + catch (Exception e) { handle.rollback(); throw e; } diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java index 9706b3b06b70..c180ab48e32a 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java @@ -170,7 +170,7 @@ public void testUpdateProperties() throws CatalogException // Transform properties by adding a new one final long version2 = manager.updateProperties(table.id(), t -> { - TableSpec target = table.spec(); + TableSpec target = t.spec(); Map updated = new HashMap<>(target.properties()); updated.put("foo", "bar"); return target.withProperties(updated); @@ -220,7 +220,7 @@ public void testUpdateColumns() throws CatalogException // Transform columns by adding a new one final long version2 = manager.updateColumns(table.id(), t -> { - TableSpec target = table.spec(); + TableSpec target = t.spec(); List updated = new ArrayList<>(target.columns()); updated.add(colC); return target.withColumns(updated); diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogSyncTest.java similarity index 86% rename from extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java rename to extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogSyncTest.java index 629bf4f1b4a6..1d7ab3041f21 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogMetadataTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogSyncTest.java @@ -20,10 +20,12 @@ package org.apache.druid.catalog.sync; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.catalog.CatalogException; import org.apache.druid.catalog.CatalogException.DuplicateKeyException; import org.apache.druid.catalog.CatalogException.NotFoundException; import org.apache.druid.catalog.model.ColumnSpec; import org.apache.druid.catalog.model.Columns; +import org.apache.druid.catalog.model.ResolvedTable; import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; @@ -43,7 +45,10 @@ import org.junit.Rule; import org.junit.Test; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -56,7 +61,7 @@ * cached (holds a copy of the DB, based on update events) and remote * (like cached, but receives events over HTTP.) */ -public class CatalogMetadataTest +public class CatalogSyncTest { @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); @@ -126,7 +131,7 @@ public void testDirect() throws DuplicateKeyException, NotFoundException } @Test - public void testCached() throws DuplicateKeyException, NotFoundException + public void testCached() throws CatalogException { populateCatalog(); CachedMetadataCatalog catalog = new CachedMetadataCatalog(storage, storage.schemaRegistry(), jsonMapper); @@ -134,6 +139,8 @@ public void testCached() throws DuplicateKeyException, NotFoundException verifyInitial(catalog); alterCatalog(); verifyAltered(catalog); + editCatalogTable(); + verifyEdited(catalog); // Also test the deletion case TableId table2 = TableId.datasource("table2"); @@ -147,7 +154,7 @@ public void testCached() throws DuplicateKeyException, NotFoundException } @Test - public void testRemoteWithJson() throws DuplicateKeyException, NotFoundException + public void testRemoteWithJson() throws CatalogException { populateCatalog(); MockCatalogSync sync = new MockCatalogSync(storage, jsonMapper); @@ -156,6 +163,8 @@ public void testRemoteWithJson() throws DuplicateKeyException, NotFoundException verifyInitial(catalog); alterCatalog(); verifyAltered(catalog); + editCatalogTable(); + verifyEdited(catalog); // Also test the deletion case TableId table2 = TableId.datasource("table2"); @@ -326,4 +335,42 @@ private void verifyAltered(MetadataCatalog catalog) assertEquals("table2", tables.get(1).id().name()); assertEquals("table3", tables.get(2).id().name()); } + + private void editCatalogTable() throws CatalogException + { + // Edit table1: add a property + TableId id = TableId.datasource("table1"); + storage.tables().updateProperties(id, t -> { + TableSpec target = t.spec(); + Map updated = new HashMap<>(target.properties()); + updated.put("foo", "bar"); + return target.withProperties(updated); + }); + + // Edit table3: add a column + id = TableId.datasource("table3"); + storage.tables().updateColumns(id, t -> { + TableSpec target = t.spec(); + List updated = new ArrayList<>(target.columns()); + ColumnSpec colC = new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "c", Columns.DOUBLE, null); + updated.add(colC); + return target.withColumns(updated); + }); + } + + private void verifyEdited(MetadataCatalog catalog) + { + { + TableId id = TableId.datasource("table1"); + DatasourceFacade ds = new DatasourceFacade(catalog.resolveTable(id)); + assertEquals("P1D", ds.segmentGranularityString()); + assertEquals("bar", ds.stringProperty("foo")); + } + { + TableId id = TableId.datasource("table3"); + ResolvedTable table = catalog.resolveTable(id); + assertEquals(3, table.spec().columns().size()); + assertEquals("c", table.spec().columns().get(2).name()); + } + } } diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java index a62fd94dfcaf..e09de234c83f 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java @@ -20,8 +20,11 @@ package org.apache.druid.server.http.catalog; import org.apache.druid.catalog.http.CatalogResource; -import org.apache.druid.catalog.http.HideColumns; import org.apache.druid.catalog.http.MoveColumn; +import org.apache.druid.catalog.http.TableEditRequest; +import org.apache.druid.catalog.http.TableEditRequest.DropColumns; +import org.apache.druid.catalog.http.TableEditRequest.HideColumns; +import org.apache.druid.catalog.http.TableEditRequest.UnhideColumns; import org.apache.druid.catalog.model.CatalogUtils; import org.apache.druid.catalog.model.Columns; import org.apache.druid.catalog.model.TableId; @@ -33,6 +36,7 @@ import org.apache.druid.catalog.model.table.TableBuilder; import org.apache.druid.catalog.storage.CatalogTests; import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.server.security.ForbiddenException; import org.junit.After; import org.junit.Before; import org.junit.Rule; @@ -50,6 +54,7 @@ import static org.apache.druid.server.http.catalog.DummyRequest.postBy; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; /** @@ -90,51 +95,50 @@ public void testCreate() TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D").buildSpec(); // Blank schema name: infer the schema. - Response resp = resource.postTable("", tableName, dsSpec, null, 0, postBy(CatalogTests.SUPER_USER)); + Response resp = resource.postTable("", tableName, dsSpec, 0, false, postBy(CatalogTests.SUPER_USER)); assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); // Missing table name - resp = resource.postTable(TableId.DRUID_SCHEMA, "", dsSpec, null, 0, postBy(CatalogTests.SUPER_USER)); + resp = resource.postTable(TableId.DRUID_SCHEMA, "", dsSpec, 0, false, postBy(CatalogTests.SUPER_USER)); assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); // Invalid table name - resp = resource.postTable(TableId.DRUID_SCHEMA, " bogus ", dsSpec, null, 0, postBy(CatalogTests.SUPER_USER)); + resp = resource.postTable(TableId.DRUID_SCHEMA, " bogus ", dsSpec, 0, false, postBy(CatalogTests.SUPER_USER)); assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); // Unknown schema - resp = resource.postTable("bogus", tableName, dsSpec, null, 0, postBy(CatalogTests.SUPER_USER)); + resp = resource.postTable("bogus", tableName, dsSpec, 0, false, postBy(CatalogTests.SUPER_USER)); assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); // Immutable schema - resp = resource.postTable(TableId.CATALOG_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.SUPER_USER)); + resp = resource.postTable(TableId.CATALOG_SCHEMA, tableName, dsSpec, 0, false, postBy(CatalogTests.SUPER_USER)); assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); // Wrong definition type. - resp = resource.postTable(TableId.EXTERNAL_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.SUPER_USER)); + resp = resource.postTable(TableId.EXTERNAL_SCHEMA, tableName, dsSpec, 0, false, postBy(CatalogTests.SUPER_USER)); assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); // No permissions - resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.DENY_USER)); - assertEquals(Response.Status.FORBIDDEN.getStatusCode(), resp.getStatus()); + assertThrows( + ForbiddenException.class, + () -> resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, 0, false, postBy(CatalogTests.DENY_USER)) + ); // Read permission - resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.READER_USER)); - assertEquals(Response.Status.FORBIDDEN.getStatusCode(), resp.getStatus()); + assertThrows( + ForbiddenException.class, + () -> resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, 0, false, postBy(CatalogTests.READER_USER)) + ); // Write permission - resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, 0, false, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); assertTrue(getVersion(resp) > 0); // Duplicate - resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, 0, false, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); - // Duplicate, "if not exists" - resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "ifnew", 0, postBy(CatalogTests.WRITER_USER)); - assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - assertEquals(0, getVersion(resp)); - // Inline input source TableSpec inputSpec = TableBuilder.externalTable(InlineTableDefn.TABLE_TYPE, "inline") .format(InputFormats.CSV_FORMAT_TYPE) @@ -143,11 +147,11 @@ public void testCreate() .column("b", Columns.VARCHAR) .column("c", Columns.BIGINT) .buildSpec(); - resp = resource.postTable(TableId.EXTERNAL_SCHEMA, "inline", inputSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + resp = resource.postTable(TableId.EXTERNAL_SCHEMA, "inline", inputSpec, 0, false, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); // Wrong spec type - resp = resource.postTable(TableId.DRUID_SCHEMA, "invalid", inputSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + resp = resource.postTable(TableId.DRUID_SCHEMA, "invalid", inputSpec, 0, false, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); } @@ -158,47 +162,32 @@ public void testUpdate() TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D").buildSpec(); // Does not exist - Response resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "replace", 0, postBy(CatalogTests.SUPER_USER)); + Response resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, 10, false, postBy(CatalogTests.SUPER_USER)); assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); // Create the table - resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, 0, false, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); long version = getVersion(resp); // No update permission - resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "replace", 0, postBy(CatalogTests.READER_USER)); - assertEquals(Response.Status.FORBIDDEN.getStatusCode(), resp.getStatus()); + assertThrows( + ForbiddenException.class, + () -> resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, 0, false, postBy(CatalogTests.READER_USER)) + ); // Out-of-date version - resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "replace", 10, postBy(CatalogTests.WRITER_USER)); - assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, 10, false, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); // Valid version - resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "replace", version, postBy(CatalogTests.WRITER_USER)); + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, version, false, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); assertTrue(getVersion(resp) > version); version = getVersion(resp); // Overwrite - resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "replace", 0, postBy(CatalogTests.WRITER_USER)); - assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - assertTrue(getVersion(resp) > version); - } - - @Test - public void testForce() - { - final String tableName = "force"; - TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D").buildSpec(); - - // Create the table - Response resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "force", 0, postBy(CatalogTests.WRITER_USER)); - assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - long version = getVersion(resp); - - // Overwrite - resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, "force", 0, postBy(CatalogTests.WRITER_USER)); + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, 0, true, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); assertTrue(getVersion(resp) > version); } @@ -226,13 +215,15 @@ public void testRead() assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); // Create the table - resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, 0, false, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); long version = getVersion(resp); // No read permission - resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, getBy(CatalogTests.DENY_USER)); - assertEquals(Response.Status.FORBIDDEN.getStatusCode(), resp.getStatus()); + assertThrows( + ForbiddenException.class, + () -> resource.getTable(TableId.DRUID_SCHEMA, tableName, getBy(CatalogTests.DENY_USER)) + ); // Valid resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, getBy(CatalogTests.READER_USER)); @@ -292,7 +283,7 @@ public void testList() // Create a table final String tableName = "list"; TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D").buildSpec(); - resp = resource.postTable(TableId.DRUID_SCHEMA, "list", dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + resp = resource.postTable(TableId.DRUID_SCHEMA, "list", dsSpec, 0, false, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); // No read access @@ -338,46 +329,42 @@ public void testDelete() { // Missing schema name String tableName = "delete"; - Response resp = resource.deleteTable("", tableName, false, deleteBy(CatalogTests.SUPER_USER)); + Response resp = resource.deleteTable("", tableName, deleteBy(CatalogTests.SUPER_USER)); assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); // Missing table name - resp = resource.deleteTable(TableId.DRUID_SCHEMA, null, false, deleteBy(CatalogTests.SUPER_USER)); + resp = resource.deleteTable(TableId.DRUID_SCHEMA, null, deleteBy(CatalogTests.SUPER_USER)); assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); // Unknown schema - resp = resource.deleteTable("bogus", tableName, false, deleteBy(CatalogTests.SUPER_USER)); + resp = resource.deleteTable("bogus", tableName, deleteBy(CatalogTests.SUPER_USER)); assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); // Immutable schema - resp = resource.deleteTable(TableId.CATALOG_SCHEMA, tableName, false, deleteBy(CatalogTests.SUPER_USER)); + resp = resource.deleteTable(TableId.CATALOG_SCHEMA, tableName, deleteBy(CatalogTests.SUPER_USER)); assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); // Does not exist - resp = resource.deleteTable(TableId.DRUID_SCHEMA, tableName, false, deleteBy(CatalogTests.SUPER_USER)); + resp = resource.deleteTable(TableId.DRUID_SCHEMA, tableName, deleteBy(CatalogTests.SUPER_USER)); assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); - resp = resource.deleteTable(TableId.DRUID_SCHEMA, tableName, true, deleteBy(CatalogTests.SUPER_USER)); - assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - // Create the table TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D").buildSpec(); - resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, 0, false, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); // No write permission - resp = resource.deleteTable(TableId.DRUID_SCHEMA, tableName, false, deleteBy(CatalogTests.READER_USER)); - assertEquals(Response.Status.FORBIDDEN.getStatusCode(), resp.getStatus()); + assertThrows( + ForbiddenException.class, + () -> resource.deleteTable(TableId.DRUID_SCHEMA, tableName, deleteBy(CatalogTests.READER_USER)) + ); // Write permission - resp = resource.deleteTable(TableId.DRUID_SCHEMA, tableName, false, deleteBy(CatalogTests.WRITER_USER)); + resp = resource.deleteTable(TableId.DRUID_SCHEMA, tableName, deleteBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - resp = resource.deleteTable(TableId.DRUID_SCHEMA, tableName, false, deleteBy(CatalogTests.WRITER_USER)); + resp = resource.deleteTable(TableId.DRUID_SCHEMA, tableName, deleteBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); - - resp = resource.deleteTable(TableId.DRUID_SCHEMA, tableName, true, deleteBy(CatalogTests.WRITER_USER)); - assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); } @Test @@ -386,7 +373,7 @@ public void testLifecycle() // Operations for one table - create String table1Name = "lifecycle1"; TableSpec dsSpec = TableBuilder.datasource(table1Name, "P1D").buildSpec(); - Response resp = resource.postTable(TableId.DRUID_SCHEMA, table1Name, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + Response resp = resource.postTable(TableId.DRUID_SCHEMA, table1Name, dsSpec, 0, true, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); long version = getVersion(resp); @@ -414,7 +401,7 @@ public void testLifecycle() // update TableSpec table2Spec = TableBuilder.datasource(table1Name, "PT1H").buildSpec(); - resp = resource.postTable(TableId.DRUID_SCHEMA, table1Name, table2Spec, "replace", version, postBy(CatalogTests.WRITER_USER)); + resp = resource.postTable(TableId.DRUID_SCHEMA, table1Name, table2Spec, version, false, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); assertTrue(getVersion(resp) > version); version = getVersion(resp); @@ -429,7 +416,7 @@ public void testLifecycle() // add second table String table2Name = "lifecycle2"; - resp = resource.postTable(TableId.DRUID_SCHEMA, table2Name, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + resp = resource.postTable(TableId.DRUID_SCHEMA, table2Name, dsSpec, 0, false, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); TableId id2 = TableId.of(TableId.DRUID_SCHEMA, table2Name); @@ -449,7 +436,7 @@ public void testLifecycle() assertEquals(id2.name(), tables.get(1)); // delete and verify - resp = resource.deleteTable(TableId.DRUID_SCHEMA, table1Name, false, deleteBy(CatalogTests.WRITER_USER)); + resp = resource.deleteTable(TableId.DRUID_SCHEMA, table1Name, deleteBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); resp = resource.listTableNamesForSchema(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); @@ -457,7 +444,7 @@ public void testLifecycle() tables = getTableList(resp); assertEquals(1, tables.size()); - resp = resource.deleteTable(TableId.DRUID_SCHEMA, table2Name, false, deleteBy(CatalogTests.WRITER_USER)); + resp = resource.deleteTable(TableId.DRUID_SCHEMA, table2Name, deleteBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); resp = resource.listTableNamesForSchema(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); @@ -475,32 +462,32 @@ public void testMoveColumn() .column("b", "BIGINT") .column("c", "FLOAT") .buildSpec(); - Response resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + Response resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, 0, false, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); long version = getVersion(resp); // Bad schema MoveColumn cmd = new MoveColumn("foo", MoveColumn.Position.FIRST, null); - resp = resource.moveColumn("bogus", tableName, cmd, postBy(CatalogTests.WRITER_USER)); + resp = resource.editTable("bogus", tableName, cmd, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); // Bad table - resp = resource.moveColumn(TableId.DRUID_SCHEMA, "bogus", cmd, postBy(CatalogTests.WRITER_USER)); + resp = resource.editTable(TableId.DRUID_SCHEMA, "bogus", cmd, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); // No target column cmd = new MoveColumn(null, MoveColumn.Position.FIRST, null); - resp = resource.moveColumn(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); + resp = resource.editTable(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); // No anchor column cmd = new MoveColumn("a", MoveColumn.Position.BEFORE, null); - resp = resource.moveColumn(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); + resp = resource.editTable(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); // Move first cmd = new MoveColumn("c", MoveColumn.Position.FIRST, null); - resp = resource.moveColumn(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); + resp = resource.editTable(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); assertTrue(getVersion(resp) > version); @@ -511,8 +498,6 @@ public void testMoveColumn() Arrays.asList("c", "a", "b"), CatalogUtils.columnNames(read.spec().columns()) ); - - // Other cases are tested in CommandTest since all the REST plumbing is the same } @Test @@ -521,21 +506,21 @@ public void testHideColumns() String tableName = "hide"; TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D") .buildSpec(); - Response resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + Response resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, 0, false, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); long version = getVersion(resp); // Bad schema - HideColumns cmd = new HideColumns(null, null); - resp = resource.hideColumns("bogus", tableName, cmd, postBy(CatalogTests.WRITER_USER)); + TableEditRequest cmd = new HideColumns(null); + resp = resource.editTable("bogus", tableName, cmd, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); - // Bad table - resp = resource.hideColumns(TableId.DRUID_SCHEMA, "bogus", cmd, postBy(CatalogTests.WRITER_USER)); - assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + // Bad table. OK because there is nothing to do. + resp = resource.editTable(TableId.DRUID_SCHEMA, "bogus", cmd, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); // Nothing to do - resp = resource.hideColumns(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); + resp = resource.editTable(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, postBy(CatalogTests.READER_USER)); @@ -543,8 +528,11 @@ public void testHideColumns() assertNull(read.spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY)); // Hide - cmd = new HideColumns(Arrays.asList("a", "b"), null); - resp = resource.hideColumns(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); + cmd = new HideColumns(Arrays.asList("a", "b")); + resp = resource.editTable(TableId.DRUID_SCHEMA, "bogus", cmd, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + resp = resource.editTable(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, postBy(CatalogTests.READER_USER)); @@ -555,20 +543,18 @@ public void testHideColumns() ); assertTrue(read.updateTime() > version); - // Unhide + hide - cmd = new HideColumns(Arrays.asList("b", "c"), Arrays.asList("a", "e")); - resp = resource.hideColumns(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); + // Unhide + cmd = new UnhideColumns(Arrays.asList("a", "e")); + resp = resource.editTable(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, postBy(CatalogTests.READER_USER)); read = (TableMetadata) resp.getEntity(); assertEquals( - Arrays.asList("b", "c"), + Arrays.asList("b"), read.spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) ); assertTrue(read.updateTime() > version); - - // Other cases are tested in CommandTest } @Test @@ -581,20 +567,25 @@ public void testDropColumns() .column("c", "FLOAT") .buildSpec(); - Response resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, null, 0, postBy(CatalogTests.WRITER_USER)); + Response resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, 0, false, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); long version = getVersion(resp); // Bad schema - resp = resource.dropColumns("bogus", tableName, Collections.emptyList(), postBy(CatalogTests.WRITER_USER)); + DropColumns cmd = new DropColumns(Collections.emptyList()); + resp = resource.editTable("bogus", tableName, cmd, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); // Bad table - resp = resource.dropColumns(TableId.DRUID_SCHEMA, "bogus", Collections.emptyList(), postBy(CatalogTests.WRITER_USER)); + resp = resource.editTable(TableId.DRUID_SCHEMA, "bogus", cmd, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); // Nothing to do - resp = resource.dropColumns(TableId.DRUID_SCHEMA, tableName, Collections.emptyList(), postBy(CatalogTests.WRITER_USER)); + resp = resource.editTable(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + cmd = new DropColumns(null); + resp = resource.editTable(TableId.DRUID_SCHEMA, "bogus", cmd, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, postBy(CatalogTests.READER_USER)); @@ -605,7 +596,8 @@ public void testDropColumns() ); // Drop - resp = resource.dropColumns(TableId.DRUID_SCHEMA, tableName, Arrays.asList("a", "c"), postBy(CatalogTests.WRITER_USER)); + cmd = new DropColumns(Arrays.asList("a", "c")); + resp = resource.editTable(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, postBy(CatalogTests.READER_USER)); From 5890a4902f22baa68b58549929982f322c6f567c Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Mon, 31 Oct 2022 13:04:18 -0700 Subject: [PATCH 14/22] Snapshot --- .../server/http/catalog/CommandTest.java | 149 ---------- .../druid/server/http/catalog/EditorTest.java | 259 ++++++++++++++++++ .../druid/catalog/model/TableMetadata.java | 12 + 3 files changed, 271 insertions(+), 149 deletions(-) delete mode 100644 extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java create mode 100644 extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/EditorTest.java diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java deleted file mode 100644 index cf4054d5ff74..000000000000 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CommandTest.java +++ /dev/null @@ -1,149 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.server.http.catalog; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.catalog.http.HideColumns; -import org.apache.druid.catalog.http.MoveColumn; -import org.apache.druid.catalog.model.CatalogUtils; -import org.apache.druid.catalog.model.ColumnSpec; -import org.apache.druid.catalog.model.TableSpec; -import org.apache.druid.catalog.model.table.TableBuilder; -import org.apache.druid.java.util.common.jackson.JacksonUtils; -import org.junit.Test; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; - -public class CommandTest -{ - private ObjectMapper mapper = new ObjectMapper(); - - @Test - public void testMoveColumn() - { - TableSpec dsSpec = TableBuilder.datasource("foo", "P1D") - .column("a", "VARCHAR") - .column("b", "BIGINT") - .column("c", "FLOAT") - .buildSpec(); - - // Move first - MoveColumn cmd = new MoveColumn("c", MoveColumn.Position.FIRST, null); - List revised = cmd.perform(dsSpec.columns()); - assertEquals( - Arrays.asList("c", "a", "b"), - CatalogUtils.columnNames(revised) - ); - - // Move last - cmd = new MoveColumn("a", MoveColumn.Position.LAST, null); - revised = cmd.perform(dsSpec.columns()); - assertEquals( - Arrays.asList("b", "c", "a"), - CatalogUtils.columnNames(revised) - ); - - // Move before, earlier anchor - cmd = new MoveColumn("c", MoveColumn.Position.BEFORE, "b"); - revised = cmd.perform(dsSpec.columns()); - assertEquals( - Arrays.asList("a", "c", "b"), - CatalogUtils.columnNames(revised) - ); - - // Move before, later anchor - cmd = new MoveColumn("a", MoveColumn.Position.BEFORE, "c"); - revised = cmd.perform(dsSpec.columns()); - assertEquals( - Arrays.asList("b", "a", "c"), - CatalogUtils.columnNames(revised) - ); - - // Move after, earlier anchor - cmd = new MoveColumn("c", MoveColumn.Position.AFTER, "a"); - revised = cmd.perform(dsSpec.columns()); - assertEquals( - Arrays.asList("a", "c", "b"), - CatalogUtils.columnNames(revised) - ); - - // Move after, later anchor - cmd = new MoveColumn("a", MoveColumn.Position.AFTER, "b"); - revised = cmd.perform(dsSpec.columns()); - assertEquals( - Arrays.asList("b", "a", "c"), - CatalogUtils.columnNames(revised) - ); - } - - @Test - public void testHideColumns() - { - // Everything is null - HideColumns cmd = new HideColumns(null, null); - List revised = cmd.perform(null); - assertNull(revised); - - // Unhide from null list - cmd = new HideColumns(null, Collections.singletonList("a")); - revised = cmd.perform(null); - assertNull(revised); - - // And from an empty list - cmd = new HideColumns(null, Collections.singletonList("a")); - revised = cmd.perform(Collections.emptyList()); - assertNull(revised); - - // Hide starting from a null list. - cmd = new HideColumns(Arrays.asList("a", "b"), null); - revised = cmd.perform(null); - assertEquals(Arrays.asList("a", "b"), revised); - - // Hide starting from an empty list. - cmd = new HideColumns(Arrays.asList("a", "b"), Collections.emptyList()); - revised = cmd.perform(Collections.emptyList()); - assertEquals(Arrays.asList("a", "b"), revised); - - // Hide with existing columns - cmd = new HideColumns(Arrays.asList("b", "d"), null); - revised = cmd.perform(Arrays.asList("a", "b", "c")); - assertEquals(Arrays.asList("a", "b", "c", "d"), revised); - - // Hide with existing columns - cmd = new HideColumns(Arrays.asList("b", "d"), null); - revised = cmd.perform(Arrays.asList("a", "b", "c")); - assertEquals(Arrays.asList("a", "b", "c", "d"), revised); - - // Unhide existing columns - cmd = new HideColumns(null, Arrays.asList("b", "d")); - revised = cmd.perform(Arrays.asList("a", "b", "c")); - assertEquals(Arrays.asList("a", "c"), revised); - - // Both hide and unhide. Hide takes precedence. - cmd = new HideColumns(Arrays.asList("b", "d", "e"), Arrays.asList("c", "d")); - revised = cmd.perform(Arrays.asList("a", "b", "c")); - assertEquals(Arrays.asList("a", "b", "d", "e"), revised); - } -} diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/EditorTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/EditorTest.java new file mode 100644 index 000000000000..518f75708267 --- /dev/null +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/EditorTest.java @@ -0,0 +1,259 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.http.catalog; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.vavr.collection.Map; +import org.apache.druid.catalog.CatalogException; +import org.apache.druid.catalog.http.CatalogResource; +import org.apache.druid.catalog.http.MoveColumn; +import org.apache.druid.catalog.http.TableEditRequest; +import org.apache.druid.catalog.http.TableEditRequest.HideColumns; +import org.apache.druid.catalog.http.TableEditor; +import org.apache.druid.catalog.model.CatalogUtils; +import org.apache.druid.catalog.model.ColumnSpec; +import org.apache.druid.catalog.model.TableId; +import org.apache.druid.catalog.model.TableMetadata; +import org.apache.druid.catalog.model.TableSpec; +import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.catalog.model.table.TableBuilder; +import org.apache.druid.catalog.storage.CatalogStorage; +import org.apache.druid.catalog.storage.CatalogTests; +import org.apache.druid.catalog.storage.sql.CatalogManager; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.metadata.TestDerbyConnector; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import javax.ws.rs.core.Response; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; + +import static org.apache.druid.server.http.catalog.DummyRequest.getBy; +import static org.apache.druid.server.http.catalog.DummyRequest.postBy; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +public class EditorTest +{ + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + + private CatalogTests.DbFixture dbFixture; + private CatalogStorage catalog; + + @Before + public void setUp() + { + dbFixture = new CatalogTests.DbFixture(derbyConnectorRule); + catalog = dbFixture.storage; + } + + @After + public void tearDown() + { + CatalogTests.tearDown(dbFixture); + } + + private TableMetadata doEdit(String tableName, TableEditRequest cmd) throws CatalogException + { + final TableId id = TableId.datasource(tableName); + new TableEditor(catalog, id, cmd).go(); + return catalog.tables().read(id); + } + + @Test + public void testMoveColumn() throws CatalogException + { + final String tableName = "table1"; + TableMetadata table = TableBuilder.datasource(tableName, "P1D") + .column("a", "VARCHAR") + .column("b", "BIGINT") + .column("c", "FLOAT") + .build(); + catalog.tables().create(table); + + // Move first + MoveColumn cmd = new MoveColumn("c", MoveColumn.Position.FIRST, null); + List revised = doEdit(tableName, cmd).spec().columns(); + assertEquals( + Arrays.asList("c", "a", "b"), + CatalogUtils.columnNames(revised) + ); + + // Move last + catalog.tables().replace(table); + cmd = new MoveColumn("a", MoveColumn.Position.LAST, null); + revised = doEdit(tableName, cmd).spec().columns(); + assertEquals( + Arrays.asList("b", "c", "a"), + CatalogUtils.columnNames(revised) + ); + + // Move before, earlier anchor + catalog.tables().replace(table); + cmd = new MoveColumn("c", MoveColumn.Position.BEFORE, "b"); + revised = doEdit(tableName, cmd).spec().columns(); + assertEquals( + Arrays.asList("a", "c", "b"), + CatalogUtils.columnNames(revised) + ); + + // Move before, later anchor + catalog.tables().replace(table); + cmd = new MoveColumn("a", MoveColumn.Position.BEFORE, "c"); + revised = doEdit(tableName, cmd).spec().columns(); + assertEquals( + Arrays.asList("b", "a", "c"), + CatalogUtils.columnNames(revised) + ); + + // Move after, earlier anchor + catalog.tables().replace(table); + cmd = new MoveColumn("c", MoveColumn.Position.AFTER, "a"); + revised = doEdit(tableName, cmd).spec().columns(); + assertEquals( + Arrays.asList("a", "c", "b"), + CatalogUtils.columnNames(revised) + ); + + // Move after, later anchor + catalog.tables().replace(table); + cmd = new MoveColumn("a", MoveColumn.Position.AFTER, "b"); + revised = doEdit(tableName, cmd).spec().columns(); + assertEquals( + Arrays.asList("b", "a", "c"), + CatalogUtils.columnNames(revised) + ); + } + + @Test + public void testHideColumns() + { + final String tableName = "table2"; + TableMetadata table = TableBuilder.datasource(tableName, "P1D") + .build(); + catalog.tables().create(table); + + // Null list + HideColumns cmd = new HideColumns(null); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); + + // Empty list + cmd = new HideColumns(Collections.emptyList()); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); + + // Hide starting from a null list. + cmd = new HideColumns(Arrays.asList("a", "b")); + assertEquals( + Arrays.asList("a", "b"), + doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + + Map props = new HashMap<>(table.spec().properties()); + props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList()); + catalog.tables().replace(table.withProperties(props)); + // Hide starting from an empty list. + cmd = new HideColumns(Arrays.asList("a", "b"), Collections.emptyList()); + revised = cmd.perform(Collections.emptyList()); + assertEquals(Arrays.asList("a", "b"), revised); + + // Hide with existing columns + cmd = new HideColumns(Arrays.asList("b", "d"), null); + revised = cmd.perform(Arrays.asList("a", "b", "c")); + assertEquals(Arrays.asList("a", "b", "c", "d"), revised); + + // Hide with existing columns + cmd = new HideColumns(Arrays.asList("b", "d"), null); + revised = cmd.perform(Arrays.asList("a", "b", "c")); + assertEquals(Arrays.asList("a", "b", "c", "d"), revised); + + // Unhide existing columns + cmd = new HideColumns(null, Arrays.asList("b", "d")); + revised = cmd.perform(Arrays.asList("a", "b", "c")); + assertEquals(Arrays.asList("a", "c"), revised); + + // Both hide and unhide. Hide takes precedence. + cmd = new HideColumns(Arrays.asList("b", "d", "e"), Arrays.asList("c", "d")); + revised = cmd.perform(Arrays.asList("a", "b", "c")); + assertEquals(Arrays.asList("a", "b", "d", "e"), revised); + + // Duplicates + } + + @Test + public void testUnhideColumns() + { + final String tableName = "table2"; + TableMetadata table = TableBuilder.datasource(tableName, "P1D") + .build(); + catalog.tables().create(table); + + // Everything is null + HideColumns cmd = new HideColumns(null, null); + List revised = cmd.perform(null); + assertNull(revised); + + // Unhide from null list + cmd = new HideColumns(null, Collections.singletonList("a")); + revised = cmd.perform(null); + assertNull(revised); + + // And from an empty list + cmd = new HideColumns(null, Collections.singletonList("a")); + revised = cmd.perform(Collections.emptyList()); + assertNull(revised); + + // Hide starting from a null list. + cmd = new HideColumns(Arrays.asList("a", "b"), null); + revised = cmd.perform(null); + assertEquals(Arrays.asList("a", "b"), revised); + + // Hide starting from an empty list. + cmd = new HideColumns(Arrays.asList("a", "b"), Collections.emptyList()); + revised = cmd.perform(Collections.emptyList()); + assertEquals(Arrays.asList("a", "b"), revised); + + // Hide with existing columns + cmd = new HideColumns(Arrays.asList("b", "d"), null); + revised = cmd.perform(Arrays.asList("a", "b", "c")); + assertEquals(Arrays.asList("a", "b", "c", "d"), revised); + + // Hide with existing columns + cmd = new HideColumns(Arrays.asList("b", "d"), null); + revised = cmd.perform(Arrays.asList("a", "b", "c")); + assertEquals(Arrays.asList("a", "b", "c", "d"), revised); + + // Unhide existing columns + cmd = new HideColumns(null, Arrays.asList("b", "d")); + revised = cmd.perform(Arrays.asList("a", "b", "c")); + assertEquals(Arrays.asList("a", "c"), revised); + + // Both hide and unhide. Hide takes precedence. + cmd = new HideColumns(Arrays.asList("b", "d", "e"), Arrays.asList("c", "d")); + revised = cmd.perform(Arrays.asList("a", "b", "c")); + assertEquals(Arrays.asList("a", "b", "d", "e"), revised); + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java index b2b6e6c962cd..8af5b1135c2b 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java @@ -25,6 +25,7 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; +import java.util.Map; import java.util.Objects; /** @@ -199,6 +200,17 @@ public TableMetadata withProperties(TableMetadata update) ); } + public TableMetadata withProperties(Map props) + { + return new TableMetadata( + id, + creationTime, + updateTime, + state, + spec.withProperties(props) + ); + } + @JsonProperty("id") public TableId id() { From 7adc308f1f5c163fea48cbf91547ac032c2ad77a Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Tue, 1 Nov 2022 09:28:25 -0700 Subject: [PATCH 15/22] Updated UTs and ITs --- .../druid/catalog/CatalogException.java | 19 + .../druid/catalog/http/CatalogResource.java | 285 +++++++++----- .../druid/catalog/http/TableEditRequest.java | 29 +- .../druid/catalog/http/TableEditor.java | 78 +++- .../druid/catalog/storage/CatalogStorage.java | 8 +- .../storage/sql/SQLCatalogManager.java | 10 +- .../catalog/sync/CachedMetadataCatalog.java | 2 +- .../catalog/sync/CatalogUpdateListener.java | 21 +- .../druid/catalog/sync/UpdateEvent.java | 19 + .../druid/catalog/sync/CatalogSyncTest.java | 12 +- .../http/catalog/CatalogResourceTest.java | 162 ++++++-- .../druid/server/http/catalog/EditorTest.java | 352 ++++++++++++++---- .../testsEx/catalog/ITCatalogRestTest.java | 30 +- .../druid/testsEx/cluster/CatalogClient.java | 62 +-- .../druid/catalog/model/SchemaRegistry.java | 2 + .../catalog/model/SchemaRegistryImpl.java | 13 + .../apache/druid/catalog/model/TableDefn.java | 11 +- .../catalog/model/TableDefnRegistry.java | 5 + .../catalog/model/table/TableBuilder.java | 2 +- .../catalog/model/TableMetadataTest.java | 4 - .../model/table/HttpInputTableTest.java | 1 - .../catalog/model/table/InlineTableTest.java | 1 - .../catalog/model/table/LocalTableTest.java | 1 - 23 files changed, 815 insertions(+), 314 deletions(-) diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/CatalogException.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/CatalogException.java index 253b00d53c35..c09fe35a5a14 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/CatalogException.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/CatalogException.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.catalog; import com.google.common.collect.ImmutableMap; diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java index d226ec6cdfa1..5f5de8861f1e 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.catalog.http; import com.google.common.base.Strings; @@ -13,6 +32,8 @@ import org.apache.druid.catalog.storage.CatalogStorage; import org.apache.druid.common.utils.IdUtils; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthorizationUtils; @@ -36,8 +57,10 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.stream.Collectors; /** * REST endpoint for user and internal catalog actions. Catalog actions @@ -52,6 +75,11 @@ public class CatalogResource { public static final String ROOT_PATH = "/druid/coordinator/v1/catalog"; + public static final String NAME_FORMAT = "name"; + public static final String PATH_FORMAT = "path"; + public static final String METADATA_FORMAT = "metadata"; + public static final String STATUS_FORMAT = "status"; + private final CatalogStorage catalog; private final AuthorizerMapper authorizerMapper; @@ -90,7 +118,7 @@ public CatalogResource( * @param req the HTTP request used for authorization. */ @POST - @Path("/schemas/{schema}/{name}") + @Path("/schemas/{schema}/tables/{name}") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) public Response postTable( @@ -136,8 +164,7 @@ public Response postTable( } return okWithVersion(newVersion); } - catch (CatalogException e) - { + catch (CatalogException e) { return e.toResponse(); } } @@ -158,7 +185,7 @@ public Response postTable( * @return the definition for the table, if any. */ @GET - @Path("/schemas/{schema}/{name}") + @Path("/schemas/{schema}/tables/{name}") @Produces(MediaType.APPLICATION_JSON) public Response getTable( @PathParam("schema") String dbSchema, @@ -172,8 +199,7 @@ public Response getTable( final TableMetadata table = catalog.tables().read(new TableId(dbSchema, name)); return Response.ok().entity(table).build(); } - catch (CatalogException e) - { + catch (CatalogException e) { return e.toResponse(); } } @@ -187,7 +213,7 @@ public Response getTable( * write access. */ @DELETE - @Path("/schemas/{schema}/{name}") + @Path("/schemas/{schema}/tables/{name}") @Produces(MediaType.APPLICATION_JSON) public Response deleteTable( @PathParam("schema") String dbSchema, @@ -201,8 +227,7 @@ public Response deleteTable( catalog.tables().delete(new TableId(dbSchema, name)); return ok(); } - catch (CatalogException e) - { + catch (CatalogException e) { return e.toResponse(); } } @@ -211,7 +236,7 @@ public Response deleteTable( // Modify a table within the catalog @POST - @Path("/schemas/{schema}/{name}/edit") + @Path("/schemas/{schema}/tables/{name}/edit") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) public Response editTable( @@ -227,59 +252,44 @@ public Response editTable( final long newVersion = new TableEditor(catalog, TableId.of(dbSchema, name), editRequest).go(); return okWithVersion(newVersion); } - catch (CatalogException e) - { + catch (CatalogException e) { return e.toResponse(); } } // --------------------------------------------------------------------- - // Query names and table specs + // Retrieval /** - * Retrieves the list of all Druid schema names. At present, Druid does - * not impose security on schemas, only tables within schemas. + * Retrieves the list of all Druid schema names, all table names, or + * all table metadata. */ @GET - @Path("/names/schemas") + @Path("/schemas") @Produces(MediaType.APPLICATION_JSON) - public Response listSchemas( + public Response getSchemas( + @QueryParam("format") String format, @Context final HttpServletRequest req ) { - // No good resource to use: we really need finer-grain control. - authorizeAccess(ResourceType.STATE, "schemas", Action.READ, req); - return Response.ok().entity(catalog.schemaRegistry().names()).build(); - } - - - /** - * Retrieves the list of all Druid table names for which the user has at - * least read access. - */ - @GET - @Path("/names/tables") - @Produces(MediaType.APPLICATION_JSON) - public Response listTableNames( - @Context final HttpServletRequest req - ) - { - List tables = catalog.tables().allTablePaths(); - Iterable filtered = AuthorizationUtils.filterAuthorizedResources( - req, - tables, - tableId -> { - SchemaSpec schema = catalog.resolveSchema(tableId.schema()); - if (schema == null) { - // Should never occur. - return null; - } - return Collections.singletonList( - resourceAction(schema, tableId.name(), Action.READ)); - }, - authorizerMapper - ); - return Response.ok().entity(Lists.newArrayList(filtered)).build(); + try { + format = Strings.isNullOrEmpty(format) ? NAME_FORMAT : StringUtils.toLowerCase(format); + switch (format) { + case NAME_FORMAT: + // No good resource to use: we really need finer-grain control. + authorizeAccess(ResourceType.STATE, "schemas", Action.READ, req); + return Response.ok().entity(catalog.schemaRegistry().names()).build(); + case PATH_FORMAT: + return listTablePaths(req); + case METADATA_FORMAT: + return listAllTableMetadata(req); + default: + throw CatalogException.badRequest("Unknown format: [%s]", format); + } + } + catch (CatalogException e) { + return e.toResponse(); + } } /** @@ -291,62 +301,29 @@ public Response listTableNames( * @param dbSchema The Druid schema to query. The user must have read access. */ @GET - @Path("/names/schemas/{schema}") + @Path("/schemas/{schema}/tables") @Produces(MediaType.APPLICATION_JSON) - public Response listTableNamesForSchema( + public Response getSchemaTables( @PathParam("schema") String dbSchema, + @QueryParam("format") String format, @Context final HttpServletRequest req ) { try { SchemaSpec schema = validateSchema(dbSchema, false); - List tables = catalog.tables().tableNamesInSchema(dbSchema); - Iterable filtered = AuthorizationUtils.filterAuthorizedResources( - req, - tables, - name -> - Collections.singletonList( - resourceAction(schema, name, Action.READ)), - authorizerMapper - ); - return Response.ok().entity(Lists.newArrayList(filtered)).build(); - } - catch (CatalogException e) - { - return e.toResponse(); - } - } - - /** - * Retrieves the list of all Druid table metadata for which the user has at - * least read access. - */ - @GET - @Path("/schemas/{dbSchema}") - @Produces(MediaType.APPLICATION_JSON) - public Response listTableMetadataForSchema( - @PathParam("dbSchema") String dbSchema, - @Context final HttpServletRequest req - ) - { - try { - SchemaSpec schema = validateSchema(dbSchema, false); - List tables = catalog.tables().tablesInSchema(schema.name()); - Iterable filtered = AuthorizationUtils.filterAuthorizedResources( - req, - tables, - table -> { - TableId tableId = table.id(); - return Collections.singletonList( - resourceAction(schema, tableId.name(), Action.READ)); - }, - authorizerMapper - ); - - return Response.ok().entity(Lists.newArrayList(filtered)).build(); + format = Strings.isNullOrEmpty(format) ? NAME_FORMAT : StringUtils.toLowerCase(format); + switch (format) { + case NAME_FORMAT: + return tableNamesInSchema(schema, req); + case METADATA_FORMAT: + return Response.ok().entity(getTableMetadataForSchema(schema, req)).build(); + case STATUS_FORMAT: + return Response.ok().entity(getTableStatusForSchema(schema, req)).build(); + default: + throw CatalogException.badRequest("Unknown format: [%s]", format); + } } - catch (CatalogException e) - { + catch (CatalogException e) { return e.toResponse(); } } @@ -373,9 +350,15 @@ public Response syncSchema( @Context final HttpServletRequest req ) { - // Same as the user-command for now. This endpoint reserves the right to change + // Same as the list schemas endpoint for now. This endpoint reserves the right to change // over time as needed, while the user endpoint cannot easily change. - return listTableMetadataForSchema(dbSchema, req); + try { + SchemaSpec schema = validateSchema(dbSchema, false); + return Response.ok().entity(getTableMetadataForSchema(schema, req)).build(); + } + catch (CatalogException e) { + return e.toResponse(); + } } public static final String TABLE_SYNC = "/sync/schemas/{schema}/{name}"; @@ -400,6 +383,108 @@ public Response syncTable( // --------------------------------------------------------------------- // Helper methods + + /** + * Retrieves the list of all Druid table names for which the user has at + * least read access. + */ + private Response listTablePaths(final HttpServletRequest req) + { + List tables = catalog.tables().allTablePaths(); + Iterable filtered = AuthorizationUtils.filterAuthorizedResources( + req, + tables, + tableId -> { + SchemaSpec schema = catalog.resolveSchema(tableId.schema()); + if (schema == null) { + // Should never occur. + return null; + } + return Collections.singletonList( + resourceAction(schema, tableId.name(), Action.READ)); + }, + authorizerMapper + ); + return Response.ok().entity(Lists.newArrayList(filtered)).build(); + } + + private List getTableMetadataForSchema( + final SchemaSpec schema, + final HttpServletRequest req + ) throws CatalogException + { + List tables = catalog.tables().tablesInSchema(schema.name()); + Iterable filtered = AuthorizationUtils.filterAuthorizedResources( + req, + tables, + table -> { + TableId tableId = table.id(); + return Collections.singletonList( + resourceAction(schema, tableId.name(), Action.READ)); + }, + authorizerMapper + ); + + return Lists.newArrayList(filtered); + } + + private List getTableStatusForSchema( + final SchemaSpec schema, + final HttpServletRequest req + ) throws CatalogException + { + // Crude but effective, assuming low volume: get all the data, and throw away + // the columns and properties. + return getTableMetadataForSchema(schema, req) + .stream() + .map(table -> table.withSpec(new TableSpec(table.spec().type(), null, null))) + .collect(Collectors.toList()); + } + + private Response listAllTableMetadata(final HttpServletRequest req) throws CatalogException + { + List> tables = new ArrayList<>(); + for (SchemaSpec schema : catalog.schemaRegistry().schemas()) { + tables.addAll(catalog.tables().tablesInSchema(schema.name()) + .stream() + .map(table -> Pair.of(schema, table)) + .collect(Collectors.toList())); + + } + Iterable> filtered = AuthorizationUtils.filterAuthorizedResources( + req, + tables, + entry -> { + return Collections.singletonList( + resourceAction(entry.lhs, entry.rhs.id().name(), Action.READ)); + }, + authorizerMapper + ); + + List metadata = Lists.newArrayList(filtered) + .stream() + .map(pair -> pair.rhs) + .collect(Collectors.toList()); + return Response.ok().entity(metadata).build(); + } + + private Response tableNamesInSchema( + final SchemaSpec schema, + final HttpServletRequest req + ) + { + List tables = catalog.tables().tableNamesInSchema(schema.name()); + Iterable filtered = AuthorizationUtils.filterAuthorizedResources( + req, + tables, + name -> + Collections.singletonList( + resourceAction(schema, name, Action.READ)), + authorizerMapper + ); + return Response.ok().entity(Lists.newArrayList(filtered)).build(); + } + private void validateTableName(String name) throws CatalogException { try { diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditRequest.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditRequest.java index 9947a0b2fb0f..63adb4a0cfc2 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditRequest.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditRequest.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.catalog.http; import com.fasterxml.jackson.annotation.JsonCreator; @@ -28,7 +47,7 @@ public static class HideColumns extends TableEditRequest public final List columns; @JsonCreator - public HideColumns( @JsonProperty("columns") List columns) + public HideColumns(@JsonProperty("columns") List columns) { this.columns = columns; } @@ -40,7 +59,7 @@ public static class UnhideColumns extends TableEditRequest public final List columns; @JsonCreator - public UnhideColumns( @JsonProperty("columns") List columns) + public UnhideColumns(@JsonProperty("columns") List columns) { this.columns = columns; } @@ -52,7 +71,7 @@ public static class DropColumns extends TableEditRequest public final List columns; @JsonCreator - public DropColumns( @JsonProperty("columns") List columns) + public DropColumns(@JsonProperty("columns") List columns) { this.columns = columns; } @@ -64,7 +83,7 @@ public static class UpdateProperties extends TableEditRequest public final Map properties; @JsonCreator - public UpdateProperties( @JsonProperty("properties") Map properties) + public UpdateProperties(@JsonProperty("properties") Map properties) { this.properties = properties; } @@ -76,7 +95,7 @@ public static class UpdateColumns extends TableEditRequest public final List columns; @JsonCreator - public UpdateColumns( @JsonProperty("columns") List columns) + public UpdateColumns(@JsonProperty("columns") List columns) { this.columns = columns; } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java index b22732a1cd63..f018a1067e5d 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.catalog.http; import com.google.common.base.Strings; @@ -15,6 +34,7 @@ import org.apache.druid.catalog.model.TableSpec; import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; import org.apache.druid.catalog.storage.CatalogStorage; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.utils.CollectionUtils; import javax.ws.rs.core.Response; @@ -33,7 +53,11 @@ public class TableEditor private final TableId id; private final TableEditRequest editRequest; - public TableEditor(CatalogStorage catalog, TableId id, TableEditRequest editRequest) + public TableEditor( + final CatalogStorage catalog, + final TableId id, + final TableEditRequest editRequest + ) { this.catalog = catalog; this.id = id; @@ -66,7 +90,7 @@ public long go() throws CatalogException private long hideColumns(List columns) throws CatalogException { - if (columns == null) { + if (CollectionUtils.isNullOrEmpty(columns)) { return 0; } return catalog.tables().updateProperties( @@ -102,8 +126,13 @@ private TableSpec applyHiddenColumns(TableMetadata table, List columns) for (String col : columns) { if (!existing.contains(col)) { revised.add(col); + existing.add(col); } } + if (revised.size() == hiddenColumns.size()) { + // Nothing changed + return null; + } Map revisedProps = new HashMap<>(props); revisedProps.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, revised); return spec.withProperties(revisedProps); @@ -111,7 +140,7 @@ private TableSpec applyHiddenColumns(TableMetadata table, List columns) private long unHideColumns(List columns) throws CatalogException { - if (columns == null) { + if (CollectionUtils.isNullOrEmpty(columns)) { return 0; } return catalog.tables().updateProperties( @@ -137,7 +166,7 @@ private TableSpec applyUnhideColumns(TableMetadata table, List columns) final Map props = existingSpec.properties(); @SuppressWarnings("unchecked") List hiddenColumns = (List) props.get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); - if (CollectionUtils.isNullOrEmpty(hiddenColumns) || columns.isEmpty()) { + if (hiddenColumns == null || columns.isEmpty()) { return null; } Set removals = new HashSet<>(columns); @@ -147,6 +176,10 @@ private TableSpec applyUnhideColumns(TableMetadata table, List columns) revisedHiddenCols.add(col); } } + if (revisedHiddenCols.size() == hiddenColumns.size() && !hiddenColumns.isEmpty()) { + // Nothing changed + return null; + } final Map revisedProps = new HashMap<>(props); if (revisedHiddenCols.isEmpty()) { revisedProps.remove(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); @@ -158,7 +191,7 @@ private TableSpec applyUnhideColumns(TableMetadata table, List columns) private long dropColumns(List columnsToDrop) throws CatalogException { - if (columnsToDrop == null) { + if (CollectionUtils.isNullOrEmpty(columnsToDrop)) { return 0; } return catalog.tables().updateColumns( @@ -171,8 +204,8 @@ private TableSpec applyDropColumns(final TableMetadata table, final List { final TableSpec existingSpec = table.spec(); List existingColumns = existingSpec.columns(); - if (toDrop.isEmpty() || existingColumns.isEmpty()) { - return existingSpec; + if (CollectionUtils.isNullOrEmpty(existingColumns)) { + return null; } Set drop = new HashSet(toDrop); List revised = new ArrayList<>(); @@ -181,13 +214,17 @@ private TableSpec applyDropColumns(final TableMetadata table, final List revised.add(col); } } + if (revised.size() == existingColumns.size()) { + // Nothing changed + return null; + } return existingSpec.withColumns(revised); } private long updateProperties(Map updates) throws CatalogException { - if (updates == null) { + if (updates == null || updates.isEmpty()) { return 0; } return catalog.tables().updateProperties( @@ -203,9 +240,17 @@ private TableSpec applyUpdateProperties( { final TableSpec existingSpec = table.spec(); final TableDefn defn = resolveDefn(existingSpec.type()); - return existingSpec.withProperties( - defn.mergeProperties(existingSpec.properties(), updates) + final Map revised = defn.mergeProperties( + existingSpec.properties(), + updates ); + try { + defn.validate(revised, catalog.jsonMapper()); + } + catch (IAE e) { + throw CatalogException.badRequest(e.getMessage()); + } + return existingSpec.withProperties(revised); } private TableDefn resolveDefn(String tableType) throws CatalogException @@ -225,7 +270,7 @@ private TableDefn resolveDefn(String tableType) throws CatalogException private long updateColumns(final List updates) throws CatalogException { - if (updates == null) { + if (CollectionUtils.isNullOrEmpty(updates)) { return 0; } return catalog.tables().updateColumns( @@ -241,9 +286,14 @@ private TableSpec applyUpdateColumns( { final TableSpec existingSpec = table.spec(); final TableDefn defn = resolveDefn(existingSpec.type()); - return existingSpec.withColumns( - defn.mergeColumns(existingSpec.columns(), updates) - ); + final List revised = defn.mergeColumns(existingSpec.columns(), updates); + try { + defn.validateColumns(revised, catalog.jsonMapper()); + } + catch (IAE e) { + throw CatalogException.badRequest(e.getMessage()); + } + return existingSpec.withColumns(revised); } private long moveColumn(MoveColumn moveColumn) throws CatalogException diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java index abc99d1f28cd..e1202d295663 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java @@ -78,6 +78,11 @@ public SchemaRegistry schemaRegistry() return schemaRegistry; } + public ObjectMapper jsonMapper() + { + return tableRegistry.jsonMapper(); + } + public SchemaSpec resolveSchema(String dbSchema) { return schemaRegistry.schema(dbSchema); @@ -100,7 +105,8 @@ public List tablesForSchema(String dbSchema) { try { return tables().read(id); - } catch (NotFoundException e) { + } + catch (NotFoundException e) { return null; } } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java index 0146d0341516..40b66c17e149 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java @@ -407,8 +407,8 @@ public TableMetadata withHandle(Handle handle) throws CatalogException return result.updateTime(); } catch (CallbackFailedException e) { - if (e.getCause() instanceof NotFoundException) { - throw (NotFoundException) e.getCause(); + if (e.getCause() instanceof CatalogException) { + throw (CatalogException) e.getCause(); } throw e; } @@ -432,7 +432,7 @@ public TableMetadata withHandle(Handle handle) throws CatalogException public long updateColumns( final TableId id, final TableTransform transform - ) throws NotFoundException + ) throws CatalogException { try { final TableMetadata result = dbi.withHandle( @@ -499,8 +499,8 @@ public TableMetadata withHandle(Handle handle) throws CatalogException return result.updateTime(); } catch (CallbackFailedException e) { - if (e.getCause() instanceof NotFoundException) { - throw (NotFoundException) e.getCause(); + if (e.getCause() instanceof CatalogException) { + throw (CatalogException) e.getCause(); } throw e; } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java index a0afff343e2c..4f5da642ec46 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java @@ -179,7 +179,7 @@ private TableEntry computeUpdate(TableEntry entry, TableMetadata update) if (!checkVersion(entry, update)) { return entry; } - return new TableEntry(update) ; + return new TableEntry(update); } private boolean checkExists(TableEntry entry, TableMetadata update) diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateListener.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateListener.java index 6db90fb0ed01..afddeb00221a 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateListener.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateListener.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.catalog.sync; /** @@ -9,4 +28,4 @@ public interface CatalogUpdateListener { void updated(UpdateEvent event); -} \ No newline at end of file +} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/UpdateEvent.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/UpdateEvent.java index 946e549f27ca..19cc383d40ad 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/UpdateEvent.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/UpdateEvent.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.druid.catalog.sync; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogSyncTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogSyncTest.java index 1d7ab3041f21..aaeda99f3b05 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogSyncTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogSyncTest.java @@ -93,7 +93,7 @@ public void testInputValidation() { // Valid definition { - TableMetadata table = TableBuilder.externalTable(InlineTableDefn.TABLE_TYPE, "externTable") + TableMetadata table = TableBuilder.external(InlineTableDefn.TABLE_TYPE, "externTable") .format(InputFormats.CSV_FORMAT_TYPE) .data("a", "c") .column("a", Columns.VARCHAR) @@ -103,7 +103,7 @@ public void testInputValidation() // No columns { - TableMetadata table = TableBuilder.externalTable(InlineTableDefn.TABLE_TYPE, "externTable") + TableMetadata table = TableBuilder.external(InlineTableDefn.TABLE_TYPE, "externTable") .format(InputFormats.CSV_FORMAT_TYPE) .data("a", "c") .build(); @@ -112,7 +112,7 @@ public void testInputValidation() // No format { - TableMetadata table = TableBuilder.externalTable(InlineTableDefn.TABLE_TYPE, "externTable") + TableMetadata table = TableBuilder.external(InlineTableDefn.TABLE_TYPE, "externTable") .data("a", "c") .column("a", Columns.VARCHAR) .build(); @@ -145,7 +145,7 @@ public void testCached() throws CatalogException // Also test the deletion case TableId table2 = TableId.datasource("table2"); storage.tables().delete(table2); - assertThrows(NotFoundException.class, () -> storage.tables().read(table2)); + assertThrows(NotFoundException.class, () -> storage.tables().read(table2)); List tables = catalog.tables(TableId.DRUID_SCHEMA); assertEquals(2, tables.size()); @@ -169,7 +169,7 @@ public void testRemoteWithJson() throws CatalogException // Also test the deletion case TableId table2 = TableId.datasource("table2"); storage.tables().delete(table2); - assertThrows(NotFoundException.class, () -> storage.tables().read(table2)); + assertThrows(NotFoundException.class, () -> storage.tables().read(table2)); List tables = catalog.tables(TableId.DRUID_SCHEMA); assertEquals(2, tables.size()); @@ -198,7 +198,7 @@ private void populateCatalog() throws DuplicateKeyException storage.validate(table2); storage.tables().create(table2); - TableMetadata table3 = TableBuilder.externalTable(InlineTableDefn.TABLE_TYPE, "table3") + TableMetadata table3 = TableBuilder.external(InlineTableDefn.TABLE_TYPE, "table3") .format(InputFormats.CSV_FORMAT_TYPE) .data("a", "c") .column("a", Columns.VARCHAR) diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java index e09de234c83f..7b2fbb184967 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java @@ -31,6 +31,7 @@ import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; +import org.apache.druid.catalog.model.table.DatasourceDefn; import org.apache.druid.catalog.model.table.InlineTableDefn; import org.apache.druid.catalog.model.table.InputFormats; import org.apache.druid.catalog.model.table.TableBuilder; @@ -48,6 +49,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Set; import static org.apache.druid.server.http.catalog.DummyRequest.deleteBy; import static org.apache.druid.server.http.catalog.DummyRequest.getBy; @@ -140,7 +142,7 @@ public void testCreate() assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); // Inline input source - TableSpec inputSpec = TableBuilder.externalTable(InlineTableDefn.TABLE_TYPE, "inline") + TableSpec inputSpec = TableBuilder.external(InlineTableDefn.TABLE_TYPE, "inline") .format(InputFormats.CSV_FORMAT_TYPE) .data("a,b,1", "c,d,2") .column("a", Columns.VARCHAR) @@ -240,6 +242,12 @@ public void testRead() assertEquals(dsSpec, read.spec()); } + @SuppressWarnings("unchecked") + private Set getSchemaSet(Response resp) + { + return (Set) resp.getEntity(); + } + @SuppressWarnings("unchecked") private List getTableIdList(Response resp) { @@ -259,25 +267,81 @@ private List getDetailsList(Response resp) } @Test - public void testList() + public void testGetSchemas() { - // No entries - Response resp = resource.listTableNames(getBy(CatalogTests.READER_USER)); + // Invalid format + Response resp = resource.getSchemas("bogus", getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Schema names (default) + resp = resource.getSchemas(null, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - List tableIds = getTableIdList(resp); - assertTrue(tableIds.isEmpty()); + assertTrue(getSchemaSet(resp).contains("druid")); - resp = resource.listTableNamesForSchema(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + resp = resource.getSchemas("", getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - List tables = getTableList(resp); - assertTrue(tables.isEmpty()); + assertTrue(getSchemaSet(resp).contains("druid")); + + // Schema names + resp = resource.getSchemas(CatalogResource.NAME_FORMAT, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertTrue(getSchemaSet(resp).contains("druid")); + + // Table paths - no entries + resp = resource.getSchemas(CatalogResource.PATH_FORMAT, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertTrue(getTableIdList(resp).isEmpty()); + + // Table metadata - no entries + resp = resource.getSchemas(CatalogResource.METADATA_FORMAT, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertTrue(getTableIdList(resp).isEmpty()); + + // Create a table + final String tableName = "list"; + TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D").buildSpec(); + resp = resource.postTable(TableId.DRUID_SCHEMA, "list", dsSpec, 0, false, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + // Table paths - no read access + resp = resource.getSchemas(CatalogResource.PATH_FORMAT, getBy(CatalogTests.DENY_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertTrue(getTableIdList(resp).isEmpty()); + + // Table metadata - no read access + resp = resource.getSchemas(CatalogResource.METADATA_FORMAT, getBy(CatalogTests.DENY_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertTrue(getDetailsList(resp).isEmpty()); + + // Table paths - read access + resp = resource.getSchemas(CatalogResource.PATH_FORMAT, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertEquals(1, getTableIdList(resp).size()); + assertEquals(tableName, getTableIdList(resp).get(0).name()); + + // Table metadata - read access + resp = resource.getSchemas(CatalogResource.METADATA_FORMAT, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + List tables = getDetailsList(resp); + assertEquals(1, tables.size()); + assertEquals(tableName, tables.get(0).id().name()); + assertEquals(1, tables.get(0).spec().properties().size()); + } + + @Test + public void testGetSchemaTables() + { + // No entries + Response resp = resource.getSchemaTables(TableId.DRUID_SCHEMA, CatalogResource.NAME_FORMAT, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertTrue(getTableList(resp).isEmpty()); // Missing schema - resp = resource.listTableNamesForSchema(null, getBy(CatalogTests.READER_USER)); + resp = resource.getSchemaTables(null, CatalogResource.NAME_FORMAT, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); // Invalid schema - resp = resource.listTableNamesForSchema("bogus", getBy(CatalogTests.READER_USER)); + resp = resource.getSchemaTables("bogus", CatalogResource.NAME_FORMAT, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); // Create a table @@ -286,32 +350,51 @@ public void testList() resp = resource.postTable(TableId.DRUID_SCHEMA, "list", dsSpec, 0, false, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - // No read access - resp = resource.listTableNames(getBy(CatalogTests.DENY_USER)); + // No read access - name + resp = resource.getSchemaTables(TableId.DRUID_SCHEMA, CatalogResource.NAME_FORMAT, getBy(CatalogTests.DENY_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - tableIds = getTableIdList(resp); - assertTrue(tableIds.isEmpty()); + assertTrue(getTableIdList(resp).isEmpty()); - resp = resource.listTableNamesForSchema(TableId.DRUID_SCHEMA, getBy(CatalogTests.DENY_USER)); + // No read access - metadata + resp = resource.getSchemaTables(TableId.DRUID_SCHEMA, CatalogResource.METADATA_FORMAT, getBy(CatalogTests.DENY_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - tables = getTableList(resp); - assertTrue(tables.isEmpty()); + assertTrue(getDetailsList(resp).isEmpty()); - // Read access - resp = resource.listTableNames(getBy(CatalogTests.READER_USER)); + // No read access - status + resp = resource.getSchemaTables(TableId.DRUID_SCHEMA, CatalogResource.STATUS_FORMAT, getBy(CatalogTests.DENY_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - tableIds = getTableIdList(resp); - assertEquals(1, tableIds.size()); + assertTrue(getDetailsList(resp).isEmpty()); - resp = resource.listTableNamesForSchema(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + // Read access - name + resp = resource.getSchemaTables(TableId.DRUID_SCHEMA, CatalogResource.NAME_FORMAT, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - tables = getTableList(resp); - assertEquals(1, tables.size()); + assertEquals(Collections.singletonList(tableName), getTableList(resp)); - resp = resource.listTableNamesForSchema(TableId.SYSTEM_SCHEMA, getBy(CatalogTests.READER_USER)); + // Read access - metadata + resp = resource.getSchemaTables(TableId.DRUID_SCHEMA, CatalogResource.METADATA_FORMAT, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertEquals(1, getDetailsList(resp).size()); + TableMetadata table = getDetailsList(resp).get(0); + assertEquals(TableId.datasource(tableName), table.id()); + assertEquals(1, table.spec().properties().size()); + + // Read access - status + resp = resource.getSchemaTables(TableId.DRUID_SCHEMA, CatalogResource.STATUS_FORMAT, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertEquals(1, getDetailsList(resp).size()); + table = getDetailsList(resp).get(0); + assertEquals(TableId.datasource(tableName), table.id()); + assertEquals(DatasourceDefn.TABLE_TYPE, table.spec().type()); + assertTrue(table.spec().properties().isEmpty()); + } + + @Test + public void testSync() + { + final String tableName = "sync"; + TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D").buildSpec(); + Response resp = resource.postTable(TableId.DRUID_SCHEMA, "list", dsSpec, 0, false, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - tables = getTableList(resp); - assertTrue(tables.isEmpty()); // Internal sync schema API resp = resource.syncSchema(TableId.SYSTEM_SCHEMA, getBy(CatalogTests.SUPER_USER)); @@ -387,13 +470,13 @@ public void testLifecycle() assertEquals(dsSpec, read1.spec()); // list - resp = resource.listTableNames(getBy(CatalogTests.READER_USER)); + resp = resource.getSchemas(CatalogResource.PATH_FORMAT, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); List tableIds = getTableIdList(resp); assertEquals(1, tableIds.size()); assertEquals(id1, tableIds.get(0)); - resp = resource.listTableNamesForSchema(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + resp = resource.getSchemaTables(TableId.DRUID_SCHEMA, CatalogResource.NAME_FORMAT, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); List tables = getTableList(resp); assertEquals(1, tables.size()); @@ -421,14 +504,14 @@ public void testLifecycle() TableId id2 = TableId.of(TableId.DRUID_SCHEMA, table2Name); // verify lists - resp = resource.listTableNames(getBy(CatalogTests.READER_USER)); + resp = resource.getSchemas(CatalogResource.PATH_FORMAT, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); tableIds = getTableIdList(resp); assertEquals(2, tableIds.size()); assertEquals(id1, tableIds.get(0)); assertEquals(id2, tableIds.get(1)); - resp = resource.listTableNamesForSchema(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + resp = resource.getSchemaTables(TableId.DRUID_SCHEMA, CatalogResource.NAME_FORMAT, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); tables = getTableList(resp); assertEquals(2, tables.size()); @@ -439,7 +522,7 @@ public void testLifecycle() resp = resource.deleteTable(TableId.DRUID_SCHEMA, table1Name, deleteBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - resp = resource.listTableNamesForSchema(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + resp = resource.getSchemaTables(TableId.DRUID_SCHEMA, CatalogResource.NAME_FORMAT, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); tables = getTableList(resp); assertEquals(1, tables.size()); @@ -447,7 +530,7 @@ public void testLifecycle() resp = resource.deleteTable(TableId.DRUID_SCHEMA, table2Name, deleteBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - resp = resource.listTableNamesForSchema(TableId.DRUID_SCHEMA, getBy(CatalogTests.READER_USER)); + resp = resource.getSchemaTables(TableId.DRUID_SCHEMA, CatalogResource.NAME_FORMAT, getBy(CatalogTests.READER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); tables = getTableList(resp); assertEquals(0, tables.size()); @@ -576,11 +659,10 @@ public void testDropColumns() resp = resource.editTable("bogus", tableName, cmd, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); - // Bad table + // Nothing to do resp = resource.editTable(TableId.DRUID_SCHEMA, "bogus", cmd, postBy(CatalogTests.WRITER_USER)); - assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); - // Nothing to do resp = resource.editTable(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); @@ -595,8 +677,12 @@ public void testDropColumns() CatalogUtils.columnNames(read.spec().columns()) ); - // Drop + // Bad table cmd = new DropColumns(Arrays.asList("a", "c")); + resp = resource.editTable(TableId.DRUID_SCHEMA, "bogus", cmd, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // Drop resp = resource.editTable(TableId.DRUID_SCHEMA, tableName, cmd, postBy(CatalogTests.WRITER_USER)); assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/EditorTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/EditorTest.java index 518f75708267..bde9249f385c 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/EditorTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/EditorTest.java @@ -19,42 +19,41 @@ package org.apache.druid.server.http.catalog; -import com.fasterxml.jackson.databind.ObjectMapper; -import io.vavr.collection.Map; +import org.apache.curator.shaded.com.google.common.collect.ImmutableMap; import org.apache.druid.catalog.CatalogException; -import org.apache.druid.catalog.http.CatalogResource; import org.apache.druid.catalog.http.MoveColumn; import org.apache.druid.catalog.http.TableEditRequest; +import org.apache.druid.catalog.http.TableEditRequest.DropColumns; import org.apache.druid.catalog.http.TableEditRequest.HideColumns; +import org.apache.druid.catalog.http.TableEditRequest.UnhideColumns; +import org.apache.druid.catalog.http.TableEditRequest.UpdateColumns; +import org.apache.druid.catalog.http.TableEditRequest.UpdateProperties; import org.apache.druid.catalog.http.TableEditor; import org.apache.druid.catalog.model.CatalogUtils; import org.apache.druid.catalog.model.ColumnSpec; +import org.apache.druid.catalog.model.Columns; import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; -import org.apache.druid.catalog.model.TableSpec; import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.catalog.model.table.DatasourceDefn.DatasourceColumnDefn; import org.apache.druid.catalog.model.table.TableBuilder; import org.apache.druid.catalog.storage.CatalogStorage; import org.apache.druid.catalog.storage.CatalogTests; -import org.apache.druid.catalog.storage.sql.CatalogManager; -import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.metadata.TestDerbyConnector; import org.junit.After; import org.junit.Before; import org.junit.Rule; import org.junit.Test; -import javax.ws.rs.core.Response; - import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; +import java.util.Map; -import static org.apache.druid.server.http.catalog.DummyRequest.getBy; -import static org.apache.druid.server.http.catalog.DummyRequest.postBy; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; public class EditorTest { @@ -150,7 +149,7 @@ public void testMoveColumn() throws CatalogException } @Test - public void testHideColumns() + public void testHideColumns() throws CatalogException { final String tableName = "table2"; TableMetadata table = TableBuilder.datasource(tableName, "P1D") @@ -172,88 +171,299 @@ public void testHideColumns() doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) ); + // Hide starting from an empty list. Map props = new HashMap<>(table.spec().properties()); props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList()); catalog.tables().replace(table.withProperties(props)); - // Hide starting from an empty list. - cmd = new HideColumns(Arrays.asList("a", "b"), Collections.emptyList()); - revised = cmd.perform(Collections.emptyList()); - assertEquals(Arrays.asList("a", "b"), revised); + assertEquals( + Arrays.asList("a", "b"), + doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); - // Hide with existing columns - cmd = new HideColumns(Arrays.asList("b", "d"), null); - revised = cmd.perform(Arrays.asList("a", "b", "c")); - assertEquals(Arrays.asList("a", "b", "c", "d"), revised); + // Hide, but all are duplicates + props = new HashMap<>(table.spec().properties()); + props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b", "c")); + catalog.tables().replace(table.withProperties(props)); + cmd = new HideColumns(Arrays.asList("b", "c")); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); // Hide with existing columns - cmd = new HideColumns(Arrays.asList("b", "d"), null); - revised = cmd.perform(Arrays.asList("a", "b", "c")); - assertEquals(Arrays.asList("a", "b", "c", "d"), revised); + cmd = new HideColumns(Arrays.asList("b", "d")); + assertEquals( + Arrays.asList("a", "b", "c", "d"), + doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + + // Hide with duplicates + props = new HashMap<>(table.spec().properties()); + props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b")); + catalog.tables().replace(table.withProperties(props)); + cmd = new HideColumns(Arrays.asList("b", "d", "b", "d")); + assertEquals( + Arrays.asList("a", "b", "d"), + doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + } + + @Test + public void testUnhideColumns() throws CatalogException + { + final String tableName = "table3"; + TableMetadata table = TableBuilder.datasource(tableName, "P1D") + .build(); + catalog.tables().create(table); + + // Null unhide list + UnhideColumns cmd = new UnhideColumns(null); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); - // Unhide existing columns - cmd = new HideColumns(null, Arrays.asList("b", "d")); - revised = cmd.perform(Arrays.asList("a", "b", "c")); - assertEquals(Arrays.asList("a", "c"), revised); + // Empty list + cmd = new UnhideColumns(Collections.emptyList()); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); - // Both hide and unhide. Hide takes precedence. - cmd = new HideColumns(Arrays.asList("b", "d", "e"), Arrays.asList("c", "d")); - revised = cmd.perform(Arrays.asList("a", "b", "c")); - assertEquals(Arrays.asList("a", "b", "d", "e"), revised); + // Unhide starting from a null list. + cmd = new UnhideColumns(Arrays.asList("a", "b")); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); - // Duplicates + // Unhide starting from an empty list. + Map props = new HashMap<>(table.spec().properties()); + props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList()); + catalog.tables().replace(table.withProperties(props)); + assertNull( + doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + + // Unhide starting with an empty list with (non-existing) columns to unhide + props = new HashMap<>(table.spec().properties()); + props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList()); + catalog.tables().replace(table.withProperties(props)); + cmd = new UnhideColumns(Arrays.asList("a")); + assertNull( + doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + + // Unhide columns which are not actually hidden. + props = new HashMap<>(table.spec().properties()); + props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b", "c")); + catalog.tables().replace(table.withProperties(props)); + cmd = new UnhideColumns(Arrays.asList("d", "e")); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); + + // Unhide some hidden columns. + cmd = new UnhideColumns(Arrays.asList("a", "c", "a", "d")); + assertEquals( + Arrays.asList("b"), + doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + + // Unhide all hidden columns + props = new HashMap<>(table.spec().properties()); + props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b", "c")); + catalog.tables().replace(table.withProperties(props)); + cmd = new UnhideColumns(Arrays.asList("a", "c", "b", "d")); + assertNull( + doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); } @Test - public void testUnhideColumns() + public void testDropColumns() throws CatalogException { - final String tableName = "table2"; + final String tableName = "table4"; TableMetadata table = TableBuilder.datasource(tableName, "P1D") + .column("a", "VARCHAR") + .column("b", "BIGINT") + .column("c", "FLOAT") .build(); catalog.tables().create(table); - // Everything is null - HideColumns cmd = new HideColumns(null, null); - List revised = cmd.perform(null); - assertNull(revised); + // Null drop list + DropColumns cmd = new DropColumns(null); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); - // Unhide from null list - cmd = new HideColumns(null, Collections.singletonList("a")); - revised = cmd.perform(null); - assertNull(revised); + // Empty list + cmd = new DropColumns(Collections.emptyList()); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); - // And from an empty list - cmd = new HideColumns(null, Collections.singletonList("a")); - revised = cmd.perform(Collections.emptyList()); - assertNull(revised); + // Drop non-existent columns + cmd = new DropColumns(Arrays.asList("d", "e")); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); - // Hide starting from a null list. - cmd = new HideColumns(Arrays.asList("a", "b"), null); - revised = cmd.perform(null); - assertEquals(Arrays.asList("a", "b"), revised); + // Drop some columns, including dups. + cmd = new DropColumns(Arrays.asList("c", "a", "c", "d")); + assertEquals( + Arrays.asList("b"), + CatalogUtils.columnNames(doEdit(tableName, cmd).spec().columns()) + ); - // Hide starting from an empty list. - cmd = new HideColumns(Arrays.asList("a", "b"), Collections.emptyList()); - revised = cmd.perform(Collections.emptyList()); - assertEquals(Arrays.asList("a", "b"), revised); + // Drop all columns + catalog.tables().replace(table); + cmd = new DropColumns(Arrays.asList("c", "a", "c", "b")); + assertEquals( + Collections.emptyList(), + doEdit(tableName, cmd).spec().columns() + ); - // Hide with existing columns - cmd = new HideColumns(Arrays.asList("b", "d"), null); - revised = cmd.perform(Arrays.asList("a", "b", "c")); - assertEquals(Arrays.asList("a", "b", "c", "d"), revised); + // Drop from a null column list + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); + } - // Hide with existing columns - cmd = new HideColumns(Arrays.asList("b", "d"), null); - revised = cmd.perform(Arrays.asList("a", "b", "c")); - assertEquals(Arrays.asList("a", "b", "c", "d"), revised); - - // Unhide existing columns - cmd = new HideColumns(null, Arrays.asList("b", "d")); - revised = cmd.perform(Arrays.asList("a", "b", "c")); - assertEquals(Arrays.asList("a", "c"), revised); - - // Both hide and unhide. Hide takes precedence. - cmd = new HideColumns(Arrays.asList("b", "d", "e"), Arrays.asList("c", "d")); - revised = cmd.perform(Arrays.asList("a", "b", "c")); - assertEquals(Arrays.asList("a", "b", "d", "e"), revised); + @Test + public void testUpdateProperties() throws CatalogException + { + final String tableName = "table5"; + TableMetadata table = TableBuilder.datasource(tableName, "P1D") + .build(); + catalog.tables().create(table); + + // Null merge map + UpdateProperties cmd = new UpdateProperties(null); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); + + // Empty merge map + cmd = new UpdateProperties(Collections.emptyMap()); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); + + // Can't test an empty property set: no table type allows empty + // properties. + + // Remove a required property + Map updates1 = new HashMap<>(); + updates1.put(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, null); + assertThrows( + CatalogException.class, + () -> new TableEditor( + catalog, + table.id(), + new UpdateProperties(updates1) + ) + .go() + ); + + // Add and update properties + Map updates = new HashMap<>(); + updates.put(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H"); + updates.put("foo", "bar"); + cmd = new UpdateProperties(updates); + Map expected = ImmutableMap.of( + DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H", + "foo", "bar" + ); + assertEquals( + expected, + doEdit(tableName, cmd).spec().properties() + ); + + // Update only + updates = new HashMap<>(); + updates.put("foo", "mumble"); + cmd = new UpdateProperties(updates); + expected = ImmutableMap.of( + DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H", + "foo", "mumble" + ); + assertEquals( + expected, + doEdit(tableName, cmd).spec().properties() + ); + + // Remove a property + updates = new HashMap<>(); + updates.put("foo", null); + cmd = new UpdateProperties(updates); + expected = ImmutableMap.of( + DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H" + ); + assertEquals( + expected, + doEdit(tableName, cmd).spec().properties() + ); + } + + @Test + public void testUpdateColumns() throws CatalogException + { + final String tableName = "table4"; + TableMetadata table = TableBuilder.datasource(tableName, "P1D") + .column("a", "VARCHAR") + .column("b", "BIGINT") + .column("c", "FLOAT") + .build(); + catalog.tables().create(table); + + // Null update list + UpdateColumns cmd = new UpdateColumns(null); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); + + // Empty list + cmd = new UpdateColumns(Collections.emptyList()); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); + + // Add a column + cmd = new UpdateColumns( + Collections.singletonList( + new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "d", Columns.VARCHAR, null) + ) + ); + TableMetadata revised = doEdit(tableName, cmd); + assertEquals( + Arrays.asList("a", "b", "c", "d"), + CatalogUtils.columnNames(revised.spec().columns()) + ); + ColumnSpec colD = revised.spec().columns().get(3); + assertEquals(DatasourceColumnDefn.COLUMN_TYPE, colD.type()); + assertEquals("d", colD.name()); + assertEquals(Columns.VARCHAR, colD.sqlType()); + + // Update a column + cmd = new UpdateColumns( + Collections.singletonList( + new ColumnSpec( + null, + "a", + Columns.BIGINT, + ImmutableMap.of("foo", "bar") + ) + ) + ); + revised = doEdit(tableName, cmd); + assertEquals( + Arrays.asList("a", "b", "c", "d"), + CatalogUtils.columnNames(revised.spec().columns()) + ); + ColumnSpec colA = revised.spec().columns().get(0); + assertEquals(DatasourceColumnDefn.COLUMN_TYPE, colA.type()); + assertEquals("a", colA.name()); + assertEquals(Columns.BIGINT, colA.sqlType()); + assertEquals(ImmutableMap.of("foo", "bar"), colA.properties()); + + // Duplicates + UpdateColumns cmd2 = new UpdateColumns( + Arrays.asList( + new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "e", Columns.VARCHAR, null), + new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "e", null, null) + ) + ); + assertThrows(CatalogException.class, () -> doEdit(tableName, cmd2)); + + // Invalid __time column type + UpdateColumns cmd3 = new UpdateColumns( + Collections.singletonList( + new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, Columns.TIME_COLUMN, Columns.VARCHAR, null) + ) + ); + assertThrows(CatalogException.class, () -> doEdit(tableName, cmd3)); + + // Valid time column type + cmd = new UpdateColumns( + Collections.singletonList( + new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, Columns.TIME_COLUMN, Columns.TIMESTAMP, null) + ) + ); + revised = doEdit(tableName, cmd); + assertEquals( + Arrays.asList("a", "b", "c", "d", "__time"), + CatalogUtils.columnNames(revised.spec().columns()) + ); } } diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java index 7c4527531e2a..f18c78c3e170 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java @@ -20,14 +20,17 @@ package org.apache.druid.testsEx.catalog; import com.google.inject.Inject; -import org.apache.druid.catalog.http.HideColumns; import org.apache.druid.catalog.http.MoveColumn; +import org.apache.druid.catalog.http.TableEditRequest.DropColumns; +import org.apache.druid.catalog.http.TableEditRequest.HideColumns; +import org.apache.druid.catalog.http.TableEditRequest.UnhideColumns; import org.apache.druid.catalog.model.CatalogUtils; import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; import org.apache.druid.catalog.model.table.TableBuilder; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.testsEx.categories.Catalog; import org.apache.druid.testsEx.cluster.CatalogClient; import org.apache.druid.testsEx.cluster.DruidClusterClient; @@ -74,7 +77,7 @@ public void testErrors() assertThrows( Exception.class, - () -> client.createTable(table) + () -> client.createTable(table, false) ); } @@ -86,7 +89,7 @@ public void testErrors() .build(); assertThrows( Exception.class, - () -> client.createTable(table) + () -> client.createTable(table, false) ); } @@ -96,7 +99,7 @@ public void testErrors() .build(); assertThrows( Exception.class, - () -> client.createTable(table) + () -> client.createTable(table, false) ); } } @@ -119,7 +122,7 @@ public void testLifecycle() // Use force action so test is reentrant if it fails part way through // when debugging. - long version = client.createTable(table, "force"); + long version = client.createTable(table, true); // Update the datasource TableSpec dsSpec2 = TableBuilder.copyOf(table) @@ -128,7 +131,7 @@ public void testLifecycle() .buildSpec(); // First, optimistic locking, wrong version - client.updateTable(table.id(), dsSpec2, 1); + assertThrows(ISE.class, () -> client.updateTable(table.id(), dsSpec2, 1)); // Optimistic locking, correct version long newVersion = client.updateTable(table.id(), dsSpec2, version); @@ -140,19 +143,19 @@ public void testLifecycle() // Move a column MoveColumn moveCmd = new MoveColumn("d", MoveColumn.Position.BEFORE, "a"); - client.moveColumn(table.id(), moveCmd); + client.editTable(table.id(), moveCmd); // Drop a column - client.dropColumns(table.id(), Collections.singletonList("b")); + DropColumns dropCmd = new DropColumns(Collections.singletonList("b")); + client.editTable(table.id(), dropCmd); read = client.readTable(table.id()); assertEquals(Arrays.asList("d", "a", "c"), CatalogUtils.columnNames(read.spec().columns())); // Hide columns HideColumns hideCmd = new HideColumns( - Arrays.asList("e", "f"), - Collections.singletonList("g") + Arrays.asList("e", "f") ); - client.hideColumns(table.id(), hideCmd); + client.editTable(table.id(), hideCmd); read = client.readTable(table.id()); assertEquals( Arrays.asList("e", "f"), @@ -160,11 +163,10 @@ public void testLifecycle() ); // Unhide - hideCmd = new HideColumns( - null, + UnhideColumns unhideCmd = new UnhideColumns( Collections.singletonList("e") ); - client.hideColumns(table.id(), hideCmd); + client.editTable(table.id(), unhideCmd); read = client.readTable(table.id()); assertEquals( Collections.singletonList("f"), diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/CatalogClient.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/CatalogClient.java index 4788f419beea..eee5b3d79730 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/CatalogClient.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/CatalogClient.java @@ -23,8 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import org.apache.druid.catalog.http.CatalogResource; -import org.apache.druid.catalog.http.HideColumns; -import org.apache.druid.catalog.http.MoveColumn; +import org.apache.druid.catalog.http.TableEditRequest; import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; import org.apache.druid.catalog.model.TableSpec; @@ -55,23 +54,18 @@ public CatalogClient(final DruidClusterClient clusterClient) this.clusterClient = clusterClient; } - public long createTable(TableMetadata table) - { - return createTable(table, null); - } - - public long createTable(TableMetadata table, String action) + public long createTable(TableMetadata table, boolean overwrite) { // Use action= String url = StringUtils.format( - "%s%s/tables/%s/%s", + "%s%s/schemas/%s/tables/%s", clusterClient.leadCoordinatorUrl(), CatalogResource.ROOT_PATH, table.id().schema(), table.id().name() ); - if (action != null) { - url += "?action=" + action; + if (overwrite) { + url += "?overwrite=true"; } VersionResponse response = clusterClient.post(url, table.spec(), VersionResponse.class); return response.version; @@ -80,7 +74,7 @@ public long createTable(TableMetadata table, String action) public long updateTable(TableId tableId, TableSpec tableSpec, long version) { String url = StringUtils.format( - "%s%s/tables/%s/%s", + "%s%s/schemas/%s/tables/%s", clusterClient.leadCoordinatorUrl(), CatalogResource.ROOT_PATH, tableId.schema(), @@ -89,14 +83,14 @@ public long updateTable(TableId tableId, TableSpec tableSpec, long version) if (version > 0) { url += "?version=" + version; } - VersionResponse response = clusterClient.put(url, tableSpec, VersionResponse.class); + VersionResponse response = clusterClient.post(url, tableSpec, VersionResponse.class); return response.version; } public TableMetadata readTable(TableId tableId) { String url = StringUtils.format( - "%s%s/tables/%s/%s", + "%s%s/schemas/%s/tables/%s", clusterClient.leadCoordinatorUrl(), CatalogResource.ROOT_PATH, tableId.schema(), @@ -108,7 +102,7 @@ public TableMetadata readTable(TableId tableId) public void dropTable(TableId tableId) { String url = StringUtils.format( - "%s%s/tables/%s/%s", + "%s%s/schemas/%s/tables/%s", clusterClient.leadCoordinatorUrl(), CatalogResource.ROOT_PATH, tableId.schema(), @@ -117,23 +111,10 @@ public void dropTable(TableId tableId) clusterClient.send(HttpMethod.DELETE, url); } - public long moveColumn(TableId tableId, MoveColumn cmd) - { - String url = StringUtils.format( - "%s%s/tables/%s/%s/moveColumn", - clusterClient.leadCoordinatorUrl(), - CatalogResource.ROOT_PATH, - tableId.schema(), - tableId.name() - ); - VersionResponse response = clusterClient.post(url, cmd, VersionResponse.class); - return response.version; - } - - public long hideColumns(TableId tableId, HideColumns cmd) + public long editTable(TableId tableId, TableEditRequest cmd) { String url = StringUtils.format( - "%s%s/tables/%s/%s/hideColumns", + "%s%s/schemas/%s/tables/%s/edit", clusterClient.leadCoordinatorUrl(), CatalogResource.ROOT_PATH, tableId.schema(), @@ -143,23 +124,10 @@ public long hideColumns(TableId tableId, HideColumns cmd) return response.version; } - public long dropColumns(TableId tableId, List columns) - { - String url = StringUtils.format( - "%s%s/tables/%s/%s/dropColumns", - clusterClient.leadCoordinatorUrl(), - CatalogResource.ROOT_PATH, - tableId.schema(), - tableId.name() - ); - VersionResponse response = clusterClient.post(url, columns, VersionResponse.class); - return response.version; - } - public List listSchemas() { String url = StringUtils.format( - "%s%s/list/schemas/names", + "%s%s/schemas?format=name", clusterClient.leadCoordinatorUrl(), CatalogResource.ROOT_PATH ); @@ -169,7 +137,7 @@ public List listSchemas() public List listTables() { String url = StringUtils.format( - "%s%s/list/tables/names", + "%s%s/schemas?format=path", clusterClient.leadCoordinatorUrl(), CatalogResource.ROOT_PATH ); @@ -179,7 +147,7 @@ public List listTables() public List listTableNamesInSchema(String schemaName) { String url = StringUtils.format( - "%s%s/schemas/%s/names", + "%s%s/schemas/%s/tables?format=name", clusterClient.leadCoordinatorUrl(), CatalogResource.ROOT_PATH, schemaName @@ -190,7 +158,7 @@ public List listTableNamesInSchema(String schemaName) public List listTablesInSchema(String schemaName) { String url = StringUtils.format( - "%s%s/schemas/%s/tables", + "%s%s/schemas/%s/tables?format=metadata", clusterClient.leadCoordinatorUrl(), CatalogResource.ROOT_PATH, schemaName diff --git a/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistry.java b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistry.java index 582c280d8226..6654ceea74d8 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistry.java +++ b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistry.java @@ -19,6 +19,7 @@ package org.apache.druid.catalog.model; +import java.util.List; import java.util.Set; /** @@ -39,4 +40,5 @@ interface SchemaSpec SchemaSpec schema(String name); Set names(); + List schemas(); } diff --git a/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java index a984502b316c..fbdfc7fc76a3 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java +++ b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.java @@ -19,11 +19,14 @@ package org.apache.druid.catalog.model; +import com.google.common.collect.Lists; import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; import org.apache.druid.catalog.model.table.ExternalTableDefn; import org.apache.druid.server.security.ResourceType; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeSet; @@ -137,4 +140,14 @@ public Set names() { return new TreeSet(builtIns.keySet()); } + + @Override + public List schemas() + { + // No real need to sort every time. However, this is used infrequently, + // so OK for now. + List schemas = Lists.newArrayList(builtIns.values()); + Collections.sort(schemas, (s1, s2) -> s1.name().compareTo(s2.name())); + return schemas; + } } diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java index 6cf2e3f013af..2916aabe885f 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java @@ -86,16 +86,21 @@ public static Map toColumnMap(final List colType public void validate(ResolvedTable table) { validate(table.properties(), table.jsonMapper()); - if (table.spec().columns() == null) { + validateColumns(table.spec().columns(), table.jsonMapper()); + } + + public void validateColumns(List columns, ObjectMapper jsonMapper) + { + if (columns == null) { return; } Set names = new HashSet<>(); - for (ColumnSpec colSpec : table.spec().columns()) { + for (ColumnSpec colSpec : columns) { if (!names.add(colSpec.name())) { throw new IAE("Duplicate column name: " + colSpec.name()); } ColumnDefn.ResolvedColumn resolvedCol = resolveColumn(colSpec); - resolvedCol.validate(table.jsonMapper()); + resolvedCol.validate(jsonMapper); } } diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java b/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java index 532df200eb8a..75102683c2e3 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java @@ -67,6 +67,11 @@ public TableDefn defnFor(String type) return defns.get(type); } + public ObjectMapper jsonMapper() + { + return jsonMapper; + } + public ResolvedTable resolve(TableSpec spec) { String type = spec.type(); diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java b/server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java index 63e00de50d63..b18aee4efc40 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java @@ -61,7 +61,7 @@ public static TableBuilder datasource(String name, String granularity) .segmentGranularity(granularity); } - public static TableBuilder externalTable(String type, String name) + public static TableBuilder external(String type, String name) { return new TableBuilder() .external(name) diff --git a/server/src/test/java/org/apache/druid/catalog/model/TableMetadataTest.java b/server/src/test/java/org/apache/druid/catalog/model/TableMetadataTest.java index 92d33aa4a071..5ae0bfd4ceba 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/TableMetadataTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/TableMetadataTest.java @@ -19,7 +19,6 @@ package org.apache.druid.catalog.model; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.catalog.CatalogTest; @@ -27,7 +26,6 @@ import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; import org.apache.druid.catalog.model.table.DatasourceDefn; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -41,8 +39,6 @@ @Category(CatalogTest.class) public class TableMetadataTest { - private ObjectMapper mapper = new ObjectMapper(); - @Test public void testId() { diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java index 1fcfd551142b..a6a4b6935fb2 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java @@ -27,7 +27,6 @@ import org.apache.druid.catalog.model.ParameterizedDefn; import org.apache.druid.catalog.model.ResolvedTable; import org.apache.druid.catalog.model.TableDefnRegistry; -import org.apache.druid.catalog.model.TableSpec; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.HttpInputSource; import org.apache.druid.data.input.impl.HttpInputSourceConfig; diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/InlineTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/InlineTableTest.java index f3037e16e1b7..f8d4057a4ce1 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/InlineTableTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/InlineTableTest.java @@ -24,7 +24,6 @@ import org.apache.druid.catalog.model.Columns; import org.apache.druid.catalog.model.ResolvedTable; import org.apache.druid.catalog.model.TableDefnRegistry; -import org.apache.druid.catalog.model.TableSpec; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.InlineInputSource; import org.apache.druid.java.util.common.IAE; diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java b/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java index 5f247429bfac..9246078fe64f 100644 --- a/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java +++ b/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java @@ -26,7 +26,6 @@ import org.apache.druid.catalog.model.ParameterizedDefn; import org.apache.druid.catalog.model.ResolvedTable; import org.apache.druid.catalog.model.TableDefnRegistry; -import org.apache.druid.catalog.model.TableSpec; import org.apache.druid.data.input.impl.CsvInputFormat; import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.java.util.common.IAE; From 9fc76e182e8e153cba4906c64c885045f874d3ca Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Tue, 1 Nov 2022 11:19:06 -0700 Subject: [PATCH 16/22] Added logging --- .../main/java/org/apache/druid/catalog/sync/CacheNotifier.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CacheNotifier.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CacheNotifier.java index 3b74432338b4..8118076c103b 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CacheNotifier.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CacheNotifier.java @@ -113,6 +113,9 @@ public void stopGracefully() public void stop() { + if (!updates.isEmpty()) { + LOG.warn("Shutting down Catalog sync with %d unsent notifications", updates.size()); + } exec.shutdownNow(); } } From af8171da0b90fa12af9d4fd4a3126e87fd6e4464 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Tue, 1 Nov 2022 18:01:37 -0700 Subject: [PATCH 17/22] IntelliJ inspections fixes --- .../druid/catalog/http/CatalogResource.java | 4 +- .../catalog/storage/sql/CatalogManager.java | 2 +- .../http/catalog/CatalogResourceTest.java | 2 +- .../druid/server/http/catalog/EditorTest.java | 48 +++++++++---------- 4 files changed, 28 insertions(+), 28 deletions(-) diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java index 5f5de8861f1e..e21007643cf2 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java @@ -411,7 +411,7 @@ private Response listTablePaths(final HttpServletRequest req) private List getTableMetadataForSchema( final SchemaSpec schema, final HttpServletRequest req - ) throws CatalogException + ) { List tables = catalog.tables().tablesInSchema(schema.name()); Iterable filtered = AuthorizationUtils.filterAuthorizedResources( @@ -441,7 +441,7 @@ private List getTableStatusForSchema( .collect(Collectors.toList()); } - private Response listAllTableMetadata(final HttpServletRequest req) throws CatalogException + private Response listAllTableMetadata(final HttpServletRequest req) { List> tables = new ArrayList<>(); for (SchemaSpec schema : catalog.schemaRegistry().schemas()) { diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java index a723e199e454..98927ae5d418 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java @@ -39,7 +39,7 @@ */ public interface CatalogManager { - public interface TableTransform + interface TableTransform { TableSpec apply(TableMetadata spec) throws CatalogException; } diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java index 7b2fbb184967..dc225d1621e7 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java @@ -634,7 +634,7 @@ public void testHideColumns() resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, postBy(CatalogTests.READER_USER)); read = (TableMetadata) resp.getEntity(); assertEquals( - Arrays.asList("b"), + Collections.singletonList("b"), read.spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) ); assertTrue(read.updateTime() > version); diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/EditorTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/EditorTest.java index bde9249f385c..617ad813e75d 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/EditorTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/EditorTest.java @@ -34,7 +34,7 @@ import org.apache.druid.catalog.model.Columns; import org.apache.druid.catalog.model.TableId; import org.apache.druid.catalog.model.TableMetadata; -import org.apache.druid.catalog.model.table.DatasourceDefn; +import org.apache.druid.catalog.model.table.AbstractDatasourceDefn; import org.apache.druid.catalog.model.table.DatasourceDefn.DatasourceColumnDefn; import org.apache.druid.catalog.model.table.TableBuilder; import org.apache.druid.catalog.storage.CatalogStorage; @@ -168,21 +168,21 @@ public void testHideColumns() throws CatalogException cmd = new HideColumns(Arrays.asList("a", "b")); assertEquals( Arrays.asList("a", "b"), - doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + doEdit(tableName, cmd).spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) ); // Hide starting from an empty list. Map props = new HashMap<>(table.spec().properties()); - props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList()); + props.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList()); catalog.tables().replace(table.withProperties(props)); assertEquals( Arrays.asList("a", "b"), - doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + doEdit(tableName, cmd).spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) ); // Hide, but all are duplicates props = new HashMap<>(table.spec().properties()); - props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b", "c")); + props.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b", "c")); catalog.tables().replace(table.withProperties(props)); cmd = new HideColumns(Arrays.asList("b", "c")); assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); @@ -191,17 +191,17 @@ public void testHideColumns() throws CatalogException cmd = new HideColumns(Arrays.asList("b", "d")); assertEquals( Arrays.asList("a", "b", "c", "d"), - doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + doEdit(tableName, cmd).spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) ); // Hide with duplicates props = new HashMap<>(table.spec().properties()); - props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b")); + props.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b")); catalog.tables().replace(table.withProperties(props)); cmd = new HideColumns(Arrays.asList("b", "d", "b", "d")); assertEquals( Arrays.asList("a", "b", "d"), - doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + doEdit(tableName, cmd).spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) ); } @@ -227,24 +227,24 @@ public void testUnhideColumns() throws CatalogException // Unhide starting from an empty list. Map props = new HashMap<>(table.spec().properties()); - props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList()); + props.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList()); catalog.tables().replace(table.withProperties(props)); assertNull( - doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + doEdit(tableName, cmd).spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) ); // Unhide starting with an empty list with (non-existing) columns to unhide props = new HashMap<>(table.spec().properties()); - props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList()); + props.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList()); catalog.tables().replace(table.withProperties(props)); - cmd = new UnhideColumns(Arrays.asList("a")); + cmd = new UnhideColumns(Collections.singletonList("a")); assertNull( - doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + doEdit(tableName, cmd).spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) ); // Unhide columns which are not actually hidden. props = new HashMap<>(table.spec().properties()); - props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b", "c")); + props.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b", "c")); catalog.tables().replace(table.withProperties(props)); cmd = new UnhideColumns(Arrays.asList("d", "e")); assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); @@ -252,17 +252,17 @@ public void testUnhideColumns() throws CatalogException // Unhide some hidden columns. cmd = new UnhideColumns(Arrays.asList("a", "c", "a", "d")); assertEquals( - Arrays.asList("b"), - doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + Collections.singletonList("b"), + doEdit(tableName, cmd).spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) ); // Unhide all hidden columns props = new HashMap<>(table.spec().properties()); - props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b", "c")); + props.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b", "c")); catalog.tables().replace(table.withProperties(props)); cmd = new UnhideColumns(Arrays.asList("a", "c", "b", "d")); assertNull( - doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + doEdit(tableName, cmd).spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) ); } @@ -292,7 +292,7 @@ public void testDropColumns() throws CatalogException // Drop some columns, including dups. cmd = new DropColumns(Arrays.asList("c", "a", "c", "d")); assertEquals( - Arrays.asList("b"), + Collections.singletonList("b"), CatalogUtils.columnNames(doEdit(tableName, cmd).spec().columns()) ); @@ -329,7 +329,7 @@ public void testUpdateProperties() throws CatalogException // Remove a required property Map updates1 = new HashMap<>(); - updates1.put(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, null); + updates1.put(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, null); assertThrows( CatalogException.class, () -> new TableEditor( @@ -342,11 +342,11 @@ public void testUpdateProperties() throws CatalogException // Add and update properties Map updates = new HashMap<>(); - updates.put(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H"); + updates.put(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H"); updates.put("foo", "bar"); cmd = new UpdateProperties(updates); Map expected = ImmutableMap.of( - DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H", + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H", "foo", "bar" ); assertEquals( @@ -359,7 +359,7 @@ public void testUpdateProperties() throws CatalogException updates.put("foo", "mumble"); cmd = new UpdateProperties(updates); expected = ImmutableMap.of( - DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H", + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H", "foo", "mumble" ); assertEquals( @@ -372,7 +372,7 @@ public void testUpdateProperties() throws CatalogException updates.put("foo", null); cmd = new UpdateProperties(updates); expected = ImmutableMap.of( - DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H" + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H" ); assertEquals( expected, From 5de07ce9341ac324b8bf1acd1844fa8398e6a8a3 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Thu, 3 Nov 2022 14:22:20 -0700 Subject: [PATCH 18/22] Revision from review comments --- .../druid/catalog/http/CatalogResource.java | 32 ++++- .../apache/druid/catalog/http/MoveColumn.java | 79 ------------ .../druid/catalog/http/TableEditRequest.java | 75 ++++++++++- .../druid/catalog/http/TableEditor.java | 25 +++- .../catalog/storage/sql/CatalogManager.java | 30 ++++- .../storage/sql/SQLCatalogManager.java | 119 ++++++++++-------- .../druid/catalog/sync/CacheNotifier.java | 2 + .../catalog/sync/CachedMetadataCatalog.java | 34 +++-- .../catalog/sync/CatalogUpdateNotifier.java | 9 +- .../http/catalog/CatalogResourceTest.java | 2 +- .../druid/server/http/catalog/EditorTest.java | 2 +- .../testsEx/catalog/ITCatalogRestTest.java | 2 +- .../druid/catalog/model/ColumnSpec.java | 26 ++++ .../druid/catalog/model/TableMetadata.java | 18 +++ .../apache/druid/catalog/model/TableSpec.java | 6 +- 15 files changed, 295 insertions(+), 166 deletions(-) delete mode 100644 extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/MoveColumn.java diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java index e21007643cf2..91306ac6ba28 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java @@ -100,7 +100,8 @@ public CatalogResource( * Create or update a new table containing the given table specification. * Supports three use cases: *

    - *
  • "create if not exists": default use case with no options.
  • + *
  • "create": default use case with no options: returns an error if + * a table of the same name already exists.
  • *
  • "create or update": set {@code overwrite=true}.
  • *
  • "update": set {@code version} to the expected current version. * This form enforces optimistic locking.
  • @@ -115,6 +116,7 @@ public CatalogResource( * match. If not (or if the table does not exist), returns an error. * @param overwrite if {@code true}, then overwrites any existing table. * If {@code false}, then the operation fails if the table already exists. + * Ignored if a version is specified. * @param req the HTTP request used for authorization. */ @POST @@ -134,7 +136,7 @@ public Response postTable( final SchemaSpec schema = validateSchema(dbSchema, true); validateTableName(name); authorizeTable(schema, name, Action.WRITE, req); - validateTableSpec(schema, name, spec); + validateTableSpec(schema, spec); final TableMetadata table = TableMetadata.newTable(TableId.of(dbSchema, name), spec); try { catalog.validate(table); @@ -235,6 +237,21 @@ public Response deleteTable( // --------------------------------------------------------------------- // Modify a table within the catalog + /** + * Modify an existing table. The edit operations perform incremental changes + * on a table spec, avoiding the need for the client to download the entire + * spec to make common changes. The incremental nature avoids the need for + * optimistic concurrency control using versions: the request applies the + * change within a transaction using actual locking. The operations are + * designed so that, in most cases, the results are easy to predict even if + * the table spec changed between the time it was retrieve and the edit operation + * is submitted. + * + * @param dbSchema The name of the schema that holds the table. + * @param name The name of the table definition to delete. The user must have + * write access. + * @param editRequest The operation to perform. See the classes for details. + */ @POST @Path("/schemas/{schema}/tables/{name}/edit") @Consumes(MediaType.APPLICATION_JSON) @@ -263,6 +280,9 @@ public Response editTable( /** * Retrieves the list of all Druid schema names, all table names, or * all table metadata. + * + * @param format the format of the response. See the code for the + * available formats */ @GET @Path("/schemas") @@ -298,7 +318,9 @@ public Response getSchemas( * which will probably differ from the list of actual tables. For example, for * the read-only schemas, there will be no table definitions. * - * @param dbSchema The Druid schema to query. The user must have read access. + * @param schema The Druid schema to query. The user must have read access. + * @param format the format of the response. See the code for the + * available formats */ @GET @Path("/schemas/{schema}/tables") @@ -431,7 +453,7 @@ private List getTableMetadataForSchema( private List getTableStatusForSchema( final SchemaSpec schema, final HttpServletRequest req - ) throws CatalogException + ) { // Crude but effective, assuming low volume: get all the data, and throw away // the columns and properties. @@ -498,7 +520,7 @@ private void validateTableName(String name) throws CatalogException } } - private void validateTableSpec(SchemaSpec schema, String name, TableSpec spec) throws CatalogException + private void validateTableSpec(SchemaSpec schema, TableSpec spec) throws CatalogException { // The given table spec has to be valid for the given schema. try { diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/MoveColumn.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/MoveColumn.java deleted file mode 100644 index 941ce4862f4b..000000000000 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/MoveColumn.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.catalog.http; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import javax.annotation.Nullable; - -import java.util.Objects; - -/** - * JSON payload for the reorder column API. - */ -public class MoveColumn extends TableEditRequest -{ - public enum Position - { - FIRST, - LAST, - BEFORE, - AFTER - } - - @JsonProperty - public final String column; - @JsonProperty - public final Position where; - @Nullable - @JsonProperty - public final String anchor; - - @JsonCreator - public MoveColumn( - @JsonProperty("column") final String column, - @JsonProperty("where") final Position where, - @JsonProperty("anchor") @Nullable final String anchor - ) - { - this.column = column; - this.where = where; - this.anchor = anchor; - } - - @Override - public boolean equals(Object o) - { - if (o == null || o.getClass() != getClass()) { - return false; - } - MoveColumn other = (MoveColumn) o; - return Objects.equals(this.column, other.column) - && this.where == other.where - && Objects.equals(this.anchor, other.anchor); - } - - @Override - public int hashCode() - { - return Objects.hash(column, where, anchor); - } -} diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditRequest.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditRequest.java index 63adb4a0cfc2..79136d4a0595 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditRequest.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditRequest.java @@ -25,22 +25,40 @@ import com.fasterxml.jackson.annotation.JsonSubTypes.Type; import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.catalog.model.ColumnSpec; -import org.apache.druid.data.input.InputSource; + +import javax.annotation.Nullable; import java.util.List; import java.util.Map; -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = InputSource.TYPE_PROPERTY) +/** + * Set of "edit" operations that can be applied to a table via the REST API + * "edit" message. Allows the user to apply selected changes without the + * need to download the entire spec and without the need for optimistic + * locking. See the subclasses for the specific set of supported operations. + *

    + * These operations are used to deserialize the operation. The Jackson-provided + * type property identifies the kind of operation. Validation of each request + * is done in {@link TableEditor} to allow control over any errors returned to + * the REST caller. (If the caller provide an invalid type, or a badly-formed + * JSON object, then the error message will, unfortunately, be generic.) + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = TableEditRequest.TYPE_PROPERTY) @JsonSubTypes(value = { @Type(name = "hideColumns", value = TableEditRequest.HideColumns.class), @Type(name = "unhideColumns", value = TableEditRequest.UnhideColumns.class), @Type(name = "dropColumns", value = TableEditRequest.DropColumns.class), @Type(name = "updateProperties", value = TableEditRequest.UpdateProperties.class), @Type(name = "updateColumns", value = TableEditRequest.UpdateColumns.class), - @Type(name = "moveColumn", value = MoveColumn.class), + @Type(name = "moveColumn", value = TableEditRequest.MoveColumn.class), }) public class TableEditRequest { + public static final String TYPE_PROPERTY = "type"; + + /** + * Add the given column names to the property that lists the hidden columns. + */ public static class HideColumns extends TableEditRequest { @JsonProperty("columns") @@ -53,6 +71,9 @@ public HideColumns(@JsonProperty("columns") List columns) } } + /** + * Remove the given column names from the property that lists the hidden columns. + */ public static class UnhideColumns extends TableEditRequest { @JsonProperty("columns") @@ -65,6 +86,9 @@ public UnhideColumns(@JsonProperty("columns") List columns) } } + /** + * Remove one or more columns from the list of columns from this table. + */ public static class DropColumns extends TableEditRequest { @JsonProperty("columns") @@ -77,6 +101,10 @@ public DropColumns(@JsonProperty("columns") List columns) } } + /** + * Update the set of properties with the given changes. Properties with a null + * value are removed from the table spec. Those with non-null values are updated. + */ public static class UpdateProperties extends TableEditRequest { @JsonProperty("properties") @@ -89,6 +117,13 @@ public UpdateProperties(@JsonProperty("properties") Map properti } } + /** + * Update the list of columns. Columns that match the name of existing columns + * are used to update the column. Values provided here replace those in the table + * spec in the DB. Properties are merged as in {@link UpdateProperties}. If the + * column given here does not yet exist in the table spec, then it is added at the + * end of the existing columns list. + */ public static class UpdateColumns extends TableEditRequest { @JsonProperty("columns") @@ -100,4 +135,38 @@ public UpdateColumns(@JsonProperty("columns") List columns) this.columns = columns; } } + + /** + * Move a column within the list of columns for a TableSpec. + */ + public static class MoveColumn extends TableEditRequest + { + public enum Position + { + FIRST, + LAST, + BEFORE, + AFTER + } + + @JsonProperty + public final String column; + @JsonProperty + public final Position where; + @Nullable + @JsonProperty + public final String anchor; + + @JsonCreator + public MoveColumn( + @JsonProperty("column") final String column, + @JsonProperty("where") final Position where, + @JsonProperty("anchor") @Nullable final String anchor + ) + { + this.column = column; + this.where = where; + this.anchor = anchor; + } + } } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java index f018a1067e5d..b16995123514 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java @@ -21,9 +21,10 @@ import com.google.common.base.Strings; import org.apache.druid.catalog.CatalogException; -import org.apache.druid.catalog.http.MoveColumn.Position; import org.apache.druid.catalog.http.TableEditRequest.DropColumns; import org.apache.druid.catalog.http.TableEditRequest.HideColumns; +import org.apache.druid.catalog.http.TableEditRequest.MoveColumn; +import org.apache.druid.catalog.http.TableEditRequest.MoveColumn.Position; import org.apache.druid.catalog.http.TableEditRequest.UnhideColumns; import org.apache.druid.catalog.http.TableEditRequest.UpdateColumns; import org.apache.druid.catalog.http.TableEditRequest.UpdateProperties; @@ -47,6 +48,18 @@ import java.util.Map; import java.util.Set; +/** + * Performs an incremental update of an existing table. Allows the user to + * provide very specific "edit" commands without the need to download the + * entire table spec to make a typical change. + *

    + * The process is that the edit "request" is in the form of a subclass of + * {@link TableEditRequest}. The DB layer starts a transaction, then calls + * back to this class to perform the edit. The DB layer then updates the + * table spec in the DB with the returned information. This ensures that + * a) we use actual (non-optimistic) locking, and b) allows the edit + * operation itself to reside here and not in the DB layer. + */ public class TableEditor { private final CatalogStorage catalog; @@ -64,6 +77,14 @@ public TableEditor( this.editRequest = editRequest; } + /** + * Perform the edit operation. + * + * @return the updated table version, or 0 if the operation was not applied. + * the 0 value indicates, say, an empty or no-op edit request + * @throws CatalogException for errors. Mapped to an HTTP response in the + * {@link CatalogResource} class + */ public long go() throws CatalogException { if (editRequest instanceof HideColumns) { @@ -207,7 +228,7 @@ private TableSpec applyDropColumns(final TableMetadata table, final List if (CollectionUtils.isNullOrEmpty(existingColumns)) { return null; } - Set drop = new HashSet(toDrop); + Set drop = new HashSet<>(toDrop); List revised = new ArrayList<>(); for (ColumnSpec col : existingColumns) { if (!drop.contains(col.name())) { diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java index 98927ae5d418..cb1733b54f0a 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java @@ -27,6 +27,8 @@ import org.apache.druid.catalog.model.TableSpec; import org.apache.druid.catalog.sync.CatalogUpdateListener; +import javax.annotation.Nullable; + import java.util.List; /** @@ -39,9 +41,20 @@ */ public interface CatalogManager { + /** + * Perform a transform (edit) of the table metadata. The metadata + * argument provided contains the table ID and the portion of the + * spec requested for the given operation: either properties or + * columns. + *

    + * The return value is either the updated spec to be written to the + * DB, or {@link null}, which indicates that no change was needed. + * When non-null, the spec will contain a revised version of either + * the columns or the properties, as determined by the specific operation. + */ interface TableTransform { - TableSpec apply(TableMetadata spec) throws CatalogException; + @Nullable TableSpec apply(TableMetadata metadata) throws CatalogException; } /** @@ -97,9 +110,22 @@ interface TableTransform * * @param id the table to update * @param transform the transform to apply to the table properties - * @return the update timestamp (version) of the updated record + * @return the update timestamp (version) of the updated record, or 0 if + * the transform returns {@code null}, which indicates no change + * is needed */ long updateProperties(TableId id, TableTransform transform) throws CatalogException; + + /** + * Update the table columns incrementally using the transform provided. Performs the update + * in a transaction to ensure the read and write are atomic. + * + * @param id the table to update + * @param transform the transform to apply to the table columns + * @return the update timestamp (version) of the updated record, or 0 if + * the transform returns {@code null}, which indicates no change + * is needed + */ long updateColumns(TableId id, TableTransform transform) throws CatalogException; /** diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java index 40b66c17e149..fa24bd825613 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java @@ -62,6 +62,15 @@ public class SQLCatalogManager implements CatalogManager { public static final String TABLES_TABLE = "tableDefs"; + private static final String SCHEMA_NAME_COL = "schemaName"; + private static final String TABLE_NAME_COL = "name"; + private static final String CREATION_TIME_COL = "creationTime"; + private static final String UPDATE_TIME_COL = "updateTime"; + private static final String STATE_COL = "state"; + private static final String TABLE_TYPE_COL = "tableType"; + private static final String PROPERTIES_COL = "properties"; + private static final String COLUMNS_COL = "columns"; + private final MetadataStorageManager metastoreManager; private final SQLMetadataConnector connector; private final ObjectMapper jsonMapper; @@ -142,14 +151,14 @@ public Long withHandle(Handle handle) throws DuplicateKeyException final long updateTime = System.currentTimeMillis(); final Update stmt = handle .createStatement(statement(INSERT_TABLE)) - .bind("schemaName", table.id().schema()) - .bind("name", table.id().name()) - .bind("creationTime", updateTime) - .bind("updateTime", updateTime) - .bind("state", TableMetadata.TableState.ACTIVE.code()) - .bind("tableType", spec.type()) - .bind("properties", JacksonUtils.toBytes(jsonMapper, spec.properties())) - .bind("columns", JacksonUtils.toBytes(jsonMapper, spec.columns())); + .bind(SCHEMA_NAME_COL, table.id().schema()) + .bind(TABLE_NAME_COL, table.id().name()) + .bind(CREATION_TIME_COL, updateTime) + .bind(UPDATE_TIME_COL, updateTime) + .bind(STATE_COL, TableMetadata.TableState.ACTIVE.code()) + .bind(TABLE_TYPE_COL, spec.type()) + .bind(PROPERTIES_COL, JacksonUtils.toBytes(jsonMapper, spec.properties())) + .bind(COLUMNS_COL, JacksonUtils.toBytes(jsonMapper, spec.columns())); try { stmt.execute(); } @@ -196,8 +205,8 @@ public TableMetadata withHandle(Handle handle) throws NotFoundException final Query> query = handle .createQuery(statement(SELECT_TABLE)) .setFetchSize(connector.getStreamingFetchSize()) - .bind("schemaName", id.schema()) - .bind("name", id.name()); + .bind(SCHEMA_NAME_COL, id.schema()) + .bind(TABLE_NAME_COL, id.name()); final ResultIterator resultIterator = query.map((index, r, ctx) -> new TableMetadata( @@ -249,17 +258,17 @@ public TableMetadata withHandle(Handle handle) throws NotFoundException final long updateTime = System.currentTimeMillis(); final int updateCount = handle .createStatement(statement(REPLACE_SPEC_STMT)) - .bind("schemaName", id.schema()) - .bind("name", id.name()) - .bind("tableType", spec.type()) - .bind("properties", JacksonUtils.toBytes(jsonMapper, spec.properties())) - .bind("columns", JacksonUtils.toBytes(jsonMapper, spec.columns())) - .bind("updateTime", updateTime) + .bind(SCHEMA_NAME_COL, id.schema()) + .bind(TABLE_NAME_COL, id.name()) + .bind(TABLE_TYPE_COL, spec.type()) + .bind(PROPERTIES_COL, JacksonUtils.toBytes(jsonMapper, spec.properties())) + .bind(COLUMNS_COL, JacksonUtils.toBytes(jsonMapper, spec.columns())) + .bind(UPDATE_TIME_COL, updateTime) .execute(); if (updateCount == 0) { throw tableNotFound(id); } - return table.fromInsert(updateTime); + return table.asUpdate(updateTime); } } ); @@ -274,6 +283,7 @@ public TableMetadata withHandle(Handle handle) throws NotFoundException } } + private static final String OLD_VERSION_PARAM = "oldVersion"; private static final String UPDATE_SPEC_STMT = REPLACE_SPEC_STMT + " AND updateTime = :oldVersion"; @@ -293,13 +303,13 @@ public TableMetadata withHandle(Handle handle) throws NotFoundException final long updateTime = System.currentTimeMillis(); final int updateCount = handle .createStatement(statement(UPDATE_SPEC_STMT)) - .bind("schemaName", id.schema()) - .bind("name", id.name()) - .bind("tableType", spec.type()) - .bind("properties", JacksonUtils.toBytes(jsonMapper, spec.properties())) - .bind("columns", JacksonUtils.toBytes(jsonMapper, spec.columns())) - .bind("updateTime", updateTime) - .bind("oldVersion", oldVersion) + .bind(SCHEMA_NAME_COL, id.schema()) + .bind(TABLE_NAME_COL, id.name()) + .bind(TABLE_TYPE_COL, spec.type()) + .bind(PROPERTIES_COL, JacksonUtils.toBytes(jsonMapper, spec.properties())) + .bind(COLUMNS_COL, JacksonUtils.toBytes(jsonMapper, spec.columns())) + .bind(UPDATE_TIME_COL, updateTime) + .bind(OLD_VERSION_PARAM, oldVersion) .execute(); if (updateCount == 0) { throw new NotFoundException( @@ -354,8 +364,8 @@ public TableMetadata withHandle(Handle handle) throws CatalogException final Query> query = handle .createQuery(statement(SELECT_PROPERTIES_STMT)) .setFetchSize(connector.getStreamingFetchSize()) - .bind("schemaName", id.schema()) - .bind("name", id.name()); + .bind(SCHEMA_NAME_COL, id.schema()) + .bind(TABLE_NAME_COL, id.name()); final ResultIterator resultIterator = query .map((index, r, ctx) -> @@ -381,10 +391,10 @@ public TableMetadata withHandle(Handle handle) throws CatalogException final long updateTime = System.currentTimeMillis(); final int updateCount = handle .createStatement(statement(UPDATE_PROPERTIES_STMT)) - .bind("schemaName", id.schema()) - .bind("name", id.name()) - .bind("properties", JacksonUtils.toBytes(jsonMapper, revised.properties())) - .bind("updateTime", updateTime) + .bind(SCHEMA_NAME_COL, id.schema()) + .bind(TABLE_NAME_COL, id.name()) + .bind(PROPERTIES_COL, JacksonUtils.toBytes(jsonMapper, revised.properties())) + .bind(UPDATE_TIME_COL, updateTime) .execute(); if (updateCount == 0) { // Should never occur because we're holding a lock. @@ -446,8 +456,8 @@ public TableMetadata withHandle(Handle handle) throws CatalogException final Query> query = handle .createQuery(statement(SELECT_COLUMNS_STMT)) .setFetchSize(connector.getStreamingFetchSize()) - .bind("schemaName", id.schema()) - .bind("name", id.name()); + .bind(SCHEMA_NAME_COL, id.schema()) + .bind(TABLE_NAME_COL, id.name()); final ResultIterator resultIterator = query .map((index, r, ctx) -> @@ -473,10 +483,10 @@ public TableMetadata withHandle(Handle handle) throws CatalogException final long updateTime = System.currentTimeMillis(); final int updateCount = handle .createStatement(statement(UPDATE_COLUMNS_STMT)) - .bind("schemaName", id.schema()) - .bind("name", id.name()) - .bind("columns", JacksonUtils.toBytes(jsonMapper, revised.columns())) - .bind("updateTime", updateTime) + .bind(SCHEMA_NAME_COL, id.schema()) + .bind(TABLE_NAME_COL, id.name()) + .bind(COLUMNS_COL, JacksonUtils.toBytes(jsonMapper, revised.columns())) + .bind(UPDATE_TIME_COL, updateTime) .execute(); if (updateCount == 0) { // Should never occur because we're holding a lock. @@ -506,9 +516,9 @@ public TableMetadata withHandle(Handle handle) throws CatalogException } } - private static final String UPDATE_STATE = + private static final String MARK_DELETING_STMT = "UPDATE %s\n SET\n" + - " state = :state,\n" + + " state = 'D',\n" + " updateTime = :updateTime\n" + "WHERE schemaName = :schemaName\n" + " AND name = :name\n"; @@ -524,11 +534,10 @@ public Long withHandle(Handle handle) { long updateTime = System.currentTimeMillis(); int updateCount = handle - .createStatement(statement(UPDATE_STATE)) - .bind("schemaName", id.schema()) - .bind("name", id.name()) - .bind("updateTime", updateTime) - .bind("state", TableMetadata.TableState.DELETING.code()) + .createStatement(statement(MARK_DELETING_STMT)) + .bind(SCHEMA_NAME_COL, id.schema()) + .bind(TABLE_NAME_COL, id.name()) + .bind(UPDATE_TIME_COL, updateTime) .execute(); sendDeletion(id); return updateCount == 1 ? updateTime : 0; @@ -537,7 +546,7 @@ public Long withHandle(Handle handle) ); } - private static final String DELETE_TABLE = + private static final String DELETE_TABLE_STMT = "DELETE FROM %s\n" + "WHERE schemaName = :schemaName\n" + " AND name = :name\n"; @@ -553,9 +562,9 @@ public void delete(TableId id) throws NotFoundException public Void withHandle(Handle handle) throws NotFoundException { int updateCount = handle - .createStatement(statement(DELETE_TABLE)) - .bind("schemaName", id.schema()) - .bind("name", id.name()) + .createStatement(statement(DELETE_TABLE_STMT)) + .bind(SCHEMA_NAME_COL, id.schema()) + .bind(TABLE_NAME_COL, id.name()) .execute(); if (updateCount == 0) { throw tableNotFound(id); @@ -575,7 +584,7 @@ public Void withHandle(Handle handle) throws NotFoundException } } - private static final String SELECT_ALL_TABLE_PATHS = + private static final String SELECT_ALL_TABLE_PATHS_STMT = "SELECT schemaName, name\n" + "FROM %s\n" + "ORDER BY schemaName, name"; @@ -590,7 +599,7 @@ public List allTablePaths() public List withHandle(Handle handle) { Query> query = handle - .createQuery(statement(SELECT_ALL_TABLE_PATHS)) + .createQuery(statement(SELECT_ALL_TABLE_PATHS_STMT)) .setFetchSize(connector.getStreamingFetchSize()); final ResultIterator resultIterator = query.map((index, r, ctx) -> @@ -602,7 +611,7 @@ public List withHandle(Handle handle) ); } - private static final String SELECT_TABLE_NAMES_IN_SCHEMA = + private static final String SELECT_TABLE_NAMES_IN_SCHEMA_STMT = "SELECT name\n" + "FROM %s\n" + "WHERE schemaName = :schemaName\n" + @@ -618,8 +627,8 @@ public List tableNamesInSchema(String dbSchema) public List withHandle(Handle handle) { Query> query = handle - .createQuery(statement(SELECT_TABLE_NAMES_IN_SCHEMA)) - .bind("schemaName", dbSchema) + .createQuery(statement(SELECT_TABLE_NAMES_IN_SCHEMA_STMT)) + .bind(SCHEMA_NAME_COL, dbSchema) .setFetchSize(connector.getStreamingFetchSize()); final ResultIterator resultIterator = query.map((index, r, ctx) -> @@ -631,7 +640,7 @@ public List withHandle(Handle handle) ); } - private static final String SELECT_TABLES_IN_SCHEMA = + private static final String SELECT_TABLES_IN_SCHEMA_STMT = "SELECT name, creationTime, updateTime, state, tableType, properties, columns\n" + "FROM %s\n" + "WHERE schemaName = :schemaName\n" + @@ -647,8 +656,8 @@ public List tablesInSchema(String dbSchema) public List withHandle(Handle handle) { Query> query = handle - .createQuery(statement(SELECT_TABLES_IN_SCHEMA)) - .bind("schemaName", dbSchema) + .createQuery(statement(SELECT_TABLES_IN_SCHEMA_STMT)) + .bind(SCHEMA_NAME_COL, dbSchema) .setFetchSize(connector.getStreamingFetchSize()); final ResultIterator resultIterator = query.map((index, r, ctx) -> diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CacheNotifier.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CacheNotifier.java index 8118076c103b..be7a7281de4d 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CacheNotifier.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CacheNotifier.java @@ -77,6 +77,7 @@ public CacheNotifier( public void start() { + LOG.info("Starting Catalog sync"); exec.submit(() -> { while (!Thread.interrupted()) { try { @@ -117,5 +118,6 @@ public void stop() LOG.warn("Shutting down Catalog sync with %d unsent notifications", updates.size()); } exec.shutdownNow(); + LOG.info("Catalog sync stopped"); } } diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java index 4f5da642ec46..2305a1278bc9 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java @@ -35,7 +35,9 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; +import java.util.TreeMap; import java.util.concurrent.ConcurrentHashMap; /** @@ -52,7 +54,7 @@ * Both tables and schemas are cached. In particular, if a table or * schema is requested, and does not exist in the base catalog, then * that schema is marked as not existing and won't be fetched again. - * + *

    * The cache is updated via an update facility which either flushes * the cache (crude) or listens to the base catalog for updates and * populates the cache with updates. For a local cache, the DB layer @@ -62,7 +64,18 @@ public class CachedMetadataCatalog implements MetadataCatalog, CatalogUpdateList { private static final Logger LOG = new Logger(CachedMetadataCatalog.class); + /** + * Indicates that the schema entry has not yet been fetched from the catalog. + */ public static final int NOT_FETCHED = -1; + + /** + * Indicates that the schema was requested from the catalog, and the catalog + * reported that no such schema exists. Saves pinging the catalog over and + * over for an undefined schema. The catalog will let us know if the schema + * becomes defined (which it won't because, at present, schemas are hard-coded + * to a fixed set.) + */ public static final int UNDEFINED = 0; /** @@ -84,14 +97,14 @@ private class SchemaEntry { private final SchemaSpec schema; private long version = NOT_FETCHED; - private final ConcurrentHashMap cache = new ConcurrentHashMap<>(); + private final Map cache = new TreeMap<>(); protected SchemaEntry(SchemaSpec schema) { this.schema = schema; } - protected TableMetadata resolveTable(TableId tableId) + protected synchronized TableMetadata resolveTable(TableId tableId) { TableEntry entry = cache.computeIfAbsent( tableId.name(), @@ -111,15 +124,14 @@ public synchronized List tables() cache.put(table.id().name(), new TableEntry(table)); } } - List orderedTables = new ArrayList<>(); // Get the list of actual tables; excluding any cached "misses". - cache.forEach((k, v) -> { - if (v.table != null) { - orderedTables.add(v.table); + List orderedTables = new ArrayList<>(cache.size()); + for (TableEntry entry : cache.values()) { + if (entry.table != null) { + orderedTables.add(entry.table); } - }); - orderedTables.sort((e1, e2) -> e1.id().name().compareTo(e2.id().name())); + } return orderedTables; } @@ -162,7 +174,7 @@ public synchronized void update(UpdateEvent event) version = Math.max(version, table.updateTime()); } - protected TableEntry computeCreate(TableEntry entry, TableMetadata update) + private TableEntry computeCreate(TableEntry entry, TableMetadata update) { if (entry != null && entry.table != null) { LOG.warn("Received creation event for existing entry: %s", update.id().sqlName()); @@ -248,7 +260,7 @@ private boolean checkVersion(TableEntry entry, TableMetadata update) return true; } - public Set tableNames() + public synchronized Set tableNames() { Set tables = new HashSet<>(); cache.forEach((k, v) -> { diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateNotifier.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateNotifier.java index deeab56cd256..f60507c8e2c9 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateNotifier.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateNotifier.java @@ -31,6 +31,7 @@ import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.server.DruidNode; import org.joda.time.Duration; @@ -44,14 +45,12 @@ * Global update notifier for the catalog. Registers itself as a catalog * listener, then uses the common cache notifier to send Smile-encoded JSON * updates to broker nodes discovered from node discovery (typically ZooKeeper.) - *

    - * Deletes are encoded as a table update with a table definition of a special - * tombstone type. This saves having the need for two endpoints, or having - * a wrapper class to handle deletes. */ @ManageLifecycle public class CatalogUpdateNotifier implements CatalogUpdateListener { + private static final EmittingLogger LOG = new EmittingLogger(CatalogUpdateNotifier.class); + private static final String CALLER_NAME = "Catalog Sync"; private static final long TIMEOUT_MS = 5000; @@ -88,12 +87,14 @@ public CatalogUpdateNotifier( public void start() { notifier.start(); + LOG.info("Catalog catalog update notifier started"); } @LifecycleStop public void stop() { notifier.stop(); + LOG.info("Catalog catalog update notifier stopped"); } @Override diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java index dc225d1621e7..00cced8b9759 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java @@ -20,10 +20,10 @@ package org.apache.druid.server.http.catalog; import org.apache.druid.catalog.http.CatalogResource; -import org.apache.druid.catalog.http.MoveColumn; import org.apache.druid.catalog.http.TableEditRequest; import org.apache.druid.catalog.http.TableEditRequest.DropColumns; import org.apache.druid.catalog.http.TableEditRequest.HideColumns; +import org.apache.druid.catalog.http.TableEditRequest.MoveColumn; import org.apache.druid.catalog.http.TableEditRequest.UnhideColumns; import org.apache.druid.catalog.model.CatalogUtils; import org.apache.druid.catalog.model.Columns; diff --git a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/EditorTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/EditorTest.java index 617ad813e75d..ab12a8e2455e 100644 --- a/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/EditorTest.java +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/EditorTest.java @@ -21,10 +21,10 @@ import org.apache.curator.shaded.com.google.common.collect.ImmutableMap; import org.apache.druid.catalog.CatalogException; -import org.apache.druid.catalog.http.MoveColumn; import org.apache.druid.catalog.http.TableEditRequest; import org.apache.druid.catalog.http.TableEditRequest.DropColumns; import org.apache.druid.catalog.http.TableEditRequest.HideColumns; +import org.apache.druid.catalog.http.TableEditRequest.MoveColumn; import org.apache.druid.catalog.http.TableEditRequest.UnhideColumns; import org.apache.druid.catalog.http.TableEditRequest.UpdateColumns; import org.apache.druid.catalog.http.TableEditRequest.UpdateProperties; diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java index f18c78c3e170..a43886424070 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java @@ -20,9 +20,9 @@ package org.apache.druid.testsEx.catalog; import com.google.inject.Inject; -import org.apache.druid.catalog.http.MoveColumn; import org.apache.druid.catalog.http.TableEditRequest.DropColumns; import org.apache.druid.catalog.http.TableEditRequest.HideColumns; +import org.apache.druid.catalog.http.TableEditRequest.MoveColumn; import org.apache.druid.catalog.http.TableEditRequest.UnhideColumns; import org.apache.druid.catalog.model.CatalogUtils; import org.apache.druid.catalog.model.TableId; diff --git a/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java b/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java index 119ee182140c..9830ad8a3640 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java +++ b/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java @@ -40,9 +40,35 @@ @UnstableApi public class ColumnSpec { + /** + * The type of column spec (not the column data type). For example, in a + * cube, dimensions and measures have distinct sets of properties. + */ private final String type; + + /** + * The name of the column as known to the SQL layer. At present, there is no + * support for column aliases, so this is also the column name as physically + * stored in segments. + */ private final String name; + + /** + * The data type of the column expressed as a supported SQL type. The data type here must + * directly match a Druid storage type. So, {@code BIGINT} for {code long}, say. + * This usage does not support Druid's usual "fudging": one cannot use {@link INTEGER} + * to mean {@code long}. The type will likely encode complex and aggregation types + * in the future, though that is not yet supported. The set of valid mappings is + * defined in the {@link Columns} class. + */ private final String sqlType; + + /** + * Properties for the column. At present, these are all user and application defined. + * For example, a UI layer might want to store a display format. Druid may define + * properties in the future. Candidates would be indexing options if/when there are + * choices available per-column. + */ private final Map properties; @JsonCreator diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java index 8af5b1135c2b..d1145e4621af 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java @@ -32,12 +32,30 @@ * REST API level description of a table. Tables have multiple types * as described by subclasses. Stores the operational aspects of a * table, such as its name, creation time, state and spec. + *

    + * Doubles as a "holder" of partial collections of table metadata + * internally. For example, in the "edit" operation, the id and + * a partial spec will be available, the other fields are implicitly + * unset. The set of provided fields is implicit in the code that uses + * the object. (Providing partial information avoids the need to query + * the DB for information that won't actually be used.) * * @see {@link ResolvedTable} for the semantic representation. */ @PublicApi public class TableMetadata { + /** + * State of the metadata table entry (not necessarily of the underlying + * datasource.) A table entry will be Active normally. The Deleting state + * is provided to handle one very specific case: a request to delete a + * datasource. Since datasources are large, and consist of a large number of + * segments, it takes time to unload segments from data nodes, then physically + * delete those segments. The Deleting state says that this process has started. + * It tell the Broker to act as if the table no longer exists in queries, but + * not to allow creation of a new table of the same name until the deletion + * process completes and the table metadata entry is deleted. + */ public enum TableState { ACTIVE("A"), diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java b/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java index 1785a136b1ac..ae0ddf9b993a 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java @@ -26,6 +26,8 @@ import com.google.common.base.Strings; import org.apache.druid.java.util.common.IAE; +import javax.annotation.Nullable; + import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -49,8 +51,8 @@ public class TableSpec @JsonCreator public TableSpec( @JsonProperty("type") final String type, - @JsonProperty("properties") final Map properties, - @JsonProperty("columns") final List columns + @JsonProperty("properties") @Nullable final Map properties, + @JsonProperty("columns") @Nullable final List columns ) { this.type = type; From e4109984f4ac733a9ca30e1c0614b52bdb1e45cf Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Fri, 4 Nov 2022 15:07:35 -0700 Subject: [PATCH 19/22] Revisions from review comments --- .../druid/catalog/http/CatalogResource.java | 101 +++++++++--------- .../catalog/sync/CachedMetadataCatalog.java | 33 +++--- .../druid/catalog/model/ColumnDefn.java | 28 +++-- .../druid/catalog/model/ColumnSpec.java | 2 +- .../apache/druid/catalog/model/Columns.java | 10 +- .../druid/catalog/model/ModelProperties.java | 7 ++ .../catalog/model/ParameterizedDefn.java | 7 ++ .../catalog/model/TableDefnRegistry.java | 20 ++++ .../druid/catalog/model/TableMetadata.java | 10 -- .../apache/druid/catalog/model/TableSpec.java | 8 -- .../model/table/AbstractDatasourceDefn.java | 15 ++- .../catalog/model/table/ClusterKeySpec.java | 10 +- .../catalog/model/table/DatasourceDefn.java | 13 +++ .../model/table/ExternalTableDefn.java | 25 +++-- .../catalog/model/table/HttpTableDefn.java | 6 +- .../catalog/model/table/LocalTableDefn.java | 4 + .../catalog/model/table/TableBuilder.java | 0 17 files changed, 188 insertions(+), 111 deletions(-) rename server/src/{main => test}/java/org/apache/druid/catalog/model/table/TableBuilder.java (100%) diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java index 91306ac6ba28..551ba490c183 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java @@ -107,9 +107,9 @@ public CatalogResource( * This form enforces optimistic locking. *

* - * @param dbSchema The name of the Druid schema, which must be writable + * @param schemaName The name of the Druid schema, which must be writable * and the user must have at least read access. - * @param name The name of the table definition to modify. The user must + * @param tableName The name of the table definition to modify. The user must * have write access to the table. * @param spec The new table definition. * @param version the expected version of an existing table. The version must @@ -124,8 +124,8 @@ public CatalogResource( @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) public Response postTable( - @PathParam("schema") String dbSchema, - @PathParam("name") String name, + @PathParam("schema") String schemaName, + @PathParam("name") String tableName, TableSpec spec, @QueryParam("version") long version, @QueryParam("overwrite") boolean overwrite, @@ -133,11 +133,11 @@ public Response postTable( ) { try { - final SchemaSpec schema = validateSchema(dbSchema, true); - validateTableName(name); - authorizeTable(schema, name, Action.WRITE, req); + final SchemaSpec schema = validateSchema(schemaName, true); + validateTableName(tableName); + authorizeTable(schema, tableName, Action.WRITE, req); validateTableSpec(schema, spec); - final TableMetadata table = TableMetadata.newTable(TableId.of(dbSchema, name), spec); + final TableMetadata table = TableMetadata.newTable(TableId.of(schemaName, tableName), spec); try { catalog.validate(table); } @@ -180,8 +180,8 @@ public Response postTable( * even if there is no datasource of the same name (typically occurs when * the definition is created before the datasource itself.) * - * @param dbSchema The Druid schema. The user must have read access. - * @param name The name of the table within the schema. The user must have + * @param schemaName The Druid schema. The user must have read access. + * @param tableName The name of the table within the schema. The user must have * read access. * @param req the HTTP request used for authorization. * @return the definition for the table, if any. @@ -190,15 +190,15 @@ public Response postTable( @Path("/schemas/{schema}/tables/{name}") @Produces(MediaType.APPLICATION_JSON) public Response getTable( - @PathParam("schema") String dbSchema, - @PathParam("name") String name, + @PathParam("schema") String schemaName, + @PathParam("name") String tableName, @Context final HttpServletRequest req ) { try { - final SchemaSpec schema = validateSchema(dbSchema, false); - authorizeTable(schema, name, Action.READ, req); - final TableMetadata table = catalog.tables().read(new TableId(dbSchema, name)); + final SchemaSpec schema = validateSchema(schemaName, false); + authorizeTable(schema, tableName, Action.READ, req); + final TableMetadata table = catalog.tables().read(new TableId(schemaName, tableName)); return Response.ok().entity(table).build(); } catch (CatalogException e) { @@ -210,23 +210,23 @@ public Response getTable( * Deletes the table definition (but not the underlying table or datasource) * for the given schema and table. * - * @param dbSchema The name of the schema that holds the table. - * @param name The name of the table definition to delete. The user must have + * @param schemaName The name of the schema that holds the table. + * @param tableName The name of the table definition to delete. The user must have * write access. */ @DELETE @Path("/schemas/{schema}/tables/{name}") @Produces(MediaType.APPLICATION_JSON) public Response deleteTable( - @PathParam("schema") String dbSchema, - @PathParam("name") String name, + @PathParam("schema") String schemaName, + @PathParam("name") String tableName, @Context final HttpServletRequest req ) { try { - final SchemaSpec schema = validateSchema(dbSchema, true); - authorizeTable(schema, name, Action.WRITE, req); - catalog.tables().delete(new TableId(dbSchema, name)); + final SchemaSpec schema = validateSchema(schemaName, true); + authorizeTable(schema, tableName, Action.WRITE, req); + catalog.tables().delete(new TableId(schemaName, tableName)); return ok(); } catch (CatalogException e) { @@ -247,8 +247,8 @@ public Response deleteTable( * the table spec changed between the time it was retrieve and the edit operation * is submitted. * - * @param dbSchema The name of the schema that holds the table. - * @param name The name of the table definition to delete. The user must have + * @param schemaName The name of the schema that holds the table. + * @param tableName The name of the table definition to delete. The user must have * write access. * @param editRequest The operation to perform. See the classes for details. */ @@ -257,16 +257,16 @@ public Response deleteTable( @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) public Response editTable( - @PathParam("schema") String dbSchema, - @PathParam("name") String name, + @PathParam("schema") String schemaName, + @PathParam("name") String tableName, TableEditRequest editRequest, @Context final HttpServletRequest req ) { try { - final SchemaSpec schema = validateSchema(dbSchema, true); - authorizeTable(schema, name, Action.WRITE, req); - final long newVersion = new TableEditor(catalog, TableId.of(dbSchema, name), editRequest).go(); + final SchemaSpec schema = validateSchema(schemaName, true); + authorizeTable(schema, tableName, Action.WRITE, req); + final long newVersion = new TableEditor(catalog, TableId.of(schemaName, tableName), editRequest).go(); return okWithVersion(newVersion); } catch (CatalogException e) { @@ -318,7 +318,8 @@ public Response getSchemas( * which will probably differ from the list of actual tables. For example, for * the read-only schemas, there will be no table definitions. * - * @param schema The Druid schema to query. The user must have read access. + * @param schemaName The name of the Druid schema to query. The user must + * have read access. * @param format the format of the response. See the code for the * available formats */ @@ -326,13 +327,13 @@ public Response getSchemas( @Path("/schemas/{schema}/tables") @Produces(MediaType.APPLICATION_JSON) public Response getSchemaTables( - @PathParam("schema") String dbSchema, + @PathParam("schema") String schemaName, @QueryParam("format") String format, @Context final HttpServletRequest req ) { try { - SchemaSpec schema = validateSchema(dbSchema, false); + SchemaSpec schema = validateSchema(schemaName, false); format = Strings.isNullOrEmpty(format) ? NAME_FORMAT : StringUtils.toLowerCase(format); switch (format) { case NAME_FORMAT: @@ -368,14 +369,14 @@ public Response getSchemaTables( @Path(SCHEMA_SYNC) @Produces(MediaType.APPLICATION_JSON) public Response syncSchema( - @PathParam("schema") String dbSchema, + @PathParam("schema") String schemaName, @Context final HttpServletRequest req ) { // Same as the list schemas endpoint for now. This endpoint reserves the right to change // over time as needed, while the user endpoint cannot easily change. try { - SchemaSpec schema = validateSchema(dbSchema, false); + SchemaSpec schema = validateSchema(schemaName, false); return Response.ok().entity(getTableMetadataForSchema(schema, req)).build(); } catch (CatalogException e) { @@ -394,18 +395,17 @@ public Response syncSchema( @Path(TABLE_SYNC) @Produces(MediaType.APPLICATION_JSON) public Response syncTable( - @PathParam("schema") String dbSchema, - @PathParam("name") String name, + @PathParam("schema") String schemaName, + @PathParam("name") String tableName, @Context final HttpServletRequest req ) { - return getTable(dbSchema, name, req); + return getTable(schemaName, tableName, req); } // --------------------------------------------------------------------- // Helper methods - /** * Retrieves the list of all Druid table names for which the user has at * least read access. @@ -539,40 +539,45 @@ private void validateTableSpec(SchemaSpec schema, TableSpec spec) throws Catalog } } - private SchemaSpec validateSchema(String dbSchema, boolean forWrite) throws CatalogException + private SchemaSpec validateSchema(String schemaName, boolean forWrite) throws CatalogException { - if (Strings.isNullOrEmpty(dbSchema)) { + if (Strings.isNullOrEmpty(schemaName)) { throw CatalogException.badRequest("Schema name is required"); } - SchemaSpec schema = catalog.resolveSchema(dbSchema); + SchemaSpec schema = catalog.resolveSchema(schemaName); if (schema == null) { - throw new NotFoundException("Unknown schema %s", dbSchema); + throw new NotFoundException("Unknown schema %s", schemaName); } if (forWrite && !schema.writable()) { throw CatalogException.badRequest( "Cannot modify schema %s", - dbSchema + schemaName ); } return schema; } - private static ResourceAction resourceAction(SchemaSpec schema, String name, Action action) + private static ResourceAction resourceAction(SchemaSpec schema, String tableName, Action action) { - return new ResourceAction(new Resource(name, schema.securityResource()), action); + return new ResourceAction(new Resource(tableName, schema.securityResource()), action); } - private void authorizeTable(SchemaSpec schema, String name, Action action, HttpServletRequest request) throws CatalogException + private void authorizeTable( + final SchemaSpec schema, + final String tableName, + final Action action, + final HttpServletRequest request + ) throws CatalogException { - if (Strings.isNullOrEmpty(name)) { + if (Strings.isNullOrEmpty(tableName)) { throw CatalogException.badRequest("Table name is required"); } if (action == Action.WRITE && !schema.writable()) { throw new ForbiddenException( "Cannot create table definitions in schema: " + schema.name()); } - authorize(schema.securityResource(), name, action, request); + authorize(schema.securityResource(), tableName, action, request); } private void authorize(String resource, String key, Action action, HttpServletRequest request) diff --git a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java index 2305a1278bc9..b30a12ee9274 100644 --- a/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java @@ -35,9 +35,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; -import java.util.Map; import java.util.Set; -import java.util.TreeMap; import java.util.concurrent.ConcurrentHashMap; /** @@ -97,14 +95,17 @@ private class SchemaEntry { private final SchemaSpec schema; private long version = NOT_FETCHED; - private final Map cache = new TreeMap<>(); + private final ConcurrentHashMap cache = new ConcurrentHashMap<>(); protected SchemaEntry(SchemaSpec schema) { this.schema = schema; } - protected synchronized TableMetadata resolveTable(TableId tableId) + /** + * High-frequency by-name table lookup called for every table in every SQL query. + */ + protected TableMetadata resolveTable(TableId tableId) { TableEntry entry = cache.computeIfAbsent( tableId.name(), @@ -113,25 +114,31 @@ protected synchronized TableMetadata resolveTable(TableId tableId) return entry.table; } - public synchronized List tables() + /** + * Low-frequency list of tables sorted by name. + */ + public List tables() { if (version == UNDEFINED) { return Collections.emptyList(); } if (version == NOT_FETCHED) { - List catalogTables = base.tablesForSchema(schema.name()); - for (TableMetadata table : catalogTables) { - cache.put(table.id().name(), new TableEntry(table)); + synchronized (this) { + List catalogTables = base.tablesForSchema(schema.name()); + for (TableMetadata table : catalogTables) { + cache.put(table.id().name(), new TableEntry(table)); + } } } + List orderedTables = new ArrayList<>(); // Get the list of actual tables; excluding any cached "misses". - List orderedTables = new ArrayList<>(cache.size()); - for (TableEntry entry : cache.values()) { - if (entry.table != null) { - orderedTables.add(entry.table); + cache.forEach((k, v) -> { + if (v.table != null) { + orderedTables.add(v.table); } - } + }); + orderedTables.sort((e1, e2) -> e1.id().name().compareTo(e2.id().name())); return orderedTables; } diff --git a/server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.java b/server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.java index a7f4b9ceab30..dd23522deb82 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.java @@ -22,11 +22,15 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.catalog.model.ModelProperties.PropertyDefn; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.StringUtils; import java.util.List; import java.util.Map; +/** + * The definition (meta-meta-data) for a column. Defines the set of allowed + * column types and the set of known properties (of which there are none + * at present.) + */ public class ColumnDefn extends ObjectDefn { /** @@ -55,6 +59,9 @@ public ColumnSpec spec() return spec; } + /** + * Merges an updated version of this column with an existing version. + */ public ResolvedColumn merge(ColumnSpec update) { return new ResolvedColumn(defn, defn.merge(spec, update)); @@ -75,6 +82,16 @@ public ColumnDefn( super(name, typeValue, fields); } + /** + * Merges an updated version of this column with an existing version. + *

+ * The name cannot be changed (it is what links the existing column and the + * update). The SQL type will be that provided in the update, if non-null, else + * the original type. Properties are merged using standard rules: those in the + * update take precedence. Null values in the update remove the existing property, + * non-null values update the property. Any properties in the update but not in + * the existing set, are inserted (if non-null). + */ public ColumnSpec merge(ColumnSpec spec, ColumnSpec update) { String updateType = update.type(); @@ -97,14 +114,5 @@ public void validate(ColumnSpec spec, ObjectMapper jsonMapper) public void validateScalarColumn(ColumnSpec spec) { Columns.validateScalarColumn(spec.name(), spec.sqlType()); - if (Columns.isTimeColumn(spec.name())) { - if (spec.sqlType() != null && !Columns.TIMESTAMP.equalsIgnoreCase(spec.sqlType())) { - throw new IAE(StringUtils.format( - "%s column must have no SQL type or SQL type %s", - Columns.TIME_COLUMN, - Columns.TIMESTAMP - )); - } - } } } diff --git a/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java b/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java index 9830ad8a3640..c8baf082dc43 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java +++ b/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java @@ -56,7 +56,7 @@ public class ColumnSpec /** * The data type of the column expressed as a supported SQL type. The data type here must * directly match a Druid storage type. So, {@code BIGINT} for {code long}, say. - * This usage does not support Druid's usual "fudging": one cannot use {@link INTEGER} + * This usage does not support Druid's usual "fudging": one cannot use {@code INTEGER} * to mean {@code long}. The type will likely encode complex and aggregation types * in the future, though that is not yet supported. The set of valid mappings is * defined in the {@link Columns} class. diff --git a/server/src/main/java/org/apache/druid/catalog/model/Columns.java b/server/src/main/java/org/apache/druid/catalog/model/Columns.java index 23f2db9c9e88..0092584c431f 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/Columns.java +++ b/server/src/main/java/org/apache/druid/catalog/model/Columns.java @@ -52,6 +52,10 @@ public class Columns .put(VARCHAR, ColumnType.STRING) .build(); + private Columns() + { + } + public static boolean isTimestamp(String type) { return TIMESTAMP.equalsIgnoreCase(type.trim()); @@ -75,11 +79,7 @@ public static void validateScalarColumn(String name, String type) if (type == null) { return; } - if (Columns.TIME_COLUMN.equals(name)) { - if (!Columns.isTimestamp(type)) { - throw new IAE("__time column must have type TIMESTAMP"); - } - } else if (!Columns.isScalar(type)) { + if (!Columns.isScalar(type)) { throw new IAE("Not a supported SQL type: " + type); } } diff --git a/server/src/main/java/org/apache/druid/catalog/model/ModelProperties.java b/server/src/main/java/org/apache/druid/catalog/model/ModelProperties.java index c6c786534830..07c9be973e00 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/ModelProperties.java +++ b/server/src/main/java/org/apache/druid/catalog/model/ModelProperties.java @@ -49,6 +49,13 @@ interface PropertyDefn String name(); String typeName(); void validate(Object value, ObjectMapper jsonMapper); + + /** + * Merge a property value with an update. Validation of the update + * is typically done later, once all the updates are applied. The most + * typical merge is just: use the new value if provided, else the old + * value. + */ Object merge(Object existing, Object update); T decode(Object value, ObjectMapper jsonMapper); } diff --git a/server/src/main/java/org/apache/druid/catalog/model/ParameterizedDefn.java b/server/src/main/java/org/apache/druid/catalog/model/ParameterizedDefn.java index e6841007848a..007fd43766b6 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/ParameterizedDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/ParameterizedDefn.java @@ -24,6 +24,13 @@ import java.util.List; import java.util.Map; +/** + * Defines a parameter for a catalog entry. A parameter is an item that can appear + * in a SQL table function as a named SQL argument. Example, for a local file, + * the file name list could be a parameter to allow using the same definition for + * a variety of local files (that is, to name today's update which is + * different from yesterday's update.) + */ public interface ParameterizedDefn { interface ParameterDefn diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java b/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java index 75102683c2e3..7e952a4393a3 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java @@ -30,8 +30,28 @@ import java.util.Map; +/** + * Registry of the table types supported in the catalog. This registry + * is used to validate insertions and updates. A table spec has a type field. + * That spec is said to be "resolved" when we use that type field to look up + * the definition for that type, and combine them into a {@link ResolvedTable}. + *

+ * Note an inherent race condition: + * once a record is written into the metadata DB, that record has a life + * independent of what happens in this table. It may be that a catalog entry + * is created for a table type defined in an extension. Later, that extension + * is removed. The system must still work with the now "unknown" table types + * in the DB. But, we won't allow the use of, changes to, or new instances of + * that type. The choice is to delete the now undefined table, or restore the + * extension. + *

+ * Holds onto the JSON mapper to simplify the resolution process: the + * {@link ResolvedTable} provides the mapper used to serialize the table spec. + */ public class TableDefnRegistry { + // Temporary list of Druid-define table definitions. This should come from + // Guice later to allow extensions to define table types. private static final TableDefn[] TABLE_DEFNS = { new DatasourceDefn(), new InlineTableDefn(), diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java index d1145e4621af..1a09171f81d3 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java @@ -118,16 +118,6 @@ public static TableMetadata newTable( ); } - public static TableMetadata newSegmentTable( - String name, - TableSpec defn - ) - { - return newTable( - TableId.datasource(name), - defn); - } - public static TableMetadata of( TableId id, TableSpec spec diff --git a/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java b/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java index ae0ddf9b993a..e65e1821bc35 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java +++ b/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java @@ -29,7 +29,6 @@ import javax.annotation.Nullable; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -73,13 +72,6 @@ public TableSpec withColumns(final List columns) return new TableSpec(type, properties, columns); } - public TableSpec withProperty(String key, Object value) - { - Map revised = new HashMap<>(properties); - revised.put(key, value); - return withProperties(revised); - } - @JsonProperty("type") public String type() { diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.java index 4619de9048c9..9fc863f98753 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.java @@ -53,7 +53,17 @@ public class AbstractDatasourceDefn extends TableDefn * If unset, then the system setting is used. */ public static final String TARGET_SEGMENT_ROWS_PROPERTY = "targetSegmentRows"; + + /** + * The clustering column names and sort order for each new segment. + */ public static final String CLUSTER_KEYS_PROPERTY = "clusterKeys"; + + /** + * The set of existing columns to "delete" (actually, just hide) from the + * SQL layer. Used to "remove" unwanted columns to avoid the need to rewrite + * existing segments to accomplish the task. + */ public static final String HIDDEN_COLUMNS_PROPERTY = "hiddenColumns"; public static class SegmentGranularityFieldDefn extends GranularityPropertyDefn @@ -125,11 +135,6 @@ public AbstractDatasourceDefn( ); } - public static boolean isDatasource(String tableType) - { - return DatasourceDefn.TABLE_TYPE.equals(tableType); - } - public static boolean isDatasource(ResolvedTable table) { return table.defn() instanceof AbstractDatasourceDefn; diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/ClusterKeySpec.java b/server/src/main/java/org/apache/druid/catalog/model/table/ClusterKeySpec.java index 7f2429da5434..6e2267ecd160 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/ClusterKeySpec.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/ClusterKeySpec.java @@ -26,6 +26,14 @@ import java.util.Objects; +/** + * Description of one clustering key (column) for a datasource. Clustering is + * the process of physically sorting data by a sort key. This class represents + * one column of that sort key. The key consists of a name and a sort direction. + * Sort direction is optional: omitted, ascending is assumed. + * (In Druid, clustering is always {@code NULLS LOW} in SQL parlance, so that attribute + * does not appear here. + */ public class ClusterKeySpec { private final String expr; @@ -38,7 +46,7 @@ public ClusterKeySpec( ) { this.expr = expr; - this.desc = desc != null && desc == true; + this.desc = desc != null && desc; } @JsonProperty("column") diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/DatasourceDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/DatasourceDefn.java index de09ece33f06..5bbecbc1aa06 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/DatasourceDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/DatasourceDefn.java @@ -22,6 +22,9 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.catalog.model.ColumnDefn; import org.apache.druid.catalog.model.ColumnSpec; +import org.apache.druid.catalog.model.Columns; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; import java.util.Collections; @@ -53,6 +56,16 @@ public void validate(ColumnSpec spec, ObjectMapper jsonMapper) { super.validate(spec, jsonMapper); validateScalarColumn(spec); + if (Columns.isTimeColumn(spec.name()) && + spec.sqlType() != null && + !Columns.TIMESTAMP.equalsIgnoreCase(spec.sqlType())) { + throw new IAE(StringUtils.format( + "%s column must have no SQL type or SQL type %s", + Columns.TIME_COLUMN, + Columns.TIMESTAMP + ) + ); + } } } diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java index 8eb004311421..e808d1833e64 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java @@ -63,7 +63,7 @@ public abstract static class FormattedExternalTableDefn extends ExternalTableDef { public static final String FORMAT_PROPERTY = "format"; - private Map formats; + private final Map formats; public FormattedExternalTableDefn( final String name, @@ -183,7 +183,6 @@ public void validate(ColumnSpec spec, ObjectMapper jsonMapper) private final List parameterList; private final Map parameterMap; - public ExternalTableDefn( final String name, final String typeValue, @@ -218,6 +217,11 @@ public ParameterDefn parameter(String key) return parameterMap.get(key); } + /** + * Merge parameters provided by a SQL table function with the catalog information + * provided in the resolved table to produce a new resolved table used for a + * specific query. + */ public abstract ResolvedTable mergeParameters(ResolvedTable table, Map values); public ExternalTableSpec convertToExtern(ResolvedTable table) @@ -229,13 +233,21 @@ public ExternalTableSpec convertToExtern(ResolvedTable table) ); } + /** + * Convert a resolved table to the Druid internal {@link InputSource} + * object required by an MSQ query. + */ + protected abstract InputSource convertSource(ResolvedTable table); + + /** + * Convert a resolved table to the Druid internal {@link InputFormat} + * object required by an MSQ query. Not all input sources require a format. + */ protected InputFormat convertFormat(ResolvedTable table) { return null; } - protected abstract InputSource convertSource(ResolvedTable table); - protected InputSource convertObject( final ObjectMapper jsonMapper, final Map jsonMap, @@ -256,11 +268,6 @@ public ExternalTableSpec applyParameters(ResolvedTable table, Map tableTypes() { // Known input tables. Get this from a registry later. diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.java index 08e107be1f9b..eb97cc744f20 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.java @@ -60,7 +60,11 @@ public class HttpTableDefn extends FormattedExternalTableDefn implements ParameterizedDefn { public static final String TABLE_TYPE = HttpInputSource.TYPE_KEY; - public static final String URI_TEMPLATE_PROPERTY = "template"; + + // Catalog properties that map to fields in the HttpInputSource. See + // that class for the meaning of these properties. + + public static final String URI_TEMPLATE_PROPERTY = "uriTemplate"; public static final String USER_PROPERTY = "user"; public static final String PASSWORD_PROPERTY = "password"; public static final String PASSWORD_ENV_VAR_PROPERTY = "passwordEnvVar"; diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/LocalTableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/LocalTableDefn.java index d6ef86959786..48992483dd53 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/LocalTableDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/LocalTableDefn.java @@ -35,6 +35,10 @@ import java.util.List; import java.util.Map; +/** + * Definition for a catalog table object that represents a Druid + * {@link LocalInputSource}. + */ public class LocalTableDefn extends FormattedExternalTableDefn implements ParameterizedDefn { public static final String TABLE_TYPE = LocalInputSource.TYPE_KEY; diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java b/server/src/test/java/org/apache/druid/catalog/model/table/TableBuilder.java similarity index 100% rename from server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java rename to server/src/test/java/org/apache/druid/catalog/model/table/TableBuilder.java From 883405d7224c52f031faf3657a158966a70b9ca1 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Mon, 7 Nov 2022 12:29:42 -0800 Subject: [PATCH 20/22] Fix --- .../druid/catalog/model/table/AbstractDatasourceDefn.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.java index 9fc863f98753..bbfc2d5042cd 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.java +++ b/server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.java @@ -135,6 +135,11 @@ public AbstractDatasourceDefn( ); } + public static boolean isDatasource(String tableType) + { + return DatasourceDefn.TABLE_TYPE.equals(tableType); + } + public static boolean isDatasource(ResolvedTable table) { return table.defn() instanceof AbstractDatasourceDefn; From aa0d2724d146afacb3cd77c6e3c5188aa8c12275 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Mon, 7 Nov 2022 13:14:20 -0800 Subject: [PATCH 21/22] Build fix --- .../java/org/apache/druid/catalog/model/table/TableBuilder.java | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename server/src/{test => main}/java/org/apache/druid/catalog/model/table/TableBuilder.java (100%) diff --git a/server/src/test/java/org/apache/druid/catalog/model/table/TableBuilder.java b/server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java similarity index 100% rename from server/src/test/java/org/apache/druid/catalog/model/table/TableBuilder.java rename to server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java From e1b05d8eb35a1b9909119e1299a9e79a7a2c6ccf Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Tue, 8 Nov 2022 18:44:39 -0800 Subject: [PATCH 22/22] Test fix --- .../main/java/org/apache/druid/catalog/model/Columns.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/catalog/model/Columns.java b/server/src/main/java/org/apache/druid/catalog/model/Columns.java index 0092584c431f..f16ed3bedb28 100644 --- a/server/src/main/java/org/apache/druid/catalog/model/Columns.java +++ b/server/src/main/java/org/apache/druid/catalog/model/Columns.java @@ -40,8 +40,10 @@ public class Columns public static final String DOUBLE = "DOUBLE"; public static final String TIMESTAMP = "TIMESTAMP"; - public static final Set NUMERIC_TYPES = ImmutableSet.of(BIGINT, FLOAT, DOUBLE); - public static final Set SCALAR_TYPES = ImmutableSet.of(VARCHAR, BIGINT, FLOAT, DOUBLE); + public static final Set NUMERIC_TYPES = + ImmutableSet.of(BIGINT, FLOAT, DOUBLE); + public static final Set SCALAR_TYPES = + ImmutableSet.of(TIMESTAMP, VARCHAR, BIGINT, FLOAT, DOUBLE); public static final Map SQL_TO_DRUID_TYPES = new ImmutableMap.Builder()