diff --git a/docs/multi-stage-query/concepts.md b/docs/multi-stage-query/concepts.md index d9385061d79f..27b7d12c91c9 100644 --- a/docs/multi-stage-query/concepts.md +++ b/docs/multi-stage-query/concepts.md @@ -115,6 +115,14 @@ When deciding whether to use `REPLACE` or `INSERT`, keep in mind that segments g with dimension-based pruning but those generated with `INSERT` cannot. For more information about the requirements for dimension-based pruning, see [Clustering](#clustering). +### Write to an external destination with `EXTERN` + +Query tasks can write data to an external destination through the `EXTERN` function, when it is used with the `INTO` +clause, such as `INSERT INTO EXTERN(...)`. The EXTERN function takes arguments that specify where to write the files. +The format can be specified using an `AS` clause. + +For more information about the syntax, see [`EXTERN`](./reference.md#extern-function). + ### Primary timestamp Druid tables always include a primary timestamp named `__time`. diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index 1087c1564fd8..25f55b31f74f 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -45,8 +45,11 @@ making it easy to reuse the same SQL statement for each ingest: just specify the ### `EXTERN` Function -Use the `EXTERN` function to read external data. The function has two variations. +Use the `EXTERN` function to read external data or write to an external location. +#### `EXTERN` as an input source + +The function has two variations. Function variation 1, with the input schema expressed as JSON: ```sql @@ -90,6 +93,91 @@ can precede the column list: `EXTEND (timestamp VARCHAR...)`. For more information, see [Read external data with EXTERN](concepts.md#read-external-data-with-extern). +#### `EXTERN` to export to a destination + +`EXTERN` can be used to specify a destination where you want to export data to. +This variation of EXTERN requires one argument, the details of the destination as specified below. +This variation additionally requires an `AS` clause to specify the format of the exported rows. + +Keep the following in mind when using EXTERN to export rows: +- Only INSERT statements are supported. +- Only `CSV` format is supported as an export format. +- Partitioning (`PARTITIONED BY`) and clustering (`CLUSTERED BY`) aren't supported with export statements. +- You can export to Amazon S3 or local storage. +- The destination provided should contain no other files or directories. + +When you export data, use the `rowsPerPage` context parameter to control how many rows get exported. The default is 100,000. + +```sql +INSERT INTO + EXTERN() +AS CSV +SELECT + +FROM +``` + +##### S3 + +Export results to S3 by passing the function `S3()` as an argument to the `EXTERN` function. Note that this requires the `druid-s3-extensions`. +The `S3()` function is a Druid function that configures the connection. Arguments for `S3()` should be passed as named parameters with the value in single quotes like the following example: + +```sql +INSERT INTO + EXTERN( + S3(bucket => 'your_bucket', prefix => 'prefix/to/files') + ) +AS CSV +SELECT + +FROM
+``` + +Supported arguments for the function: + +| Parameter | Required | Description | Default | +|-------------|----------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|---------| +| `bucket` | Yes | The S3 bucket to which the files are exported to. The bucket and prefix combination should be whitelisted in `druid.export.storage.s3.allowedExportPaths`. | n/a | +| `prefix` | Yes | Path where the exported files would be created. The export query expects the destination to be empty. If the location includes other files, then the query will fail. The bucket and prefix combination should be whitelisted in `druid.export.storage.s3.allowedExportPaths`. | n/a | + +The following runtime parameters must be configured to export into an S3 destination: + +| Runtime Parameter | Required | Description | Default | +|----------------------------------------------|----------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-----| +| `druid.export.storage.s3.tempLocalDir` | Yes | Directory used on the local storage of the worker to store temporary files required while uploading the data. | n/a | +| `druid.export.storage.s3.allowedExportPaths` | Yes | An array of S3 prefixes that are whitelisted as export destinations. Export queries fail if the export destination does not match any of the configured prefixes. Example: `[\"s3://bucket1/export/\", \"s3://bucket2/export/\"]` | n/a | +| `druid.export.storage.s3.maxRetry` | No | Defines the max number times to attempt S3 API calls to avoid failures due to transient errors. | 10 | +| `druid.export.storage.s3.chunkSize` | No | Defines the size of each chunk to temporarily store in `tempDir`. The chunk size must be between 5 MiB and 5 GiB. A large chunk size reduces the API calls to S3, however it requires more disk space to store the temporary chunks. | 100MiB | + +##### LOCAL + +You can export to the local storage, which exports the results to the filesystem of the MSQ worker. +This is useful in a single node setup or for testing but is not suitable for production use cases. + +Export results to local storage by passing the function `LOCAL()` as an argument for the `EXTERN FUNCTION`. +To use local storage as an export destination, the runtime property `druid.export.storage.baseDir` must be configured on the Indexer/Middle Manager. +This value must be set to an absolute path on the local machine. Exporting data will be allowed to paths which match the prefix set by this value. +Arguments to `LOCAL()` should be passed as named parameters with the value in single quotes in the following example: + +```sql +INSERT INTO + EXTERN( + local(exportPath => 'exportLocation/query1') + ) +AS CSV +SELECT + +FROM
+``` + +Supported arguments to the function: + +| Parameter | Required | Description | Default | +|-------------|--------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| --| +| `exportPath` | Yes | Absolute path to a subdirectory of `druid.export.storage.baseDir` used as the destination to export the results to. The export query expects the destination to be empty. If the location includes other files or directories, then the query will fail. | n/a | + +For more information, see [Read external data with EXTERN](concepts.md#write-to-an-external-destination-with-extern). + ### `INSERT` Use the `INSERT` statement to insert data. diff --git a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java index f33a3b1f44db..49856a9c1eff 100644 --- a/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java +++ b/extensions-core/google-extensions/src/main/java/org/apache/druid/storage/google/output/GoogleStorageConnectorProvider.java @@ -60,5 +60,4 @@ public StorageConnector get() { return new GoogleStorageConnector(this, googleStorage, googleInputDataConfig); } - } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index f31f66c5ef3a..d62bcce04ddc 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -106,6 +106,7 @@ import org.apache.druid.msq.indexing.client.ControllerChatHandler; import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; import org.apache.druid.msq.indexing.destination.DurableStorageMSQDestination; +import org.apache.druid.msq.indexing.destination.ExportMSQDestination; import org.apache.druid.msq.indexing.destination.MSQSelectDestination; import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; import org.apache.druid.msq.indexing.error.CanceledFault; @@ -165,9 +166,9 @@ import org.apache.druid.msq.querykit.MultiQueryKit; import org.apache.druid.msq.querykit.QueryKit; import org.apache.druid.msq.querykit.QueryKitUtils; -import org.apache.druid.msq.querykit.ShuffleSpecFactories; import org.apache.druid.msq.querykit.ShuffleSpecFactory; import org.apache.druid.msq.querykit.groupby.GroupByQueryKit; +import org.apache.druid.msq.querykit.results.ExportResultsFrameProcessorFactory; import org.apache.druid.msq.querykit.results.QueryResultFrameProcessorFactory; import org.apache.druid.msq.querykit.scan.ScanQueryKit; import org.apache.druid.msq.shuffle.input.DurableStorageInputChannelFactory; @@ -201,6 +202,8 @@ import org.apache.druid.sql.calcite.planner.ColumnMapping; import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.calcite.rel.DruidQuery; +import org.apache.druid.sql.http.ResultFormat; +import org.apache.druid.storage.ExportStorageProvider; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentTimeline; import org.apache.druid.timeline.partition.DimensionRangeShardSpec; @@ -220,6 +223,7 @@ import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; @@ -1756,7 +1760,8 @@ private static QueryDefinition makeQueryDefinition( final ShuffleSpecFactory shuffleSpecFactory; if (MSQControllerTask.isIngestion(querySpec)) { - shuffleSpecFactory = ShuffleSpecFactories.getGlobalSortWithTargetSize(tuningConfig.getRowsPerSegment()); + shuffleSpecFactory = querySpec.getDestination() + .getShuffleSpecFactory(tuningConfig.getRowsPerSegment()); if (!columnMappings.hasUniqueOutputColumnNames()) { // We do not expect to hit this case in production, because the SQL validator checks that column names @@ -1777,16 +1782,10 @@ private static QueryDefinition makeQueryDefinition( } else { queryToPlan = querySpec.getQuery(); } - } else if (querySpec.getDestination() instanceof TaskReportMSQDestination) { - shuffleSpecFactory = ShuffleSpecFactories.singlePartition(); - queryToPlan = querySpec.getQuery(); - } else if (querySpec.getDestination() instanceof DurableStorageMSQDestination) { - shuffleSpecFactory = ShuffleSpecFactories.getGlobalSortWithTargetSize( - MultiStageQueryContext.getRowsPerPage(querySpec.getQuery().context()) - ); - queryToPlan = querySpec.getQuery(); } else { - throw new ISE("Unsupported destination [%s]", querySpec.getDestination()); + shuffleSpecFactory = querySpec.getDestination() + .getShuffleSpecFactory(MultiStageQueryContext.getRowsPerPage(querySpec.getQuery().context())); + queryToPlan = querySpec.getQuery(); } final QueryDefinition queryDef; @@ -1877,6 +1876,43 @@ private static QueryDefinition makeQueryDefinition( } else { return queryDef; } + } else if (querySpec.getDestination() instanceof ExportMSQDestination) { + final ExportMSQDestination exportMSQDestination = (ExportMSQDestination) querySpec.getDestination(); + final ExportStorageProvider exportStorageProvider = exportMSQDestination.getExportStorageProvider(); + + try { + // Check that the export destination is empty as a sanity check. We want to avoid modifying any other files with export. + Iterator filesIterator = exportStorageProvider.get().listDir(""); + if (filesIterator.hasNext()) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build("Found files at provided export destination[%s]. Export is only allowed to " + + "an empty path. Please provide an empty path/subdirectory or move the existing files.", + exportStorageProvider.getBasePath()); + } + } + catch (IOException e) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build(e, "Exception occurred while connecting to export destination."); + } + + + final ResultFormat resultFormat = exportMSQDestination.getResultFormat(); + final QueryDefinitionBuilder builder = QueryDefinition.builder(); + builder.addAll(queryDef); + builder.add(StageDefinition.builder(queryDef.getNextStageNumber()) + .inputs(new StageInputSpec(queryDef.getFinalStageDefinition().getStageNumber())) + .maxWorkerCount(tuningConfig.getMaxNumWorkers()) + .signature(queryDef.getFinalStageDefinition().getSignature()) + .shuffleSpec(null) + .processorFactory(new ExportResultsFrameProcessorFactory( + queryId, + exportStorageProvider, + resultFormat + )) + ); + return builder.build(); } else { throw new ISE("Unsupported destination [%s]", querySpec.getDestination()); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java index 4af832705c3f..c2b50f172677 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/guice/MSQIndexingModule.java @@ -88,6 +88,7 @@ import org.apache.druid.msq.querykit.common.SortMergeJoinFrameProcessorFactory; import org.apache.druid.msq.querykit.groupby.GroupByPostShuffleFrameProcessorFactory; import org.apache.druid.msq.querykit.groupby.GroupByPreShuffleFrameProcessorFactory; +import org.apache.druid.msq.querykit.results.ExportResultsFrameProcessorFactory; import org.apache.druid.msq.querykit.results.QueryResultFrameProcessorFactory; import org.apache.druid.msq.querykit.scan.ScanQueryFrameProcessorFactory; import org.apache.druid.msq.util.PassthroughAggregatorFactory; @@ -158,6 +159,7 @@ public List getJacksonModules() NilExtraInfoHolder.class, SortMergeJoinFrameProcessorFactory.class, QueryResultFrameProcessorFactory.class, + ExportResultsFrameProcessorFactory.class, // DataSource classes (note: ExternalDataSource is in MSQSqlModule) InputNumberDataSource.class, diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java index 3cdf706ba163..7eb455ca8424 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQControllerTask.java @@ -57,6 +57,7 @@ import org.apache.druid.rpc.StandardRetryPolicy; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.sql.calcite.run.SqlResults; import org.joda.time.Interval; @@ -65,6 +66,7 @@ import javax.annotation.Nullable; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; @JsonTypeName(MSQControllerTask.TYPE) @@ -273,6 +275,12 @@ private static String getDataSourceForTaskMetadata(final MSQSpec querySpec) } } + @Override + public Optional getDestinationResource() + { + return querySpec.getDestination().getDestinationResource(); + } + public static boolean isIngestion(final MSQSpec querySpec) { return querySpec.getDestination() instanceof DataSourceMSQDestination; diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DataSourceMSQDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DataSourceMSQDestination.java index 5bf033a1aa03..0854582a733c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DataSourceMSQDestination.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DataSourceMSQDestination.java @@ -26,12 +26,17 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.msq.querykit.ShuffleSpecFactories; +import org.apache.druid.msq.querykit.ShuffleSpecFactory; +import org.apache.druid.server.security.Resource; +import org.apache.druid.server.security.ResourceType; import org.joda.time.Interval; import javax.annotation.Nullable; import java.util.Collections; import java.util.List; import java.util.Objects; +import java.util.Optional; public class DataSourceMSQDestination implements MSQDestination { @@ -160,4 +165,16 @@ public String toString() ", replaceTimeChunks=" + replaceTimeChunks + '}'; } + + @Override + public ShuffleSpecFactory getShuffleSpecFactory(int targetSize) + { + return ShuffleSpecFactories.getGlobalSortWithTargetSize(targetSize); + } + + @Override + public Optional getDestinationResource() + { + return Optional.of(new Resource(getDataSource(), ResourceType.DATASOURCE)); + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DurableStorageMSQDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DurableStorageMSQDestination.java index c401d2eee6e4..e522243b60d2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DurableStorageMSQDestination.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/DurableStorageMSQDestination.java @@ -20,6 +20,13 @@ package org.apache.druid.msq.indexing.destination; import com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.querykit.ShuffleSpecFactories; +import org.apache.druid.msq.querykit.ShuffleSpecFactory; +import org.apache.druid.server.security.Resource; +import org.apache.druid.server.security.ResourceType; + +import java.util.Optional; public class DurableStorageMSQDestination implements MSQDestination { @@ -45,4 +52,15 @@ public String toString() return "DurableStorageDestination{}"; } + @Override + public ShuffleSpecFactory getShuffleSpecFactory(int targetSize) + { + return ShuffleSpecFactories.getGlobalSortWithTargetSize(targetSize); + } + + @Override + public Optional getDestinationResource() + { + return Optional.of(new Resource(MSQControllerTask.DUMMY_DATASOURCE_FOR_SELECT, ResourceType.DATASOURCE)); + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/ExportMSQDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/ExportMSQDestination.java new file mode 100644 index 000000000000..3187ace349b1 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/ExportMSQDestination.java @@ -0,0 +1,109 @@ +/* + * 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.msq.indexing.destination; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.msq.querykit.ShuffleSpecFactories; +import org.apache.druid.msq.querykit.ShuffleSpecFactory; +import org.apache.druid.server.security.Resource; +import org.apache.druid.server.security.ResourceType; +import org.apache.druid.sql.http.ResultFormat; +import org.apache.druid.storage.ExportStorageProvider; + +import java.util.Objects; +import java.util.Optional; + +/** + * Destination used by tasks that write the results as files to an external destination. {@link #resultFormat} denotes + * the format of the file created and {@link #exportStorageProvider} denotes the type of external + * destination. + */ +public class ExportMSQDestination implements MSQDestination +{ + public static final String TYPE = "export"; + private final ExportStorageProvider exportStorageProvider; + private final ResultFormat resultFormat; + + @JsonCreator + public ExportMSQDestination( + @JsonProperty("exportStorageProvider") ExportStorageProvider exportStorageProvider, + @JsonProperty("resultFormat") ResultFormat resultFormat + ) + { + this.exportStorageProvider = exportStorageProvider; + this.resultFormat = resultFormat; + } + + + + @JsonProperty("exportStorageProvider") + public ExportStorageProvider getExportStorageProvider() + { + return exportStorageProvider; + } + + @JsonProperty("resultFormat") + public ResultFormat getResultFormat() + { + return resultFormat; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ExportMSQDestination that = (ExportMSQDestination) o; + return Objects.equals(exportStorageProvider, that.exportStorageProvider) + && resultFormat == that.resultFormat; + } + + @Override + public int hashCode() + { + return Objects.hash(exportStorageProvider, resultFormat); + } + + @Override + public String toString() + { + return "ExportMSQDestination{" + + "exportStorageProvider=" + exportStorageProvider + + ", resultFormat=" + resultFormat + + '}'; + } + + @Override + public ShuffleSpecFactory getShuffleSpecFactory(int targetSize) + { + return ShuffleSpecFactories.getGlobalSortWithTargetSize(targetSize); + } + + @Override + public Optional getDestinationResource() + { + return Optional.of(new Resource(getExportStorageProvider().getResourceType(), ResourceType.EXTERNAL)); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/MSQDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/MSQDestination.java index 52489d15a343..39460b15194c 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/MSQDestination.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/MSQDestination.java @@ -21,14 +21,21 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.msq.querykit.ShuffleSpecFactory; +import org.apache.druid.server.security.Resource; + +import java.util.Optional; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @JsonSubTypes.Type(name = DataSourceMSQDestination.TYPE, value = DataSourceMSQDestination.class), @JsonSubTypes.Type(name = TaskReportMSQDestination.TYPE, value = TaskReportMSQDestination.class), + @JsonSubTypes.Type(name = ExportMSQDestination.TYPE, value = ExportMSQDestination.class), @JsonSubTypes.Type(name = DurableStorageMSQDestination.TYPE, value = DurableStorageMSQDestination.class) }) public interface MSQDestination { - // No methods. Just a marker interface for deserialization. + ShuffleSpecFactory getShuffleSpecFactory(int targetSize); + + Optional getDestinationResource(); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/MSQSelectDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/MSQSelectDestination.java index db57fdc5dc07..e32705462470 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/MSQSelectDestination.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/MSQSelectDestination.java @@ -30,6 +30,10 @@ public enum MSQSelectDestination * Writes all the results directly to the report. */ TASKREPORT("taskReport", false), + /** + * Writes all the results as files in a specified format to an external location outside druid. + */ + EXPORT("export", false), /** * Writes the results as frame files to durable storage. Task report can be truncated to a preview. */ diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/TaskReportMSQDestination.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/TaskReportMSQDestination.java index ac1254abfe33..3f199255ac76 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/TaskReportMSQDestination.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/TaskReportMSQDestination.java @@ -20,6 +20,13 @@ package org.apache.druid.msq.indexing.destination; import com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.druid.msq.indexing.MSQControllerTask; +import org.apache.druid.msq.querykit.ShuffleSpecFactories; +import org.apache.druid.msq.querykit.ShuffleSpecFactory; +import org.apache.druid.server.security.Resource; +import org.apache.druid.server.security.ResourceType; + +import java.util.Optional; public class TaskReportMSQDestination implements MSQDestination { @@ -43,4 +50,15 @@ public String toString() return "TaskReportMSQDestination{}"; } + @Override + public ShuffleSpecFactory getShuffleSpecFactory(int targetSize) + { + return ShuffleSpecFactories.singlePartition(); + } + + @Override + public Optional getDestinationResource() + { + return Optional.of(new Resource(MSQControllerTask.DUMMY_DATASOURCE_FOR_SELECT, ResourceType.DATASOURCE)); + } } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java new file mode 100644 index 000000000000..de65d3e9d7ad --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessor.java @@ -0,0 +1,180 @@ +/* + * 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.msq.querykit.results; + +import com.fasterxml.jackson.databind.ObjectMapper; +import it.unimi.dsi.fastutil.ints.IntSet; +import org.apache.druid.error.DruidException; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.channel.ReadableFrameChannel; +import org.apache.druid.frame.channel.WritableFrameChannel; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.FrameProcessors; +import org.apache.druid.frame.processor.ReturnOrAwait; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.segment.FrameStorageAdapter; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Unit; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.msq.querykit.QueryKitUtils; +import org.apache.druid.msq.util.SequenceUtils; +import org.apache.druid.segment.BaseObjectColumnValueSelector; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.sql.http.ResultFormat; +import org.apache.druid.storage.StorageConnector; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public class ExportResultsFrameProcessor implements FrameProcessor +{ + private final ReadableFrameChannel inputChannel; + private final ResultFormat exportFormat; + private final FrameReader frameReader; + private final StorageConnector storageConnector; + private final ObjectMapper jsonMapper; + private final ChannelCounters channelCounter; + final String exportFilePath; + + public ExportResultsFrameProcessor( + final ReadableFrameChannel inputChannel, + final ResultFormat exportFormat, + final FrameReader frameReader, + final StorageConnector storageConnector, + final ObjectMapper jsonMapper, + final ChannelCounters channelCounter, + final String exportFilePath + ) + { + this.inputChannel = inputChannel; + this.exportFormat = exportFormat; + this.frameReader = frameReader; + this.storageConnector = storageConnector; + this.jsonMapper = jsonMapper; + this.channelCounter = channelCounter; + this.exportFilePath = exportFilePath; + } + + @Override + public List inputChannels() + { + return Collections.singletonList(inputChannel); + } + + @Override + public List outputChannels() + { + return Collections.emptyList(); + } + + @Override + public ReturnOrAwait runIncrementally(IntSet readableInputs) throws IOException + { + if (readableInputs.isEmpty()) { + return ReturnOrAwait.awaitAll(1); + } + + if (inputChannel.isFinished()) { + return ReturnOrAwait.returnObject(Unit.instance()); + } else { + exportFrame(inputChannel.read()); + return ReturnOrAwait.awaitAll(1); + } + } + + private void exportFrame(final Frame frame) throws IOException + { + final RowSignature exportRowSignature = createRowSignatureForExport(frameReader.signature()); + + final Sequence cursorSequence = + new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY) + .makeCursors(null, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, false, null); + + // Add headers if we are writing to a new file. + final boolean writeHeader = !storageConnector.pathExists(exportFilePath); + + try (OutputStream stream = storageConnector.write(exportFilePath)) { + ResultFormat.Writer formatter = exportFormat.createFormatter(stream, jsonMapper); + formatter.writeResponseStart(); + + if (writeHeader) { + formatter.writeHeaderFromRowSignature(exportRowSignature, false); + } + + SequenceUtils.forEach( + cursorSequence, + cursor -> { + try { + final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + //noinspection rawtypes + @SuppressWarnings("rawtypes") + final List selectors = + exportRowSignature + .getColumnNames() + .stream() + .map(columnSelectorFactory::makeColumnValueSelector) + .collect(Collectors.toList()); + + while (!cursor.isDone()) { + formatter.writeRowStart(); + for (int j = 0; j < exportRowSignature.size(); j++) { + formatter.writeRowField(exportRowSignature.getColumnName(j), selectors.get(j).getObject()); + } + channelCounter.incrementRowCount(); + formatter.writeRowEnd(); + cursor.advance(); + } + } + catch (IOException e) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build(e, "Exception occurred while writing file to the export location [%s].", exportFilePath); + } + } + ); + formatter.writeResponseEnd(); + } + } + + private static RowSignature createRowSignatureForExport(RowSignature inputRowSignature) + { + RowSignature.Builder exportRowSignatureBuilder = RowSignature.builder(); + inputRowSignature.getColumnNames() + .stream() + .filter(name -> !QueryKitUtils.PARTITION_BOOST_COLUMN.equals(name)) + .forEach(name -> exportRowSignatureBuilder.add(name, inputRowSignature.getColumnType(name).orElse(null))); + return exportRowSignatureBuilder.build(); + } + + @Override + public void cleanup() throws IOException + { + FrameProcessors.closeAll(inputChannels(), outputChannels()); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessorFactory.java new file mode 100644 index 000000000000..c9f9b6a40a81 --- /dev/null +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/results/ExportResultsFrameProcessorFactory.java @@ -0,0 +1,139 @@ +/* + * 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.msq.querykit.results; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.error.DruidException; +import org.apache.druid.frame.processor.FrameProcessor; +import org.apache.druid.frame.processor.OutputChannelFactory; +import org.apache.druid.frame.processor.OutputChannels; +import org.apache.druid.frame.processor.manager.ProcessorManagers; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.msq.counters.ChannelCounters; +import org.apache.druid.msq.counters.CounterNames; +import org.apache.druid.msq.counters.CounterTracker; +import org.apache.druid.msq.input.InputSlice; +import org.apache.druid.msq.input.InputSliceReader; +import org.apache.druid.msq.input.ReadableInput; +import org.apache.druid.msq.input.stage.StageInputSlice; +import org.apache.druid.msq.kernel.FrameContext; +import org.apache.druid.msq.kernel.ProcessorsAndChannels; +import org.apache.druid.msq.kernel.StageDefinition; +import org.apache.druid.msq.querykit.BaseFrameProcessorFactory; +import org.apache.druid.sql.http.ResultFormat; +import org.apache.druid.storage.ExportStorageProvider; +import org.apache.druid.utils.CollectionUtils; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.function.Consumer; + +@JsonTypeName("exportResults") +public class ExportResultsFrameProcessorFactory extends BaseFrameProcessorFactory +{ + private final String queryId; + private final ExportStorageProvider exportStorageProvider; + private final ResultFormat exportFormat; + + @JsonCreator + public ExportResultsFrameProcessorFactory( + @JsonProperty("queryId") String queryId, + @JsonProperty("exportStorageProvider") ExportStorageProvider exportStorageProvider, + @JsonProperty("exportFormat") ResultFormat exportFormat + ) + { + this.queryId = queryId; + this.exportStorageProvider = exportStorageProvider; + this.exportFormat = exportFormat; + } + + @JsonProperty("queryId") + public String getQueryId() + { + return queryId; + } + + @JsonProperty("exportFormat") + public ResultFormat getExportFormat() + { + return exportFormat; + } + + + @JsonProperty("exportStorageProvider") + public ExportStorageProvider getExportStorageProvider() + { + return exportStorageProvider; + } + + @Override + public ProcessorsAndChannels makeProcessors( + StageDefinition stageDefinition, + int workerNumber, + List inputSlices, + InputSliceReader inputSliceReader, + @Nullable Object extra, + OutputChannelFactory outputChannelFactory, + FrameContext frameContext, + int maxOutstandingProcessors, + CounterTracker counters, + Consumer warningPublisher + ) + { + final StageInputSlice slice = (StageInputSlice) CollectionUtils.getOnlyElement( + inputSlices, + x -> DruidException.defensive().build("Expected only a single input slice but found [%s]", inputSlices) + ); + + if (inputSliceReader.numReadableInputs(slice) == 0) { + return new ProcessorsAndChannels<>(ProcessorManagers.none(), OutputChannels.none()); + } + + ChannelCounters channelCounter = counters.channel(CounterNames.outputChannel()); + final Sequence readableInputs = + Sequences.simple(inputSliceReader.attach(0, slice, counters, warningPublisher)); + + final Sequence> processors = readableInputs.map( + readableInput -> new ExportResultsFrameProcessor( + readableInput.getChannel(), + exportFormat, + readableInput.getChannelFrameReader(), + exportStorageProvider.get(), + frameContext.jsonMapper(), + channelCounter, + getExportFilePath(queryId, workerNumber, readableInput.getStagePartition().getPartitionNumber(), exportFormat) + ) + ); + + return new ProcessorsAndChannels<>( + ProcessorManagers.of(processors), + OutputChannels.none() + ); + } + + private static String getExportFilePath(String queryId, int workerNumber, int partitionNumber, ResultFormat exportFormat) + { + return StringUtils.format("%s-worker%s-partition%s.%s", queryId, workerNumber, partitionNumber, exportFormat.toString()); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java index d38fa1a8dc64..f5a1fd8c90d9 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskQueryMaker.java @@ -39,6 +39,7 @@ import org.apache.druid.msq.indexing.MSQTuningConfig; import org.apache.druid.msq.indexing.destination.DataSourceMSQDestination; import org.apache.druid.msq.indexing.destination.DurableStorageMSQDestination; +import org.apache.druid.msq.indexing.destination.ExportMSQDestination; import org.apache.druid.msq.indexing.destination.MSQDestination; import org.apache.druid.msq.indexing.destination.MSQSelectDestination; import org.apache.druid.msq.indexing.destination.TaskReportMSQDestination; @@ -51,6 +52,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.server.QueryResponse; +import org.apache.druid.sql.calcite.parser.DruidSqlIngest; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; import org.apache.druid.sql.calcite.parser.DruidSqlReplace; import org.apache.druid.sql.calcite.planner.ColumnMapping; @@ -62,6 +64,10 @@ import org.apache.druid.sql.calcite.run.QueryMaker; import org.apache.druid.sql.calcite.run.SqlResults; import org.apache.druid.sql.calcite.table.RowSignatures; +import org.apache.druid.sql.destination.ExportDestination; +import org.apache.druid.sql.destination.IngestDestination; +import org.apache.druid.sql.destination.TableDestination; +import org.apache.druid.sql.http.ResultFormat; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -80,7 +86,7 @@ public class MSQTaskQueryMaker implements QueryMaker private static final Granularity DEFAULT_SEGMENT_GRANULARITY = Granularities.ALL; - private final String targetDataSource; + private final IngestDestination targetDataSource; private final OverlordClient overlordClient; private final PlannerContext plannerContext; private final ObjectMapper jsonMapper; @@ -88,7 +94,7 @@ public class MSQTaskQueryMaker implements QueryMaker MSQTaskQueryMaker( - @Nullable final String targetDataSource, + @Nullable final IngestDestination targetDataSource, final OverlordClient overlordClient, final PlannerContext plannerContext, final ObjectMapper jsonMapper, @@ -203,7 +209,15 @@ public QueryResponse runQuery(final DruidQuery druidQuery) final MSQDestination destination; - if (targetDataSource != null) { + if (targetDataSource instanceof ExportDestination) { + ExportDestination exportDestination = ((ExportDestination) targetDataSource); + ResultFormat format = ResultFormat.fromString(sqlQueryContext.getString(DruidSqlIngest.SQL_EXPORT_FILE_FORMAT)); + + destination = new ExportMSQDestination( + exportDestination.getStorageConnectorProvider(), + format + ); + } else if (targetDataSource instanceof TableDestination) { Granularity segmentGranularityObject; try { segmentGranularityObject = jsonMapper.readValue((String) segmentGranularity, Granularity.class); @@ -227,7 +241,7 @@ public QueryResponse runQuery(final DruidQuery druidQuery) ); final DataSourceMSQDestination dataSourceMSQDestination = new DataSourceMSQDestination( - targetDataSource, + targetDataSource.getType(), segmentGranularityObject, segmentSortOrder, replaceTimeChunks diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java index cb331760ca34..6f4f109ffa4b 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/sql/MSQTaskSqlEngine.java @@ -40,6 +40,7 @@ import org.apache.druid.msq.util.MultiStageQueryContext; import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.sql.calcite.parser.DruidSqlIngest; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.PlannerContext; @@ -48,6 +49,7 @@ import org.apache.druid.sql.calcite.run.QueryMaker; import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.run.SqlEngines; +import org.apache.druid.sql.destination.IngestDestination; import java.util.HashSet; import java.util.List; @@ -60,6 +62,7 @@ public class MSQTaskSqlEngine implements SqlEngine ImmutableSet.builder() .addAll(NativeSqlEngine.SYSTEM_CONTEXT_PARAMETERS) .add(QueryKitUtils.CTX_TIME_COLUMN_NAME) + .add(DruidSqlIngest.SQL_EXPORT_FILE_FORMAT) .add(MultiStageQueryContext.CTX_IS_REINDEX) .build(); @@ -121,6 +124,7 @@ public boolean featureAvailable(EngineFeature feature, PlannerContext plannerCon case CAN_INSERT: case CAN_REPLACE: case READ_EXTERNAL_DATA: + case WRITE_EXTERNAL_DATA: case SCAN_ORDER_BY_NON_TIME: case SCAN_NEEDS_SIGNATURE: return true; @@ -153,7 +157,7 @@ public OverlordClient overlordClient() @Override public QueryMaker buildQueryMakerForInsert( - final String targetDataSource, + final IngestDestination destination, final RelRoot relRoot, final PlannerContext plannerContext ) @@ -161,7 +165,7 @@ public QueryMaker buildQueryMakerForInsert( validateInsert(relRoot.rel, relRoot.fields, plannerContext); return new MSQTaskQueryMaker( - targetDataSource, + destination, overlordClient, plannerContext, jsonMapper, diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQExportTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQExportTest.java new file mode 100644 index 000000000000..e6c3b5e2931a --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQExportTest.java @@ -0,0 +1,130 @@ +/* + * 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.msq.exec; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.msq.test.MSQTestBase; +import org.apache.druid.msq.util.MultiStageQueryContext; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.sql.calcite.export.TestExportStorageConnector; +import org.apache.druid.sql.http.ResultFormat; +import org.junit.Assert; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class MSQExportTest extends MSQTestBase +{ + @Test + public void testExport() throws IOException + { + TestExportStorageConnector storageConnector = (TestExportStorageConnector) exportStorageConnectorProvider.get(); + + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .add("cnt", ColumnType.LONG).build(); + + final String sql = StringUtils.format("insert into extern(%s()) as csv select cnt, dim1 from foo", TestExportStorageConnector.TYPE_NAME); + + testIngestQuery().setSql(sql) + .setExpectedDataSource("foo1") + .setQueryContext(DEFAULT_MSQ_CONTEXT) + .setExpectedRowSignature(rowSignature) + .setExpectedSegment(ImmutableSet.of()) + .setExpectedResultRows(ImmutableList.of()) + .verifyResults(); + + List objects = expectedFooFileContents(); + + Assert.assertEquals( + convertResultsToString(objects), + new String(storageConnector.getByteArrayOutputStream().toByteArray(), Charset.defaultCharset()) + ); + } + + @Test + public void testNumberOfRowsPerFile() throws IOException + { + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("dim1", ColumnType.STRING) + .add("cnt", ColumnType.LONG).build(); + + File exportDir = temporaryFolder.newFolder("export/"); + + Map queryContext = new HashMap<>(DEFAULT_MSQ_CONTEXT); + queryContext.put(MultiStageQueryContext.CTX_ROWS_PER_PAGE, 1); + + final String sql = StringUtils.format("insert into extern(local(exportPath=>'%s')) as csv select cnt, dim1 from foo", exportDir.getAbsolutePath()); + + testIngestQuery().setSql(sql) + .setExpectedDataSource("foo1") + .setQueryContext(queryContext) + .setExpectedRowSignature(rowSignature) + .setExpectedSegment(ImmutableSet.of()) + .setExpectedResultRows(ImmutableList.of()) + .verifyResults(); + + Assert.assertEquals( + expectedFooFileContents().size(), + Objects.requireNonNull(new File(exportDir.getAbsolutePath()).listFiles()).length + ); + } + + private List expectedFooFileContents() + { + return new ArrayList<>(ImmutableList.of( + new Object[]{"1", null}, + new Object[]{"1", 10.1}, + new Object[]{"1", 2}, + new Object[]{"1", 1}, + new Object[]{"1", "def"}, + new Object[]{"1", "abc"} + )); + } + + private String convertResultsToString(List expectedRows) throws IOException + { + ByteArrayOutputStream expectedResult = new ByteArrayOutputStream(); + ResultFormat.Writer formatter = ResultFormat.CSV.createFormatter(expectedResult, objectMapper); + formatter.writeResponseStart(); + for (Object[] row : expectedRows) { + formatter.writeRowStart(); + for (Object object : row) { + formatter.writeRowField("", object); + } + formatter.writeRowEnd(); + } + formatter.writeResponseEnd(); + return new String(expectedResult.toByteArray(), Charset.defaultCharset()); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/destination/ExportMSQDestinationTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/destination/ExportMSQDestinationTest.java new file mode 100644 index 000000000000..697866bc9122 --- /dev/null +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/destination/ExportMSQDestinationTest.java @@ -0,0 +1,54 @@ +/* + * 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.msq.indexing.destination; + +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.sql.http.ResultFormat; +import org.apache.druid.storage.StorageConfig; +import org.apache.druid.storage.StorageConnectorModule; +import org.apache.druid.storage.local.LocalFileExportStorageProvider; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +public class ExportMSQDestinationTest +{ + @Test + public void testSerde() throws IOException + { + ExportMSQDestination exportDestination = new ExportMSQDestination( + new LocalFileExportStorageProvider("/path"), + ResultFormat.CSV + ); + ObjectMapper objectMapper = new DefaultObjectMapper(); + new StorageConnectorModule().getJacksonModules().forEach(objectMapper::registerModule); + String string = objectMapper.writeValueAsString(exportDestination); + objectMapper.setInjectableValues( + new InjectableValues.Std() + .addValue(StorageConfig.class, new StorageConfig("/")) + ); + + ExportMSQDestination newDest = objectMapper.readValue(string, ExportMSQDestination.class); + Assert.assertEquals(exportDestination, newDest); + } +} diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index 0146fcf9bd8b..f8fc01b9369f 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -158,6 +158,8 @@ import org.apache.druid.sql.SqlStatementFactory; import org.apache.druid.sql.SqlToolbox; import org.apache.druid.sql.calcite.BaseCalciteQueryTest; +import org.apache.druid.sql.calcite.export.TestExportStorageConnector; +import org.apache.druid.sql.calcite.export.TestExportStorageConnectorProvider; import org.apache.druid.sql.calcite.external.ExternalDataSource; import org.apache.druid.sql.calcite.external.ExternalOperatorConversion; import org.apache.druid.sql.calcite.planner.CalciteRulesManager; @@ -174,7 +176,9 @@ import org.apache.druid.sql.calcite.util.SqlTestFramework; import org.apache.druid.sql.calcite.view.InProcessViewManager; import org.apache.druid.sql.guice.SqlBindings; +import org.apache.druid.storage.StorageConfig; import org.apache.druid.storage.StorageConnector; +import org.apache.druid.storage.StorageConnectorModule; import org.apache.druid.storage.StorageConnectorProvider; import org.apache.druid.storage.local.LocalFileStorageConnector; import org.apache.druid.timeline.DataSegment; @@ -306,7 +310,7 @@ public class MSQTestBase extends BaseCalciteQueryTest protected SqlStatementFactory sqlStatementFactory; protected AuthorizerMapper authorizerMapper; private IndexIO indexIO; - + protected TestExportStorageConnectorProvider exportStorageConnectorProvider = new TestExportStorageConnectorProvider(); // Contains the metadata of loaded segments protected List loadedSegmentsMetadata = new ArrayList<>(); // Mocks the return of data from data servers @@ -470,6 +474,7 @@ public String getFormatString() ); binder.bind(Key.get(StorageConnector.class, MultiStageQuery.class)) .toProvider(() -> localFileStorageConnector); + binder.bind(StorageConfig.class).toInstance(new StorageConfig("/")); } catch (IOException e) { throw new ISE(e, "Unable to create setup storage connector"); @@ -508,6 +513,13 @@ public String getFormatString() .build(); objectMapper = setupObjectMapper(injector); + objectMapper.registerModule( + new SimpleModule(StorageConnector.class.getSimpleName()) + .registerSubtypes( + new NamedType(TestExportStorageConnectorProvider.class, TestExportStorageConnector.TYPE_NAME) + ) + ); + objectMapper.registerModules(new StorageConnectorModule().getJacksonModules()); objectMapper.registerModules(sqlModule.getJacksonModules()); doReturn(mock(Request.class)).when(brokerClient).makeRequest(any(), anyString()); @@ -895,7 +907,7 @@ public Builder setExpectedRowSignature(RowSignature expectedRowSignature) public Builder setExpectedSegment(Set expectedSegments) { - Preconditions.checkArgument(!expectedSegments.isEmpty(), "Segments cannot be empty"); + Preconditions.checkArgument(expectedSegments != null, "Segments cannot be null"); this.expectedSegments = expectedSegments; return asBuilder(); } diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportConfig.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportConfig.java new file mode 100644 index 000000000000..d5477c2998e0 --- /dev/null +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportConfig.java @@ -0,0 +1,72 @@ +/* + * 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.storage.s3.output; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.java.util.common.HumanReadableBytes; + +import javax.annotation.Nullable; +import java.util.List; + +public class S3ExportConfig +{ + @JsonProperty("tempLocalDir") + private final String tempLocalDir; + @JsonProperty("chunkSize") + private final HumanReadableBytes chunkSize; + @JsonProperty("maxRetry") + private final Integer maxRetry; + @JsonProperty("allowedExportPaths") + private final List allowedExportPaths; + + @JsonCreator + public S3ExportConfig( + @JsonProperty("tempLocalDir") final String tempLocalDir, + @JsonProperty("chunkSize") @Nullable final HumanReadableBytes chunkSize, + @JsonProperty("maxRetry") @Nullable final Integer maxRetry, + @JsonProperty("allowedExportPaths") final List allowedExportPaths) + { + this.tempLocalDir = tempLocalDir; + this.chunkSize = chunkSize; + this.maxRetry = maxRetry; + this.allowedExportPaths = allowedExportPaths; + } + + public String getTempLocalDir() + { + return tempLocalDir; + } + + public HumanReadableBytes getChunkSize() + { + return chunkSize; + } + + public Integer getMaxRetry() + { + return maxRetry; + } + + public List getAllowedExportPaths() + { + return allowedExportPaths; + } +} diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java new file mode 100644 index 000000000000..7577f56f76f2 --- /dev/null +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3ExportStorageProvider.java @@ -0,0 +1,146 @@ +/* + * 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.storage.s3.output; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.annotations.VisibleForTesting; +import org.apache.druid.data.input.impl.CloudObjectLocation; +import org.apache.druid.data.input.s3.S3InputSource; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.storage.ExportStorageProvider; +import org.apache.druid.storage.StorageConnector; +import org.apache.druid.storage.s3.S3StorageDruidModule; +import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3; + +import javax.validation.constraints.NotNull; +import java.io.File; +import java.net.URI; +import java.util.List; + +@JsonTypeName(S3ExportStorageProvider.TYPE_NAME) +public class S3ExportStorageProvider implements ExportStorageProvider +{ + public static final String TYPE_NAME = S3InputSource.TYPE_KEY; + @JsonProperty + private final String bucket; + @JsonProperty + private final String prefix; + + @JacksonInject + S3ExportConfig s3ExportConfig; + @JacksonInject + ServerSideEncryptingAmazonS3 s3; + + @JsonCreator + public S3ExportStorageProvider( + @JsonProperty(value = "bucket", required = true) String bucket, + @JsonProperty(value = "prefix", required = true) String prefix + ) + { + this.bucket = bucket; + this.prefix = prefix; + } + + @Override + public StorageConnector get() + { + final String tempDir = s3ExportConfig.getTempLocalDir(); + if (tempDir == null) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.NOT_FOUND) + .build("The runtime property `druid.export.storage.s3.tempLocalDir` must be configured for S3 export."); + } + final List allowedExportPaths = s3ExportConfig.getAllowedExportPaths(); + if (allowedExportPaths == null) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.NOT_FOUND) + .build( + "The runtime property `druid.export.storage.s3.allowedExportPaths` must be configured for S3 export."); + } + validateS3Prefix(allowedExportPaths, bucket, prefix); + final S3OutputConfig s3OutputConfig = new S3OutputConfig( + bucket, + prefix, + new File(tempDir), + s3ExportConfig.getChunkSize(), + s3ExportConfig.getMaxRetry() + ); + return new S3StorageConnector(s3OutputConfig, s3); + } + + @VisibleForTesting + static void validateS3Prefix(@NotNull final List allowedExportPaths, final String bucket, final String prefix) + { + final URI providedUri = new CloudObjectLocation(bucket, prefix).toUri(S3StorageDruidModule.SCHEME); + for (final String path : allowedExportPaths) { + final URI allowedUri = URI.create(path); + if (validateUri(allowedUri, providedUri)) { + return; + } + } + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("None of the allowed prefixes matched the input path [%s]. " + + "Please reach out to the cluster admin for the whitelisted paths for export. " + + "The paths are controlled via the property `druid.export.storage.s3.allowedExportPaths`.", + providedUri); + } + + private static boolean validateUri(final URI allowedUri, final URI providedUri) + { + if (!allowedUri.getHost().equals(providedUri.getHost())) { + return false; + } + final String allowedPath = StringUtils.maybeAppendTrailingSlash(allowedUri.getPath()); + final String providedPath = StringUtils.maybeAppendTrailingSlash(providedUri.getPath()); + return providedPath.startsWith(allowedPath); + } + + @JsonProperty("bucket") + public String getBucket() + { + return bucket; + } + + @JsonProperty("prefix") + public String getPrefix() + { + return prefix; + } + + @Override + @JsonIgnore + public String getResourceType() + { + return TYPE_NAME; + } + + @Override + @JsonIgnore + public String getBasePath() + { + return new CloudObjectLocation(bucket, prefix).toUri(S3StorageDruidModule.SCHEME).toString(); + } +} diff --git a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorModule.java b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorModule.java index c51f63adfc5b..a57a93a525fb 100644 --- a/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorModule.java +++ b/extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/output/S3StorageConnectorModule.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.inject.Binder; +import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.initialization.DruidModule; import java.util.Collections; @@ -33,12 +34,15 @@ public class S3StorageConnectorModule implements DruidModule public List getJacksonModules() { return Collections.singletonList( - new SimpleModule(this.getClass().getSimpleName()).registerSubtypes(S3StorageConnectorProvider.class) + new SimpleModule(this.getClass().getSimpleName()) + .registerSubtypes(S3StorageConnectorProvider.class) + .registerSubtypes(S3ExportStorageProvider.class) ); } @Override public void configure(Binder binder) { + JsonConfigProvider.bind(binder, "druid.export.storage.s3", S3ExportConfig.class); } } diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3ExportStorageProviderTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3ExportStorageProviderTest.java new file mode 100644 index 000000000000..362f8583fd13 --- /dev/null +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/output/S3ExportStorageProviderTest.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.storage.s3.output; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.error.DruidException; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class S3ExportStorageProviderTest +{ + private final List validPrefixes = ImmutableList.of( + "s3://bucket-name/validPath1", + "s3://bucket-name/validPath2" + ); + + @Test + public void testValidatePaths() + { + S3ExportStorageProvider.validateS3Prefix(validPrefixes, "bucket-name", "validPath1/"); + S3ExportStorageProvider.validateS3Prefix(validPrefixes, "bucket-name", "validPath1"); + S3ExportStorageProvider.validateS3Prefix(validPrefixes, "bucket-name", "validPath1/validSubPath/"); + + S3ExportStorageProvider.validateS3Prefix(ImmutableList.of("s3://bucket-name"), "bucket-name", ""); + S3ExportStorageProvider.validateS3Prefix(ImmutableList.of("s3://bucket-name"), "bucket-name", "validPath"); + S3ExportStorageProvider.validateS3Prefix(validPrefixes, "bucket-name", "validPath1/../validPath2/"); + + Assert.assertThrows( + DruidException.class, + () -> S3ExportStorageProvider.validateS3Prefix(validPrefixes, "incorrect-bucket", "validPath1/") + ); + Assert.assertThrows( + DruidException.class, + () -> S3ExportStorageProvider.validateS3Prefix(validPrefixes, "bucket-name", "invalidPath1") + ); + Assert.assertThrows( + DruidException.class, + () -> S3ExportStorageProvider.validateS3Prefix(validPrefixes, "bucket-name", "validPath123") + ); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index 81a55aae1b4c..a5f34f9bbbfe 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -41,10 +41,13 @@ import org.apache.druid.java.util.common.UOE; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; +import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.ResourceAction; +import org.apache.druid.server.security.ResourceType; import javax.annotation.Nonnull; import java.util.Map; +import java.util.Optional; import java.util.Set; /** @@ -297,6 +300,11 @@ default Map addToContextIfAbsent(String key, Object val) Map getContext(); + default Optional getDestinationResource() + { + return Optional.of(new Resource(getDataSource(), ResourceType.DATASOURCE)); + } + default ContextValueType getContextValue(String key) { return (ContextValueType) getContext().get(key); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index b589dc5ffd81..b529aa45854c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -1115,9 +1115,9 @@ private Response asLeaderWith(Optional x, Function f) @VisibleForTesting Set getNeededResourceActionsForTask(Task task) throws UOE { - final String dataSource = task.getDataSource(); final Set resourceActions = new HashSet<>(); - resourceActions.add(new ResourceAction(new Resource(dataSource, ResourceType.DATASOURCE), Action.WRITE)); + java.util.Optional destinationResource = task.getDestinationResource(); + destinationResource.ifPresent(resource -> resourceActions.add(new ResourceAction(resource, Action.WRITE))); if (authConfig.isEnableInputSourceSecurity()) { resourceActions.addAll(task.getInputSourceResources()); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java index 6897e69c26b9..8c1b6765431c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordResourceTest.java @@ -1699,6 +1699,7 @@ public void testResourceActionsForTaskWithInputTypeAndInputSecurityEnabled() EasyMock.expect(authConfig.isEnableInputSourceSecurity()).andReturn(true); EasyMock.expect(task.getDataSource()).andReturn(dataSource); + EasyMock.expect(task.getDestinationResource()).andReturn(java.util.Optional.of(new Resource(dataSource, ResourceType.DATASOURCE))); EasyMock.expect(task.getInputSourceResources()) .andReturn(ImmutableSet.of(new ResourceAction( new Resource(inputSourceType, ResourceType.EXTERNAL), @@ -1735,6 +1736,7 @@ public void testResourceActionsForTaskWithFirehoseAndInputSecurityEnabled() EasyMock.expect(authConfig.isEnableInputSourceSecurity()).andReturn(true); EasyMock.expect(task.getId()).andReturn("taskId"); EasyMock.expect(task.getDataSource()).andReturn(dataSource); + EasyMock.expect(task.getDestinationResource()).andReturn(java.util.Optional.of(new Resource(dataSource, ResourceType.DATASOURCE))); EasyMock.expect(task.getInputSourceResources()).andThrow(expectedException); EasyMock.replay( @@ -1767,6 +1769,7 @@ public void testResourceActionsForTaskWithInputTypeAndInputSecurityDisabled() EasyMock.expect(authConfig.isEnableInputSourceSecurity()).andReturn(false); EasyMock.expect(task.getDataSource()).andReturn(dataSource); + EasyMock.expect(task.getDestinationResource()).andReturn(java.util.Optional.of(new Resource(dataSource, ResourceType.DATASOURCE))); EasyMock.expect(task.getInputSourceResources()) .andReturn(ImmutableSet.of(new ResourceAction( new Resource(inputSourceType, ResourceType.EXTERNAL), diff --git a/integration-tests-ex/cases/cluster/MultiStageQuery/docker-compose.py b/integration-tests-ex/cases/cluster/MultiStageQuery/docker-compose.py index bb88aa6de2d8..159a0638dd1e 100644 --- a/integration-tests-ex/cases/cluster/MultiStageQuery/docker-compose.py +++ b/integration-tests-ex/cases/cluster/MultiStageQuery/docker-compose.py @@ -22,5 +22,6 @@ def define_indexer(self): self.add_property(service, 'druid.msq.intermediate.storage.enable', 'true') self.add_property(service, 'druid.msq.intermediate.storage.type', 'local') self.add_property(service, 'druid.msq.intermediate.storage.basePath', '/shared/durablestorage/') + self.add_property(service, 'druid.export.storage.baseDir', '/') generate(__file__, Template()) diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java index ac3f8415488c..6f1e336c935f 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/config/Initializer.java @@ -71,6 +71,7 @@ import org.apache.druid.metadata.storage.mysql.MySQLConnectorDriverConfig; import org.apache.druid.metadata.storage.mysql.MySQLConnectorSslConfig; import org.apache.druid.metadata.storage.mysql.MySQLMetadataStorageModule; +import org.apache.druid.msq.guice.MSQExternalDataSourceModule; import org.apache.druid.server.DruidNode; import org.apache.druid.testing.IntegrationTestingConfig; import org.apache.druid.testing.IntegrationTestingConfigProvider; @@ -499,6 +500,7 @@ private static Injector makeInjector( new LegacyBrokerParallelMergeConfigModule(), // Dependencies from other modules new StorageNodeModule(), + new MSQExternalDataSourceModule(), // Test-specific items, including bits copy/pasted // from modules that don't play well in a client setting. diff --git a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java index b70329bd5683..a8677a53e734 100644 --- a/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java +++ b/integration-tests-ex/cases/src/test/java/org/apache/druid/testsEx/msq/ITMultiStageQuery.java @@ -19,9 +19,17 @@ package org.apache.druid.testsEx.msq; +import com.google.api.client.util.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.inject.Inject; +import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.msq.indexing.report.MSQResultsReport; +import org.apache.druid.msq.indexing.report.MSQTaskReport; +import org.apache.druid.msq.indexing.report.MSQTaskReportPayload; import org.apache.druid.msq.sql.SqlTaskStatus; +import org.apache.druid.storage.local.LocalFileExportStorageProvider; import org.apache.druid.testing.clients.CoordinatorResourceTestClient; import org.apache.druid.testing.utils.DataLoaderHelper; import org.apache.druid.testing.utils.MsqTestQueryHelper; @@ -33,6 +41,11 @@ import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + @RunWith(DruidTestRunner.class) @Category(MultiStageQuery.class) public class ITMultiStageQuery @@ -176,4 +189,83 @@ public void testMsqIngestionAndQueryingWithLocalFn() throws Exception msqHelper.testQueriesFromFile(QUERY_FILE, datasource); } + + @Test + public void testExport() throws Exception + { + String exportQuery = + StringUtils.format( + "INSERT INTO extern(%s(exportPath => '%s'))\n" + + "AS CSV\n" + + "SELECT page, added, delta\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"local\",\"files\":[\"/resources/data/batch_index/json/wikipedia_index_data1.json\"]}',\n" + + " '{\"type\":\"json\"}',\n" + + " '[{\"type\":\"string\",\"name\":\"timestamp\"},{\"type\":\"string\",\"name\":\"isRobot\"},{\"type\":\"string\",\"name\":\"diffUrl\"},{\"type\":\"long\",\"name\":\"added\"},{\"type\":\"string\",\"name\":\"countryIsoCode\"},{\"type\":\"string\",\"name\":\"regionName\"},{\"type\":\"string\",\"name\":\"channel\"},{\"type\":\"string\",\"name\":\"flags\"},{\"type\":\"long\",\"name\":\"delta\"},{\"type\":\"string\",\"name\":\"isUnpatrolled\"},{\"type\":\"string\",\"name\":\"isNew\"},{\"type\":\"double\",\"name\":\"deltaBucket\"},{\"type\":\"string\",\"name\":\"isMinor\"},{\"type\":\"string\",\"name\":\"isAnonymous\"},{\"type\":\"long\",\"name\":\"deleted\"},{\"type\":\"string\",\"name\":\"cityName\"},{\"type\":\"long\",\"name\":\"metroCode\"},{\"type\":\"string\",\"name\":\"namespace\"},{\"type\":\"string\",\"name\":\"comment\"},{\"type\":\"string\",\"name\":\"page\"},{\"type\":\"long\",\"name\":\"commentLength\"},{\"type\":\"string\",\"name\":\"countryName\"},{\"type\":\"string\",\"name\":\"user\"},{\"type\":\"string\",\"name\":\"regionIsoCode\"}]'\n" + + " )\n" + + ")\n", + LocalFileExportStorageProvider.TYPE_NAME, "/shared/export/" + ); + + SqlTaskStatus exportTask = msqHelper.submitMsqTask(exportQuery); + + msqHelper.pollTaskIdForSuccess(exportTask.getTaskId()); + + if (exportTask.getState().isFailure()) { + Assert.fail(StringUtils.format( + "Unable to start the task successfully.\nPossible exception: %s", + exportTask.getError() + )); + } + + String resultQuery = StringUtils.format( + "SELECT page, delta, added\n" + + " FROM TABLE(\n" + + " EXTERN(\n" + + " '{\"type\":\"local\",\"baseDir\":\"/shared/export/\",\"filter\":\"*.csv\"}',\n" + + " '{\"type\":\"csv\",\"findColumnsFromHeader\":true}'\n" + + " )\n" + + " ) EXTEND (\"added\" BIGINT, \"delta\" BIGINT, \"page\" VARCHAR)\n" + + " WHERE delta != 0\n" + + " ORDER BY page"); + + SqlTaskStatus resultTaskStatus = msqHelper.submitMsqTask(resultQuery); + + msqHelper.pollTaskIdForSuccess(resultTaskStatus.getTaskId()); + + Map statusReport = msqHelper.fetchStatusReports(resultTaskStatus.getTaskId()); + MSQTaskReport taskReport = statusReport.get(MSQTaskReport.REPORT_KEY); + if (taskReport == null) { + throw new ISE("Unable to fetch the status report for the task [%]", resultTaskStatus.getTaskId()); + } + MSQTaskReportPayload taskReportPayload = Preconditions.checkNotNull( + taskReport.getPayload(), + "payload" + ); + MSQResultsReport resultsReport = Preconditions.checkNotNull( + taskReportPayload.getResults(), + "Results report for the task id is empty" + ); + + Yielder yielder = resultsReport.getResultYielder(); + List> actualResults = new ArrayList<>(); + + while (!yielder.isDone()) { + Object[] row = yielder.get(); + actualResults.add(Arrays.asList(row)); + yielder = yielder.next(null); + } + + ImmutableList> expectedResults = ImmutableList.of( + ImmutableList.of("Cherno Alpha", 111, 123), + ImmutableList.of("Gypsy Danger", -143, 57), + ImmutableList.of("Striker Eureka", 330, 459) + ); + + Assert.assertEquals( + expectedResults, + actualResults + ); + } } diff --git a/processing/src/main/java/org/apache/druid/java/util/common/StringUtils.java b/processing/src/main/java/org/apache/druid/java/util/common/StringUtils.java index e86d51b8d965..f8808b1cc807 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/StringUtils.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/StringUtils.java @@ -460,6 +460,11 @@ public static String maybeRemoveTrailingSlash(String s) return s != null && s.endsWith("/") ? s.substring(0, s.length() - 1) : s; } + public static String maybeAppendTrailingSlash(String s) + { + return s != null && !s.endsWith("/") ? s + "/" : s; + } + /** * Removes all occurrences of the given char from the given string. This method is an optimal version of * {@link String#replace(CharSequence, CharSequence) s.replace("c", "")}. diff --git a/processing/src/main/java/org/apache/druid/storage/ExportStorageProvider.java b/processing/src/main/java/org/apache/druid/storage/ExportStorageProvider.java new file mode 100644 index 000000000000..890ac577b1a5 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/storage/ExportStorageProvider.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.storage; + +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.inject.Provider; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +public interface ExportStorageProvider extends Provider +{ + String getResourceType(); + + /** + * Return a URI representation of the base path. This is used to be used for logging and error messages. + */ + String getBasePath(); +} diff --git a/processing/src/main/java/org/apache/druid/storage/StorageConfig.java b/processing/src/main/java/org/apache/druid/storage/StorageConfig.java new file mode 100644 index 000000000000..0f49afd5f757 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/storage/StorageConfig.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.storage; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.annotation.Nullable; + +public class StorageConfig +{ + @JsonProperty("baseDir") + @Nullable + private final String baseDir; + + @JsonCreator + public StorageConfig(@JsonProperty("baseDir") @Nullable String baseDir) + { + this.baseDir = baseDir; + } + + @Nullable + public String getBaseDir() + { + return baseDir; + } +} diff --git a/processing/src/main/java/org/apache/druid/storage/StorageConnectorModule.java b/processing/src/main/java/org/apache/druid/storage/StorageConnectorModule.java index a0bf3a91f0cc..a919742dc226 100644 --- a/processing/src/main/java/org/apache/druid/storage/StorageConnectorModule.java +++ b/processing/src/main/java/org/apache/druid/storage/StorageConnectorModule.java @@ -23,7 +23,9 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.collect.ImmutableList; import com.google.inject.Binder; +import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.initialization.DruidModule; +import org.apache.druid.storage.local.LocalFileExportStorageProvider; import org.apache.druid.storage.local.LocalFileStorageConnectorProvider; import java.util.List; @@ -36,12 +38,13 @@ public List getJacksonModules() return ImmutableList.of( new SimpleModule(StorageConnector.class.getSimpleName()) .registerSubtypes(LocalFileStorageConnectorProvider.class) + .registerSubtypes(LocalFileExportStorageProvider.class) ); } @Override public void configure(Binder binder) { - + JsonConfigProvider.bind(binder, "druid.export.storage", StorageConfig.class); } } diff --git a/processing/src/main/java/org/apache/druid/storage/local/LocalFileExportStorageProvider.java b/processing/src/main/java/org/apache/druid/storage/local/LocalFileExportStorageProvider.java new file mode 100644 index 000000000000..f0d4c87b41f3 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/storage/local/LocalFileExportStorageProvider.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.storage.local; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.data.input.impl.LocalInputSource; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.storage.ExportStorageProvider; +import org.apache.druid.storage.StorageConfig; +import org.apache.druid.storage.StorageConnector; + +import java.io.File; +import java.io.IOException; +import java.util.Objects; + +@JsonTypeName(LocalFileExportStorageProvider.TYPE_NAME) +public class LocalFileExportStorageProvider implements ExportStorageProvider +{ + public static final String TYPE_NAME = LocalInputSource.TYPE_KEY; + + @JacksonInject + StorageConfig storageConfig; + + @JsonProperty + private final String exportPath; + + @JsonCreator + public LocalFileExportStorageProvider(@JsonProperty(value = "exportPath", required = true) String exportPath) + { + this.exportPath = exportPath; + } + + @Override + public StorageConnector get() + { + final File exportDestination = validateAndGetPath(storageConfig.getBaseDir(), exportPath); + try { + return new LocalFileStorageConnector(exportDestination); + } + catch (IOException e) { + throw new IAE( + e, + "Unable to create storage connector [%s] for base path [%s]", + LocalFileStorageConnector.class.getSimpleName(), + exportDestination.toPath() + ); + } + } + + @Override + @JsonIgnore + public String getResourceType() + { + return TYPE_NAME; + } + + @Override + @JsonIgnore + public String getBasePath() + { + return exportPath; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LocalFileExportStorageProvider that = (LocalFileExportStorageProvider) o; + return Objects.equals(exportPath, that.exportPath); + } + + @Override + public int hashCode() + { + return Objects.hash(exportPath); + } + + @Override + public String toString() + { + return "LocalFileExportStorageProvider{" + + "exportPath=" + exportPath + + '}'; + } + + public static File validateAndGetPath(String basePath, String customPath) + { + if (basePath == null) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.NOT_FOUND) + .build( + "The runtime property `druid.export.storage.baseDir` must be configured for local export."); + } + final File baseDir = new File(basePath); + if (!baseDir.isAbsolute()) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build( + "The runtime property `druid.export.storage.baseDir` must be an absolute path."); + } + final File exportFile = new File(customPath); + if (!exportFile.toPath().normalize().startsWith(baseDir.toPath())) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("The provided destination [%s] must be within the path configured by runtime property `druid.export.storage.baseDir` " + + "Please reach out to the cluster admin for the allowed path. ", customPath); + } + return exportFile; + } +} diff --git a/processing/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnector.java b/processing/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnector.java index 3d96f8d43b1d..225f3eb85373 100644 --- a/processing/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnector.java +++ b/processing/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnector.java @@ -101,7 +101,7 @@ public OutputStream write(String path) throws IOException { File toWrite = fileWithBasePath(path); FileUtils.mkdirp(toWrite.getParentFile()); - return Files.newOutputStream(toWrite.toPath()); + return Files.newOutputStream(toWrite.toPath(), StandardOpenOption.CREATE, StandardOpenOption.APPEND); } /** diff --git a/processing/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnectorProvider.java b/processing/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnectorProvider.java index 07966378c455..82d1623f8404 100644 --- a/processing/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnectorProvider.java +++ b/processing/src/main/java/org/apache/druid/storage/local/LocalFileStorageConnectorProvider.java @@ -28,12 +28,15 @@ import java.io.File; import java.io.IOException; +import java.util.Objects; -@JsonTypeName("local") +@JsonTypeName(LocalFileStorageConnectorProvider.TYPE_NAME) public class LocalFileStorageConnectorProvider implements StorageConnectorProvider { + public static final String TYPE_NAME = "local"; + @JsonProperty - File basePath; + final File basePath; @JsonCreator public LocalFileStorageConnectorProvider(@JsonProperty(value = "basePath", required = true) File basePath) @@ -56,4 +59,23 @@ public StorageConnector get() ); } } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LocalFileStorageConnectorProvider that = (LocalFileStorageConnectorProvider) o; + return Objects.equals(basePath, that.basePath); + } + + @Override + public int hashCode() + { + return Objects.hash(basePath); + } } diff --git a/processing/src/test/java/org/apache/druid/storage/local/LocalFileExportStorageProviderTest.java b/processing/src/test/java/org/apache/druid/storage/local/LocalFileExportStorageProviderTest.java new file mode 100644 index 000000000000..4daef2f9cd9e --- /dev/null +++ b/processing/src/test/java/org/apache/druid/storage/local/LocalFileExportStorageProviderTest.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.storage.local; + +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.error.DruidException; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.storage.ExportStorageProvider; +import org.apache.druid.storage.StorageConfig; +import org.apache.druid.storage.StorageConnectorModule; +import org.junit.Assert; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; + +public class LocalFileExportStorageProviderTest +{ + @Test + public void testSerde() throws IOException + { + ExportStorageProvider exportDestination = new LocalFileExportStorageProvider("/basepath/export"); + + ObjectMapper objectMapper = new DefaultObjectMapper(); + objectMapper.registerModules(new StorageConnectorModule().getJacksonModules()); + objectMapper.setInjectableValues( + new InjectableValues.Std() + .addValue(StorageConfig.class, new StorageConfig("/")) + ); + byte[] bytes = objectMapper.writeValueAsBytes(exportDestination); + + ExportStorageProvider deserialized = objectMapper.readValue(bytes, LocalFileExportStorageProvider.class); + Assert.assertEquals(exportDestination, deserialized); + } + + @Test + public void testEqualsAndHashCode() + { + EqualsVerifier.forClass(LocalFileExportStorageProvider.class) + .withNonnullFields("exportPath") + .withIgnoredFields("storageConfig") + .usingGetClass() + .verify(); + } + + @Test + public void testEmptyPath() + { + Assert.assertThrows( + DruidException.class, + () -> LocalFileExportStorageProvider.validateAndGetPath(null, "path") + ); + } + + @Test + public void testValidate() + { + File file = LocalFileExportStorageProvider.validateAndGetPath("/base", "/base/path"); + Assert.assertEquals("/base/path", file.toPath().toString()); + } + + @Test + public void testWithNonSubdir() + { + Assert.assertThrows( + DruidException.class, + () -> LocalFileExportStorageProvider.validateAndGetPath("/base", "/base/../path") + ); + Assert.assertThrows( + DruidException.class, + () -> LocalFileExportStorageProvider.validateAndGetPath("/base", "/base1") + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/storage/local/LocalFileStorageConnectorProviderTest.java b/processing/src/test/java/org/apache/druid/storage/local/LocalFileStorageConnectorProviderTest.java new file mode 100644 index 000000000000..eb4c4242f96b --- /dev/null +++ b/processing/src/test/java/org/apache/druid/storage/local/LocalFileStorageConnectorProviderTest.java @@ -0,0 +1,35 @@ +/* + * 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.storage.local; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; + +public class LocalFileStorageConnectorProviderTest +{ + @Test + public void testEqualsAndHashCode() + { + EqualsVerifier.forClass(LocalFileStorageConnectorProvider.class) + .withNonnullFields("basePath") + .usingGetClass() + .verify(); + } +} diff --git a/sql/src/main/codegen/config.fmpp b/sql/src/main/codegen/config.fmpp index 87195131b574..19ff322406eb 100644 --- a/sql/src/main/codegen/config.fmpp +++ b/sql/src/main/codegen/config.fmpp @@ -51,12 +51,16 @@ data: { "java.util.List" "org.apache.calcite.sql.SqlNode" "org.apache.calcite.sql.SqlInsert" + "org.apache.calcite.sql.SqlNodeList" + "org.apache.calcite.sql.SqlBasicCall" "org.apache.druid.java.util.common.granularity.Granularity" "org.apache.druid.java.util.common.granularity.Granularities" "org.apache.druid.sql.calcite.parser.DruidSqlInsert" "org.apache.druid.sql.calcite.parser.DruidSqlParserUtils" "org.apache.druid.sql.calcite.external.ExtendOperator" "org.apache.druid.sql.calcite.external.ParameterizeOperator" + "org.apache.druid.sql.calcite.parser.ExternalDestinationSqlIdentifier" + "java.util.HashMap" ] # List of new keywords. Example: "DATABASES", "TABLES". If the keyword is not a reserved @@ -65,10 +69,12 @@ data: { "CLUSTERED" "OVERWRITE" "PARTITIONED" + "EXTERN" ] nonReservedKeywordsToAdd: [ "OVERWRITE" + "EXTERN" ] # List of methods for parsing custom SQL statements. diff --git a/sql/src/main/codegen/includes/common.ftl b/sql/src/main/codegen/includes/common.ftl index 59915bf09ed4..757a869003ed 100644 --- a/sql/src/main/codegen/includes/common.ftl +++ b/sql/src/main/codegen/includes/common.ftl @@ -106,3 +106,83 @@ SqlTypeNameSpec DruidType() : return new SqlUserDefinedTypeNameSpec(typeName, span().pos()); } } + +// Parses the supported file formats for export. +String FileFormat() : +{ + SqlNode format; +} +{ + format = SimpleIdentifier() + { + return format.toString(); + } +} + +SqlIdentifier ExternalDestination() : +{ + final Span s; + SqlIdentifier destinationType = null; + String destinationTypeString = null; + Map properties = new HashMap(); +} +{ + ( + destinationType = SimpleIdentifier() + { + destinationTypeString = destinationType.toString(); + } + | + + { + // local is a reserved keyword in calcite. However, local is also a supported input source / destination and + // keeping the name is preferred for consistency in other places, and so that permission checks are applied + // correctly, so this is handled as a special case. + destinationTypeString = "local"; + } + ) + [ [ properties = ExternProperties() ] ] + { + s = span(); + return new ExternalDestinationSqlIdentifier( + destinationTypeString, + s.pos(), + properties + ); + } +} + +Map ExternProperties() : +{ + final Span s; + final Map properties = new HashMap(); + SqlIdentifier identifier; + String value; + SqlNodeList commaList = SqlNodeList.EMPTY; +} +{ + ( + identifier = SimpleIdentifier() value = SimpleStringLiteral() + { + properties.put(identifier.toString(), value); + } + ) + ( + + identifier = SimpleIdentifier() value = SimpleStringLiteral() + { + properties.put(identifier.toString(), value); + } + )* + { + return properties; + } +} + +SqlNode testRule(): +{ + final SqlNode e; +} +{ + e = SimpleIdentifier() { return e; } +} \ No newline at end of file diff --git a/sql/src/main/codegen/includes/insert.ftl b/sql/src/main/codegen/includes/insert.ftl index 00133496d240..1e74cd3e6bf8 100644 --- a/sql/src/main/codegen/includes/insert.ftl +++ b/sql/src/main/codegen/includes/insert.ftl @@ -21,68 +21,68 @@ * Parses an INSERT statement. This function is copied from SqlInsert in core/src/main/codegen/templates/Parser.jj, * with some changes to allow a custom error message if an OVERWRITE clause is present. */ -SqlNode DruidSqlInsert() : +// Using fully qualified name for Pair class, since Calcite also has a same class name being used in the Parser.jj +SqlNode DruidSqlInsertEof() : { - final List keywords = new ArrayList(); - final SqlNodeList keywordList; - final SqlIdentifier tableName; - SqlNode tableRef; - SqlNode source; - final SqlNodeList columnList; - final Span s; - final Pair p; + SqlNode insertNode; + final List keywords = new ArrayList(); + final SqlNodeList keywordList; + final SqlIdentifier destination; + SqlNode tableRef = null; + SqlNode source; + final SqlNodeList columnList; + final Span s; + final Pair p; + org.apache.druid.java.util.common.Pair partitionedBy = new org.apache.druid.java.util.common.Pair(null, null); + SqlNodeList clusteredBy = null; + String exportFileFormat = null; } { - ( - + ( + | - { keywords.add(SqlInsertKeyword.UPSERT.symbol(getPos())); } - ) - { s = span(); } - SqlInsertKeywords(keywords) { - keywordList = new SqlNodeList(keywords, s.addAll(keywords).pos()); - } - tableName = CompoundTableIdentifier() - ( tableRef = TableHints(tableName) | { tableRef = tableName; } ) + { keywords.add(SqlInsertKeyword.UPSERT.symbol(getPos())); } + ) + { s = span(); } + SqlInsertKeywords(keywords) { + keywordList = new SqlNodeList(keywords, s.addAll(keywords).pos()); + } + + ( + LOOKAHEAD(2) + destination = ExternalDestination() + | + destination = CompoundTableIdentifier() + ( tableRef = TableHints(destination) | { tableRef = destination; } ) [ LOOKAHEAD(5) tableRef = ExtendTable(tableRef) ] - ( - LOOKAHEAD(2) - p = ParenthesizedCompoundIdentifierList() { - if (p.right.size() > 0) { - tableRef = extend(tableRef, p.right); - } - if (p.left.size() > 0) { - columnList = p.left; - } else { - columnList = null; - } - } - | { columnList = null; } - ) - ( + ) + ( + LOOKAHEAD(2) + p = ParenthesizedCompoundIdentifierList() { + if (p.right.size() > 0) { + tableRef = extend(tableRef, p.right); + } + if (p.left.size() > 0) { + columnList = p.left; + } else { + columnList = null; + } + } + | { columnList = null; } + ) + [ + exportFileFormat = FileFormat() + ] + ( { - throw org.apache.druid.sql.calcite.parser.DruidSqlParserUtils.problemParsing( - "An OVERWRITE clause is not allowed with INSERT statements. Use REPLACE statements if overwriting existing segments is required or remove the OVERWRITE clause." - ); + throw org.apache.druid.sql.calcite.parser.DruidSqlParserUtils.problemParsing( + "An OVERWRITE clause is not allowed with INSERT statements. Use REPLACE statements if overwriting existing segments is required or remove the OVERWRITE clause." + ); } | - source = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY) { - return new SqlInsert(s.end(source), keywordList, tableRef, source, - columnList); - } - ) -} - -// Using fully qualified name for Pair class, since Calcite also has a same class name being used in the Parser.jj -SqlNode DruidSqlInsertEof() : -{ - SqlNode insertNode; - org.apache.druid.java.util.common.Pair partitionedBy = new org.apache.druid.java.util.common.Pair(null, null); - SqlNodeList clusteredBy = null; -} -{ - insertNode = DruidSqlInsert() + source = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY) + ) // PARTITIONED BY is necessary, but is kept optional in the grammar. It is asserted that it is not missing in the // DruidSqlInsert constructor so that we can return a custom error message. [ @@ -105,12 +105,13 @@ SqlNode DruidSqlInsertEof() : // actual error message. { + insertNode = new SqlInsert(s.end(source), keywordList, destination, source, columnList); if (!(insertNode instanceof SqlInsert)) { // This shouldn't be encountered, but done as a defensive practice. SqlInsert() always returns a node of type // SqlInsert return insertNode; } SqlInsert sqlInsert = (SqlInsert) insertNode; - return new DruidSqlInsert(sqlInsert, partitionedBy.lhs, partitionedBy.rhs, clusteredBy); + return new DruidSqlInsert(sqlInsert, partitionedBy.lhs, partitionedBy.rhs, clusteredBy, exportFileFormat); } } diff --git a/sql/src/main/codegen/includes/replace.ftl b/sql/src/main/codegen/includes/replace.ftl index f3ea3a567610..b2c5e55ff956 100644 --- a/sql/src/main/codegen/includes/replace.ftl +++ b/sql/src/main/codegen/includes/replace.ftl @@ -20,33 +20,52 @@ // Taken from syntax of SqlInsert statement from calcite parser, edited for replace syntax SqlNode DruidSqlReplaceEof() : { - SqlNode table; + final SqlIdentifier destination; SqlNode source; SqlNodeList columnList = null; final Span s; + SqlNode tableRef = null; SqlInsert sqlInsert; // Using fully qualified name for Pair class, since Calcite also has a same class name being used in the Parser.jj org.apache.druid.java.util.common.Pair partitionedBy = new org.apache.druid.java.util.common.Pair(null, null); SqlNodeList clusteredBy = null; final Pair p; SqlNode replaceTimeQuery = null; + String exportFileFormat = null; } { { s = span(); } - table = CompoundIdentifier() - [ - p = ParenthesizedCompoundIdentifierList() { - if (p.left.size() > 0) { - columnList = p.left; - } + ( + LOOKAHEAD(2) + destination = ExternalDestination() + | + destination = CompoundTableIdentifier() + ( tableRef = TableHints(destination) | { tableRef = destination; } ) + [ LOOKAHEAD(5) tableRef = ExtendTable(tableRef) ] + ) + ( + LOOKAHEAD(2) + p = ParenthesizedCompoundIdentifierList() { + if (p.right.size() > 0) { + tableRef = extend(tableRef, p.right); } + if (p.left.size() > 0) { + columnList = p.left; + } else { + columnList = null; + } + } + | { columnList = null; } + ) + [ + exportFileFormat = FileFormat() ] [ - - [ - replaceTimeQuery = ReplaceTimeQuery() - ] + + [ + replaceTimeQuery = ReplaceTimeQuery() + ] ] source = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY) // PARTITIONED BY is necessary, but is kept optional in the grammar. It is asserted that it is not missing in the @@ -71,8 +90,8 @@ SqlNode DruidSqlReplaceEof() : // actual error message. { - sqlInsert = new SqlInsert(s.end(source), SqlNodeList.EMPTY, table, source, columnList); - return new DruidSqlReplace(sqlInsert, partitionedBy.lhs, partitionedBy.rhs, clusteredBy, replaceTimeQuery); + sqlInsert = new SqlInsert(s.end(source), SqlNodeList.EMPTY, destination, source, columnList); + return new DruidSqlReplace(sqlInsert, partitionedBy.lhs, partitionedBy.rhs, clusteredBy, replaceTimeQuery, exportFileFormat); } } diff --git a/sql/src/main/java/org/apache/druid/sql/avatica/DruidMeta.java b/sql/src/main/java/org/apache/druid/sql/avatica/DruidMeta.java index fa3dbdfa22cc..0dde72e4830c 100644 --- a/sql/src/main/java/org/apache/druid/sql/avatica/DruidMeta.java +++ b/sql/src/main/java/org/apache/druid/sql/avatica/DruidMeta.java @@ -56,7 +56,6 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; - import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlIngest.java b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlIngest.java index 146d13673bde..56c2766b0fa7 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlIngest.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlIngest.java @@ -34,6 +34,9 @@ */ public abstract class DruidSqlIngest extends SqlInsert { + public static final String SQL_EXPORT_FILE_FORMAT = "__exportFileFormat"; + + @Nullable protected final Granularity partitionedBy; // Used in the unparse function to generate the original query since we convert the string to an enum @@ -41,6 +44,8 @@ public abstract class DruidSqlIngest extends SqlInsert @Nullable protected final SqlNodeList clusteredBy; + @Nullable + private final String exportFileFormat; public DruidSqlIngest( SqlParserPos pos, @@ -50,7 +55,8 @@ public DruidSqlIngest( SqlNodeList columnList, @Nullable Granularity partitionedBy, @Nullable String partitionedByStringForUnparse, - @Nullable SqlNodeList clusteredBy + @Nullable SqlNodeList clusteredBy, + @Nullable String exportFileFormat ) { super(pos, keywords, targetTable, source, columnList); @@ -58,8 +64,10 @@ public DruidSqlIngest( this.partitionedByStringForUnparse = partitionedByStringForUnparse; this.partitionedBy = partitionedBy; this.clusteredBy = clusteredBy; + this.exportFileFormat = exportFileFormat; } + @Nullable public Granularity getPartitionedBy() { return partitionedBy; @@ -70,4 +78,10 @@ public SqlNodeList getClusteredBy() { return clusteredBy; } + + @Nullable + public String getExportFileFormat() + { + return exportFileFormat; + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlInsert.java b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlInsert.java index c2eeb2ed1e4d..148caeb6d4bd 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlInsert.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlInsert.java @@ -51,7 +51,8 @@ public DruidSqlInsert( @Nonnull SqlInsert insertNode, @Nullable Granularity partitionedBy, @Nullable String partitionedByStringForUnparse, - @Nullable SqlNodeList clusteredBy + @Nullable SqlNodeList clusteredBy, + @Nullable String exportFileFormat ) { super( @@ -62,7 +63,8 @@ public DruidSqlInsert( insertNode.getTargetColumnList(), partitionedBy, partitionedByStringForUnparse, - clusteredBy + clusteredBy, + exportFileFormat ); } @@ -76,9 +78,28 @@ public SqlOperator getOperator() @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { - super.unparse(writer, leftPrec, rightPrec); - writer.keyword("PARTITIONED BY"); - writer.keyword(partitionedByStringForUnparse); + writer.startList(SqlWriter.FrameTypeEnum.SELECT); + writer.sep(isUpsert() ? "UPSERT INTO" : "INSERT INTO"); + final int opLeft = getOperator().getLeftPrec(); + final int opRight = getOperator().getRightPrec(); + getTargetTable().unparse(writer, opLeft, opRight); + if (getTargetColumnList() != null) { + getTargetColumnList().unparse(writer, opLeft, opRight); + } + writer.newlineAndIndent(); + if (getExportFileFormat() != null) { + writer.keyword("AS"); + writer.print(getExportFileFormat()); + writer.newlineAndIndent(); + } + getSource().unparse(writer, 0, 0); + writer.newlineAndIndent(); + + if (partitionedByStringForUnparse != null) { + writer.keyword("PARTITIONED BY"); + writer.keyword(partitionedByStringForUnparse); + } + if (getClusteredBy() != null) { writer.keyword("CLUSTERED BY"); SqlWriter.Frame frame = writer.startList("", ""); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlReplace.java b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlReplace.java index d527a08b59ec..543300b97f56 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlReplace.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlReplace.java @@ -56,7 +56,8 @@ public DruidSqlReplace( @Nullable Granularity partitionedBy, @Nullable String partitionedByStringForUnparse, @Nullable SqlNodeList clusteredBy, - @Nullable SqlNode replaceTimeQuery + @Nullable SqlNode replaceTimeQuery, + @Nullable String exportFileFormat ) { super( @@ -67,7 +68,8 @@ public DruidSqlReplace( insertNode.getTargetColumnList(), partitionedBy, partitionedByStringForUnparse, - clusteredBy + clusteredBy, + exportFileFormat ); this.replaceTimeQuery = replaceTimeQuery; @@ -99,6 +101,12 @@ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) } writer.newlineAndIndent(); + if (getExportFileFormat() != null) { + writer.keyword("AS"); + writer.print(getExportFileFormat()); + writer.newlineAndIndent(); + } + writer.keyword("OVERWRITE"); if (replaceTimeQuery instanceof SqlLiteral) { writer.keyword("ALL"); @@ -110,8 +118,10 @@ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) getSource().unparse(writer, 0, 0); writer.newlineAndIndent(); - writer.keyword("PARTITIONED BY"); - writer.keyword(partitionedByStringForUnparse); + if (partitionedByStringForUnparse != null) { + writer.keyword("PARTITIONED BY"); + writer.keyword(partitionedByStringForUnparse); + } if (getClusteredBy() != null) { writer.keyword("CLUSTERED BY"); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/parser/ExternalDestinationSqlIdentifier.java b/sql/src/main/java/org/apache/druid/sql/calcite/parser/ExternalDestinationSqlIdentifier.java new file mode 100644 index 000000000000..60d88f3c2df5 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/parser/ExternalDestinationSqlIdentifier.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.sql.calcite.parser; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.storage.ExportStorageProvider; +import org.apache.druid.utils.CollectionUtils; + +import java.util.HashMap; +import java.util.Map; + +/** + * Extends the {@link SqlIdentifier} to hold parameters for an external destination. + */ +public class ExternalDestinationSqlIdentifier extends SqlIdentifier +{ + private final Map properties; + + public ExternalDestinationSqlIdentifier( + String name, + SqlParserPos pos, + Map properties + ) + { + super(name, pos); + this.properties = properties; + } + + public String getDestinationType() + { + return CollectionUtils.getOnlyElement( + names, + x -> DruidException.defensive("Expected single name in external destination identifier, but got [%s]", names) + ); + } + + public Map getProperties() + { + return properties; + } + + @Override + public void unparse(SqlWriter writer, int leftPrec, int rightPrec) + { + SqlWriter.Frame externFrame = writer.startFunCall("EXTERN"); + SqlWriter.Frame frame = writer.startFunCall(getDestinationType()); + for (Map.Entry property : properties.entrySet()) { + writer.sep(","); + writer.print(StringUtils.format("%s => '%s'", property.getKey(), property.getValue())); + } + writer.endFunCall(frame); + writer.endFunCall(externFrame); + } + + @Override + public SqlNode clone(SqlParserPos pos) + { + final String name = CollectionUtils.getOnlyElement( + names, + x -> DruidException.defensive("Expected single name in external destination identifier, but got [%s]", names) + ); + return new ExternalDestinationSqlIdentifier(name, pos, properties); + } + + @Override + @Deprecated + public Object clone() + { + throw DruidException.defensive("Function is deprecated, please use clone(SqlNode) instead."); + } + + public ExportStorageProvider toExportStorageProvider(ObjectMapper objectMapper) + { + final HashMap storageConnectorProperties = new HashMap<>(properties); + storageConnectorProperties.put("type", getDestinationType()); + + return objectMapper.convertValue( + storageConnectorProperties, + ExportStorageProvider.class + ); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java index 145886686f4d..d1851c511c2d 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidOperatorTable.java @@ -128,7 +128,6 @@ import org.apache.druid.sql.calcite.planner.convertlet.DruidConvertletTable; import javax.annotation.Nullable; - import java.util.ArrayList; import java.util.HashMap; import java.util.List; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/ExplainAttributes.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/ExplainAttributes.java index e2ae4fa7a10c..49c1861f6ab5 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/ExplainAttributes.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/ExplainAttributes.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.sql.destination.IngestDestination; import javax.annotation.Nullable; import java.util.List; @@ -34,7 +35,7 @@ public final class ExplainAttributes private final String statementType; @Nullable - private final String targetDataSource; + private final IngestDestination targetDataSource; @Nullable private final Granularity partitionedBy; @@ -47,7 +48,7 @@ public final class ExplainAttributes public ExplainAttributes( @JsonProperty("statementType") final String statementType, - @JsonProperty("targetDataSource") @Nullable final String targetDataSource, + @JsonProperty("targetDataSource") @Nullable final IngestDestination targetDataSource, @JsonProperty("partitionedBy") @Nullable final Granularity partitionedBy, @JsonProperty("clusteredBy") @Nullable final List clusteredBy, @JsonProperty("replaceTimeChunks") @Nullable final String replaceTimeChunks @@ -76,7 +77,7 @@ public String getStatementType() @Nullable @JsonProperty @JsonInclude(JsonInclude.Include.NON_NULL) - public String getTargetDataSource() + public IngestDestination getTargetDataSource() { return targetDataSource; } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java index 3d38c6b3f2c1..67ad85f24fbc 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/IngestHandler.java @@ -46,8 +46,13 @@ import org.apache.druid.sql.calcite.parser.DruidSqlInsert; import org.apache.druid.sql.calcite.parser.DruidSqlParserUtils; import org.apache.druid.sql.calcite.parser.DruidSqlReplace; +import org.apache.druid.sql.calcite.parser.ExternalDestinationSqlIdentifier; import org.apache.druid.sql.calcite.run.EngineFeature; import org.apache.druid.sql.calcite.run.QueryMaker; +import org.apache.druid.sql.destination.ExportDestination; +import org.apache.druid.sql.destination.IngestDestination; +import org.apache.druid.sql.destination.TableDestination; +import org.apache.druid.storage.ExportStorageProvider; import java.util.List; import java.util.regex.Pattern; @@ -57,7 +62,7 @@ public abstract class IngestHandler extends QueryHandler private static final Pattern UNNAMED_COLUMN_PATTERN = Pattern.compile("^EXPR\\$\\d+$", Pattern.CASE_INSENSITIVE); protected final Granularity ingestionGranularity; - protected String targetDatasource; + protected IngestDestination targetDatasource; IngestHandler( HandlerContext handlerContext, @@ -103,15 +108,56 @@ protected String operationName() protected abstract DruidSqlIngest ingestNode(); - @Override - public void validate() + private void validateExport() { - if (ingestNode().getPartitionedBy() == null) { + if (!handlerContext.plannerContext().featureAvailable(EngineFeature.WRITE_EXTERNAL_DATA)) { + throw InvalidSqlInput.exception( + "Writing to external sources are not supported by requested SQL engine [%s], consider using MSQ.", + handlerContext.engine().name() + ); + } + + if (ingestNode().getPartitionedBy() != null) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.UNSUPPORTED) + .build("Export statements do not support a PARTITIONED BY or CLUSTERED BY clause."); + } + + final String exportFileFormat = ingestNode().getExportFileFormat(); + if (exportFileFormat == null) { throw InvalidSqlInput.exception( - "Operation [%s] requires a PARTITIONED BY to be explicitly defined, but none was found.", + "Exporting rows into an EXTERN destination requires an AS clause to specify the format, but none was found.", operationName() ); + } else { + handlerContext.plannerContext().queryContextMap().put( + DruidSqlIngest.SQL_EXPORT_FILE_FORMAT, + exportFileFormat + ); + } + } + + @Override + public void validate() + { + if (ingestNode().getTargetTable() instanceof ExternalDestinationSqlIdentifier) { + validateExport(); + } else { + if (ingestNode().getPartitionedBy() == null) { + throw InvalidSqlInput.exception( + "Operation [%s] requires a PARTITIONED BY to be explicitly defined, but none was found.", + operationName() + ); + } + + if (ingestNode().getExportFileFormat() != null) { + throw InvalidSqlInput.exception( + "The AS clause should only be specified while exporting rows into an EXTERN destination.", + operationName() + ); + } } + try { PlannerContext plannerContext = handlerContext.plannerContext(); if (ingestionGranularity != null) { @@ -135,7 +181,6 @@ public void validate() ); } targetDatasource = validateAndGetDataSourceForIngest(); - resourceActions.add(new ResourceAction(new Resource(targetDatasource, ResourceType.DATASOURCE), Action.WRITE)); } @Override @@ -149,10 +194,12 @@ protected RelDataType returnedRowType() } /** - * Extract target datasource from a {@link SqlInsert}, and also validate that the ingestion is of a form we support. - * Expects the target datasource to be either an unqualified name, or a name qualified by the default schema. + * Extract target destination from a {@link SqlInsert}, validates that the ingestion is of a form we support, and + * adds the resource action required (if the destination is a druid datasource). + * Expects the target datasource to be an unqualified name, a name qualified by the default schema or an external + * destination. */ - private String validateAndGetDataSourceForIngest() + private IngestDestination validateAndGetDataSourceForIngest() { final SqlInsert insert = ingestNode(); if (insert.isUpsert()) { @@ -168,23 +215,34 @@ private String validateAndGetDataSourceForIngest() } final SqlIdentifier tableIdentifier = (SqlIdentifier) insert.getTargetTable(); - final String dataSource; + final IngestDestination dataSource; if (tableIdentifier.names.isEmpty()) { // I don't think this can happen, but include a branch for it just in case. throw DruidException.forPersona(DruidException.Persona.USER) .ofCategory(DruidException.Category.DEFENSIVE) .build("Operation [%s] requires a target table", operationName()); + } else if (tableIdentifier instanceof ExternalDestinationSqlIdentifier) { + ExternalDestinationSqlIdentifier externalDestination = ((ExternalDestinationSqlIdentifier) tableIdentifier); + ExportStorageProvider storageProvider = externalDestination.toExportStorageProvider(handlerContext.jsonMapper()); + dataSource = new ExportDestination(storageProvider); + resourceActions.add(new ResourceAction(new Resource(externalDestination.getDestinationType(), ResourceType.EXTERNAL), Action.WRITE)); } else if (tableIdentifier.names.size() == 1) { // Unqualified name. - dataSource = Iterables.getOnlyElement(tableIdentifier.names); + String tableName = Iterables.getOnlyElement(tableIdentifier.names); + IdUtils.validateId("table", tableName); + dataSource = new TableDestination(tableName); + resourceActions.add(new ResourceAction(new Resource(tableName, ResourceType.DATASOURCE), Action.WRITE)); } else { // Qualified name. final String defaultSchemaName = Iterables.getOnlyElement(CalciteSchema.from(handlerContext.defaultSchema()).path(null)); if (tableIdentifier.names.size() == 2 && defaultSchemaName.equals(tableIdentifier.names.get(0))) { - dataSource = tableIdentifier.names.get(1); + String tableName = tableIdentifier.names.get(1); + IdUtils.validateId("table", tableName); + dataSource = new TableDestination(tableName); + resourceActions.add(new ResourceAction(new Resource(tableName, ResourceType.DATASOURCE), Action.WRITE)); } else { throw InvalidSqlInput.exception( "Table [%s] does not support operation [%s] because it is not a Druid datasource", @@ -194,8 +252,6 @@ private String validateAndGetDataSourceForIngest() } } - IdUtils.validateId("table", dataSource); - return dataSource; } @@ -315,6 +371,11 @@ protected DruidSqlIngest ingestNode() @Override public void validate() { + if (ingestNode().getTargetTable() instanceof ExternalDestinationSqlIdentifier) { + throw InvalidSqlInput.exception( + "REPLACE operations do no support EXTERN destinations. Use INSERT statements to write to an external destination." + ); + } if (!handlerContext.plannerContext().featureAvailable(EngineFeature.CAN_REPLACE)) { throw InvalidSqlInput.exception( "REPLACE operations are not supported by the requested SQL engine [%s]. Consider using MSQ.", diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/InputAccessor.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/InputAccessor.java index 57b81c685368..12c81d887567 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/InputAccessor.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/InputAccessor.java @@ -28,7 +28,6 @@ import org.apache.druid.sql.calcite.expression.Expressions; import javax.annotation.Nullable; - import java.util.List; import java.util.stream.Collectors; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidUnionDataSourceRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidUnionDataSourceRule.java index e4a72776315d..31a6d38785de 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidUnionDataSourceRule.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/DruidUnionDataSourceRule.java @@ -36,7 +36,6 @@ import org.apache.druid.sql.calcite.table.DruidTable; import javax.annotation.Nullable; - import java.util.ArrayList; import java.util.Collections; import java.util.List; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/run/EngineFeature.java b/sql/src/main/java/org/apache/druid/sql/calcite/run/EngineFeature.java index 778c7ec03b6f..79883d505931 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/run/EngineFeature.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/run/EngineFeature.java @@ -118,5 +118,9 @@ public enum EngineFeature * and cannot concat the results together (as * the result for broker is the query id). Therefore, we don't get the * correct result back, while the MSQ engine is executing the partial query */ - ALLOW_TOP_LEVEL_UNION_ALL; + ALLOW_TOP_LEVEL_UNION_ALL, + /** + * Queries can write to an external datasource using {@link org.apache.druid.sql.destination.ExportDestination} + */ + WRITE_EXTERNAL_DATA; } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java index 164e02a0ca8d..e7fdf9f7c33f 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeSqlEngine.java @@ -35,6 +35,7 @@ import org.apache.druid.sql.calcite.planner.JoinAlgorithm; import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.rel.DruidQuery; +import org.apache.druid.sql.destination.IngestDestination; import java.util.Map; import java.util.Set; @@ -112,6 +113,7 @@ public boolean featureAvailable(EngineFeature feature, PlannerContext plannerCon case CAN_INSERT: case CAN_REPLACE: case READ_EXTERNAL_DATA: + case WRITE_EXTERNAL_DATA: case SCAN_ORDER_BY_NON_TIME: case SCAN_NEEDS_SIGNATURE: return false; @@ -133,7 +135,7 @@ public QueryMaker buildQueryMakerForSelect(final RelRoot relRoot, final PlannerC @Override public QueryMaker buildQueryMakerForInsert( - final String targetDataSource, + final IngestDestination destination, final RelRoot relRoot, final PlannerContext plannerContext ) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlEngine.java b/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlEngine.java index 678ded23e9da..1ff52f84d0c5 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlEngine.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/run/SqlEngine.java @@ -24,6 +24,7 @@ import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.tools.ValidationException; import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.destination.IngestDestination; import java.util.Map; @@ -82,7 +83,7 @@ public interface SqlEngine /** * Create a {@link QueryMaker} for an INSERT ... SELECT query. * - * @param targetDataSource datasource for the INSERT portion of the query + * @param destination destination for the INSERT portion of the query * @param relRoot planned and validated rel for the SELECT portion of the query * @param plannerContext context for this query * @@ -92,7 +93,7 @@ public interface SqlEngine */ @SuppressWarnings("RedundantThrows") QueryMaker buildQueryMakerForInsert( - String targetDataSource, + IngestDestination destination, RelRoot relRoot, PlannerContext plannerContext ) throws ValidationException; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/view/ViewSqlEngine.java b/sql/src/main/java/org/apache/druid/sql/calcite/view/ViewSqlEngine.java index e2ce813a37f7..ae4cf9639549 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/view/ViewSqlEngine.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/view/ViewSqlEngine.java @@ -27,6 +27,7 @@ import org.apache.druid.sql.calcite.run.QueryMaker; import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.run.SqlEngines; +import org.apache.druid.sql.destination.IngestDestination; import java.util.Map; @@ -59,6 +60,7 @@ public boolean featureAvailable(EngineFeature feature, PlannerContext plannerCon case CAN_SELECT: case ALLOW_BINDABLE_PLAN: case READ_EXTERNAL_DATA: + case WRITE_EXTERNAL_DATA: case SCAN_ORDER_BY_NON_TIME: case GROUPING_SETS: case WINDOW_FUNCTIONS: @@ -109,7 +111,7 @@ public QueryMaker buildQueryMakerForSelect(RelRoot relRoot, PlannerContext plann } @Override - public QueryMaker buildQueryMakerForInsert(String targetDataSource, RelRoot relRoot, PlannerContext plannerContext) + public QueryMaker buildQueryMakerForInsert(IngestDestination destination, RelRoot relRoot, PlannerContext plannerContext) { // Can't have views of INSERT or REPLACE statements. throw new UnsupportedOperationException(); diff --git a/sql/src/main/java/org/apache/druid/sql/destination/ExportDestination.java b/sql/src/main/java/org/apache/druid/sql/destination/ExportDestination.java new file mode 100644 index 000000000000..300998e64458 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/destination/ExportDestination.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.sql.destination; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.storage.ExportStorageProvider; + +import java.util.Objects; + +/** + * Destination that represents an ingestion to an external source. + */ +@JsonTypeName(ExportDestination.TYPE_KEY) +public class ExportDestination implements IngestDestination +{ + public static final String TYPE_KEY = "external"; + private final ExportStorageProvider storageConnectorProvider; + + public ExportDestination(@JsonProperty("storageConnectorProvider") ExportStorageProvider storageConnectorProvider) + { + this.storageConnectorProvider = storageConnectorProvider; + } + + @JsonProperty("storageConnectorProvider") + public ExportStorageProvider getStorageConnectorProvider() + { + return storageConnectorProvider; + } + + @Override + public String getType() + { + return TYPE_KEY; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ExportDestination that = (ExportDestination) o; + return Objects.equals(storageConnectorProvider, that.storageConnectorProvider); + } + + @Override + public int hashCode() + { + return Objects.hash(storageConnectorProvider); + } + + @Override + public String toString() + { + return "ExportDestination{" + + "storageConnectorProvider=" + storageConnectorProvider + + '}'; + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/destination/IngestDestination.java b/sql/src/main/java/org/apache/druid/sql/destination/IngestDestination.java new file mode 100644 index 000000000000..0b1931c847d4 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/destination/IngestDestination.java @@ -0,0 +1,33 @@ +/* + * 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.sql.destination; + +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.guice.annotations.UnstableApi; + +/** + * Represents the destination where the data is ingested. + */ +@UnstableApi +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +public interface IngestDestination +{ + String getType(); +} diff --git a/sql/src/main/java/org/apache/druid/sql/destination/TableDestination.java b/sql/src/main/java/org/apache/druid/sql/destination/TableDestination.java new file mode 100644 index 000000000000..587622ee2c4d --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/destination/TableDestination.java @@ -0,0 +1,79 @@ +/* + * 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.sql.destination; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +import java.util.Objects; + +/** + * Destination that represents an ingestion to a druid table. + */ +@JsonTypeName(TableDestination.TYPE_KEY) +public class TableDestination implements IngestDestination +{ + public static final String TYPE_KEY = "table"; + private final String tableName; + + @JsonCreator + public TableDestination(@JsonProperty("tableName") String tableName) + { + this.tableName = tableName; + } + + @Override + public String getType() + { + return tableName; + } + + @JsonProperty("tableName") + public String getTableName() + { + return tableName; + } + + @Override + public String toString() + { + return tableName; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TableDestination that = (TableDestination) o; + return Objects.equals(tableName, that.tableName); + } + + @Override + public int hashCode() + { + return Objects.hash(tableName); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/http/ArrayLinesWriter.java b/sql/src/main/java/org/apache/druid/sql/http/ArrayLinesWriter.java index beda6deceaba..55b04a193338 100644 --- a/sql/src/main/java/org/apache/druid/sql/http/ArrayLinesWriter.java +++ b/sql/src/main/java/org/apache/druid/sql/http/ArrayLinesWriter.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.SerializerProvider; import org.apache.calcite.rel.type.RelDataType; import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; import java.io.IOException; @@ -70,6 +71,15 @@ public void writeHeader( ArrayWriter.writeHeader(jsonGenerator, rowType, includeTypes, includeSqlTypes); } + @Override + public void writeHeaderFromRowSignature( + final RowSignature rowSignature, + final boolean includeTypes + ) throws IOException + { + ArrayWriter.writeHeader(jsonGenerator, rowSignature, includeTypes); + } + @Override public void writeRowStart() throws IOException { diff --git a/sql/src/main/java/org/apache/druid/sql/http/ArrayWriter.java b/sql/src/main/java/org/apache/druid/sql/http/ArrayWriter.java index cd863d5bf175..e70f7ecfdf65 100644 --- a/sql/src/main/java/org/apache/druid/sql/http/ArrayWriter.java +++ b/sql/src/main/java/org/apache/druid/sql/http/ArrayWriter.java @@ -74,6 +74,12 @@ public void writeHeader( writeHeader(jsonGenerator, rowType, includeTypes, includeSqlTypes); } + @Override + public void writeHeaderFromRowSignature(final RowSignature rowSignature, final boolean includeTypes) throws IOException + { + writeHeader(jsonGenerator, rowSignature, includeTypes); + } + @Override public void writeRowStart() throws IOException { @@ -129,4 +135,25 @@ static void writeHeader( jsonGenerator.writeEndArray(); } } + + static void writeHeader( + final JsonGenerator jsonGenerator, + final RowSignature signature, + final boolean includeTypes + ) throws IOException + { + jsonGenerator.writeStartArray(); + for (String columnName : signature.getColumnNames()) { + jsonGenerator.writeString(columnName); + } + jsonGenerator.writeEndArray(); + + if (includeTypes) { + jsonGenerator.writeStartArray(); + for (int i = 0; i < signature.size(); i++) { + jsonGenerator.writeString(signature.getColumnType(i).map(TypeSignature::asTypeString).orElse(null)); + } + jsonGenerator.writeEndArray(); + } + } } diff --git a/sql/src/main/java/org/apache/druid/sql/http/CsvWriter.java b/sql/src/main/java/org/apache/druid/sql/http/CsvWriter.java index e5e997306845..060cf40da900 100644 --- a/sql/src/main/java/org/apache/druid/sql/http/CsvWriter.java +++ b/sql/src/main/java/org/apache/druid/sql/http/CsvWriter.java @@ -94,6 +94,22 @@ public void writeHeader( } } + @Override + public void writeHeaderFromRowSignature(final RowSignature signature, final boolean includeTypes) + { + writer.writeNext(signature.getColumnNames().toArray(new String[0]), false); + + if (includeTypes) { + final String[] types = new String[signature.size()]; + + for (int i = 0; i < signature.size(); i++) { + types[i] = signature.getColumnType(i).map(TypeSignature::asTypeString).orElse(null); + } + + writer.writeNext(types, false); + } + } + @Override public void writeRowStart() { diff --git a/sql/src/main/java/org/apache/druid/sql/http/ObjectLinesWriter.java b/sql/src/main/java/org/apache/druid/sql/http/ObjectLinesWriter.java index a593b9b21b2a..93fc01e0ed3b 100644 --- a/sql/src/main/java/org/apache/druid/sql/http/ObjectLinesWriter.java +++ b/sql/src/main/java/org/apache/druid/sql/http/ObjectLinesWriter.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.databind.SerializerProvider; import org.apache.calcite.rel.type.RelDataType; import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; import java.io.IOException; @@ -70,6 +71,15 @@ public void writeHeader( ObjectWriter.writeHeader(jsonGenerator, rowType, includeTypes, includeSqlTypes); } + @Override + public void writeHeaderFromRowSignature( + final RowSignature rowSignature, + final boolean includeTypes + ) throws IOException + { + ObjectWriter.writeHeader(jsonGenerator, rowSignature, includeTypes); + } + @Override public void writeRowStart() throws IOException { diff --git a/sql/src/main/java/org/apache/druid/sql/http/ObjectWriter.java b/sql/src/main/java/org/apache/druid/sql/http/ObjectWriter.java index bdab65a1f7e6..6545ce80eacb 100644 --- a/sql/src/main/java/org/apache/druid/sql/http/ObjectWriter.java +++ b/sql/src/main/java/org/apache/druid/sql/http/ObjectWriter.java @@ -77,6 +77,12 @@ public void writeHeader( writeHeader(jsonGenerator, rowType, includeTypes, includeSqlTypes); } + @Override + public void writeHeaderFromRowSignature(final RowSignature signature, final boolean includeTypes) throws IOException + { + writeHeader(jsonGenerator, signature, includeTypes); + } + @Override public void writeRowStart() throws IOException { @@ -141,4 +147,32 @@ static void writeHeader( jsonGenerator.writeEndObject(); } + + static void writeHeader( + final JsonGenerator jsonGenerator, + final RowSignature signature, + final boolean includeTypes + ) throws IOException + { + jsonGenerator.writeStartObject(); + + for (int i = 0; i < signature.size(); i++) { + jsonGenerator.writeFieldName(signature.getColumnName(i)); + + if (!includeTypes) { + jsonGenerator.writeNull(); + } else { + jsonGenerator.writeStartObject(); + + jsonGenerator.writeStringField( + ObjectWriter.TYPE_HEADER_NAME, + signature.getColumnType(i).map(TypeSignature::asTypeString).orElse(null) + ); + + jsonGenerator.writeEndObject(); + } + } + + jsonGenerator.writeEndObject(); + } } diff --git a/sql/src/main/java/org/apache/druid/sql/http/ResultFormat.java b/sql/src/main/java/org/apache/druid/sql/http/ResultFormat.java index 7a16fb7b885a..9f2f37dd7eb7 100644 --- a/sql/src/main/java/org/apache/druid/sql/http/ResultFormat.java +++ b/sql/src/main/java/org/apache/druid/sql/http/ResultFormat.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.calcite.rel.type.RelDataType; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; import javax.ws.rs.core.MediaType; @@ -132,6 +133,8 @@ public interface Writer extends Closeable void writeHeader(RelDataType rowType, boolean includeTypes, boolean includeSqlTypes) throws IOException; + void writeHeaderFromRowSignature(RowSignature rowSignature, boolean includeTypes) throws IOException; + /** * Start of each result row. */ diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 83a863e33f03..4f580d17b89f 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -124,7 +124,6 @@ import org.junit.rules.TemporaryFolder; import javax.annotation.Nullable; - import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteExportTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteExportTest.java new file mode 100644 index 000000000000..cc4b2a0fec49 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteExportTest.java @@ -0,0 +1,265 @@ +/* + * 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.sql.calcite; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import org.apache.druid.error.DruidException; +import org.apache.druid.guice.DruidInjectorBuilder; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.Druids; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.server.security.ForbiddenException; +import org.apache.druid.sql.calcite.export.TestExportModule; +import org.apache.druid.sql.calcite.export.TestExportStorageConnector; +import org.apache.druid.sql.calcite.filtration.Filtration; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.destination.ExportDestination; +import org.apache.druid.storage.StorageConfig; +import org.apache.druid.storage.StorageConnector; +import org.apache.druid.storage.local.LocalFileExportStorageProvider; +import org.apache.druid.storage.local.LocalFileStorageConnectorProvider; +import org.hamcrest.CoreMatchers; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; + +import java.util.List; + +public class CalciteExportTest extends CalciteIngestionDmlTest +{ + @Override + public void configureGuice(DruidInjectorBuilder builder) + { + super.configureGuice(builder); + builder.addModule(new TestExportModule()); + builder.addModule(new DruidModule() + { + @Override + public List getJacksonModules() + { + return ImmutableList.of( + new SimpleModule(StorageConnector.class.getSimpleName()) + .registerSubtypes(LocalFileStorageConnectorProvider.class) + .registerSubtypes(LocalFileExportStorageProvider.class) + ); + } + + @Override + public void configure(Binder binder) + { + binder.bind(StorageConfig.class).toInstance(new StorageConfig("/tmp/export")); + } + }); + } + + // Disabled until replace supports external destinations. To be enabled after that point. + @Test + @Ignore + public void testReplaceIntoExtern() + { + testIngestionQuery() + .sql(StringUtils.format("REPLACE INTO EXTERN(%s(basePath => 'export')) " + + "AS CSV " + + "OVERWRITE ALL " + + "SELECT dim2 FROM foo", TestExportStorageConnector.TYPE_NAME)) + .expectQuery( + Druids.newScanQueryBuilder() + .dataSource( + "foo" + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim2") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .build() + ) + .expectResources(dataSourceRead("foo"), externalWrite(TestExportStorageConnector.TYPE_NAME)) + .expectTarget(ExportDestination.TYPE_KEY, RowSignature.builder().add("dim2", ColumnType.STRING).build()) + .verify(); + } + + @Test + public void testReplaceIntoExternShouldThrowUnsupportedException() + { + testIngestionQuery() + .sql(StringUtils.format("REPLACE INTO EXTERN(%s(basePath => 'export')) " + + "AS CSV " + + "OVERWRITE ALL " + + "SELECT dim2 FROM foo", TestExportStorageConnector.TYPE_NAME)) + .expectValidationError( + CoreMatchers.allOf( + CoreMatchers.instanceOf(DruidException.class), + ThrowableMessageMatcher.hasMessage( + CoreMatchers.containsString( + "REPLACE operations do no support EXTERN destinations. Use INSERT statements to write to an external destination." + ) + ) + ) + ) + .verify(); + } + + @Test + public void testExportWithoutRequiredParameter() + { + testIngestionQuery() + .sql(StringUtils.format("INSERT INTO EXTERN(%s()) " + + "AS CSV " + + "SELECT dim2 FROM foo", LocalFileExportStorageProvider.TYPE_NAME)) + .expectValidationError( + CoreMatchers.allOf( + CoreMatchers.instanceOf(IllegalArgumentException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("Missing required creator property 'exportPath'")) + ) + ) + .verify(); + } + + @Test + public void testExportWithPartitionedBy() + { + testIngestionQuery() + .sql(StringUtils.format("INSERT INTO EXTERN(%s()) " + + "AS CSV " + + "SELECT dim2 FROM foo " + + "PARTITIONED BY ALL", TestExportStorageConnector.TYPE_NAME)) + .expectValidationError( + DruidException.class, + "Export statements do not support a PARTITIONED BY or CLUSTERED BY clause." + ) + .verify(); + } + + @Test + public void testInsertIntoExtern() + { + testIngestionQuery() + .sql(StringUtils.format("INSERT INTO EXTERN(%s()) " + + "AS CSV " + + "SELECT dim2 FROM foo", TestExportStorageConnector.TYPE_NAME)) + .expectQuery( + Druids.newScanQueryBuilder() + .dataSource( + "foo" + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim2") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .build() + ) + .expectResources(dataSourceRead("foo"), externalWrite(TestExportStorageConnector.TYPE_NAME)) + .expectTarget(ExportDestination.TYPE_KEY, RowSignature.builder().add("dim2", ColumnType.STRING).build()) + .verify(); + } + + @Test + public void testExportWithoutFormat() + { + testIngestionQuery() + .sql("INSERT INTO EXTERN(testStorage(bucket=>'bucket1',prefix=>'prefix1',tempDir=>'/tempdir',chunkSize=>'5242880',maxRetry=>'1')) " + + "SELECT dim2 FROM foo") + .expectValidationError( + DruidException.class, + "Exporting rows into an EXTERN destination requires an AS clause to specify the format, but none was found." + ) + .verify(); + } + + @Test + public void testWithUnsupportedStorageConnector() + { + testIngestionQuery() + .sql("insert into extern(nonExistent()) as csv select __time, dim1 from foo") + .expectValidationError( + CoreMatchers.allOf( + CoreMatchers.instanceOf(IllegalArgumentException.class), + ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("Could not resolve type id 'nonExistent' as a subtype")) + ) + ) + .verify(); + } + + @Test + public void testWithForbiddenDestination() + { + testIngestionQuery() + .sql(StringUtils.format("insert into extern(%s()) as csv select __time, dim1 from foo", CalciteTests.FORBIDDEN_DESTINATION)) + .expectValidationError(ForbiddenException.class) + .verify(); + } + + @Test + public void testSelectFromTableNamedExport() + { + testIngestionQuery() + .sql("INSERT INTO csv SELECT dim2 FROM foo PARTITIONED BY ALL") + .expectQuery( + Druids.newScanQueryBuilder() + .dataSource("foo") + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim2") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .build() + ) + .expectResources(dataSourceRead("foo"), dataSourceWrite("csv")) + .expectTarget("csv", RowSignature.builder().add("dim2", ColumnType.STRING).build()) + .verify(); + } + + @Test + public void testNormalInsertWithFormat() + { + testIngestionQuery() + .sql("REPLACE INTO testTable " + + "AS CSV " + + "OVERWRITE ALL " + + "SELECT dim2 FROM foo " + + "PARTITIONED BY ALL") + .expectValidationError( + DruidException.class, + "The AS clause should only be specified while exporting rows into an EXTERN destination." + ) + .verify(); + } + + @Test + public void testUnsupportedExportFormat() + { + testIngestionQuery() + .sql("REPLACE INTO testTable " + + "AS JSON " + + "OVERWRITE ALL " + + "SELECT dim2 FROM foo " + + "PARTITIONED BY ALL") + .expectValidationError( + DruidException.class, + "The AS clause should only be specified while exporting rows into an EXTERN destination." + ) + .verify(); + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java index a34c93ce9c92..35256c96de59 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java @@ -657,7 +657,7 @@ public void testExplainPlanInsertWithClusteredBy() throws JsonProcessingExceptio skipVectorize(); final String resources = "[{\"name\":\"dst\",\"type\":\"DATASOURCE\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; - final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":[\"floor_m1\",\"dim1\",\"CEIL(\\\"m2\\\")\"]}"; + final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":{\"type\":\"table\",\"tableName\":\"dst\"},\"partitionedBy\":\"DAY\",\"clusteredBy\":[\"floor_m1\",\"dim1\",\"CEIL(\\\"m2\\\")\"]}"; final String sql = "EXPLAIN PLAN FOR INSERT INTO druid.dst " + "SELECT __time, FLOOR(m1) as floor_m1, dim1, CEIL(m2) as ceil_m2 FROM foo " @@ -761,7 +761,7 @@ public void testExplainPlanInsertWithAsSubQueryClusteredBy() skipVectorize(); final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; - final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"foo\",\"partitionedBy\":{\"type\":\"all\"},\"clusteredBy\":[\"namespace\",\"country\"]}"; + final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":{\"type\":\"table\",\"tableName\":\"foo\"},\"partitionedBy\":{\"type\":\"all\"},\"clusteredBy\":[\"namespace\",\"country\"]}"; final String sql = "EXPLAIN PLAN FOR\n" + "INSERT INTO \"foo\"\n" @@ -859,7 +859,7 @@ public void testExplainPlanInsertJoinQuery() skipVectorize(); final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"my_table\",\"type\":\"DATASOURCE\"}]"; - final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"my_table\",\"partitionedBy\":\"HOUR\",\"clusteredBy\":[\"__time\",\"isRobotAlias\",\"countryCapital\",\"regionName\"]}"; + final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":{\"type\":\"table\",\"tableName\":\"my_table\"},\"partitionedBy\":\"HOUR\",\"clusteredBy\":[\"__time\",\"isRobotAlias\",\"countryCapital\",\"regionName\"]}"; final String sql = "EXPLAIN PLAN FOR\n" + "INSERT INTO my_table\n" @@ -1225,7 +1225,7 @@ public void testExplainInsertFromExternal() throws IOException + "}]"; final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"dst\",\"type\":\"DATASOURCE\"}]"; - final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"dst\",\"partitionedBy\":{\"type\":\"all\"}}"; + final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":{\"type\":\"table\",\"tableName\":\"dst\"},\"partitionedBy\":{\"type\":\"all\"}}"; // Use testQuery for EXPLAIN (not testIngestionQuery). testQuery( @@ -1329,7 +1329,7 @@ public void testExplainPlanForInsertWithClusteredBy() throws JsonProcessingExcep + "}]"; final String resources = "[{\"name\":\"dst\",\"type\":\"DATASOURCE\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; - final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":[\"floor_m1\",\"dim1\",\"CEIL(\\\"m2\\\")\"]}"; + final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":{\"type\":\"table\",\"tableName\":\"dst\"},\"partitionedBy\":\"DAY\",\"clusteredBy\":[\"floor_m1\",\"dim1\",\"CEIL(\\\"m2\\\")\"]}"; // Use testQuery for EXPLAIN (not testIngestionQuery). testQuery( diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java index 2970330e82fc..dbad37496f7c 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java @@ -654,7 +654,7 @@ public void testExplainReplaceFromExternal() throws IOException + "\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]}]"; final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"dst\",\"type\":\"DATASOURCE\"}]"; - final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":\"dst\",\"partitionedBy\":{\"type\":\"all\"},\"replaceTimeChunks\":\"all\"}"; + final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":{\"type\":\"table\",\"tableName\":\"dst\"},\"partitionedBy\":{\"type\":\"all\"},\"replaceTimeChunks\":\"all\"}"; // Use testQuery for EXPLAIN (not testIngestionQuery). testQuery( @@ -732,7 +732,7 @@ public void testExplainReplaceTimeChunksWithPartitioningAndClustering() throws I final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"dim1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"sqlReplaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; final String resources = "[{\"name\":\"dst\",\"type\":\"DATASOURCE\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; - final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":\"dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":[\"dim1\"],\"replaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\"}"; + final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":{\"type\":\"table\",\"tableName\":\"dst\"},\"partitionedBy\":\"DAY\",\"clusteredBy\":[\"dim1\"],\"replaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\"}"; final String sql = "EXPLAIN PLAN FOR" + " REPLACE INTO dst" @@ -833,7 +833,7 @@ public void testExplainReplaceWithLimitAndClusteredByOrdinals() throws IOExcepti + "{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"}," + "{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; final String resources = "[{\"name\":\"dst\",\"type\":\"DATASOURCE\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; - final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":\"dst\",\"partitionedBy\":\"HOUR\"," + final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":{\"type\":\"table\",\"tableName\":\"dst\"},\"partitionedBy\":\"HOUR\"," + "\"clusteredBy\":[\"__time\",\"dim1\",\"dim3\",\"dim2\"],\"replaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\"}"; final String sql = "EXPLAIN PLAN FOR" diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteScanSignatureTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteScanSignatureTest.java index 424a2e8895e9..954eda85b546 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteScanSignatureTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteScanSignatureTest.java @@ -36,6 +36,7 @@ import org.apache.druid.sql.calcite.run.QueryMaker; import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.sql.destination.IngestDestination; import org.junit.Test; import java.util.HashMap; @@ -168,7 +169,7 @@ public QueryMaker buildQueryMakerForSelect(RelRoot relRoot, PlannerContext plann } @Override - public QueryMaker buildQueryMakerForInsert(String targetDataSource, RelRoot relRoot, PlannerContext plannerContext) + public QueryMaker buildQueryMakerForInsert(IngestDestination destination, RelRoot relRoot, PlannerContext plannerContext) { throw new UnsupportedOperationException(); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java index 03dc361e478b..7990071467cf 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java @@ -70,7 +70,6 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; - import java.io.File; import java.io.IOException; import java.io.InputStream; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java index ff29a8743242..976e0aff59bc 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java @@ -322,7 +322,7 @@ public void testExplainHttpFn() "\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]," + "\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]}]"; final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"dst\",\"type\":\"DATASOURCE\"}]"; - final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"dst\",\"partitionedBy\":{\"type\":\"all\"}}"; + final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":{\"type\":\"table\",\"tableName\":\"dst\"},\"partitionedBy\":{\"type\":\"all\"}}"; testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/IngestionTestSqlEngine.java b/sql/src/test/java/org/apache/druid/sql/calcite/IngestionTestSqlEngine.java index b0bf0bd7b29d..0a18eb47f46a 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/IngestionTestSqlEngine.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/IngestionTestSqlEngine.java @@ -31,6 +31,7 @@ import org.apache.druid.sql.calcite.run.SqlEngine; import org.apache.druid.sql.calcite.run.SqlEngines; import org.apache.druid.sql.calcite.table.RowSignatures; +import org.apache.druid.sql.destination.IngestDestination; import java.util.Map; @@ -88,6 +89,7 @@ public boolean featureAvailable(final EngineFeature feature, final PlannerContex case CAN_INSERT: case CAN_REPLACE: case READ_EXTERNAL_DATA: + case WRITE_EXTERNAL_DATA: case SCAN_ORDER_BY_NON_TIME: case ALLOW_BROADCAST_RIGHTY_JOIN: case ALLOW_TOP_LEVEL_UNION_ALL: @@ -104,13 +106,13 @@ public QueryMaker buildQueryMakerForSelect(RelRoot relRoot, PlannerContext plann } @Override - public QueryMaker buildQueryMakerForInsert(String targetDataSource, RelRoot relRoot, PlannerContext plannerContext) + public QueryMaker buildQueryMakerForInsert(IngestDestination destination, RelRoot relRoot, PlannerContext plannerContext) { final RowSignature signature = RowSignatures.fromRelDataType( relRoot.validatedRowType.getFieldNames(), relRoot.validatedRowType ); - return new TestInsertQueryMaker(targetDataSource, signature); + return new TestInsertQueryMaker(destination, signature); } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/TestInsertQueryMaker.java b/sql/src/test/java/org/apache/druid/sql/calcite/TestInsertQueryMaker.java index 8562ce29bec5..ba1eec29d796 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/TestInsertQueryMaker.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/TestInsertQueryMaker.java @@ -26,21 +26,22 @@ import org.apache.druid.server.QueryResponse; import org.apache.druid.sql.calcite.rel.DruidQuery; import org.apache.druid.sql.calcite.run.QueryMaker; +import org.apache.druid.sql.destination.IngestDestination; /** * QueryMaker used by {@link CalciteInsertDmlTest}. */ public class TestInsertQueryMaker implements QueryMaker { - private final String targetDataSource; + private final IngestDestination destination; private final RowSignature signature; public TestInsertQueryMaker( - final String targetDataSource, + final IngestDestination destination, final RowSignature signature ) { - this.targetDataSource = targetDataSource; + this.destination = destination; this.signature = signature; } @@ -54,7 +55,7 @@ public QueryResponse runQuery(final DruidQuery druidQuery) // 2) Return the dataSource and signature of the insert operation, so tests can confirm they are correct. return QueryResponse.withEmptyContext( - Sequences.simple(ImmutableList.of(new Object[]{targetDataSource, signature})) + Sequences.simple(ImmutableList.of(new Object[]{destination.getType(), signature})) ); } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/export/TestExportModule.java b/sql/src/test/java/org/apache/druid/sql/calcite/export/TestExportModule.java new file mode 100644 index 000000000000..b6969f4c165c --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/export/TestExportModule.java @@ -0,0 +1,52 @@ +/* + * 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.sql.calcite.export; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.apache.druid.storage.StorageConnectorProvider; + +import java.util.List; + +public class TestExportModule implements DruidModule +{ + @Override + public List getJacksonModules() + { + return ImmutableList.of( + new SimpleModule(StorageConnectorProvider.class.getSimpleName()) + .registerSubtypes( + new NamedType(TestExportStorageConnectorProvider.class, TestExportStorageConnector.TYPE_NAME), + new NamedType(TestExportStorageConnectorProvider.class, CalciteTests.FORBIDDEN_DESTINATION) + ) + ); + } + + @Override + public void configure(Binder binder) + { + + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/export/TestExportStorageConnector.java b/sql/src/test/java/org/apache/druid/sql/calcite/export/TestExportStorageConnector.java new file mode 100644 index 000000000000..b81b22ceb878 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/export/TestExportStorageConnector.java @@ -0,0 +1,92 @@ +/* + * 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.sql.calcite.export; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.storage.StorageConnector; + +import java.io.ByteArrayOutputStream; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Iterator; + +public class TestExportStorageConnector implements StorageConnector +{ + public static final String TYPE_NAME = "testStorage"; + private final ByteArrayOutputStream byteArrayOutputStream; + + public TestExportStorageConnector() + { + this.byteArrayOutputStream = new ByteArrayOutputStream(); + } + + public ByteArrayOutputStream getByteArrayOutputStream() + { + return byteArrayOutputStream; + } + + @Override + public boolean pathExists(String path) + { + return true; + } + + @Override + public InputStream read(String path) + { + throw new UnsupportedOperationException(); + } + + @Override + public InputStream readRange(String path, long from, long size) + { + throw new UnsupportedOperationException(); + } + + @Override + public OutputStream write(String path) + { + return byteArrayOutputStream; + } + + @Override + public void deleteFile(String path) + { + throw new UnsupportedOperationException(); + } + + @Override + public void deleteFiles(Iterable paths) + { + throw new UnsupportedOperationException(); + } + + @Override + public void deleteRecursively(String path) + { + throw new UnsupportedOperationException(); + } + + @Override + public Iterator listDir(String dirName) + { + return ImmutableList.of().stream().iterator(); + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/export/TestExportStorageConnectorProvider.java b/sql/src/test/java/org/apache/druid/sql/calcite/export/TestExportStorageConnectorProvider.java new file mode 100644 index 000000000000..b1ca59e2ccc3 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/export/TestExportStorageConnectorProvider.java @@ -0,0 +1,46 @@ +/* + * 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.sql.calcite.export; + +import org.apache.druid.storage.ExportStorageProvider; +import org.apache.druid.storage.StorageConnector; + +public class TestExportStorageConnectorProvider implements ExportStorageProvider +{ + private static final StorageConnector STORAGE_CONNECTOR = new TestExportStorageConnector(); + + @Override + public StorageConnector get() + { + return STORAGE_CONNECTOR; + } + + @Override + public String getResourceType() + { + return "testExport"; + } + + @Override + public String getBasePath() + { + return "testExport"; + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlUnparseTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlUnparseTest.java index 0e67fdbe9641..0e843b4ca2ea 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlUnparseTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/parser/DruidSqlUnparseTest.java @@ -41,7 +41,8 @@ public void testUnparseInsert() throws ParseException String sqlQuery = "INSERT INTO dst SELECT * FROM foo PARTITIONED BY ALL TIME"; String prettySqlQuery = "INSERT INTO \"dst\"\n" + "SELECT *\n" - + " FROM \"foo\" PARTITIONED BY ALL TIME"; + + " FROM \"foo\"\n" + + "PARTITIONED BY ALL TIME"; DruidSqlParserImpl druidSqlParser = createTestParser(sqlQuery); DruidSqlInsert druidSqlReplace = (DruidSqlInsert) druidSqlParser.DruidSqlInsertEof(); @@ -95,4 +96,33 @@ private static DruidSqlParserImpl createTestParser(String parseString) druidSqlParser.setIdentifierMaxLength(20); return druidSqlParser; } + + @Test + public void testUnparseExternalSqlIdentifierReplace() throws ParseException + { + String sqlQuery = "REPLACE INTO EXTERN( s3(bucket=>'bucket1',prefix=>'prefix1') ) AS CSV OVERWRITE ALL SELECT dim2 FROM foo"; + String prettySqlQuery = "REPLACE INTO EXTERN(S3(bucket => 'bucket1', prefix => 'prefix1'))\n" + + "AS csv\n" + + "OVERWRITE ALL\n" + + "SELECT \"dim2\"\n" + + " FROM \"foo\"\n"; + DruidSqlParserImpl druidSqlParser = createTestParser(sqlQuery); + DruidSqlReplace druidSqlReplace = (DruidSqlReplace) druidSqlParser.DruidSqlReplaceEof(); + druidSqlReplace.unparse(sqlWriter, 0, 0); + assertEquals(prettySqlQuery, sqlWriter.toSqlString().getSql()); + } + + @Test + public void testUnparseExternalSqlIdentifierInsert() throws ParseException + { + String sqlQuery = "INSERT INTO EXTERN( s3(bucket=>'bucket1',prefix=>'prefix1') ) AS CSV SELECT dim2 FROM foo"; + String prettySqlQuery = "INSERT INTO EXTERN(S3(bucket => 'bucket1', prefix => 'prefix1'))\n" + + "AS csv\n" + + "SELECT \"dim2\"\n" + + " FROM \"foo\"\n"; + DruidSqlParserImpl druidSqlParser = createTestParser(sqlQuery); + DruidSqlInsert druidSqlInsert = (DruidSqlInsert) druidSqlParser.DruidSqlInsertEof(); + druidSqlInsert.unparse(sqlWriter, 0, 0); + assertEquals(prettySqlQuery, sqlWriter.toSqlString().getSql()); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/planner/ExplainAttributesTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/planner/ExplainAttributesTest.java index 67f97d64215d..052a57c6d4e5 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/planner/ExplainAttributesTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/planner/ExplainAttributesTest.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.sql.destination.TableDestination; import org.junit.Assert; import org.junit.Test; @@ -64,14 +65,14 @@ public void testSerializeInsertAttributes() throws JsonProcessingException { ExplainAttributes insertAttributes = new ExplainAttributes( "INSERT", - "foo", + new TableDestination("foo"), Granularities.DAY, null, null ); final String expectedAttributes = "{" + "\"statementType\":\"INSERT\"," - + "\"targetDataSource\":\"foo\"," + + "\"targetDataSource\":{\"type\":\"table\",\"tableName\":\"foo\"}," + "\"partitionedBy\":\"DAY\"" + "}"; Assert.assertEquals(expectedAttributes, DEFAULT_OBJECT_MAPPER.writeValueAsString(insertAttributes)); @@ -82,14 +83,14 @@ public void testSerializeInsertAllAttributes() throws JsonProcessingException { ExplainAttributes insertAttributes = new ExplainAttributes( "INSERT", - "foo", + new TableDestination("foo"), Granularities.ALL, null, null ); final String expectedAttributes = "{" + "\"statementType\":\"INSERT\"," - + "\"targetDataSource\":\"foo\"," + + "\"targetDataSource\":{\"type\":\"table\",\"tableName\":\"foo\"}," + "\"partitionedBy\":{\"type\":\"all\"}" + "}"; Assert.assertEquals(expectedAttributes, DEFAULT_OBJECT_MAPPER.writeValueAsString(insertAttributes)); @@ -100,14 +101,14 @@ public void testSerializeReplaceAttributes() throws JsonProcessingException { ExplainAttributes replaceAttributes1 = new ExplainAttributes( "REPLACE", - "foo", + new TableDestination("foo"), Granularities.HOUR, null, "ALL" ); final String expectedAttributes1 = "{" + "\"statementType\":\"REPLACE\"," - + "\"targetDataSource\":\"foo\"," + + "\"targetDataSource\":{\"type\":\"table\",\"tableName\":\"foo\"}," + "\"partitionedBy\":\"HOUR\"," + "\"replaceTimeChunks\":\"ALL\"" + "}"; @@ -116,14 +117,14 @@ public void testSerializeReplaceAttributes() throws JsonProcessingException ExplainAttributes replaceAttributes2 = new ExplainAttributes( "REPLACE", - "foo", + new TableDestination("foo"), Granularities.HOUR, null, "2019-08-25T02:00:00.000Z/2019-08-25T03:00:00.000Z" ); final String expectedAttributes2 = "{" + "\"statementType\":\"REPLACE\"," - + "\"targetDataSource\":\"foo\"," + + "\"targetDataSource\":{\"type\":\"table\",\"tableName\":\"foo\"}," + "\"partitionedBy\":\"HOUR\"," + "\"replaceTimeChunks\":\"2019-08-25T02:00:00.000Z/2019-08-25T03:00:00.000Z\"" + "}"; @@ -135,14 +136,14 @@ public void testSerializeReplaceWithClusteredByAttributes() throws JsonProcessin { ExplainAttributes replaceAttributes1 = new ExplainAttributes( "REPLACE", - "foo", + new TableDestination("foo"), Granularities.HOUR, Arrays.asList("foo", "CEIL(`f2`)"), "ALL" ); final String expectedAttributes1 = "{" + "\"statementType\":\"REPLACE\"," - + "\"targetDataSource\":\"foo\"," + + "\"targetDataSource\":{\"type\":\"table\",\"tableName\":\"foo\"}," + "\"partitionedBy\":\"HOUR\"," + "\"clusteredBy\":[\"foo\",\"CEIL(`f2`)\"]," + "\"replaceTimeChunks\":\"ALL\"" @@ -152,14 +153,14 @@ public void testSerializeReplaceWithClusteredByAttributes() throws JsonProcessin ExplainAttributes replaceAttributes2 = new ExplainAttributes( "REPLACE", - "foo", + new TableDestination("foo"), Granularities.HOUR, Arrays.asList("foo", "boo"), "2019-08-25T02:00:00.000Z/2019-08-25T03:00:00.000Z" ); final String expectedAttributes2 = "{" + "\"statementType\":\"REPLACE\"," - + "\"targetDataSource\":\"foo\"," + + "\"targetDataSource\":{\"type\":\"table\",\"tableName\":\"foo\"}," + "\"partitionedBy\":\"HOUR\"," + "\"clusteredBy\":[\"foo\",\"boo\"]," + "\"replaceTimeChunks\":\"2019-08-25T02:00:00.000Z/2019-08-25T03:00:00.000Z\"" diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidRelsTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidRelsTest.java index 199d259d92f6..e5daa3471e16 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidRelsTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/rel/DruidRelsTest.java @@ -32,7 +32,6 @@ import org.junit.Test; import javax.annotation.Nullable; - import java.util.List; import java.util.function.Consumer; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTestBase.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTestBase.java index 7e990887dd10..a10a56d31e34 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTestBase.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTestBase.java @@ -124,4 +124,9 @@ protected static ResourceAction externalRead(final String inputSourceType) { return new ResourceAction(new Resource(inputSourceType, ResourceType.EXTERNAL), Action.READ); } + + protected static ResourceAction externalWrite(final String inputSourceType) + { + return new ResourceAction(new Resource(inputSourceType, ResourceType.EXTERNAL), Action.WRITE); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index a9ca48e90a9b..342a9384decc 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -62,6 +62,7 @@ import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.security.Access; +import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AllowAllAuthenticator; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthenticationResult; @@ -112,6 +113,7 @@ public class CalciteTests public static final String DATASOURCE5 = "lotsocolumns"; public static final String BROADCAST_DATASOURCE = "broadcast"; public static final String FORBIDDEN_DATASOURCE = "forbiddenDatasource"; + public static final String FORBIDDEN_DESTINATION = "forbiddenDestination"; public static final String SOME_DATASOURCE = "some_datasource"; public static final String SOME_DATSOURCE_ESCAPED = "some\\_datasource"; public static final String SOMEXDATASOURCE = "somexdatasource"; @@ -146,6 +148,15 @@ public Authorizer getAuthorizer(String name) } case ResourceType.QUERY_CONTEXT: return Access.OK; + case ResourceType.EXTERNAL: + if (Action.WRITE.equals(action)) { + if (FORBIDDEN_DESTINATION.equals(resource.getName())) { + return new Access(false); + } else { + return Access.OK; + } + } + return new Access(false); default: return new Access(false); } diff --git a/sql/src/test/java/org/apache/druid/sql/destination/ExportDestinationTest.java b/sql/src/test/java/org/apache/druid/sql/destination/ExportDestinationTest.java new file mode 100644 index 000000000000..a06de536a2fa --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/destination/ExportDestinationTest.java @@ -0,0 +1,66 @@ +/* + * 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.sql.destination; + +import com.fasterxml.jackson.databind.BeanProperty; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.storage.StorageConfig; +import org.apache.druid.storage.StorageConnectorModule; +import org.apache.druid.storage.local.LocalFileExportStorageProvider; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +public class ExportDestinationTest +{ + @Test + public void testSerde() throws IOException + { + ExportDestination exportDestination = new ExportDestination(new LocalFileExportStorageProvider("/basepath/export")); + + ObjectMapper objectMapper = new DefaultObjectMapper(); + objectMapper.registerModules(new StorageConnectorModule().getJacksonModules()); + objectMapper.setInjectableValues(new InjectableValues() + { + @Override + public Object findInjectableValue( + Object valueId, + DeserializationContext ctxt, + BeanProperty forProperty, + Object beanInstance + ) + { + if (((String) valueId).contains("StorageConfig")) { + return new StorageConfig("/"); + } else { + throw new RuntimeException(); + } + } + }); + byte[] bytes = objectMapper.writeValueAsBytes(exportDestination); + + ExportDestination deserialized = objectMapper.readValue(bytes, ExportDestination.class); + Assert.assertEquals(exportDestination, deserialized); + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/destination/TableDestinationTest.java b/sql/src/test/java/org/apache/druid/sql/destination/TableDestinationTest.java new file mode 100644 index 000000000000..7af4ad2eb4d6 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/destination/TableDestinationTest.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.sql.destination; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +public class TableDestinationTest +{ + @Test + public void testSerde() throws IOException + { + TableDestination tableDestination = new TableDestination("tableName"); + + ObjectMapper objectMapper = new DefaultObjectMapper(); + byte[] bytes = objectMapper.writeValueAsBytes(tableDestination); + + TableDestination deserialized = objectMapper.readValue(bytes, TableDestination.class); + Assert.assertEquals(tableDestination, deserialized); + } +}