diff --git a/.travis.yml b/.travis.yml index efee82a8625e..3c37f2125b0e 100644 --- a/.travis.yml +++ b/.travis.yml @@ -688,12 +688,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..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 @@ -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 = "regex", value = RegexInputFormat.class), - @Type(name = "tsv", value = DelimitedInputFormat.class) + @Type(name = CsvInputFormat.TYPE_KEY, value = CsvInputFormat.class), + @Type(name = JsonInputFormat.TYPE_KEY, value = JsonInputFormat.class), + @Type(name = RegexInputFormat.TYPE_KEY, value = RegexInputFormat.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..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 @@ -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 = "combining", value = CombiningInputSource.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 = CombiningInputSource.TYPE_KEY, 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/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/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/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 b269544482de..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,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,17 @@ 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()); + } } } 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 ee21a634fab4..b73e4215a64a 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..0678f95ff047 --- /dev/null +++ b/extensions-core/druid-catalog/pom.xml @@ -0,0 +1,286 @@ + + + + + + 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 + + + + 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..c09fe35a5a14 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/CatalogException.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; + +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 BAD_STATE = "Invalid table spec"; + + 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/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..649f3515dcea --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/guice/CatalogCoordinatorModule.java @@ -0,0 +1,93 @@ +/* + * 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.MetadataStorageManager; +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(MetadataStorageManager.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..67b4d29fbef1 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogListenerResource.java @@ -0,0 +1,70 @@ +/* + * 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.jaxrs.smile.SmileMediaTypes; +import com.sun.jersey.spi.container.ResourceFilters; +import org.apache.druid.catalog.sync.CatalogUpdateListener; +import org.apache.druid.catalog.sync.UpdateEvent; +import org.apache.druid.server.http.security.ConfigResourceFilter; + +import javax.inject.Inject; +import javax.ws.rs.Consumes; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; + +/** + * 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 +{ + public static final String BASE_URL = "/druid/broker/v1/catalog"; + public static final String SYNC_URL = "/sync"; + + private final CatalogUpdateListener listener; + + @Inject + public CatalogListenerResource( + final CatalogUpdateListener listener + ) + { + this.listener = listener; + } + + /** + * 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}) + @ResourceFilters(ConfigResourceFilter.class) + public Response syncTable(final UpdateEvent event) + { + 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/CatalogResource.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java new file mode 100644 index 000000000000..551ba490c183 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/CatalogResource.java @@ -0,0 +1,613 @@ +/* + * 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.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.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; +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; + +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 + * 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. + */ +@Path(CatalogResource.ROOT_PATH) +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; + + @Inject + public CatalogResource( + 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: + *

+ * + * @param schemaName The name of the Druid schema, which must be writable + * and the user must have at least read access. + * @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 + * 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 + @Path("/schemas/{schema}/tables/{name}") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response postTable( + @PathParam("schema") String schemaName, + @PathParam("name") String tableName, + TableSpec spec, + @QueryParam("version") long version, + @QueryParam("overwrite") boolean overwrite, + @Context final HttpServletRequest req + ) + { + try { + final SchemaSpec schema = validateSchema(schemaName, true); + validateTableName(tableName); + authorizeTable(schema, tableName, Action.WRITE, req); + validateTableSpec(schema, spec); + final TableMetadata table = TableMetadata.newTable(TableId.of(schemaName, tableName), 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 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 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. + */ + @GET + @Path("/schemas/{schema}/tables/{name}") + @Produces(MediaType.APPLICATION_JSON) + public Response getTable( + @PathParam("schema") String schemaName, + @PathParam("name") String tableName, + @Context final HttpServletRequest req + ) + { + try { + 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) { + return e.toResponse(); + } + } + + /** + * Deletes the table definition (but not the underlying table or datasource) + * for the given schema and table. + * + * @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 schemaName, + @PathParam("name") String tableName, + @Context final HttpServletRequest req + ) + { + try { + final SchemaSpec schema = validateSchema(schemaName, true); + authorizeTable(schema, tableName, Action.WRITE, req); + catalog.tables().delete(new TableId(schemaName, tableName)); + return ok(); + } + catch (CatalogException e) { + return e.toResponse(); + } + } + + // --------------------------------------------------------------------- + // 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 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. + */ + @POST + @Path("/schemas/{schema}/tables/{name}/edit") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + public Response editTable( + @PathParam("schema") String schemaName, + @PathParam("name") String tableName, + TableEditRequest editRequest, + @Context final HttpServletRequest req + ) + { + try { + 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) { + return e.toResponse(); + } + } + + // --------------------------------------------------------------------- + // Retrieval + + /** + * 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") + @Produces(MediaType.APPLICATION_JSON) + public Response getSchemas( + @QueryParam("format") String format, + @Context final HttpServletRequest req + ) + { + 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(); + } + } + + /** + * 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 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 + */ + @GET + @Path("/schemas/{schema}/tables") + @Produces(MediaType.APPLICATION_JSON) + public Response getSchemaTables( + @PathParam("schema") String schemaName, + @QueryParam("format") String format, + @Context final HttpServletRequest req + ) + { + try { + SchemaSpec schema = validateSchema(schemaName, false); + 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) { + return e.toResponse(); + } + } + + // --------------------------------------------------------------------- + // Sync methods + + public static final String SCHEMA_SYNC = "/sync/schemas/{schema}"; + + /** + * 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("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(schemaName, 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}"; + + /** + * 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("schema") String schemaName, + @PathParam("name") String tableName, + @Context final HttpServletRequest 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. + */ + 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 + ) + { + 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 + ) + { + // 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) + { + 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 { + 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, 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 schemaName, boolean forWrite) throws CatalogException + { + if (Strings.isNullOrEmpty(schemaName)) { + throw CatalogException.badRequest("Schema name is required"); + } + SchemaSpec schema = catalog.resolveSchema(schemaName); + if (schema == null) { + throw new NotFoundException("Unknown schema %s", schemaName); + } + + if (forWrite && !schema.writable()) { + throw CatalogException.badRequest( + "Cannot modify schema %s", + schemaName + ); + } + return schema; + } + + private static ResourceAction resourceAction(SchemaSpec schema, String tableName, Action action) + { + return new ResourceAction(new Resource(tableName, schema.securityResource()), action); + } + + private void authorizeTable( + final SchemaSpec schema, + final String tableName, + final Action action, + final HttpServletRequest request + ) throws CatalogException + { + 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(), tableName, 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/TableEditRequest.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditRequest.java new file mode 100644 index 000000000000..79136d4a0595 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditRequest.java @@ -0,0 +1,172 @@ +/* + * 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 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 javax.annotation.Nullable; + +import java.util.List; +import java.util.Map; + +/** + * 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 = 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") + public final List columns; + + @JsonCreator + public HideColumns(@JsonProperty("columns") List columns) + { + this.columns = columns; + } + } + + /** + * Remove the given column names from the property that lists the hidden columns. + */ + public static class UnhideColumns extends TableEditRequest + { + @JsonProperty("columns") + public final List columns; + + @JsonCreator + public UnhideColumns(@JsonProperty("columns") List columns) + { + this.columns = columns; + } + } + + /** + * Remove one or more columns from the list of columns from this table. + */ + public static class DropColumns extends TableEditRequest + { + @JsonProperty("columns") + public final List columns; + + @JsonCreator + public DropColumns(@JsonProperty("columns") List columns) + { + this.columns = 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") + public final Map properties; + + @JsonCreator + public UpdateProperties(@JsonProperty("properties") Map properties) + { + this.properties = properties; + } + } + + /** + * 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") + public final List columns; + + @JsonCreator + 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 new file mode 100644 index 000000000000..b16995123514 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/http/TableEditor.java @@ -0,0 +1,390 @@ +/* + * 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.druid.catalog.CatalogException; +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; +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.java.util.common.IAE; +import org.apache.druid.utils.CollectionUtils; + +import javax.ws.rs.core.Response; + +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; + +/** + * 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; + private final TableId id; + private final TableEditRequest editRequest; + + public TableEditor( + final CatalogStorage catalog, + final TableId id, + final TableEditRequest editRequest + ) + { + this.catalog = catalog; + this.id = id; + 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) { + 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 + { + if (CollectionUtils.isNullOrEmpty(columns)) { + return 0; + } + 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); + 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); + } + + private long unHideColumns(List columns) throws CatalogException + { + if (CollectionUtils.isNullOrEmpty(columns)) { + return 0; + } + 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 TableSpec applyUnhideColumns(TableMetadata table, List columns) throws CatalogException + { + 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(); + @SuppressWarnings("unchecked") + List hiddenColumns = (List) props.get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); + if (hiddenColumns == null || columns.isEmpty()) { + return null; + } + Set removals = new HashSet<>(columns); + List revisedHiddenCols = new ArrayList<>(); + for (String col : hiddenColumns) { + if (!removals.contains(col)) { + 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); + } else { + revisedProps.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, revisedHiddenCols); + } + return existingSpec.withProperties(revisedProps); + } + + private long dropColumns(List columnsToDrop) throws CatalogException + { + if (CollectionUtils.isNullOrEmpty(columnsToDrop)) { + return 0; + } + return catalog.tables().updateColumns( + id, + table -> applyDropColumns(table, columnsToDrop) + ); + } + + private TableSpec applyDropColumns(final TableMetadata table, final List toDrop) + { + final TableSpec existingSpec = table.spec(); + List existingColumns = existingSpec.columns(); + if (CollectionUtils.isNullOrEmpty(existingColumns)) { + return null; + } + Set drop = new HashSet<>(toDrop); + List revised = new ArrayList<>(); + for (ColumnSpec col : existingColumns) { + if (!drop.contains(col.name())) { + 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 || updates.isEmpty()) { + return 0; + } + return catalog.tables().updateProperties( + id, + table -> applyUpdateProperties(table, updates) + ); + } + + private TableSpec applyUpdateProperties( + final TableMetadata table, + final Map updates + ) throws CatalogException + { + final TableSpec existingSpec = table.spec(); + final TableDefn defn = resolveDefn(existingSpec.type()); + 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 + { + 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(), + tableType + ); + } + return defn; + } + + private long updateColumns(final List updates) throws CatalogException + { + if (CollectionUtils.isNullOrEmpty(updates)) { + return 0; + } + return catalog.tables().updateColumns( + id, + table -> applyUpdateColumns(table, updates) + ); + } + + private TableSpec applyUpdateColumns( + final TableMetadata table, + final List updates + ) throws CatalogException + { + final TableSpec existingSpec = table.spec(); + final TableDefn defn = resolveDefn(existingSpec.type()); + 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 + { + 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, + table -> applyMoveColumn(table, moveColumn) + ); + } + + private TableSpec applyMoveColumn( + final TableMetadata table, + final MoveColumn moveColumn + ) throws CatalogException + { + 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(existingCols, 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 existingSpec.withColumns(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/CatalogStorage.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java new file mode 100644 index 000000000000..e1202d295663 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/CatalogStorage.java @@ -0,0 +1,131 @@ +/* + * 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.CatalogException.NotFoundException; +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.CatalogUpdateListener; +import org.apache.druid.catalog.sync.MetadataCatalog.CatalogSource; +import org.apache.druid.catalog.sync.MetadataCatalog.CatalogUpdateProvider; +import org.apache.druid.guice.annotations.Json; + +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: + *

+ */ +public class CatalogStorage implements CatalogUpdateProvider, CatalogSource +{ + protected final SchemaRegistry schemaRegistry; + protected final TableDefnRegistry tableRegistry; + protected final CatalogManager catalogMgr; + + @Inject + public CatalogStorage( + final CatalogManager catalogMgr, + @Json final ObjectMapper jsonMapper + ) + { + this.schemaRegistry = new SchemaRegistryImpl(); + this.tableRegistry = new TableDefnRegistry(jsonMapper); + this.catalogMgr = catalogMgr; + } + + public CatalogManager tables() + { + return catalogMgr; + } + + public SchemaRegistry schemaRegistry() + { + return schemaRegistry; + } + + public ObjectMapper jsonMapper() + { + return tableRegistry.jsonMapper(); + } + + public SchemaSpec resolveSchema(String dbSchema) + { + return schemaRegistry.schema(dbSchema); + } + + @Override + public void register(CatalogUpdateListener listener) + { + tables().register(listener); + } + + @Override + public List tablesForSchema(String dbSchema) + { + return tables().tablesInSchema(dbSchema); + } + + @Override + public @Nullable TableMetadata table(TableId id) + { + try { + return tables().read(id); + } + catch (NotFoundException e) { + return null; + } + } + + public void validate(TableMetadata table) + { + table.validate(); + tableRegistry.resolve(table.spec()).validate(); + } + + public TableDefnRegistry tableRegistry() + { + return tableRegistry; + } + + @Override + 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/MetadataStorageManager.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetadataStorageManager.java new file mode 100644 index 000000000000..b5c7173698ee --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/MetadataStorageManager.java @@ -0,0 +1,93 @@ +/* + * 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; + +/** + * 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; + private final MetadataStorageConnectorConfig config; + private final MetadataStorageTablesConfig tablesConfig; + + @Inject + public MetadataStorageManager( + @Json final ObjectMapper jsonMapper, + final MetadataStorageConnector connector, + final Supplier configSupplier, + final Supplier tablesConfigSupplier + ) + { + this.jsonMapper = jsonMapper; + this.connector = connector; + this.config = configSupplier.get(); + this.tablesConfig = tablesConfigSupplier.get(); + } + + public MetadataStorageConnectorConfig config() + { + return config; + } + + public MetadataStorageTablesConfig tablesConfig() + { + return tablesConfig; + } + + /** + * Object mapper to use for serializing and deserializing + * JSON objects stored in the metastore DB. + */ + public ObjectMapper jsonMapper() + { + return jsonMapper; + } + + /** + * Is the implementation SQL-based? + */ + public boolean isSql() + { + return connector instanceof SQLMetadataConnector; + } + + /** + * 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/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..cb1733b54f0a --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/CatalogManager.java @@ -0,0 +1,179 @@ +/* + * 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.CatalogException; +import org.apache.druid.catalog.CatalogException.DuplicateKeyException; +import org.apache.druid.catalog.CatalogException.NotFoundException; +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 javax.annotation.Nullable; + +import java.util.List; + +/** + * 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. + * + * Performs detailed CRUD operations on the catalog tables table. + * Higher-level operations appear elsewhere. + */ +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 + { + @Nullable TableSpec apply(TableMetadata metadata) throws CatalogException; + } + + /** + * 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(CatalogUpdateListener listener); + + /** + * 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. + *

+ * 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. + *

+ * Use this when the desire to replace whatever exists with the new information, + * such as configuration-as-code style updates. + */ + long replace(TableMetadata table) throws NotFoundException; + + /** + * 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 properties + * @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; + + /** + * 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. + */ + TableMetadata read(TableId id) throws NotFoundException; + + /** + * 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. + */ + void delete(TableId id) throws NotFoundException; + + /** + * Returns a list of the ids (schema, name) of all tables in the catalog. + */ + List allTablePaths(); + + /** + * Returns a list of the names of all tables within the given schema. + */ + List tableNamesInSchema(String dbSchema); + + /** + * Returns a list of the table metadata for all tables within the given + * schema. + */ + List tablesInSchema(String dbSchema); +} 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..fa24bd825613 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/storage/sql/SQLCatalogManager.java @@ -0,0 +1,786 @@ +/* + * 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.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; +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.storage.MetadataStorageManager; +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; +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; +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.io.IOException; +import java.util.Deque; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentLinkedDeque; + +@ManageLifecycle +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; + private final IDBI dbi; + private final String tableName; + private final Deque listeners = new ConcurrentLinkedDeque<>(); + + @Inject + public SQLCatalogManager(MetadataStorageManager 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(); + } + + 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() + { + if (!metastoreManager.config().isCreateTables()) { + return; + } + connector.createTable( + tableName, + ImmutableList.of( + StringUtils.format( + 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 + { + try { + return dbi.withHandle( + new HandleCallback() + { + @Override + public Long withHandle(Handle handle) throws DuplicateKeyException + { + final TableSpec spec = table.spec(); + final long updateTime = System.currentTimeMillis(); + final Update stmt = handle + .createStatement(statement(INSERT_TABLE)) + .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(); + } + catch (UnableToExecuteStatementException e) { + if (DbUtils.isDuplicateRecordException(e)) { + throw new DuplicateKeyException( + "Tried to insert a duplicate table: %s", + table.sqlName() + ); + } else { + throw e; + } + } + sendAddition(table, updateTime); + return updateTime; + } + } + ); + } + catch (CallbackFailedException e) { + if (e.getCause() instanceof DuplicateKeyException) { + throw (DuplicateKeyException) e.getCause(); + } + throw e; + } + } + + 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) throws NotFoundException + { + try { + return dbi.withHandle( + new HandleCallback() + { + @Override + public TableMetadata withHandle(Handle handle) throws NotFoundException + { + final Query> query = handle + .createQuery(statement(SELECT_TABLE)) + .setFetchSize(connector.getStreamingFetchSize()) + .bind(SCHEMA_NAME_COL, id.schema()) + .bind(TABLE_NAME_COL, 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); + } + } + ); + } + catch (CallbackFailedException e) { + if (e.getCause() instanceof NotFoundException) { + throw (NotFoundException) e.getCause(); + } + throw e; + } + } + + 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 { + final TableMetadata revised = dbi.withHandle( + new HandleCallback() + { + @Override + public TableMetadata withHandle(Handle handle) throws NotFoundException + { + 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(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.asUpdate(updateTime); + } + } + ); + sendUpdate(EventType.UPDATE, revised); + return revised.updateTime(); + } + catch (CallbackFailedException e) { + if (e.getCause() instanceof NotFoundException) { + throw (NotFoundException) e.getCause(); + } + throw e; + } + } + + private static final String OLD_VERSION_PARAM = "oldVersion"; + private static final String UPDATE_SPEC_STMT = + REPLACE_SPEC_STMT + + " AND updateTime = :oldVersion"; + + @Override + public long update(TableMetadata table, long oldVersion) throws NotFoundException + { + try { + final TableMetadata revised = dbi.withHandle( + new HandleCallback() + { + @Override + public TableMetadata withHandle(Handle handle) throws NotFoundException + { + 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(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( + "Table %s: not found, is being deleted or update version does not match DB version", + id.sqlName() + ); + } + return table.asUpdate(updateTime); + } + } + ); + sendUpdate(EventType.UPDATE, revised); + return revised.updateTime(); + } + catch (CallbackFailedException e) { + if (e.getCause() instanceof NotFoundException) { + throw (NotFoundException) e.getCause(); + } + throw e; + } + } + + 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_PROPERTIES_STMT = + "UPDATE %s\n SET\n" + + " properties = :properties,\n" + + " updateTime = :updateTime\n" + + "WHERE schemaName = :schemaName\n" + + " AND name = :name\n"; + + @Override + public long updateProperties( + final TableId id, + final TableTransform transform + ) throws CatalogException + { + try { + final TableMetadata result = dbi.withHandle( + new HandleCallback() + { + @Override + public TableMetadata withHandle(Handle handle) throws CatalogException + { + handle.begin(); + try { + final Query> query = handle + .createQuery(statement(SELECT_PROPERTIES_STMT)) + .setFetchSize(connector.getStreamingFetchSize()) + .bind(SCHEMA_NAME_COL, id.schema()) + .bind(TABLE_NAME_COL, id.name()); + + final ResultIterator resultIterator = query + .map((index, r, ctx) -> + tableSpecFromBytes( + jsonMapper, + r.getString(1), + r.getBytes(2), + null + ) + ) + .iterator(); + final TableSpec tableSpec; + if (resultIterator.hasNext()) { + tableSpec = resultIterator.next(); + } else { + throw tableNotFound(id); + } + 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_PROPERTIES_STMT)) + .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. + throw new ISE("Table %s: not found", id.sqlName()); + } + handle.commit(); + return TableMetadata.forUpdate(id, updateTime, revised); + } + catch (Exception e) { + handle.rollback(); + throw e; + } + } + } + ); + if (result == null) { + return 0; + } + sendUpdate(EventType.PROPERTY_UPDATE, result); + return result.updateTime(); + } + catch (CallbackFailedException e) { + if (e.getCause() instanceof CatalogException) { + throw (CatalogException) e.getCause(); + } + throw e; + } + } + + private static final String SELECT_COLUMNS_STMT = + "SELECT tableType, 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" + + " updateTime = :updateTime\n" + + "WHERE schemaName = :schemaName\n" + + " AND name = :name\n"; + + @Override + public long updateColumns( + final TableId id, + final TableTransform transform + ) throws CatalogException + { + try { + final TableMetadata result = dbi.withHandle( + new HandleCallback() + { + @Override + public TableMetadata withHandle(Handle handle) throws CatalogException + { + handle.begin(); + try { + final Query> query = handle + .createQuery(statement(SELECT_COLUMNS_STMT)) + .setFetchSize(connector.getStreamingFetchSize()) + .bind(SCHEMA_NAME_COL, id.schema()) + .bind(TABLE_NAME_COL, id.name()); + + final ResultIterator resultIterator = query + .map((index, r, ctx) -> + tableSpecFromBytes( + jsonMapper, + r.getString(1), + null, + r.getBytes(2) + ) + ) + .iterator(); + final TableSpec tableSpec; + if (resultIterator.hasNext()) { + tableSpec = resultIterator.next(); + } else { + throw tableNotFound(id); + } + 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)) + .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. + throw new ISE("Table %s: not found", id.sqlName()); + } + handle.commit(); + return TableMetadata.forUpdate(id, updateTime, revised); + } + catch (Exception e) { + handle.rollback(); + throw e; + } + } + } + ); + if (result == null) { + return 0; + } + sendUpdate(EventType.COLUMNS_UPDATE, result); + return result.updateTime(); + } + catch (CallbackFailedException e) { + if (e.getCause() instanceof CatalogException) { + throw (CatalogException) e.getCause(); + } + throw e; + } + } + + private static final String MARK_DELETING_STMT = + "UPDATE %s\n SET\n" + + " state = 'D',\n" + + " updateTime = :updateTime\n" + + "WHERE schemaName = :schemaName\n" + + " AND name = :name\n"; + + @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(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; + } + } + ); + } + + private static final String DELETE_TABLE_STMT = + "DELETE FROM %s\n" + + "WHERE schemaName = :schemaName\n" + + " AND name = :name\n"; + + @Override + public void delete(TableId id) throws NotFoundException + { + try { + dbi.withHandle( + new HandleCallback() + { + @Override + public Void withHandle(Handle handle) throws NotFoundException + { + int updateCount = handle + .createStatement(statement(DELETE_TABLE_STMT)) + .bind(SCHEMA_NAME_COL, id.schema()) + .bind(TABLE_NAME_COL, 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_STMT = + "SELECT schemaName, name\n" + + "FROM %s\n" + + "ORDER BY schemaName, name"; + + @Override + public List allTablePaths() + { + return dbi.withHandle( + new HandleCallback>() + { + @Override + public List withHandle(Handle handle) + { + Query> query = handle + .createQuery(statement(SELECT_ALL_TABLE_PATHS_STMT)) + .setFetchSize(connector.getStreamingFetchSize()); + final ResultIterator resultIterator = + query.map((index, r, ctx) -> + new TableId(r.getString(1), r.getString(2))) + .iterator(); + return Lists.newArrayList(resultIterator); + } + } + ); + } + + private static final String SELECT_TABLE_NAMES_IN_SCHEMA_STMT = + "SELECT name\n" + + "FROM %s\n" + + "WHERE schemaName = :schemaName\n" + + "ORDER BY name"; + + @Override + public List tableNamesInSchema(String dbSchema) + { + return dbi.withHandle( + new HandleCallback>() + { + @Override + public List withHandle(Handle handle) + { + Query> query = handle + .createQuery(statement(SELECT_TABLE_NAMES_IN_SCHEMA_STMT)) + .bind(SCHEMA_NAME_COL, dbSchema) + .setFetchSize(connector.getStreamingFetchSize()); + final ResultIterator resultIterator = + query.map((index, r, ctx) -> + r.getString(1)) + .iterator(); + return Lists.newArrayList(resultIterator); + } + } + ); + } + + 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" + + "ORDER BY name"; + + @Override + public List tablesInSchema(String dbSchema) + { + return dbi.withHandle( + new HandleCallback>() + { + @Override + public List withHandle(Handle handle) + { + Query> query = handle + .createQuery(statement(SELECT_TABLES_IN_SCHEMA_STMT)) + .bind(SCHEMA_NAME_COL, 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)), + tableSpecFromBytes(jsonMapper, r.getString(5), r.getBytes(6), r.getBytes(7)) + ) + ) + .iterator(); + return Lists.newArrayList(resultIterator); + } + } + ); + } + + @Override + public synchronized void register(CatalogUpdateListener listener) + { + listeners.add(listener); + } + + protected synchronized void sendAddition(TableMetadata table, long updateTime) + { + if (listeners.isEmpty()) { + return; + } + sendEvent(new UpdateEvent(EventType.CREATE, table.fromInsert(updateTime))); + } + + protected synchronized void sendUpdate(EventType eventType, TableMetadata table) + { + if (listeners.isEmpty()) { + return; + } + sendEvent(new UpdateEvent(eventType, table)); + } + + protected void sendDeletion(TableId id) + { + sendEvent(new UpdateEvent(EventType.DELETE, TableMetadata.empty(id))); + } + + protected synchronized void sendEvent(UpdateEvent event) + { + for (CatalogUpdateListener listener : listeners) { + listener.updated(event); + } + } + + // 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() + ); + } + + /** + * 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, + properties == null ? null : propertiesFromBytes(jsonMapper, properties), + columns == null ? null : 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/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..be7a7281de4d --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CacheNotifier.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.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() + { + LOG.info("Starting Catalog sync"); + 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() + { + if (!updates.isEmpty()) { + 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 new file mode 100644 index 000000000000..b30a12ee9274 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CachedMetadataCatalog.java @@ -0,0 +1,350 @@ +/* + * 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.guice.annotations.Json; +import org.apache.druid.java.util.common.logger.Logger; + +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: + *

    + *
  • A table is requested that has not yet been requested. + * Once requested, the entry is cached, even if the table does + * not exist in the base catalog.
  • + *
  • The contents of a schema are requested, and have not yet + * been fetched.
  • + *
+ * + * 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, CatalogUpdateListener +{ + 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; + + /** + * 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(TableMetadata table) + { + this.table = table; + } + } + + 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; + } + + /** + * 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(), + key -> new TableEntry(base.table(tableId)) + ); + return entry.table; + } + + /** + * Low-frequency list of tables sorted by name. + */ + public List tables() + { + if (version == UNDEFINED) { + return Collections.emptyList(); + } + if (version == NOT_FETCHED) { + 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". + 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(UpdateEvent event) + { + 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()); + } + + 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()); + 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) + { + 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 synchronized 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(UpdateEvent event) + { + SchemaEntry schemaEntry = entryFor(event.table.id().schema()); + if (schemaEntry != null) { + schemaEntry.update(event); + } + } + + @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/CatalogUpdateListener.java b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateListener.java new file mode 100644 index 000000000000..afddeb00221a --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateListener.java @@ -0,0 +1,31 @@ +/* + * 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; + +/** + * 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); +} 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..f60507c8e2c9 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/CatalogUpdateNotifier.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.sync; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.catalog.http.CatalogListenerResource; +import org.apache.druid.catalog.storage.CatalogStorage; +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.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; + +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.) + */ +@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; + + 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(); + LOG.info("Catalog catalog update notifier started"); + } + + @LifecycleStop + public void stop() + { + notifier.stop(); + LOG.info("Catalog catalog update notifier stopped"); + } + + @Override + public void updated(UpdateEvent event) + { + notifier.send(JacksonUtils.toBytes(smileMapper, event)); + } +} 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..89e398e56df6 --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/MetadataCatalog.java @@ -0,0 +1,76 @@ +/* + * 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 javax.annotation.Nullable; + +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 CatalogUpdateProvider + { + void register(CatalogUpdateListener 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. + */ + @Nullable TableMetadata getTable(TableId tableId); + @Nullable 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/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..19cc383d40ad --- /dev/null +++ b/extensions-core/druid-catalog/src/main/java/org/apache/druid/catalog/sync/UpdateEvent.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.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/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..d89d5f7a20ce --- /dev/null +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/CatalogTests.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.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"; + + public 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) + { + MetadataStorageManager metastoreMgr = new MetadataStorageManager( + JSON_MAPPER, + derbyConnectorRule.getConnector(), + () -> derbyConnectorRule.getMetadataConnectorConfig(), + derbyConnectorRule.metadataTablesConfigSupplier() + ); + manager = new SQLCatalogManager(metastoreMgr); + manager.start(); + storage = new CatalogStorage(manager, JSON_MAPPER); + } + + public void tearDown() + { + if (manager != null) { + 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..c180ab48e32a --- /dev/null +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/storage/TableManagerTest.java @@ -0,0 +1,308 @@ +/* + * 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.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; +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.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.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() + { + MetadataStorageManager metastoreMgr = new MetadataStorageManager( + JSON_MAPPER, + derbyConnectorRule.getConnector(), + () -> derbyConnectorRule.getMetadataConnectorConfig(), + derbyConnectorRule.metadataTablesConfigSupplier() + ); + manager = new SQLCatalogManager(metastoreMgr); + manager.start(); + } + + @After + public void tearDown() + { + if (manager != null) { + manager = null; + } + } + + @Test + public void testCreate() throws DuplicateKeyException, NotFoundException + { + 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); + + // Table does not exist, read throws an exception. + assertThrows(NotFoundException.class, () -> 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, NotFoundException + { + 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); + + // Change the definition + props = ImmutableMap.of( + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D", + AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 2_000_000 + ); + TableSpec spec2 = spec.withProperties(props); + TableMetadata table2 = table.withSpec(spec2); + 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()); + + // 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, 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 + 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 = t.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 = t.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", + 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); + + assertThrows(NotFoundException.class, () -> manager.delete(table.id())); + manager.create(table); + manager.delete(table.id()); + assertThrows(NotFoundException.class, () -> manager.delete(table.id())); + } + + @Test + public void testList() throws DuplicateKeyException + { + List list = manager.allTablePaths(); + assertTrue(list.isEmpty()); + + Map props = ImmutableMap.of( + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H", + AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000 + ); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_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.allTablePaths(); + 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.tableNamesInSchema(TableId.DRUID_SCHEMA); + assertEquals(2, names.size()); + + names = manager.tableNamesInSchema(TableId.SYSTEM_SCHEMA); + assertEquals(0, names.size()); + + 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/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/CatalogSyncTest.java b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogSyncTest.java new file mode 100644 index 000000000000..aaeda99f3b05 --- /dev/null +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/CatalogSyncTest.java @@ -0,0 +1,376 @@ +/* + * 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.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; +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; +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.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.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; +import static org.junit.Assert.assertNull; +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 CatalogSyncTest +{ + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + + private CatalogTests.DbFixture dbFixture; + private CatalogStorage storage; + private ObjectMapper jsonMapper; + + @Before + public void setUp() + { + dbFixture = new CatalogTests.DbFixture(derbyConnectorRule); + storage = dbFixture.storage; + jsonMapper = new ObjectMapper(); + } + + @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.external(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.external(InlineTableDefn.TABLE_TYPE, "externTable") + .format(InputFormats.CSV_FORMAT_TYPE) + .data("a", "c") + .build(); + assertThrows(IAE.class, () -> storage.validate(table)); + } + + // No format + { + TableMetadata table = TableBuilder.external(InlineTableDefn.TABLE_TYPE, "externTable") + .data("a", "c") + .column("a", Columns.VARCHAR) + .build(); + assertThrows(IAE.class, () -> storage.validate(table)); + } + } + + @Test + public void testDirect() throws DuplicateKeyException, NotFoundException + { + populateCatalog(); + MetadataCatalog catalog = new LocalMetadataCatalog(storage, storage.schemaRegistry()); + verifyInitial(catalog); + alterCatalog(); + verifyAltered(catalog); + } + + @Test + public void testCached() throws CatalogException + { + populateCatalog(); + CachedMetadataCatalog catalog = new CachedMetadataCatalog(storage, storage.schemaRegistry(), jsonMapper); + storage.register(catalog); + verifyInitial(catalog); + alterCatalog(); + verifyAltered(catalog); + editCatalogTable(); + verifyEdited(catalog); + + // Also test the deletion case + TableId table2 = TableId.datasource("table2"); + storage.tables().delete(table2); + assertThrows(NotFoundException.class, () -> 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 CatalogException + { + populateCatalog(); + MockCatalogSync sync = new MockCatalogSync(storage, jsonMapper); + MetadataCatalog catalog = sync.catalog(); + storage.register(sync); + verifyInitial(catalog); + alterCatalog(); + verifyAltered(catalog); + editCatalogTable(); + verifyEdited(catalog); + + // Also test the deletion case + TableId table2 = TableId.datasource("table2"); + storage.tables().delete(table2); + assertThrows(NotFoundException.class, () -> 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.datasource("table1", "P1D") + .timeColumn() + .column("a", Columns.VARCHAR) + .build(); + storage.validate(table1); + storage.tables().create(table1); + + TableMetadata table2 = TableBuilder.datasource("table2", "P1D") + .timeColumn() + .column("dim", Columns.VARCHAR) + .column("measure", "BIGINT") + .build(); + storage.validate(table2); + storage.tables().create(table2); + + TableMetadata table3 = TableBuilder.external(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.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(DatasourceColumnDefn.COLUMN_TYPE, cols.get(0).type()); + assertEquals("a", cols.get(1).name()); + assertEquals(Columns.VARCHAR, cols.get(1).sqlType()); + assertEquals(DatasourceColumnDefn.COLUMN_TYPE, cols.get(0).type()); + + DatasourceFacade ds = new DatasourceFacade(catalog.resolveTable(id)); + assertEquals("P1D", ds.segmentGranularityString()); + } + { + TableId id = TableId.datasource("table2"); + TableMetadata table = catalog.getTable(id); + assertEquals(id, table.id()); + assertTrue(table.updateTime() > 0); + + TableSpec dsSpec = table.spec(); + assertEquals(DatasourceDefn.TABLE_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(DatasourceColumnDefn.COLUMN_TYPE, cols.get(1).type()); + assertEquals("measure", cols.get(2).name()); + 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()); + } + + 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, 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.datasource("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(DatasourceColumnDefn.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()); + } + + 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/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..7a5b7f03b273 --- /dev/null +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/catalog/sync/MockCatalogSync.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.sync; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.catalog.http.CatalogListenerResource; +import org.apache.druid.catalog.storage.CatalogStorage; + +/** + * Simulates a network sync from catalog (Coordinator) to consumer (Broker). + */ +public class MockCatalogSync implements CatalogUpdateListener +{ + private final CatalogListenerResource listenerResource; + private final CachedMetadataCatalog catalog; + + public MockCatalogSync( + final CatalogStorage storage, + final ObjectMapper jsonMapper + ) + { + this.catalog = new CachedMetadataCatalog(storage, storage.schemaRegistry(), jsonMapper); + this.listenerResource = new CatalogListenerResource(catalog); + } + + @Override + public void updated(UpdateEvent update) + { + doSync(update); + } + + private void doSync(UpdateEvent event) + { + listenerResource.syncTable(event); + } + + 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..00cced8b9759 --- /dev/null +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/CatalogResourceTest.java @@ -0,0 +1,697 @@ +/* + * 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.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; +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.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.metadata.TestDerbyConnector; +import org.apache.druid.server.security.ForbiddenException; +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 java.util.Set; + +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.assertThrows; +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, CatalogTests.AUTH_MAPPER); + } + + @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.datasource(tableName, "P1D").buildSpec(); + + // Blank schema name: infer the schema. + 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, 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, 0, false, postBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Unknown schema + 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, 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, 0, false, postBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // No permissions + assertThrows( + ForbiddenException.class, + () -> resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, 0, false, postBy(CatalogTests.DENY_USER)) + ); + + // Read permission + 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, 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, 0, false, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Inline input source + TableSpec inputSpec = TableBuilder.external(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, 0, false, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + // Wrong spec type + resp = resource.postTable(TableId.DRUID_SCHEMA, "invalid", inputSpec, 0, false, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + } + + @Test + public void testUpdate() + { + final String tableName = "update"; + TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D").buildSpec(); + + // Does not exist + 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, 0, false, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + long version = getVersion(resp); + + // No update permission + 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, 10, false, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // Valid version + 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, 0, true, 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.datasource(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, 0, false, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + long version = getVersion(resp); + + // No read permission + 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)); + 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 Set getSchemaSet(Response resp) + { + return (Set) resp.getEntity(); + } + + @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 testGetSchemas() + { + // 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()); + assertTrue(getSchemaSet(resp).contains("druid")); + + resp = resource.getSchemas("", getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + 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.getSchemaTables(null, CatalogResource.NAME_FORMAT, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Invalid schema + resp = resource.getSchemaTables("bogus", CatalogResource.NAME_FORMAT, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // 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()); + + // No read access - name + resp = resource.getSchemaTables(TableId.DRUID_SCHEMA, CatalogResource.NAME_FORMAT, getBy(CatalogTests.DENY_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertTrue(getTableIdList(resp).isEmpty()); + + // No read access - metadata + resp = resource.getSchemaTables(TableId.DRUID_SCHEMA, CatalogResource.METADATA_FORMAT, getBy(CatalogTests.DENY_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertTrue(getDetailsList(resp).isEmpty()); + + // No read access - status + resp = resource.getSchemaTables(TableId.DRUID_SCHEMA, CatalogResource.STATUS_FORMAT, getBy(CatalogTests.DENY_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertTrue(getDetailsList(resp).isEmpty()); + + // Read access - name + resp = resource.getSchemaTables(TableId.DRUID_SCHEMA, CatalogResource.NAME_FORMAT, getBy(CatalogTests.READER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + assertEquals(Collections.singletonList(tableName), getTableList(resp)); + + // 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()); + + // 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, deleteBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus()); + + // Missing table name + 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, deleteBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // Immutable schema + 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, deleteBy(CatalogTests.SUPER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // Create the table + TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D").buildSpec(); + resp = resource.postTable(TableId.DRUID_SCHEMA, tableName, dsSpec, 0, false, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + // No write permission + assertThrows( + ForbiddenException.class, + () -> resource.deleteTable(TableId.DRUID_SCHEMA, tableName, deleteBy(CatalogTests.READER_USER)) + ); + + // Write permission + 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, deleteBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + } + + @Test + 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, 0, true, 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.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.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()); + assertEquals(id1.name(), tables.get(0)); + + // update + TableSpec table2Spec = TableBuilder.datasource(table1Name, "PT1H").buildSpec(); + 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); + + // 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, 0, false, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + TableId id2 = TableId.of(TableId.DRUID_SCHEMA, table2Name); + + // verify lists + 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.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()); + assertEquals(id1.name(), tables.get(0)); + assertEquals(id2.name(), tables.get(1)); + + // delete and verify + resp = resource.deleteTable(TableId.DRUID_SCHEMA, table1Name, deleteBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + 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()); + + resp = resource.deleteTable(TableId.DRUID_SCHEMA, table2Name, deleteBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + 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()); + } + + @Test + public void testMoveColumn() + { + String tableName = "move"; + TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D") + .column("a", "VARCHAR") + .column("b", "BIGINT") + .column("c", "FLOAT") + .buildSpec(); + 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.editTable("bogus", tableName, cmd, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // Bad table + 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.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.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.editTable(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()) + ); + } + + @Test + public void testHideColumns() + { + String tableName = "hide"; + TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D") + .buildSpec(); + 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 + 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. 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.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)); + TableMetadata read = (TableMetadata) resp.getEntity(); + assertNull(read.spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY)); + + // Hide + 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)); + read = (TableMetadata) resp.getEntity(); + assertEquals( + Arrays.asList("a", "b"), + read.spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + assertTrue(read.updateTime() > version); + + // 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( + Collections.singletonList("b"), + read.spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + assertTrue(read.updateTime() > version); + } + + @Test + public void testDropColumns() + { + String tableName = "drop"; + TableSpec dsSpec = TableBuilder.datasource(tableName, "P1D") + .column("a", "VARCHAR") + .column("b", "BIGINT") + .column("c", "FLOAT") + .buildSpec(); + + 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 + DropColumns cmd = new DropColumns(Collections.emptyList()); + resp = resource.editTable("bogus", tableName, cmd, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.NOT_FOUND.getStatusCode(), resp.getStatus()); + + // Nothing to do + resp = resource.editTable(TableId.DRUID_SCHEMA, "bogus", cmd, postBy(CatalogTests.WRITER_USER)); + assertEquals(Response.Status.OK.getStatusCode(), resp.getStatus()); + + 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)); + TableMetadata read = (TableMetadata) resp.getEntity(); + assertEquals( + CatalogUtils.columnNames(dsSpec.columns()), + CatalogUtils.columnNames(read.spec().columns()) + ); + + // 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()); + + 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/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/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..ab12a8e2455e --- /dev/null +++ b/extensions-core/druid-catalog/src/test/java/org/apache/druid/server/http/catalog/EditorTest.java @@ -0,0 +1,469 @@ +/* + * 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.curator.shaded.com.google.common.collect.ImmutableMap; +import org.apache.druid.catalog.CatalogException; +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; +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.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; +import org.apache.druid.catalog.storage.CatalogTests; +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.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.assertNull; +import static org.junit.Assert.assertThrows; + +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() throws CatalogException + { + 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(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + + // Hide starting from an empty list. + Map props = new HashMap<>(table.spec().properties()); + 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(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + + // Hide, but all are duplicates + props = new HashMap<>(table.spec().properties()); + 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()); + + // Hide with existing columns + cmd = new HideColumns(Arrays.asList("b", "d")); + assertEquals( + Arrays.asList("a", "b", "c", "d"), + doEdit(tableName, cmd).spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + + // Hide with duplicates + props = new HashMap<>(table.spec().properties()); + 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(AbstractDatasourceDefn.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()); + + // Empty list + cmd = new UnhideColumns(Collections.emptyList()); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); + + // Unhide starting from a null list. + cmd = new UnhideColumns(Arrays.asList("a", "b")); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); + + // Unhide starting from an empty list. + Map props = new HashMap<>(table.spec().properties()); + props.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList()); + catalog.tables().replace(table.withProperties(props)); + assertNull( + 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(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList()); + catalog.tables().replace(table.withProperties(props)); + cmd = new UnhideColumns(Collections.singletonList("a")); + assertNull( + 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(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()); + + // Unhide some hidden columns. + cmd = new UnhideColumns(Arrays.asList("a", "c", "a", "d")); + assertEquals( + 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(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(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + } + + @Test + public void testDropColumns() 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 drop list + DropColumns cmd = new DropColumns(null); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); + + // Empty list + cmd = new DropColumns(Collections.emptyList()); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); + + // Drop non-existent columns + cmd = new DropColumns(Arrays.asList("d", "e")); + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); + + // Drop some columns, including dups. + cmd = new DropColumns(Arrays.asList("c", "a", "c", "d")); + assertEquals( + Collections.singletonList("b"), + CatalogUtils.columnNames(doEdit(tableName, cmd).spec().columns()) + ); + + // Drop all columns + catalog.tables().replace(table); + cmd = new DropColumns(Arrays.asList("c", "a", "c", "b")); + assertEquals( + Collections.emptyList(), + doEdit(tableName, cmd).spec().columns() + ); + + // Drop from a null column list + assertEquals(0, new TableEditor(catalog, table.id(), cmd).go()); + } + + @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(AbstractDatasourceDefn.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(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H"); + updates.put("foo", "bar"); + cmd = new UpdateProperties(updates); + Map expected = ImmutableMap.of( + AbstractDatasourceDefn.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( + AbstractDatasourceDefn.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( + AbstractDatasourceDefn.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/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..5456d4b81b01 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 @@ -215,6 +218,10 @@ JUnitParams test + + javax.ws.rs + jsr311-api + @@ -302,6 +309,15 @@ MultiStageQuery + + IT-Catalog + + false + + + Catalog + + docker-tests 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..a43886424070 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/catalog/ITCatalogRestTest.java @@ -0,0 +1,196 @@ +/* + * 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.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; +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; +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, false) + ); + } + + // Read-only schema + { + final TableMetadata table = new TableBuilder() + .id(TableId.of(TableId.SYSTEM_SCHEMA, "foo")) + .property(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D") + .build(); + assertThrows( + Exception.class, + () -> client.createTable(table, false) + ); + } + + // Malformed table name + { + final TableMetadata table = TableBuilder.datasource(" foo ", "P1D") + .build(); + assertThrows( + Exception.class, + () -> client.createTable(table, false) + ); + } + } + + /** + * 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.datasource("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, true); + + // Update the datasource + TableSpec dsSpec2 = TableBuilder.copyOf(table) + .property(AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 3_000_000) + .column("d", "DOUBLE") + .buildSpec(); + + // First, optimistic locking, wrong version + assertThrows(ISE.class, () -> 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.editTable(table.id(), moveCmd); + + // Drop a column + 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") + ); + client.editTable(table.id(), hideCmd); + read = client.readTable(table.id()); + assertEquals( + Arrays.asList("e", "f"), + read.spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + + // Unhide + UnhideColumns unhideCmd = new UnhideColumns( + Collections.singletonList("e") + ); + client.editTable(table.id(), unhideCmd); + read = client.readTable(table.id()); + assertEquals( + Collections.singletonList("f"), + read.spec().properties().get(AbstractDatasourceDefn.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..eee5b3d79730 --- /dev/null +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/cluster/CatalogClient.java @@ -0,0 +1,168 @@ +/* + * 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.http.TableEditRequest; +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.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, boolean overwrite) + { + // Use action= + String url = StringUtils.format( + "%s%s/schemas/%s/tables/%s", + clusterClient.leadCoordinatorUrl(), + CatalogResource.ROOT_PATH, + table.id().schema(), + table.id().name() + ); + if (overwrite) { + url += "?overwrite=true"; + } + 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/schemas/%s/tables/%s", + clusterClient.leadCoordinatorUrl(), + CatalogResource.ROOT_PATH, + tableId.schema(), + tableId.name() + ); + if (version > 0) { + url += "?version=" + version; + } + VersionResponse response = clusterClient.post(url, tableSpec, VersionResponse.class); + return response.version; + } + + public TableMetadata readTable(TableId tableId) + { + String url = StringUtils.format( + "%s%s/schemas/%s/tables/%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/schemas/%s/tables/%s", + clusterClient.leadCoordinatorUrl(), + CatalogResource.ROOT_PATH, + tableId.schema(), + tableId.name() + ); + clusterClient.send(HttpMethod.DELETE, url); + } + + public long editTable(TableId tableId, TableEditRequest cmd) + { + String url = StringUtils.format( + "%s%s/schemas/%s/tables/%s/edit", + clusterClient.leadCoordinatorUrl(), + CatalogResource.ROOT_PATH, + tableId.schema(), + tableId.name() + ); + VersionResponse response = clusterClient.post(url, cmd, VersionResponse.class); + return response.version; + } + + public List listSchemas() + { + String url = StringUtils.format( + "%s%s/schemas?format=name", + clusterClient.leadCoordinatorUrl(), + CatalogResource.ROOT_PATH + ); + return clusterClient.getAs(url, new TypeReference>() { }); + } + + public List listTables() + { + String url = StringUtils.format( + "%s%s/schemas?format=path", + clusterClient.leadCoordinatorUrl(), + CatalogResource.ROOT_PATH + ); + return clusterClient.getAs(url, new TypeReference>() { }); + } + + public List listTableNamesInSchema(String schemaName) + { + String url = StringUtils.format( + "%s%s/schemas/%s/tables?format=name", + 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?format=metadata", + 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 2d7f1aea108b..9cddaf49cb3b 100644 --- a/pom.xml +++ b/pom.xml @@ -185,6 +185,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/server/pom.xml b/server/pom.xml index 70ab6e2dd7b7..d4bafbb23b9f 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..9301a931afbf --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java @@ -0,0 +1,147 @@ +/* + * 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 javax.annotation.Nullable; + +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class CatalogUtils +{ + public static List columnNames(List columns) + { + return columns + .stream() + .map(col -> col.name()) + .collect(Collectors.toList()); + } + + /** + * 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; + } + 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"; + } + + /** + * 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 + ) + { + 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 new file mode 100644 index 000000000000..dd23522deb82 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.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.model; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.catalog.model.ModelProperties.PropertyDefn; +import org.apache.druid.java.util.common.IAE; + +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 +{ + /** + * 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; + } + + /** + * Merges an updated version of this column with an existing version. + */ + 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); + } + + /** + * 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(); + 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()); + } +} 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..c8baf082dc43 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.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.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.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 +{ + /** + * 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 {@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. + */ + 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 + 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"); + } + } + + @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..f16ed3bedb28 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/Columns.java @@ -0,0 +1,107 @@ +/* + * 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(TIMESTAMP, 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(); + + private Columns() + { + } + + 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.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/ModelProperties.java b/server/src/main/java/org/apache/druid/catalog/model/ModelProperties.java new file mode 100644 index 000000000000..07c9be973e00 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/ModelProperties.java @@ -0,0 +1,312 @@ +/* + * 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 ModelProperties +{ + 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); + } + + 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. + */ + @Override + 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; + } + + @Override + 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/ObjectDefn.java b/server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java new file mode 100644 index 000000000000..895ea64c88c4 --- /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.ModelProperties.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 {@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. + *

+ * 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. + */ + public 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/ParameterizedDefn.java b/server/src/main/java/org/apache/druid/catalog/model/ParameterizedDefn.java new file mode 100644 index 000000000000..007fd43766b6 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/ParameterizedDefn.java @@ -0,0 +1,69 @@ +/* + * 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.ExternalTableSpec; + +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 + { + 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); + ExternalTableSpec applyParameters(ResolvedTable table, Map parameters); +} 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..6654ceea74d8 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistry.java @@ -0,0 +1,44 @@ +/* + * 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.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(); + 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 new file mode 100644 index 000000000000..fbdfc7fc76a3 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/SchemaRegistryImpl.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.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; + +/** + * 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, + AbstractDatasourceDefn.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()); + } + + @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 new file mode 100644 index 000000000000..2916aabe885f --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/TableDefn.java @@ -0,0 +1,176 @@ +/* + * 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.ModelProperties.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 ParameterizedDefn} 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 ModelProperties.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()); + validateColumns(table.spec().columns(), table.jsonMapper()); + } + + public void validateColumns(List columns, ObjectMapper jsonMapper) + { + if (columns == null) { + return; + } + Set names = new HashSet<>(); + for (ColumnSpec colSpec : columns) { + if (!names.add(colSpec.name())) { + throw new IAE("Duplicate column name: " + colSpec.name()); + } + ColumnDefn.ResolvedColumn resolvedCol = resolveColumn(colSpec); + resolvedCol.validate(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. + */ + public 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..7e952a4393a3 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/TableDefnRegistry.java @@ -0,0 +1,107 @@ +/* + * 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; + +/** + * 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(), + 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 TableDefn defnFor(String type) + { + return defns.get(type); + } + + public ObjectMapper jsonMapper() + { + return 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..1a09171f81d3 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/TableMetadata.java @@ -0,0 +1,308 @@ +/* + * 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.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.Map; +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. + *

+ * 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"), + 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 spec + ) + { + return new TableMetadata( + id, + 0, + 0, + TableState.ACTIVE, + spec + ); + } + + 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 static TableMetadata empty(TableId id) + { + return new TableMetadata( + id, + 0, + 0, + null, + null + ); + } + + 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 + ); + } + + 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()) + ); + } + + public TableMetadata withProperties(Map props) + { + return new TableMetadata( + id, + creationTime, + updateTime, + state, + spec.withProperties(props) + ); + } + + @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."); + } + } + + @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..e65e1821bc35 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/TableSpec.java @@ -0,0 +1,137 @@ +/* + * 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.google.common.base.Strings; +import org.apache.druid.java.util.common.IAE; + +import javax.annotation.Nullable; + +import java.util.Collections; +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") @Nullable final Map properties, + @JsonProperty("columns") @Nullable 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); + } + + @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"); + } + } + + @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..13d0384509f1 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/facade/ColumnFacade.java @@ -0,0 +1,57 @@ +/* + * 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.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(); + } + } + + 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..bed1245da203 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/facade/DatasourceFacade.java @@ -0,0 +1,71 @@ +/* + * 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.AbstractDatasourceDefn; +import org.apache.druid.catalog.model.table.ClusterKeySpec; +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 String segmentGranularityString() + { + return stringProperty(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY); + } + + public Granularity segmentGranularity() + { + String value = stringProperty(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY); + return value == null ? null : CatalogUtils.asDruidGranularity(value); + } + + public Integer targetSegmentRows() + { + return intProperty(AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY); + } + + @SuppressWarnings("unchecked") + public List clusterKeys() + { + return (List) property(AbstractDatasourceDefn.CLUSTER_KEYS_PROPERTY); + } + + @SuppressWarnings("unchecked") + public List hiddenColumns() + { + Object value = property(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY); + return value == null ? Collections.emptyList() : (List) value; + } +} diff --git a/server/src/main/java/org/apache/druid/catalog/model/facade/ExternalTableFacade.java b/server/src/main/java/org/apache/druid/catalog/model/facade/ExternalTableFacade.java new file mode 100644 index 000000000000..f2571150305e --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/facade/ExternalTableFacade.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 ExternalTableFacade extends TableFacade +{ + public ExternalTableFacade(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/AbstractDatasourceDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.java new file mode 100644 index 000000000000..bbfc2d5042cd --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/AbstractDatasourceDefn.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.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.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; +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"; + + /** + * 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 + { + 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 ModelProperties.IntPropertyDefn(TARGET_SEGMENT_ROWS_PROPERTY), + new ModelProperties.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/ClusterKeySpec.java b/server/src/main/java/org/apache/druid/catalog/model/table/ClusterKeySpec.java new file mode 100644 index 000000000000..6e2267ecd160 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/ClusterKeySpec.java @@ -0,0 +1,86 @@ +/* + * 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; + +/** + * 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; + private final boolean desc; + + @JsonCreator + public ClusterKeySpec( + @JsonProperty("column") String expr, + @JsonProperty("desc") @Nullable Boolean desc + ) + { + this.expr = expr; + this.desc = desc != null && desc; + } + + @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..5bbecbc1aa06 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/DatasourceDefn.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.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; + +/** + * 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 +{ + /** + * Definition of a column in a datasource. + */ + public static class DatasourceColumnDefn extends ColumnDefn + { + public static final String COLUMN_TYPE = "column"; + + public DatasourceColumnDefn() + { + super( + "Column", + COLUMN_TYPE, + null + ); + } + + @Override + 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 + ) + ); + } + } + } + + public static final String TABLE_TYPE = "datasource"; + + public DatasourceDefn() + { + super( + "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 new file mode 100644 index 000000000000..e808d1833e64 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableDefn.java @@ -0,0 +1,280 @@ +/* + * 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 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.ModelProperties.PropertyDefn; +import org.apache.druid.catalog.model.ParameterizedDefn; +import org.apache.druid.catalog.model.ParameterizedDefn.ParameterDefn; +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 ParameterizedDefn} 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 final 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: %s", param.name()); + } + } + this.parameterMap = ImmutableMap.copyOf(params); + } + } + + public List parameters() + { + return parameterList; + } + + 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) + { + return new ExternalTableSpec( + convertSource(table), + convertFormat(table), + Columns.convertSignature(table.spec()) + ); + } + + /** + * 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 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 ExternalTableSpec applyParameters(ResolvedTable table, Map parameters) + { + ResolvedTable revised = mergeParameters(table, parameters); + return convertToExtern(revised); + } + + public static Set tableTypes() + { + // Known input tables. Get this from a registry later. + 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/ExternalTableSpec.java b/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableSpec.java new file mode 100644 index 000000000000..588e5e6b03a2 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/ExternalTableSpec.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 ExternalTableSpec +{ + protected final InputSource inputSource; + protected final InputFormat inputFormat; + protected final RowSignature signature; + + public ExternalTableSpec( + 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/HttpTableDefn.java b/server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.java new file mode 100644 index 000000000000..eb97cc744f20 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/HttpTableDefn.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.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.ModelProperties.StringListPropertyDefn; +import org.apache.druid.catalog.model.ModelProperties.StringPropertyDefn; +import org.apache.druid.catalog.model.ParameterizedDefn; +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 ParameterizedDefn +{ + public static final String TABLE_TYPE = HttpInputSource.TYPE_KEY; + + // 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"; + 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..273aad2696d9 --- /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.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; +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..9d1235aac862 --- /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.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; +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..48992483dd53 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/LocalTableDefn.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.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.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; + +/** + * 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; + + /** + * 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/TableBuilder.java b/server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java new file mode 100644 index 000000000000..b18aee4efc40 --- /dev/null +++ b/server/src/main/java/org/apache/druid/catalog/model/table/TableBuilder.java @@ -0,0 +1,269 @@ +/* + * 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 datasource(String name, String granularity) + { + return new TableBuilder() + .datasource(name) + .type(DatasourceDefn.TABLE_TYPE) + .segmentGranularity(granularity); + } + + public static TableBuilder external(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 segmentGranularity(String segmentGranularity) + { + return property(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, segmentGranularity); + } + + public TableBuilder clusterColumns(ClusterKeySpec...clusterKeys) + { + return property(AbstractDatasourceDefn.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.TABLE_TYPE.equals(tableType)) { + colType = DatasourceDefn.DatasourceColumnDefn.COLUMN_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 hiddenColumns(List hiddenColumns) + { + return property(AbstractDatasourceDefn.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/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/PropertyDefnTest.java b/server/src/test/java/org/apache/druid/catalog/model/PropertyDefnTest.java new file mode 100644 index 000000000000..99714f0411ef --- /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.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; + +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..5ae0bfd4ceba --- /dev/null +++ b/server/src/test/java/org/apache/druid/catalog/model/TableMetadataTest.java @@ -0,0 +1,143 @@ +/* + * 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 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.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 +{ + @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()); + } + + @Test + public void testIdEquals() + { + EqualsVerifier.forClass(TableId.class) + .usingGetClass() + .verify(); + } + + @Test + public void testTableMetadata() + { + Map props = ImmutableMap.of( + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D" + ); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_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()); + } + + { + 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( + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D" + ); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_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..b544ab4ca877 --- /dev/null +++ b/server/src/test/java/org/apache/druid/catalog/model/table/DatasourceTableTest.java @@ -0,0 +1,512 @@ +/* + * 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.DatasourceColumnDefn; +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( + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D" + ); + { + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); + ResolvedTable table = registry.resolve(spec); + assertNotNull(table); + assertTrue(table.defn() instanceof DatasourceDefn); + table.validate(); + } + + { + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); + ResolvedTable table = registry.resolve(spec); + assertNotNull(table); + assertTrue(table.defn() instanceof DatasourceDefn); + 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.TABLE_TYPE, ImmutableMap.of(), null); + ResolvedTable table = registry.resolve(spec); + expectValidationFails(table); + } + + { + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, ImmutableMap.of(), null); + expectValidationFails(spec); + } + } + + @Test + public void testAllProperties() + { + Map props = ImmutableMap.builder() + .put(TableDefn.DESCRIPTION_PROPERTY, "My table") + .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.TABLE_TYPE, props, null); + expectValidationSucceeds(spec); + } + + { + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); + expectValidationSucceeds(spec); + } + } + + @Test + public void testWrongTypes() + { + { + TableSpec spec = new TableSpec("bogus", ImmutableMap.of(), null); + assertThrows(IAE.class, () -> registry.resolve(spec)); + } + + // Segment granularity + { + TableSpec spec = TableBuilder.datasource("foo", "bogus").buildSpec(); + expectValidationFails(spec); + } + + { + TableSpec spec = TableBuilder.datasource("foo", "bogus").buildSpec(); + expectValidationFails(spec); + } + + // Target segment rows + { + TableSpec spec = TableBuilder.datasource("foo", "P1D") + .property(AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, "bogus") + .buildSpec(); + expectValidationFails(spec); + } + + // Hidden columns + { + TableSpec spec = TableBuilder.datasource("foo", "P1D") + .property(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, "bogus") + .buildSpec(); + expectValidationFails(spec); + } + { + TableSpec spec = TableBuilder.datasource("foo", "P1D") + .hiddenColumns("a", Columns.TIME_COLUMN) + .buildSpec(); + expectValidationFails(spec); + } + } + + @Test + public void testExtendedProperties() + { + TableSpec spec = TableBuilder.datasource("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(DatasourceColumnDefn.COLUMN_TYPE, null, null, null); + assertThrows(IAE.class, () -> spec.validate()); + } + { + ColumnSpec spec = new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "foo", null, null); + spec.validate(); + } + + // Type is optional + { + ColumnSpec spec = new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "foo", "VARCHAR", null); + spec.validate(); + } + } + + @Test + public void testDetailTableColumns() + { + TableBuilder builder = TableBuilder.datasource("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); + } + + // 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 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.datasource("foo", "PT1H") + .description("My table") + .property(AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000) + .hiddenColumns("foo", "bar") + .property("tag1", "some value") + .property("tag2", "second value") + .column(new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "a", null, colProps)) + .column("b", Columns.VARCHAR) + .buildSpec(); + + // Sanity check + expectValidationSucceeds(spec); + return spec; + } + + 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(AbstractDatasourceDefn.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(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY), + merged.properties().get(AbstractDatasourceDefn.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(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, null); + TableSpec update = new TableSpec(null, updatedProps, null); + TableSpec merged = mergeTables(spec, update); + expectValidationSucceeds(merged); + assertFalse( + merged.properties().containsKey(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + + // Wrong type + updatedProps = ImmutableMap.of( + AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, "mumble" + ); + update = new TableSpec(null, updatedProps, null); + assertMergeFails(spec, update); + + // Merge + updatedProps = ImmutableMap.of( + AbstractDatasourceDefn.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(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY) + ); + } + + @Test + public void testMergeColsWithEmptyList() + { + Map props = ImmutableMap.of( + AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D" + ); + TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null); + + List colUpdates = Collections.singletonList( + new ColumnSpec( + DatasourceColumnDefn.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( + DatasourceColumnDefn.COLUMN_TYPE, + "a", + Columns.BIGINT, + updatedProps + ), + new ColumnSpec( + DatasourceColumnDefn.COLUMN_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..a6a4b6935fb2 --- /dev/null +++ b/server/src/test/java/org/apache/druid/catalog/model/table/HttpInputTableTest.java @@ -0,0 +1,212 @@ +/* + * 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.ParameterizedDefn; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableDefnRegistry; +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())); + + // Convert to an external spec + ExternalTableSpec 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 + ExternalTableSpec 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 + ParameterizedDefn 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 + ExternalTableSpec 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..f8d4057a4ce1 --- /dev/null +++ b/server/src/test/java/org/apache/druid/catalog/model/table/InlineTableTest.java @@ -0,0 +1,90 @@ +/* + * 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.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())); + + // Convert to an external spec + ExternalTableSpec 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..9246078fe64f --- /dev/null +++ b/server/src/test/java/org/apache/druid/catalog/model/table/LocalTableTest.java @@ -0,0 +1,204 @@ +/* + * 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.ParameterizedDefn; +import org.apache.druid.catalog.model.ResolvedTable; +import org.apache.druid.catalog.model.TableDefnRegistry; +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())); + + // Convert to an external spec + ExternalTableSpec 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 + ExternalTableSpec 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 + ExternalTableSpec 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); + + ParameterizedDefn 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 + ExternalTableSpec 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 + ExternalTableSpec 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.