Much improved table functions (#13627)

Much improved table functions

* Revises properties, definitions in the catalog
* Adds a "table function" abstraction to model such functions
* Specific functions for HTTP, inline, local and S3.
* Extended SQL types in the catalog
* Restructure external table definitions to use table functions
* EXTEND syntax for Druid's extern table function
* Support for array-valued table function parameters
* Support for array-valued SQL query parameters
* Much new documentation
This commit is contained in:
Paul Rogers 2023-01-17 08:41:57 -08:00 committed by GitHub
parent 59dfe7bed3
commit 22630b0aab
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
85 changed files with 7523 additions and 3214 deletions

View File

@ -49,37 +49,42 @@ To support ingestion, additional SQL functionality is available through the MSQ
<a name="extern"></a>
### Read external data with EXTERN
### Read external data with `EXTERN`
Query tasks can access external data through the EXTERN function, using any native batch [input
Query tasks can access external data through the `EXTERN` function, using any native batch [input
source](../ingestion/native-batch-input-source.md) and [input format](../ingestion/data-formats.md#input-format).
EXTERN can read multiple files in parallel across different worker tasks. However, EXTERN does not split individual
`EXTERN` can read multiple files in parallel across different worker tasks. However, `EXTERN` does not split individual
files across multiple worker tasks. If you have a small number of very large input files, you can increase query
parallelism by splitting up your input files.
For more information about the syntax, see [EXTERN](./reference.md#extern).
For more information about the syntax, see [`EXTERN`](./reference.md#extern-function).
See also the set of SQL-friendly input-source-specific table functions which may be more convenient
than `EXTERN`.
<a name="insert"></a>
### Load data with INSERT
### Load data with `INSERT`
INSERT statements can create a new datasource or append to an existing datasource. In Druid SQL, unlike standard SQL,
`INSERT` statements can create a new datasource or append to an existing datasource. In Druid SQL, unlike standard SQL,
there is no syntactical difference between creating a table and appending data to a table. Druid does not include a
`CREATE TABLE` statement.
Nearly all SELECT capabilities are available for `INSERT ... SELECT` queries. Certain exceptions are listed on the [Known
issues](./known-issues.md#select) page.
Nearly all `SELECT` capabilities are available for `INSERT ... SELECT` queries. Certain exceptions are listed on the [Known
issues](./known-issues.md#select-statement) page.
INSERT statements acquire a shared lock to the target datasource. Multiple INSERT statements can run at the same time,
`INSERT` statements acquire a shared lock to the target datasource. Multiple `INSERT` statements can run at the same time,
for the same datasource, if your cluster has enough task slots.
Like all other forms of [batch ingestion](../ingestion/index.md#batch), each INSERT statement generates new segments and
Like all other forms of [batch ingestion](../ingestion/index.md#batch), each `INSERT` statement generates new segments and
publishes them at the end of its run. For this reason, it is best suited to loading data in larger batches. Do not use
INSERT statements to load data in a sequence of microbatches; for that, use [streaming
`INSERT` statements to load data in a sequence of microbatches; for that, use [streaming
ingestion](../ingestion/index.md#streaming) instead.
When deciding whether to use REPLACE or INSERT, keep in mind that segments generated with REPLACE can be pruned with dimension-based pruning but those generated with INSERT cannot. For more information about the requirements for dimension-based pruning, see [Clustering](#clustering).
When deciding whether to use `REPLACE` or `INSERT`, keep in mind that segments generated with `REPLACE` can be pruned
with dimension-based pruning but those generated with `INSERT` cannot. For more information about the requirements
for dimension-based pruning, see [Clustering](#clustering).
For more information about the syntax, see [INSERT](./reference.md#insert).
@ -87,24 +92,26 @@ For more information about the syntax, see [INSERT](./reference.md#insert).
### Overwrite data with REPLACE
REPLACE statements can create a new datasource or overwrite data in an existing datasource. In Druid SQL, unlike
`REPLACE` statements can create a new datasource or overwrite data in an existing datasource. In Druid SQL, unlike
standard SQL, there is no syntactical difference between creating a table and overwriting data in a table. Druid does
not include a `CREATE TABLE` statement.
REPLACE uses an [OVERWRITE clause](reference.md#replace-specific-time-ranges) to determine which data to overwrite. You
`REPLACE` uses an [OVERWRITE clause](reference.md#replace-specific-time-ranges) to determine which data to overwrite. You
can overwrite an entire table, or a specific time range of a table. When you overwrite a specific time range, that time
range must align with the granularity specified in the PARTITIONED BY clause.
range must align with the granularity specified in the `PARTITIONED BY` clause.
REPLACE statements acquire an exclusive write lock to the target time range of the target datasource. No other ingestion
`REPLACE` statements acquire an exclusive write lock to the target time range of the target datasource. No other ingestion
or compaction operations may proceed for that time range while the task is running. However, ingestion and compaction
operations may proceed for other time ranges.
Nearly all SELECT capabilities are available for `REPLACE ... SELECT` queries. Certain exceptions are listed on the [Known
issues](./known-issues.md#select) page.
Nearly all `SELECT` capabilities are available for `REPLACE ... SELECT` queries. Certain exceptions are listed on the [Known
issues](./known-issues.md#select-statement) page.
For more information about the syntax, see [REPLACE](./reference.md#replace).
When deciding whether to use REPLACE or INSERT, keep in mind that segments generated with REPLACE can be pruned with dimension-based pruning but those generated with INSERT cannot. For more information about the requirements for dimension-based pruning, see [Clustering](#clustering).
When deciding whether to use `REPLACE` or `INSERT`, keep in mind that segments generated with `REPLACE` can be pruned
with dimension-based pruning but those generated with `INSERT` cannot. For more information about the requirements
for dimension-based pruning, see [Clustering](#clustering).
### Primary timestamp
@ -116,7 +123,7 @@ HH:mm:ss') AS __time`.
The `__time` column is used for [partitioning by time](#partitioning-by-time). If you use `PARTITIONED BY ALL` or
`PARTITIONED BY ALL TIME`, partitioning by time is disabled. In these cases, you do not need to include a `__time`
column in your INSERT statement. However, Druid still creates a `__time` column in your Druid table and sets all
column in your `INSERT` statement. However, Druid still creates a `__time` column in your Druid table and sets all
timestamps to 1970-01-01 00:00:00.
For more information, see [Primary timestamp](../ingestion/data-model.md#primary-timestamp).
@ -125,7 +132,7 @@ For more information, see [Primary timestamp](../ingestion/data-model.md#primary
### Partitioning by time
INSERT and REPLACE statements require the PARTITIONED BY clause, which determines how time-based partitioning is done.
`INSERT` and `REPLACE` statements require the `PARTITIONED BY` clause, which determines how time-based partitioning is done.
In Druid, data is split into one or more segments per time chunk, defined by the PARTITIONED BY granularity.
Partitioning by time is important for three reasons:
@ -160,8 +167,8 @@ Clustering is important for two reasons:
To activate dimension-based pruning, these requirements must be met:
- Segments were generated by a REPLACE statement, not an INSERT statement.
- All CLUSTERED BY columns are single-valued string columns.
- Segments were generated by a `REPLACE` statement, not an `INSERT` statement.
- All `CLUSTERED BY` columns are single-valued string columns.
If these requirements are _not_ met, Druid still clusters data during ingestion but will not be able to perform
dimension-based segment pruning at query time. You can tell if dimension-based segment pruning is possible by using the
@ -169,7 +176,7 @@ dimension-based segment pruning at query time. You can tell if dimension-based s
`range` or `single`, then dimension-based segment pruning is possible. Otherwise, it is not. The shard spec type is also
available in the **Segments** view under the **Partitioning** column.
For more information about syntax, see [CLUSTERED BY](./reference.md#clustered-by).
For more information about syntax, see [`CLUSTERED BY`](./reference.md#clustered-by).
### Rollup
@ -179,14 +186,14 @@ This reduces storage footprint and improves performance, often dramatically.
To perform ingestion with rollup:
1. Use GROUP BY. The columns in the GROUP BY clause become dimensions, and aggregation functions become metrics.
1. Use `GROUP BY`. The columns in the `GROUP BY` clause become dimensions, and aggregation functions become metrics.
2. Set [`finalizeAggregations: false`](reference.md#context-parameters) in your context. This causes aggregation
functions to write their internal state to the generated segments, instead of the finalized end result, and enables
further aggregation at query time.
3. Wrap all multi-value strings in `MV_TO_ARRAY(...)` and set [`groupByEnableMultiValueUnnesting:
false`](reference.md#context-parameters) in your context. This ensures that multi-value strings are left alone and
remain lists, instead of being [automatically unnested](../querying/sql-data-types.md#multi-value-strings) by the
GROUP BY operator.
`GROUP BY` operator.
When you do all of these things, Druid understands that you intend to do an ingestion with rollup, and it writes
rollup-related metadata into the generated segments. Other applications can then use [`segmentMetadata`
@ -196,8 +203,8 @@ If you see the error "Encountered multi-value dimension `x` that cannot be proce
groupByEnableMultiValueUnnesting set to false", then wrap that column in `MV_TO_ARRAY(x) AS x`.
The following [aggregation functions](../querying/sql-aggregations.md) are supported for rollup at ingestion time:
`COUNT` (but switch to `SUM` at query time), `SUM`, `MIN`, `MAX`, `EARLIEST` ([string only](known-issues.md#select)),
`LATEST` ([string only](known-issues.md#select)), `APPROX_COUNT_DISTINCT`, `APPROX_COUNT_DISTINCT_BUILTIN`,
`COUNT` (but switch to `SUM` at query time), `SUM`, `MIN`, `MAX`, `EARLIEST` ([string only](known-issues.md#select-statement)),
`LATEST` ([string only](known-issues.md#select-statement)), `APPROX_COUNT_DISTINCT`, `APPROX_COUNT_DISTINCT_BUILTIN`,
`APPROX_COUNT_DISTINCT_DS_HLL`, `APPROX_COUNT_DISTINCT_DS_THETA`, and `DS_QUANTILES_SKETCH` (but switch to
`APPROX_QUANTILE_DS` at query time). Do not use `AVG`; instead, use `SUM` and `COUNT` at ingest time and compute the
quotient at query time.
@ -214,19 +221,19 @@ happens:
1. The Broker plans your SQL query into a native query, as usual.
2. The Broker wraps the native query into a task of type `query_controller`
and submits it to the indexing service.
and submits it to the indexing service.
3. The Broker returns the task ID to you and exits.
4. The controller task launches some number of worker tasks determined by
the `maxNumTasks` and `taskAssignment` [context parameters](./reference.md#context-parameters). You can set these settings individually for each query.
the `maxNumTasks` and `taskAssignment` [context parameters](./reference.md#context-parameters). You can set these settings individually for each query.
5. Worker tasks of type `query_worker` execute the query.
6. If the query is a SELECT query, the worker tasks send the results
back to the controller task, which writes them into its task report.
If the query is an INSERT or REPLACE query, the worker tasks generate and
publish new Druid segments to the provided datasource.
6. If the query is a `SELECT` query, the worker tasks send the results
back to the controller task, which writes them into its task report.
If the query is an INSERT or REPLACE query, the worker tasks generate and
publish new Druid segments to the provided datasource.
### Parallelism

View File

@ -31,10 +31,10 @@ description: Introduces multi-stage query architecture and its task engine
Apache Druid supports SQL-based ingestion using the bundled [`druid-multi-stage-query` extension](#load-the-extension).
This extension adds a [multi-stage query task engine for SQL](concepts.md#multi-stage-query-task-engine) that allows running SQL
[INSERT](concepts.md#insert) and [REPLACE](concepts.md#replace) statements as batch tasks. As an experimental feature,
the task engine also supports running SELECT queries as batch tasks.
the task engine also supports running `SELECT` queries as batch tasks.
Nearly all SELECT capabilities are available in the multi-stage query (MSQ) task engine, with certain exceptions listed on the [Known
issues](./known-issues.md#select) page. This allows great flexibility to apply transformations, filters, JOINs,
Nearly all `SELECT` capabilities are available in the multi-stage query (MSQ) task engine, with certain exceptions listed on the [Known
issues](./known-issues.md#select-statement) page. This allows great flexibility to apply transformations, filters, JOINs,
aggregations, and so on as part of `INSERT ... SELECT` and `REPLACE ... SELECT` statements. This also allows in-database
transformation: creating new tables based on queries of other tables.
@ -64,8 +64,9 @@ To add the extension to an existing cluster, add `druid-multi-stage-query` to `d
For more information about how to load an extension, see [Loading extensions](../development/extensions.md#loading-extensions).
To use [EXTERN](reference.md#extern), you need READ permission on the resource named "EXTERNAL" of the resource type
"EXTERNAL". If you encounter a 403 error when trying to use EXTERN, verify that you have the correct permissions.
To use [EXTERN](reference.md#extern-function), you need READ permission on the resource named "EXTERNAL" of the resource type
"EXTERNAL". If you encounter a 403 error when trying to use `EXTERN`, verify that you have the correct permissions.
The same is true of any of the input-source specific table function such as `S3` or `LOCALFILES`.
## Next steps

View File

@ -35,39 +35,39 @@ sidebar_label: Known issues
generate a large amount of output data may exhaust all available disk space. In this case, the query fails with
an [UnknownError](./reference.md#error_UnknownError) with a message including "No space left on device".
## SELECT
## `SELECT` Statement
- SELECT from a Druid datasource does not include unpublished real-time data.
- `SELECT` from a Druid datasource does not include unpublished real-time data.
- GROUPING SETS and UNION ALL are not implemented. Queries using these features return a
- `GROUPING SETS` and `UNION ALL` are not implemented. Queries using these features return a
[QueryNotSupported](reference.md#error_QueryNotSupported) error.
- For some COUNT DISTINCT queries, you'll encounter a [QueryNotSupported](reference.md#error_QueryNotSupported) error
- For some `COUNT DISTINCT` queries, you'll encounter a [QueryNotSupported](reference.md#error_QueryNotSupported) error
that includes `Must not have 'subtotalsSpec'` as one of its causes. This is caused by the planner attempting to use
GROUPING SETs, which are not implemented.
`GROUPING SET`s, which are not implemented.
- The numeric varieties of the EARLIEST and LATEST aggregators do not work properly. Attempting to use the numeric
- The numeric varieties of the `EARLIEST` and `LATEST` aggregators do not work properly. Attempting to use the numeric
varieties of these aggregators lead to an error like
`java.lang.ClassCastException: class java.lang.Double cannot be cast to class org.apache.druid.collections.SerializablePair`.
The string varieties, however, do work properly.
## INSERT and REPLACE
## `INSERT` and `REPLACE` Statements
- INSERT and REPLACE with column lists, like `INSERT INTO tbl (a, b, c) SELECT ...`, is not implemented.
- The `INSERT` and `REPLACE` statements with column lists, like `INSERT INTO tbl (a, b, c) SELECT ...`, is not implemented.
- `INSERT ... SELECT` and `REPLACE ... SELECT` insert columns from the SELECT statement based on column name. This
- `INSERT ... SELECT` and `REPLACE ... SELECT` insert columns from the `SELECT` statement based on column name. This
differs from SQL standard behavior, where columns are inserted based on position.
- INSERT and REPLACE do not support all options available in [ingestion specs](../ingestion/ingestion-spec.md),
- `INSERT` and `REPLACE` do not support all options available in [ingestion specs](../ingestion/ingestion-spec.md),
including the `createBitmapIndex` and `multiValueHandling` [dimension](../ingestion/ingestion-spec.md#dimension-objects)
properties, and the `indexSpec` [`tuningConfig`](../ingestion/ingestion-spec.md#tuningconfig) property.
## EXTERN
## `EXTERN` Function
- The [schemaless dimensions](../ingestion/ingestion-spec.md#inclusions-and-exclusions)
feature is not available. All columns and their types must be specified explicitly using the `signature` parameter
of the [EXTERN function](reference.md#extern).
of the [`EXTERN` function](reference.md#extern-function).
- EXTERN with input sources that match large numbers of files may exhaust available memory on the controller task.
- `EXTERN` with input sources that match large numbers of files may exhaust available memory on the controller task.
- EXTERN does not accept `druid` input sources. Use FROM instead.
- `EXTERN` refers to external files. Use `FROM` to access `druid` input sources.

View File

@ -32,11 +32,20 @@ sidebar_label: Reference
This topic is a reference guide for the multi-stage query architecture in Apache Druid. For examples of real-world
usage, refer to the [Examples](examples.md) page.
### `EXTERN`
`INSERT` and `REPLACE` load data into a Druid datasource from either an external input source, or from another
datasource. When loading from an external datasource, you typically must provide the kind of input source,
the data format, and the schema (signature) of the input file. Druid provides *table functions* to allow you to
specify the external file. There are two kinds. `EXTERN` works with the JSON-serialized specs for the three
items, using the same JSON you would use in native ingest. A set of other, input-source-specific functions
use SQL syntax to specify the format and the input schema. There is one function for each input source. The
input-source-specific functions allow you to use SQL query parameters to specify the set of files (or URIs),
making it easy to reuse the same SQL statement for each ingest: just specify the set of files to use each time.
Use the `EXTERN` function to read external data.
### `EXTERN` Function
Function format:
Use the `EXTERN` function to read external data. The function has two variations.
Function variation 1, with the input schema expressed as JSON:
```sql
SELECT
@ -54,16 +63,41 @@ FROM TABLE(
1. Any [Druid input source](../ingestion/native-batch-input-source.md) as a JSON-encoded string.
2. Any [Druid input format](../ingestion/data-formats.md) as a JSON-encoded string.
3. A row signature, as a JSON-encoded array of column descriptors. Each column descriptor must have a `name` and a `type`. The type can be `string`, `long`, `double`, or `float`. This row signature is used to map the external data into the SQL layer.
3. A row signature, as a JSON-encoded array of column descriptors. Each column descriptor must have a
`name` and a `type`. The type can be `string`, `long`, `double`, or `float`. This row signature is
used to map the external data into the SQL layer.
Variation 2, with the input schema expressed in SQL using an `EXTEND` clause. (See the next
section for more detail on `EXTEND`). This format also uses named arguments to make the
SQL a bit easier to read:
```sql
SELECT
<column>
FROM TABLE(
EXTERN(
inputSource => '<Druid input source>',
inputFormat => '<Druid input format>'
) (<columns>)
)
```
The input source and format are as above. The columns are expressed as in a SQL `CREATE TABLE`.
Example: `(timestamp VARCHAR, metricType VARCHAR, value BIGINT)`. The optional `EXTEND` keyword
can precede the column list: `EXTEND (timestamp VARCHAR...)`.
For more information, see [Read external data with EXTERN](concepts.md#extern).
### `HTTP`, `INLINE` and `LOCALFILES`
### `HTTP`, `INLINE`, `LOCALFILES` and `S3` Functions
While `EXTERN` allows you to specify an external table using JSON, other table functions allow you
describe the external table using SQL syntax. Each function works for one specific kind of input
source. You provide properties using SQL named arguments. The row signature is given using the
Druid SQL `EXTEND` keyword using SQL syntax and types. Function format:
Druid SQL `EXTEND` keyword using SQL syntax and types.
The set of table functions and formats is preliminary in this release.
Function format:
```sql
SELECT
@ -72,12 +106,19 @@ FROM TABLE(
http(
userName => 'bob',
password => 'secret',
uris => 'http:foo.com/bar.csv',
uris => ARRAY['http:example.com/foo.csv', 'http:example.com/bar.csv'],
format => 'csv'
)
) EXTEND (x VARCHAR, y VARCHAR, z BIGINT)
```
For each function, you provide:
* The function name indicates the kind of input source: `http`, `inline` or `localfiles`.
* The function arguments correspond to a subset of the JSON fields for that input source.
* A `format` argument to indicate the desired input format.
* Additional arguments required for the selected format type.
Note that the `EXTEND` keyword is optional. The following is equally valid (and perhaps
more convenient):
@ -88,93 +129,323 @@ FROM TABLE(
http(
userName => 'bob',
password => 'secret',
uris => 'http:foo.com/bar.csv',
uris => ARRAY['http:example.com/foo.csv', 'http:example.com/bar.csv'],
format => 'csv'
)
) (x VARCHAR, y VARCHAR, z BIGINT)
```
#### Function Arguments
The set of table functions and formats is preliminary in this release.
These table functions are intended for use with the SQL by-name argument syntax
as shown above. Because the functions include all parameters for all formats,
using positional calls is both cumbersome and error-prone.
#### `HTTP`
Function argument names are generally the same as the JSON field names, except
as noted below. Each argument has a SQL type which matches the JSON type. For
arguments that take a string list in JSON, use the SQL `ARRAY[...]` syntax in
SQL as shown in the above example.
The `HTTP` table function represents the `HttpInputSource` class in Druid which allows you to
read from an HTTP server. The function accepts the following arguments:
Array parameters are good candidates for use in parameterized queries. That is:
| Name | Description | JSON equivalent | Required |
| ---- | ----------- | --------------- | -------- |
| `userName` | Basic authentication user name | `httpAuthenticationUsername` | No |
| `password` | Basic authentication password | `httpAuthenticationPassword` | No |
| `passwordEnvVar` | Environment variable that contains the basic authentication password| `httpAuthenticationPassword` | No |
| `uris` | Comma-separated list of URIs to read. | `uris` | Yes |
```sql
SELECT
<column>
FROM TABLE(
http(
userName => 'bob',
password => 'secret',
uris => ?,
format => 'csv'
)
) (x VARCHAR, y VARCHAR, z BIGINT)
```
#### `INLINE`
Provide the list of URIs (in this case) as a query parameter in each ingest. Doing
so is simpler than writing a script to insert the array into the SQL text.
The `INLINE` table function represents the `InlineInputSource` class in Druid which provides
data directly in the table function. The function accepts the following arguments:
#### `HTTP` Function
| Name | Description | JSON equivalent | Required |
| ---- | ----------- | --------------- | -------- |
| `data` | Text lines of inline data. Separate lines with a newline. | `data` | Yes |
The `HTTP` table function represents the
[HTTP input source](../ingestion/native-batch-input-source.md#http-input-source)
to read from an HTTP server. The function accepts the following arguments:
#### `LOCALFILES`
* `userName` (`VARCHAR`) - Same as JSON `httpAuthenticationUsername`.
* `password` (`VARCHAR`) - Same as`httpAuthenticationPassword` when used with the default option.
* `passwordEnvVar` (`VARCHAR`) - Same as the HTTP `httpAuthenticationPassword` when used with
the `"type": "environment"` option.
* `uris` (`ARRAY` of `VARCHAR`)
The `LOCALFILES` table function represents the `LocalInputSource` class in Druid which reads
#### `INLINE` Function
The `INLINE` table function represents the
[Inline input source](../ingestion/native-batch-input-source.md#inline-input-source)
which provides data directly in the table function. Parameter:
* `data` (`ARRAY` of `VARCHAR`) - Data lines, without a trailing newline, as an array.
Example:
```sql
SELECT ...
FROM TABLE(
inline(
data => ARRAY[
'a,b',
'c,d'],
format => 'csv'
)
) (x VARCHAR, y VARCHAR)
```
#### `LOCALFILES` Function
The `LOCALFILES` table function represents the
[Local input source](../ingestion/native-batch-input-source.md#local-input-source) which reads
files from the file system of the node running Druid. This is most useful for single-node
installations. The function accepts the following arguments:
installations. The function accepts the following parameters:
| Name | Description | JSON equivalent | Required |
| ---- | ----------- | --------------- | -------- |
| `baseDir` | Directory to read from. | `baseDir` | No |
| `filter` | Filter pattern to read. Example: `*.csv`. | `filter` | No |
| `files` | Comma-separated list of files to read. | `files` | No |
* `baseDir`
* `filter`
* `files`
You must either provide the `baseDir` or the list of `files`. You can provide both, in which case
the files are assumed relative to the `baseDir`. If you provide a `filter`, you must provide the
`baseDir`.
When the local files input source is used directly in an `extern` function, or ingestion spec, you
can provide either `baseDir` and `filter` or `files` but not both. This function, however, allows
you to provide any of the following combinations:
#### Table Function Format
* `baseDir` - Matches all files in the given directory. (Assumes the filter is `*`.)
* `baseDir` and `filter` - Match files in the given directory using the filter.
* `baseDir` and `files` - A set of files relative to `baseDir`.
* `files` - The files should be absolute paths, else they will be computed relative to Druid's
working directory (usually the Druid install directory.)
Each of the table functions above requires that you specify a format.
Examples:
| Name | Description | JSON equivalent | Required |
| ---- | ----------- | --------------- | -------- |
| `format` | The input format, using the same names as for `EXTERN`. | `inputFormat.type` | Yes |
To read All files in /tmp, which must be CSV files:
```sql
SELECT ...
FROM TABLE(
localfiles(
baseDir => '/tmp',
format => 'csv')
) (x VARCHAR, y VARCHAR)
```
Some additional variations (omitting the common bits):
```sql
-- CSV files in /tmp
localfiles(baseDir => '/tmp',
filter => '*.csv',
format => 'csv')
-- /tmp/a.csv and /tmp/b.csv
localfiles(baseDir => '/tmp',
files => ARRAY['a.csv', 'b.csv'],
format => 'csv')
-- /tmp/a.csv and /tmp/b.csv
localfiles(files => ARRAY['/tmp/a.csv', '/tmp/b.csv'],
format => 'csv')
```
#### `S3` Function
The `S3` table function represents the
[S3 input source](../ingestion/native-batch-input-source.md#s3-input-source) which reads
files from an S3 bucket. The function accepts the following parameters to specify the
objects to read:
* `uris` (`ARRAY` of `VARCHAR`)
* `prefix` (`VARCHAR`) - Corresponds to the JSON `prefixes` property, but allows a single
prefix.
* `bucket` (`VARCHAR`) - Corresponds to the `bucket` field of the `objects` JSON field. SQL
does not have syntax for an array of objects. Instead, this function takes a single bucket,
and one or more objects within that bucket.
* `paths` (`ARRAY` of `VARCHAR`) - Corresponds to the `path` fields of the `object` JSON field.
All paths are within the single `bucket` parameter.
The S3 input source accepts one of the following patterns:
* `uris` - A list of fully-qualified object URIs.
* `prefixes` - A list of fully-qualified "folder" prefixes.
* `bucket` and `paths` - A list of objects relative to the given bucket path.
The `S3` function also accepts the following security parameters:
* `accessKeyId` (`VARCHAR`)
* `secretAccessKey` (`VARCHAR`)
* `assumeRoleArn` (`VARCHAR`)
The `S3` table function does not support either the `clientConfig` or `proxyConfig`
JSON properties.
If you need the full power of the S3 input source, then consider the use of the `extern`
function, which accepts the full S3 input source serialized as JSON. Alternatively,
create a catalog external table that has the full set of properties, leaving just the
`uris` or `paths` to be provided at query time.
Examples, each of which correspond to an example on the
[S3 input source](../ingestion/native-batch-input-source.md#s3-input-source) page.
The examples omit the format and schema; however you must remember to provide those
in an actual query.
```sql
SELECT ...
FROM TABLE(S3(
uris => ARRAY['s3://foo/bar/file.json', 's3://bar/foo/file2.json'],
format => 'csv'))
) (x VARCHAR, y VARCHAR)
```
Additional variations, omitting the common bits:
```sql
S3(prefixes => ARRAY['s3://foo/bar/', 's3://bar/foo/']))
```
```sql
-- Not an exact match for the JSON example: the S3 function allows
-- only one bucket.
S3(bucket => 's3://foo`,
paths => ARRAY['bar/file1.json', 'foo/file2.json'])
```
```sql
S3(uris => ARRAY['s3://foo/bar/file.json', 's3://bar/foo/file2.json'],
accessKeyId => 'KLJ78979SDFdS2',
secretAccessKey => 'KLS89s98sKJHKJKJH8721lljkd')
```
```sql
S3(uris => ARRAY['s3://foo/bar/file.json', 's3://bar/foo/file2.json'],
accessKeyId => 'KLJ78979SDFdS2',
secretAccessKey => 'KLS89s98sKJHKJKJH8721lljkd',
assumeRoleArn => 'arn:aws:iam::2981002874992:role/role-s3')
```
#### Input Format
Each of the table functions above requires that you specify a format using the `format`
parameter which accepts a value the same as the format names used for `EXTERN` and described
for [each input source](../ingestion/native-batch-input-source.md).
#### CSV Format
Use the `csv` format to read from CSV. This choice selects the Druid `CsvInputFormat` class.
The `csv` format selects the [CSV input format](../ingestion/data-formats.md#csv).
Parameters:
| Name | Description | JSON equivalent | Required |
| ---- | ----------- | --------------- | -------- |
| `listDelimiter` | The delimiter to use for fields that represent a list of strings. | `listDelimiter` | No |
| `skipRows` | The number of rows to skip at the start of the file. Default is 0. | `skipHeaderRows` | No |
* `listDelimiter` (`VARCHAR`)
* `skipHeaderRows` (`BOOLEAN`)
MSQ does not have the ability to infer schema from a CSV, file, so the `findColumnsFromHeader` property
is unavailable. Instead, Columns are given using the `EXTEND` syntax described above.
Example for a CSV format with a list delimiter and where we want to skip the first
input row:
```sql
SELECT ...
FROM TABLE(
inline(
data => ARRAY[
'skip me',
'a;foo,b',
'c;bar,d'],
format => 'csv',
listDelimiter => ';',
skipHeaderRows => 1
)
) (x VARCHAR, y VARCHAR)
```
#### Delimited Text Format
Use the `tsv` format to read from an arbitrary delimited (CSV-like) file such as tab-delimited,
pipe-delimited, etc. This choice selects the Druid `DelimitedInputFormat` class.
The `tsv` format selects the [TSV (Delimited) input format](../ingestion/data-formats.md#tsv-delimited).
Parameters:
| Name | Description | JSON equivalent | Required |
| ---- | ----------- | --------------- | -------- |
| `delimiter` | The delimiter which separates fields. | `delimiter` | Yes |
| `listDelimiter` | The delimiter to use for fields that represent a list of strings. | `listDelimiter` | No |
| `skipRows` | The number of rows to skip at the start of the file. Default is 0. | `skipHeaderRows` | No |
* `delimiter` (`VARCHAR`)
* `listDelimiter` (`VARCHAR`)
* `skipHeaderRows` (`BOOLEAN`)
As noted above, MSQ cannot infer schema using headers. Use `EXTEND` instead.
Example for a pipe-separated format with a list delimiter and where we want to skip the first
input row:
```sql
SELECT ...
FROM TABLE(
inline(
data => ARRAY[
'skip me',
'a;foo|b',
'c;bar|d'],
format => 'tsv',
listDelimiter => ';',
skipHeaderRows => 1,
delimiter => '|'
)
) (x VARCHAR, y VARCHAR)
```
#### JSON Format
Use the `json` format to read from a JSON input source. This choice selects the Druid `JsonInputFormat` class.
The `json` format selects the
[JSON input format](../ingestion/data-formats.html#json).
The JSON format accepts no additional parameters.
| Name | Description | JSON equivalent | Required |
| ---- | ----------- | --------------- | -------- |
| `keepNulls` | Whether to keep null values. Defaults to `false`. | `keepNullColumns` | No |
Example:
```sql
SELECT ...
FROM TABLE(
inline(
data => ARRAY['{"x": "foo", "y": "bar"}'],
format => 'json')
) (x VARCHAR, y VARCHAR)
```
### Parameters
Starting with the Druid 26.0 release, you can use query parameters with MSQ queries. You may find
that you periodically ingest a new set of files into Druid. Often, the bulk of the query is identical
for each ingestion: only the list of files (or URIs or objects) changes. For example, for the `S3`
input source, you will likely ingest from the same bucket and security setup in
each query; only the specific objects will change. Consider using a query parameter
to pass the object names:
```sql
INSERT INTO ...
SELECT ...
FROM TABLE(S3(bucket => 's3://foo`,
accessKeyId => ?,
paths => ?,
format => JSON))
(a VARCHAR, b BIGINT, ...)
```
This same technique can be used with the `uris` or `prefixes` parameters instead.
Function arguments that take an array parameter require an array function in your JSON request.
For example:
```json
{
"query" : "INSERT INTO ...
SELECT ...
FROM TABLE(S3(bucket => 's3://foo`,
accessKeyId => ?,
paths => ?,
format => JSON))
(a VARCHAR, b BIGINT, ...)",
"parameters": [
{ "type": "VARCHAR", "value": "ABCD-EF01"},
{ "type": "VARCHAR", "value": [
"foo.csv", "bar.csv"
] }
]
}
```
The type in the above example is the type of each element. It must be `VARCHAR` for all the array
parameters for functions described on this page.
### `INSERT`
@ -197,7 +468,8 @@ INSERT consists of the following parts:
1. Optional [context parameters](./reference.md#context-parameters).
2. An `INSERT INTO <dataSource>` clause at the start of your query, such as `INSERT INTO your-table`.
3. A clause for the data you want to insert, such as `SELECT ... FROM ...`. You can use [EXTERN](#extern) to reference external tables using `FROM TABLE(EXTERN(...))`.
3. A clause for the data you want to insert, such as `SELECT ... FROM ...`. You can use [`EXTERN`](#extern-function)
to reference external tables using `FROM TABLE(EXTERN(...))`.
4. A [PARTITIONED BY](#partitioned-by) clause, such as `PARTITIONED BY DAY`.
5. An optional [CLUSTERED BY](#clustered-by) clause.
@ -257,14 +529,18 @@ The `PARTITIONED BY <time granularity>` clause is required for [INSERT](#insert)
The following granularity arguments are accepted:
- Time unit: `HOUR`, `DAY`, `MONTH`, or `YEAR`. Equivalent to `FLOOR(__time TO TimeUnit)`.
- Time unit keywords: `HOUR`, `DAY`, `MONTH`, or `YEAR`. Equivalent to `FLOOR(__time TO TimeUnit)`.
- Time units as ISO 8601 period strings: :`'PT1H'`, '`P1D`, etc. (Druid 26.0 and later.)
- `TIME_FLOOR(__time, 'granularity_string')`, where granularity_string is one of the ISO 8601 periods listed below. The
first argument must be `__time`.
- `FLOOR(__time TO TimeUnit)`, where `TimeUnit` is any unit supported by the [FLOOR function](../querying/sql-scalar.md#date-and-time-functions). The first argument must be `__time`.
- `ALL` or `ALL TIME`, which effectively disables time partitioning by placing all data in a single time chunk. To use
LIMIT or OFFSET at the outer level of your INSERT or REPLACE query, you must set PARTITIONED BY to ALL or ALL TIME.
LIMIT or OFFSET at the outer level of your `INSERT` or `REPLACE` query, you must set `PARTITIONED BY` to `ALL` or `ALL TIME`.
The following ISO 8601 periods are supported for `TIME_FLOOR`:
Earlier versions required the `TIME_FLOOR` notation to specify a granularity other than the keywords.
In the current version, the string constant provides a simpler equivalent solution.
The following ISO 8601 periods are supported for `TIME_FLOOR` and the string constant:
- PT1S
- PT1M

View File

@ -27,15 +27,18 @@ sidebar_label: Security
> extension, new in Druid 24.0. Refer to the [ingestion methods](../ingestion/index.md#batch) table to determine which
> ingestion method is right for you.
All authenticated users can use the multi-stage query task engine (MSQ task engine) through the UI and API if the extension is loaded. However, without additional permissions, users are not able to issue queries that read or write Druid datasources or external data. The permission needed depends on what the user is trying to do.
All authenticated users can use the multi-stage query task engine (MSQ task engine) through the UI and API if the
extension is loaded. However, without additional permissions, users are not able to issue queries that read or write
Druid datasources or external data. The permission needed depends on what the user is trying to do.
To submit a query:
- SELECT from a Druid datasource requires the READ DATASOURCE permission on that datasource.
- [INSERT](reference.md#insert) or [REPLACE](reference.md#replace) into a Druid datasource requires the WRITE DATASOURCE
permission on that datasource.
- [EXTERN](reference.md#extern) requires READ permission on a resource named "EXTERNAL" with type "EXTERNAL". Users without the correct
permission encounter a 403 error when trying to run queries that include EXTERN.
- [EXTERN](reference.md#extern-function) and the input-source-specific table functions require READ permission on a
resource named "EXTERNAL" with type "EXTERNAL". Users without the correct
permission encounter a 403 error when trying to run queries that include `EXTERN`.
Once a query is submitted, it executes as a [`query_controller`](concepts.md#execution-flow) task. Query tasks that
users submit to the MSQ task engine are Overlord tasks, so they follow the Overlord's security model. This means that
@ -45,5 +48,5 @@ SQL-based ingestion](./api.md).
To interact with a query through the Overlord API, users need the following permissions:
- INSERT or REPLACE queries: Users must have READ DATASOURCE permission on the output datasource.
- SELECT queries: Users must have read permissions on the `__query_select` datasource, which is a stub datasource that gets created.
- `INSERT` or `REPLACE` queries: Users must have READ DATASOURCE permission on the output datasource.
- `SELECT` queries: Users must have read permissions on the `__query_select` datasource, which is a stub datasource that gets created.

View File

@ -28,12 +28,13 @@ import org.apache.druid.catalog.http.TableEditRequest.MoveColumn.Position;
import org.apache.druid.catalog.http.TableEditRequest.UnhideColumns;
import org.apache.druid.catalog.http.TableEditRequest.UpdateColumns;
import org.apache.druid.catalog.http.TableEditRequest.UpdateProperties;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.TableDefn;
import org.apache.druid.catalog.model.TableId;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.TableSpec;
import org.apache.druid.catalog.model.table.AbstractDatasourceDefn;
import org.apache.druid.catalog.model.table.DatasourceDefn;
import org.apache.druid.catalog.storage.CatalogStorage;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.utils.CollectionUtils;
@ -129,7 +130,7 @@ public class TableEditor
*/
private TableSpec applyHiddenColumns(TableMetadata table, List<String> columns) throws CatalogException
{
if (!AbstractDatasourceDefn.isDatasource(table.spec().type())) {
if (!DatasourceDefn.isDatasource(table.spec().type())) {
throw CatalogException.badRequest("hideColumns is supported only for data source specs");
}
TableSpec spec = table.spec();
@ -138,7 +139,7 @@ public class TableEditor
}
Map<String, Object> props = spec.properties();
@SuppressWarnings("unchecked")
List<String> hiddenColumns = (List<String>) props.get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY);
List<String> hiddenColumns = (List<String>) props.get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY);
if (hiddenColumns == null) {
hiddenColumns = Collections.emptyList();
}
@ -155,7 +156,7 @@ public class TableEditor
return null;
}
Map<String, Object> revisedProps = new HashMap<>(props);
revisedProps.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, revised);
revisedProps.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, revised);
return spec.withProperties(revisedProps);
}
@ -180,13 +181,13 @@ public class TableEditor
private TableSpec applyUnhideColumns(TableMetadata table, List<String> columns) throws CatalogException
{
final TableSpec existingSpec = table.spec();
if (!AbstractDatasourceDefn.isDatasource(existingSpec.type())) {
if (!DatasourceDefn.isDatasource(existingSpec.type())) {
throw CatalogException.badRequest("hideColumns is supported only for data source specs");
}
final Map<String, Object> props = existingSpec.properties();
@SuppressWarnings("unchecked")
List<String> hiddenColumns = (List<String>) props.get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY);
List<String> hiddenColumns = (List<String>) props.get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY);
if (hiddenColumns == null || columns.isEmpty()) {
return null;
}
@ -203,9 +204,9 @@ public class TableEditor
}
final Map<String, Object> revisedProps = new HashMap<>(props);
if (revisedHiddenCols.isEmpty()) {
revisedProps.remove(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY);
revisedProps.remove(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY);
} else {
revisedProps.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, revisedHiddenCols);
revisedProps.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, revisedHiddenCols);
}
return existingSpec.withProperties(revisedProps);
}
@ -276,7 +277,7 @@ public class TableEditor
private TableDefn resolveDefn(String tableType) throws CatalogException
{
TableDefn defn = catalog.tableRegistry().defnFor(tableType);
TableDefn defn = catalog.tableRegistry().tableDefnFor(tableType);
if (defn == null) {
throw new CatalogException(
CatalogException.BAD_STATE,
@ -309,7 +310,7 @@ public class TableEditor
final TableDefn defn = resolveDefn(existingSpec.type());
final List<ColumnSpec> revised = defn.mergeColumns(existingSpec.columns(), updates);
try {
defn.validateColumns(revised, catalog.jsonMapper());
defn.validateColumns(revised);
}
catch (IAE e) {
throw CatalogException.badRequest(e.getMessage());
@ -342,13 +343,13 @@ public class TableEditor
final TableSpec existingSpec = table.spec();
final List<ColumnSpec> existingCols = existingSpec.columns();
final List<ColumnSpec> revised = new ArrayList<>(existingCols);
final int colPosn = findColumn(existingCols, moveColumn.column);
final int colPosn = CatalogUtils.findColumn(existingCols, moveColumn.column);
if (colPosn == -1) {
throw CatalogException.badRequest("Column [%s] is not defined", moveColumn.column);
}
int anchorPosn;
if (moveColumn.where == Position.BEFORE || moveColumn.where == Position.AFTER) {
anchorPosn = findColumn(existingCols, moveColumn.anchor);
anchorPosn = CatalogUtils.findColumn(existingCols, moveColumn.anchor);
if (anchorPosn == -1) {
throw CatalogException.badRequest("Anchor [%s] is not defined", moveColumn.column);
}
@ -377,14 +378,4 @@ public class TableEditor
return existingSpec.withColumns(revised);
}
private static int findColumn(List<ColumnSpec> columns, String colName)
{
for (int i = 0; i < columns.size(); i++) {
if (columns.get(i).name().equals(colName)) {
return i;
}
}
return -1;
}
}

View File

@ -30,9 +30,7 @@ import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.TableId;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.TableSpec;
import org.apache.druid.catalog.model.table.AbstractDatasourceDefn;
import org.apache.druid.catalog.model.table.DatasourceDefn;
import org.apache.druid.catalog.model.table.DatasourceDefn.DatasourceColumnDefn;
import org.apache.druid.catalog.storage.sql.CatalogManager;
import org.apache.druid.catalog.storage.sql.SQLCatalogManager;
import org.apache.druid.jackson.DefaultObjectMapper;
@ -88,8 +86,8 @@ public class TableManagerTest
public void testCreate() throws DuplicateKeyException, NotFoundException
{
Map<String, Object> props = ImmutableMap.of(
AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D",
AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000
DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D",
DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000
);
TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null);
TableMetadata table = TableMetadata.newTable(TableId.datasource("table1"), spec);
@ -113,8 +111,8 @@ public class TableManagerTest
public void testUpdate() throws DuplicateKeyException, NotFoundException
{
Map<String, Object> props = ImmutableMap.of(
AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D",
AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000
DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D",
DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000
);
TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null);
TableMetadata table = TableMetadata.newTable(TableId.datasource("table1"), spec);
@ -122,8 +120,8 @@ public class TableManagerTest
// Change the definition
props = ImmutableMap.of(
AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D",
AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 2_000_000
DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D",
DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 2_000_000
);
TableSpec spec2 = spec.withProperties(props);
TableMetadata table2 = table.withSpec(spec2);
@ -161,8 +159,8 @@ public class TableManagerTest
public void testUpdateProperties() throws CatalogException
{
Map<String, Object> props = ImmutableMap.of(
AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D",
AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000
DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D",
DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000
);
TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null);
TableMetadata table = TableMetadata.newTable(TableId.datasource("table1"), spec);
@ -205,14 +203,14 @@ public class TableManagerTest
public void testUpdateColumns() throws CatalogException
{
Map<String, Object> props = ImmutableMap.of(
AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D",
AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000
DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D",
DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000
);
List<ColumnSpec> cols = Arrays.asList(
new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "a", Columns.VARCHAR, null),
new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "b", Columns.BIGINT, null)
new ColumnSpec("a", Columns.VARCHAR, null),
new ColumnSpec("b", Columns.BIGINT, null)
);
ColumnSpec colC = new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "c", Columns.DOUBLE, null);
ColumnSpec colC = new ColumnSpec("c", Columns.DOUBLE, null);
TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, cols);
TableMetadata table = TableMetadata.newTable(TableId.datasource("table1"), spec);
@ -255,8 +253,8 @@ public class TableManagerTest
public void testDelete() throws DuplicateKeyException, NotFoundException
{
Map<String, Object> props = ImmutableMap.of(
AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D",
AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000
DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D",
DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000
);
TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null);
TableMetadata table = TableMetadata.newTable(TableId.datasource("table1"), spec);
@ -274,8 +272,8 @@ public class TableManagerTest
assertTrue(list.isEmpty());
Map<String, Object> props = ImmutableMap.of(
AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H",
AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000
DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H",
DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000
);
TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null);

View File

@ -30,15 +30,15 @@ import org.apache.druid.catalog.model.TableId;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.TableSpec;
import org.apache.druid.catalog.model.facade.DatasourceFacade;
import org.apache.druid.catalog.model.table.BaseExternTableTest;
import org.apache.druid.catalog.model.table.DatasourceDefn;
import org.apache.druid.catalog.model.table.DatasourceDefn.DatasourceColumnDefn;
import org.apache.druid.catalog.model.table.ExternalTableDefn;
import org.apache.druid.catalog.model.table.InlineTableDefn;
import org.apache.druid.catalog.model.table.InputFormats;
import org.apache.druid.catalog.model.table.TableBuilder;
import org.apache.druid.catalog.storage.CatalogStorage;
import org.apache.druid.catalog.storage.CatalogTests;
import org.apache.druid.data.input.impl.InlineInputSource;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.metadata.TestDerbyConnector;
import org.junit.After;
import org.junit.Before;
@ -93,9 +93,9 @@ public class CatalogSyncTest
{
// Valid definition
{
TableMetadata table = TableBuilder.external(InlineTableDefn.TABLE_TYPE, "externTable")
.format(InputFormats.CSV_FORMAT_TYPE)
.data("a", "c")
TableMetadata table = TableBuilder.external("externTable")
.inputSource(toMap(new InlineInputSource("a\nc")))
.inputFormat(BaseExternTableTest.CSV_FORMAT)
.column("a", Columns.VARCHAR)
.build();
storage.validate(table);
@ -103,23 +103,33 @@ public class CatalogSyncTest
// No columns
{
TableMetadata table = TableBuilder.external(InlineTableDefn.TABLE_TYPE, "externTable")
.format(InputFormats.CSV_FORMAT_TYPE)
.data("a", "c")
TableMetadata table = TableBuilder.external("externTable")
.inputSource(toMap(new InlineInputSource("a\nc")))
.inputFormat(BaseExternTableTest.CSV_FORMAT)
.build();
assertThrows(IAE.class, () -> storage.validate(table));
}
// No format
{
TableMetadata table = TableBuilder.external(InlineTableDefn.TABLE_TYPE, "externTable")
.data("a", "c")
.column("a", Columns.VARCHAR)
TableMetadata table = TableBuilder.external("externTable")
.inputSource(toMap(new InlineInputSource("a\nc")))
.column("a", Columns.VARCHAR)
.build();
assertThrows(IAE.class, () -> storage.validate(table));
}
}
private Map<String, Object> toMap(Object obj)
{
try {
return jsonMapper.convertValue(obj, ExternalTableDefn.MAP_TYPE_REF);
}
catch (Exception e) {
throw new ISE(e, "bad conversion");
}
}
@Test
public void testDirect() throws DuplicateKeyException, NotFoundException
{
@ -198,9 +208,9 @@ public class CatalogSyncTest
storage.validate(table2);
storage.tables().create(table2);
TableMetadata table3 = TableBuilder.external(InlineTableDefn.TABLE_TYPE, "table3")
.format(InputFormats.CSV_FORMAT_TYPE)
.data("a", "c")
TableMetadata table3 = TableBuilder.external("table3")
.inputFormat(BaseExternTableTest.CSV_FORMAT)
.inputSource(toMap(new InlineInputSource("a\nc")))
.column("a", Columns.VARCHAR)
.build();
storage.validate(table3);
@ -221,10 +231,8 @@ public class CatalogSyncTest
assertEquals(2, cols.size());
assertEquals(Columns.TIME_COLUMN, cols.get(0).name());
assertEquals(Columns.TIMESTAMP, cols.get(0).sqlType());
assertEquals(DatasourceColumnDefn.COLUMN_TYPE, cols.get(0).type());
assertEquals("a", cols.get(1).name());
assertEquals(Columns.VARCHAR, cols.get(1).sqlType());
assertEquals(DatasourceColumnDefn.COLUMN_TYPE, cols.get(0).type());
DatasourceFacade ds = new DatasourceFacade(catalog.resolveTable(id));
assertEquals("P1D", ds.segmentGranularityString());
@ -244,10 +252,8 @@ public class CatalogSyncTest
assertEquals(Columns.TIMESTAMP, cols.get(0).sqlType());
assertEquals("dim", cols.get(1).name());
assertEquals(Columns.VARCHAR, cols.get(1).sqlType());
assertEquals(DatasourceColumnDefn.COLUMN_TYPE, cols.get(1).type());
assertEquals("measure", cols.get(2).name());
assertEquals("BIGINT", cols.get(2).sqlType());
assertEquals(DatasourceColumnDefn.COLUMN_TYPE, cols.get(2).type());
DatasourceFacade ds = new DatasourceFacade(catalog.resolveTable(id));
assertEquals("P1D", ds.segmentGranularityString());
@ -263,12 +269,11 @@ public class CatalogSyncTest
assertTrue(table.updateTime() > 0);
TableSpec inputSpec = table.spec();
assertEquals(InlineTableDefn.TABLE_TYPE, inputSpec.type());
assertEquals(ExternalTableDefn.TABLE_TYPE, inputSpec.type());
List<ColumnSpec> cols = inputSpec.columns();
assertEquals(1, cols.size());
assertEquals("a", cols.get(0).name());
assertEquals(Columns.VARCHAR, cols.get(0).sqlType());
assertEquals(ExternalTableDefn.EXTERNAL_COLUMN_TYPE, cols.get(0).type());
assertNotNull(inputSpec.properties());
}
@ -316,7 +321,6 @@ public class CatalogSyncTest
assertEquals("a", cols.get(1).name());
assertEquals("b", cols.get(2).name());
assertEquals(Columns.DOUBLE, cols.get(2).sqlType());
assertEquals(DatasourceColumnDefn.COLUMN_TYPE, cols.get(2).type());
}
{
TableId id = TableId.datasource("table3");
@ -352,7 +356,7 @@ public class CatalogSyncTest
storage.tables().updateColumns(id, t -> {
TableSpec target = t.spec();
List<ColumnSpec> updated = new ArrayList<>(target.columns());
ColumnSpec colC = new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "c", Columns.DOUBLE, null);
ColumnSpec colC = new ColumnSpec("c", Columns.DOUBLE, null);
updated.add(colC);
return target.withColumns(updated);
});

View File

@ -30,12 +30,13 @@ import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.TableId;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.TableSpec;
import org.apache.druid.catalog.model.table.AbstractDatasourceDefn;
import org.apache.druid.catalog.model.table.BaseExternTableTest;
import org.apache.druid.catalog.model.table.DatasourceDefn;
import org.apache.druid.catalog.model.table.InlineTableDefn;
import org.apache.druid.catalog.model.table.InputFormats;
import org.apache.druid.catalog.model.table.ExternalTableDefn;
import org.apache.druid.catalog.model.table.TableBuilder;
import org.apache.druid.catalog.storage.CatalogTests;
import org.apache.druid.data.input.impl.InlineInputSource;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.metadata.TestDerbyConnector;
import org.apache.druid.server.mocks.MockHttpServletRequest;
import org.apache.druid.server.security.AuthConfig;
@ -47,6 +48,7 @@ import org.junit.Rule;
import org.junit.Test;
import javax.ws.rs.core.Response;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
@ -90,7 +92,7 @@ public class CatalogResourceTest
{
@SuppressWarnings("unchecked")
Map<String, Object> result = (Map<String, Object>) resp.getEntity();
return (Long) result.get("version");
return CatalogUtils.getLong(result, "version");
}
@Test
@ -145,9 +147,9 @@ public class CatalogResourceTest
assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus());
// Inline input source
TableSpec inputSpec = TableBuilder.external(InlineTableDefn.TABLE_TYPE, "inline")
.format(InputFormats.CSV_FORMAT_TYPE)
.data("a,b,1", "c,d,2")
TableSpec inputSpec = TableBuilder.external("inline")
.inputSource(toMap(new InlineInputSource("a,b,1\nc,d,2\n")))
.inputFormat(BaseExternTableTest.CSV_FORMAT)
.column("a", Columns.VARCHAR)
.column("b", Columns.VARCHAR)
.column("c", Columns.BIGINT)
@ -160,6 +162,16 @@ public class CatalogResourceTest
assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), resp.getStatus());
}
private Map<String, Object> toMap(Object obj)
{
try {
return dbFixture.storage.jsonMapper().convertValue(obj, ExternalTableDefn.MAP_TYPE_REF);
}
catch (Exception e) {
throw new ISE(e, "bad conversion");
}
}
@Test
public void testUpdate()
{
@ -611,7 +623,7 @@ public class CatalogResourceTest
resp = resource.getTable(TableId.DRUID_SCHEMA, tableName, postBy(CatalogTests.READER_USER));
TableMetadata read = (TableMetadata) resp.getEntity();
assertNull(read.spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY));
assertNull(read.spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY));
// Hide
cmd = new HideColumns(Arrays.asList("a", "b"));
@ -625,7 +637,7 @@ public class CatalogResourceTest
read = (TableMetadata) resp.getEntity();
assertEquals(
Arrays.asList("a", "b"),
read.spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
read.spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
);
assertTrue(read.updateTime() > version);
@ -638,7 +650,7 @@ public class CatalogResourceTest
read = (TableMetadata) resp.getEntity();
assertEquals(
Collections.singletonList("b"),
read.spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
read.spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
);
assertTrue(read.updateTime() > version);
}

View File

@ -34,8 +34,7 @@ import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.TableId;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.table.AbstractDatasourceDefn;
import org.apache.druid.catalog.model.table.DatasourceDefn.DatasourceColumnDefn;
import org.apache.druid.catalog.model.table.DatasourceDefn;
import org.apache.druid.catalog.model.table.TableBuilder;
import org.apache.druid.catalog.storage.CatalogStorage;
import org.apache.druid.catalog.storage.CatalogTests;
@ -168,21 +167,21 @@ public class EditorTest
cmd = new HideColumns(Arrays.asList("a", "b"));
assertEquals(
Arrays.asList("a", "b"),
doEdit(tableName, cmd).spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
);
// Hide starting from an empty list.
Map<String, Object> props = new HashMap<>(table.spec().properties());
props.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList());
props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList());
catalog.tables().replace(table.withProperties(props));
assertEquals(
Arrays.asList("a", "b"),
doEdit(tableName, cmd).spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
);
// Hide, but all are duplicates
props = new HashMap<>(table.spec().properties());
props.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b", "c"));
props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b", "c"));
catalog.tables().replace(table.withProperties(props));
cmd = new HideColumns(Arrays.asList("b", "c"));
assertEquals(0, new TableEditor(catalog, table.id(), cmd).go());
@ -191,17 +190,17 @@ public class EditorTest
cmd = new HideColumns(Arrays.asList("b", "d"));
assertEquals(
Arrays.asList("a", "b", "c", "d"),
doEdit(tableName, cmd).spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
);
// Hide with duplicates
props = new HashMap<>(table.spec().properties());
props.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b"));
props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b"));
catalog.tables().replace(table.withProperties(props));
cmd = new HideColumns(Arrays.asList("b", "d", "b", "d"));
assertEquals(
Arrays.asList("a", "b", "d"),
doEdit(tableName, cmd).spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
);
}
@ -227,24 +226,24 @@ public class EditorTest
// Unhide starting from an empty list.
Map<String, Object> props = new HashMap<>(table.spec().properties());
props.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList());
props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList());
catalog.tables().replace(table.withProperties(props));
assertNull(
doEdit(tableName, cmd).spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
);
// Unhide starting with an empty list with (non-existing) columns to unhide
props = new HashMap<>(table.spec().properties());
props.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList());
props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.emptyList());
catalog.tables().replace(table.withProperties(props));
cmd = new UnhideColumns(Collections.singletonList("a"));
assertNull(
doEdit(tableName, cmd).spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
);
// Unhide columns which are not actually hidden.
props = new HashMap<>(table.spec().properties());
props.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b", "c"));
props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b", "c"));
catalog.tables().replace(table.withProperties(props));
cmd = new UnhideColumns(Arrays.asList("d", "e"));
assertEquals(0, new TableEditor(catalog, table.id(), cmd).go());
@ -253,16 +252,16 @@ public class EditorTest
cmd = new UnhideColumns(Arrays.asList("a", "c", "a", "d"));
assertEquals(
Collections.singletonList("b"),
doEdit(tableName, cmd).spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
);
// Unhide all hidden columns
props = new HashMap<>(table.spec().properties());
props.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b", "c"));
props.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("a", "b", "c"));
catalog.tables().replace(table.withProperties(props));
cmd = new UnhideColumns(Arrays.asList("a", "c", "b", "d"));
assertNull(
doEdit(tableName, cmd).spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
doEdit(tableName, cmd).spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
);
}
@ -329,7 +328,7 @@ public class EditorTest
// Remove a required property
Map<String, Object> updates1 = new HashMap<>();
updates1.put(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, null);
updates1.put(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, null);
assertThrows(
CatalogException.class,
() -> new TableEditor(
@ -342,11 +341,11 @@ public class EditorTest
// Add and update properties
Map<String, Object> updates = new HashMap<>();
updates.put(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H");
updates.put(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H");
updates.put("foo", "bar");
cmd = new UpdateProperties(updates);
Map<String, Object> expected = ImmutableMap.of(
AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H",
DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H",
"foo", "bar"
);
assertEquals(
@ -359,7 +358,7 @@ public class EditorTest
updates.put("foo", "mumble");
cmd = new UpdateProperties(updates);
expected = ImmutableMap.of(
AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H",
DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H",
"foo", "mumble"
);
assertEquals(
@ -372,7 +371,7 @@ public class EditorTest
updates.put("foo", null);
cmd = new UpdateProperties(updates);
expected = ImmutableMap.of(
AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H"
DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "PT1H"
);
assertEquals(
expected,
@ -402,7 +401,7 @@ public class EditorTest
// Add a column
cmd = new UpdateColumns(
Collections.singletonList(
new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "d", Columns.VARCHAR, null)
new ColumnSpec("d", Columns.VARCHAR, null)
)
);
TableMetadata revised = doEdit(tableName, cmd);
@ -411,7 +410,6 @@ public class EditorTest
CatalogUtils.columnNames(revised.spec().columns())
);
ColumnSpec colD = revised.spec().columns().get(3);
assertEquals(DatasourceColumnDefn.COLUMN_TYPE, colD.type());
assertEquals("d", colD.name());
assertEquals(Columns.VARCHAR, colD.sqlType());
@ -419,7 +417,6 @@ public class EditorTest
cmd = new UpdateColumns(
Collections.singletonList(
new ColumnSpec(
null,
"a",
Columns.BIGINT,
ImmutableMap.of("foo", "bar")
@ -432,7 +429,6 @@ public class EditorTest
CatalogUtils.columnNames(revised.spec().columns())
);
ColumnSpec colA = revised.spec().columns().get(0);
assertEquals(DatasourceColumnDefn.COLUMN_TYPE, colA.type());
assertEquals("a", colA.name());
assertEquals(Columns.BIGINT, colA.sqlType());
assertEquals(ImmutableMap.of("foo", "bar"), colA.properties());
@ -440,24 +436,16 @@ public class EditorTest
// Duplicates
UpdateColumns cmd2 = new UpdateColumns(
Arrays.asList(
new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "e", Columns.VARCHAR, null),
new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "e", null, null)
new ColumnSpec("e", Columns.VARCHAR, null),
new ColumnSpec("e", null, null)
)
);
assertThrows(CatalogException.class, () -> doEdit(tableName, cmd2));
// Invalid __time column type
UpdateColumns cmd3 = new UpdateColumns(
Collections.singletonList(
new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, Columns.TIME_COLUMN, Columns.VARCHAR, null)
)
);
assertThrows(CatalogException.class, () -> doEdit(tableName, cmd3));
// Valid time column type
cmd = new UpdateColumns(
Collections.singletonList(
new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, Columns.TIME_COLUMN, Columns.TIMESTAMP, null)
new ColumnSpec(Columns.TIME_COLUMN, Columns.TIMESTAMP, null)
)
);
revised = doEdit(tableName, cmd);

View File

@ -40,6 +40,12 @@
<version>${project.parent.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-server</artifactId>
<version>${project.parent.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-aws-common</artifactId>
@ -114,7 +120,6 @@
<dependency>
<groupId>com.amazonaws</groupId>
<artifactId>aws-java-sdk-sts</artifactId>
<version>${aws.sdk.version}</version>
</dependency>
<dependency>
<groupId>it.unimi.dsi</groupId>
@ -129,12 +134,6 @@
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-server</artifactId>
<version>${project.parent.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-processing</artifactId>

View File

@ -0,0 +1,326 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.table.BaseTableFunction.Parameter;
import org.apache.druid.catalog.model.table.TableFunction.ParameterDefn;
import org.apache.druid.catalog.model.table.TableFunction.ParameterType;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.impl.CloudObjectLocation;
import org.apache.druid.data.input.s3.S3InputSource;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.storage.s3.S3StorageDruidModule;
import org.apache.druid.utils.CollectionUtils;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* Catalog definition for the S3 input source.
* <p>
* The catalog entry contains a serialized S3 input source, with some simplifying variations.
* The catalog entry can define the {@code bucket} table property which is the (single) bucket
* value to use when creating the list of objects: the catalog provides the bucket, the table
* function provides the list of objects. (Done this way since defining two correlated lists
* in SQL is awkward, and credentials make the most sense when working with a single bucket.)
* <p>
* The ad-hoc function allows the various ways to specify the objects, but not the configuration
* parameters. If the user wishes to use such parameters, they should be defined in a catalog
* entry, since providing maps in SQL is awkward.
* <p>
* The partial table function can be of various forms:
* <ul>
* <li>Fully define the table, which means providing the full set of S3 properties and not
* providing the table-level {@code bucket} property. This form is complete and does't need
* a table function. If used with a table function, the function provides the {@code glob}
* parameter (if not already provided in the table spec.)</li>
* <li>Partially define the table: using URIs with the {@code glob} to be provided later, or
* by using the {@code bucket} table property. The table function provides the {@code objects}
* parameter to specify the specific objects. This form provides both the format and the list
* of columns.</li>
* <li>Partially define the table as a connection: provide only the {@code bucket} property,
* and omit both the format and the columns. The table function requests the {@code objects}
* and the {@code format}. The user must provide the list of columns.</li>
* </ul>
*
* @see {@link S3InputSource} for details on the meaning of the various properties, and the rules
* about valid combinations
*/
public class S3InputSourceDefn extends FormattedInputSourceDefn
{
public static final String URIS_PARAMETER = "uris";
public static final String PREFIXES_PARAMETER = "prefixes";
public static final String BUCKET_PARAMETER = "bucket";
public static final String PATHS_PARAMETER = "paths";
public static final String ACCESS_KEY_ID_PARAMETER = "accessKeyId";
public static final String SECRET_ACCESS_KEY_PARAMETER = "secretAccessKey";
public static final String ASSUME_ROLE_ARN_PARAMETER = "assumeRoleArn";
/**
* The {@code objectGlob} property exists in S3, but is not documented. The corresponding
* function parameter also exists, but is not documented.
*/
public static final String OBJECT_GLOB_PARAMETER = "objectGlob";
/**
* External data table spec property that lets the user define one bucket in the catalog,
* so that the corresponding table function can supply just the relative path names within
* that bucket. That is, if the user sets this, Druid will generate the {@code objects}
* field from this entry and files provided in the table function.
*/
public static final String BUCKET_PROPERTY = "bucket";
private static final ParameterDefn URI_PARAM_DEFN = new Parameter(URIS_PARAMETER, ParameterType.VARCHAR_ARRAY, true);
private static final ParameterDefn PREFIXES_PARAM_DEFN = new Parameter(PREFIXES_PARAMETER, ParameterType.VARCHAR_ARRAY, true);
private static final ParameterDefn BUCKET_PARAM_DEFN = new Parameter(BUCKET_PARAMETER, ParameterType.VARCHAR, true);
private static final ParameterDefn PATHS_PARAM_DEFN = new Parameter(PATHS_PARAMETER, ParameterType.VARCHAR_ARRAY, true);
private static final ParameterDefn OBJECT_GLOB_PARAM_DEFN = new Parameter(OBJECT_GLOB_PARAMETER, ParameterType.VARCHAR, true);
private static final List<ParameterDefn> SECURITY_PARAMS = Arrays.asList(
new Parameter(ACCESS_KEY_ID_PARAMETER, ParameterType.VARCHAR, true),
new Parameter(SECRET_ACCESS_KEY_PARAMETER, ParameterType.VARCHAR, true),
new Parameter(ASSUME_ROLE_ARN_PARAMETER, ParameterType.VARCHAR, true)
);
// Field names in the S3InputSource
private static final String URIS_FIELD = "uris";
private static final String PREFIXES_FIELD = "prefixes";
private static final String OBJECTS_FIELD = "objects";
private static final String OBJECT_GLOB_FIELD = "objectGlob";
private static final String PROPERTIES_FIELD = "properties";
private static final String ACCESS_KEY_ID_FIELD = "accessKeyId";
private static final String SECRET_ACCESS_KEY_FIELD = "secretAccessKey";
private static final String ASSUME_ROLE_ARN_FIELD = "assumeRoleArn";
@Override
public String typeValue()
{
return S3StorageDruidModule.SCHEME;
}
@Override
protected Class<? extends InputSource> inputSourceClass()
{
return S3InputSource.class;
}
@Override
public void validate(ResolvedExternalTable table)
{
final boolean hasFormat = table.inputFormatMap != null;
final boolean hasColumns = !CollectionUtils.isNullOrEmpty(table.resolvedTable().spec().columns());
if (hasFormat && !hasColumns) {
throw new IAE(
"An external S3 table with a format must also provide the corresponding columns"
);
}
// The user can either provide a bucket, or can provide one of the valid items.
final String bucket = table.resolvedTable().stringProperty(BUCKET_PROPERTY);
final boolean hasBucket = bucket != null;
final Map<String, Object> sourceMap = table.inputSourceMap;
final boolean hasUris = sourceMap.containsKey(URIS_FIELD);
final boolean hasPrefix = sourceMap.containsKey(PREFIXES_FIELD);
final boolean hasObjects = sourceMap.containsKey(OBJECTS_FIELD);
final boolean hasGlob = sourceMap.containsKey(OBJECT_GLOB_FIELD);
if (hasBucket) {
if (hasUris || hasPrefix || hasObjects) {
throw new IAE(
"Provide either the %s property, or one of the S3 input source fields %s, %s or %s, but not both.",
BUCKET_PROPERTY,
URIS_FIELD,
PREFIXES_FIELD,
OBJECTS_FIELD
);
}
if (hasGlob) {
throw new IAE(
"The %s property cannot be provided when the %s property is set",
OBJECT_GLOB_FIELD,
BUCKET_PROPERTY
);
}
// Patch in a dummy URI so that validation of the rest of the fields
// will pass.
sourceMap.put(URIS_FIELD, Collections.singletonList(bucket));
}
super.validate(table);
}
@Override
protected List<ParameterDefn> adHocTableFnParameters()
{
return CatalogUtils.concatLists(
Arrays.asList(
URI_PARAM_DEFN,
PREFIXES_PARAM_DEFN,
BUCKET_PARAM_DEFN,
PATHS_PARAM_DEFN,
OBJECT_GLOB_PARAM_DEFN
),
SECURITY_PARAMS
);
}
@Override
protected void convertArgsToSourceMap(Map<String, Object> jsonMap, Map<String, Object> args)
{
jsonMap.put(InputSource.TYPE_PROPERTY, S3StorageDruidModule.SCHEME);
final List<String> uris = CatalogUtils.getStringArray(args, URIS_PARAMETER);
final List<String> prefixes = CatalogUtils.getStringArray(args, PREFIXES_PARAMETER);
final String bucket = CatalogUtils.getNonBlankString(args, BUCKET_PARAMETER);
final List<String> paths = CatalogUtils.getStringArray(args, PATHS_PARAMETER);
final String objectGlob = CatalogUtils.getNonBlankString(args, OBJECT_GLOB_PARAMETER);
final boolean hasUris = uris != null;
final boolean hasPrefixes = prefixes != null;
final boolean hasBucket = bucket != null;
final boolean hasPaths = !CollectionUtils.isNullOrEmpty(paths);
if (hasPaths && !hasBucket) {
throw new IAE(
"S3 requires the %s parameter if %s is set",
BUCKET_PARAMETER,
PATHS_PARAMETER
);
}
if ((hasUris && (hasPrefixes || hasBucket)) || (hasPrefixes && hasBucket)) {
throw new IAE(
"S3 accepts only one of %s, %s or %s",
PATHS_PARAMETER,
BUCKET_PARAMETER,
PREFIXES_PARAMETER
);
}
if (!hasUris && !hasPrefixes && !hasBucket) {
throw new IAE(
"S3 requires one of %s, %s or %s",
PATHS_PARAMETER,
BUCKET_PARAMETER,
PREFIXES_PARAMETER
);
}
if (hasUris) {
jsonMap.put(URIS_FIELD, CatalogUtils.stringListToUriList(uris));
}
if (hasPrefixes) {
jsonMap.put(PREFIXES_FIELD, prefixes);
}
if (hasBucket) {
if (!hasPaths) {
throw new IAE("When using the %s parameter, %s must also be provided", BUCKET_PARAMETER, PATHS_PARAMETER);
}
List<CloudObjectLocation> objects = new ArrayList<>();
for (String obj : paths) {
objects.add(new CloudObjectLocation(bucket, obj));
}
jsonMap.put(OBJECTS_FIELD, objects);
}
if (objectGlob != null) {
jsonMap.put(OBJECT_GLOB_FIELD, objectGlob);
}
applySecurityParams(jsonMap, args);
}
private void applySecurityParams(Map<String, Object> jsonMap, Map<String, Object> args)
{
final String accessKeyId = CatalogUtils.getNonBlankString(args, ACCESS_KEY_ID_PARAMETER);
final String secretAccessKey = CatalogUtils.getNonBlankString(args, SECRET_ACCESS_KEY_PARAMETER);
final String assumeRoleArn = CatalogUtils.getNonBlankString(args, ASSUME_ROLE_ARN_PARAMETER);
if (accessKeyId != null || secretAccessKey != null || assumeRoleArn != null) {
Map<String, Object> properties = new HashMap<>();
if (accessKeyId != null) {
properties.put(ACCESS_KEY_ID_FIELD, accessKeyId);
}
if (secretAccessKey != null) {
properties.put(SECRET_ACCESS_KEY_FIELD, secretAccessKey);
}
if (assumeRoleArn != null) {
properties.put(ASSUME_ROLE_ARN_FIELD, assumeRoleArn);
}
jsonMap.put(PROPERTIES_FIELD, properties);
}
}
@Override
public TableFunction partialTableFn(final ResolvedExternalTable table)
{
// Allow parameters depending on what is available.
final Map<String, Object> sourceMap = table.inputSourceMap;
List<ParameterDefn> params = new ArrayList<>();
// If a bucket is provided, then the user can specify objects.
if (table.resolvedTable().spec().properties().containsKey(BUCKET_PROPERTY)) {
params.add(PATHS_PARAM_DEFN);
// Else, if no glob is provided, the user can specify the glob.
} else if (!sourceMap.containsKey(OBJECT_GLOB_FIELD)) {
params.add(OBJECT_GLOB_PARAM_DEFN);
}
// Add security arguments if table does not provide them.
if (!sourceMap.containsKey(PROPERTIES_FIELD)) {
params.addAll(SECURITY_PARAMS);
}
// Does the table define a format?
if (table.inputFormatMap == null) {
params = addFormatParameters(params);
}
return new PartialTableFunction(table, params);
}
@Override
protected ExternalTableSpec convertCompletedTable(
final ResolvedExternalTable table,
final Map<String, Object> args,
final List<ColumnSpec> columns
)
{
final Map<String, Object> sourceMap = new HashMap<>(table.inputSourceMap);
// If a bucket parameter is provided, user provides the objects. Else, use the
// catalog input source definition.
final String bucket = table.resolvedTable().stringProperty(BUCKET_PROPERTY);
if (bucket != null) {
List<String> paths = CatalogUtils.getStringArray(args, PATHS_PARAMETER);
if (CollectionUtils.isNullOrEmpty(paths)) {
throw new IAE(
"S3 external table defines the %s property. The table function must provide the %s parameter",
BUCKET_PROPERTY,
PATHS_PARAMETER
);
}
List<CloudObjectLocation> objects = new ArrayList<>();
for (String obj : paths) {
objects.add(new CloudObjectLocation(bucket, obj));
}
sourceMap.put(OBJECTS_FIELD, objects);
}
applySecurityParams(sourceMap, args);
return convertPartialFormattedTable(table, args, columns, sourceMap);
}
}

View File

@ -0,0 +1,727 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableDefnRegistry;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.table.InputFormats.CsvFormatDefn;
import org.apache.druid.catalog.model.table.TableFunction.ParameterDefn;
import org.apache.druid.data.input.impl.CloudObjectLocation;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.s3.S3InputSource;
import org.apache.druid.data.input.s3.S3InputSourceConfig;
import org.apache.druid.data.input.s3.S3InputSourceDruidModule;
import org.apache.druid.data.input.s3.S3InputSourceTest;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.storage.s3.S3StorageDruidModule;
import org.junit.Before;
import org.junit.Test;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
/**
* Test the catalog definition on top of the S3 input source. Here we assume that
* the S3 input source itself works. That is, that the {@link S3InputSourceTest} tests
* pass.
* <p>
* Setup is a bit tricky. This test follows {@link S3InputSourceTest} in using mocks. In fact, this
* test uses the mocks defined in {@link S3InputSourceTest}. Jackson setup is also tricky: we need
* to register the right subclasses and injectables. {@link S3InputSourceTest} provides no method
* to do both, so we cobble that together here.
*/
public class S3InputSourceDefnTest
{
private static final List<ColumnSpec> COLUMNS = Arrays.asList(
new ColumnSpec("x", Columns.VARCHAR, null),
new ColumnSpec("y", Columns.BIGINT, null)
);
/**
* Minimum JSON input source format.
*/
public static final Map<String, Object> CSV_FORMAT = ImmutableMap.of("type", CsvInputFormat.TYPE_KEY);
/**
* Object mapper created using the {@link S3InputSourceTest} version, which registers
* injectables (but, sadly, not subclasses.)
*/
private final ObjectMapper mapper = S3InputSourceTest.createS3ObjectMapper();
/**
* Create a catalog table definition registry with the S3 extension added.
*/
private final TableDefnRegistry registry = new TableDefnRegistry(
null,
Collections.singletonList(new S3InputSourceDefn()),
null,
mapper
);
private final InputSourceDefn s3Defn = registry.inputSourceDefnFor(S3StorageDruidModule.SCHEME);
/**
* Finish up Jackson configuration: add the required S3 input source subtype.
*/
@Before
public void setup()
{
mapper.registerModules(new S3InputSourceDruidModule().getJacksonModules());
}
@Test
public void testValidateEmptyInputSource()
{
// No data property: not valid
TableMetadata table = TableBuilder.external("foo")
.inputSource(ImmutableMap.of("type", S3StorageDruidModule.SCHEME))
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
/**
* Utility method to handle the boilerplate required to create an S3 input source.
* These tests care only about the properties which the catalog exposes.
*/
private S3InputSource s3InputSource(
@Nullable List<String> uris,
@Nullable List<String> prefixes,
@Nullable List<CloudObjectLocation> objects,
@Nullable String objectGlob
)
{
return s3InputSource(uris, prefixes, objects, objectGlob, null);
}
private S3InputSource s3InputSource(
@Nullable List<String> uris,
@Nullable List<String> prefixes,
@Nullable List<CloudObjectLocation> objects,
@Nullable String objectGlob,
@Nullable S3InputSourceConfig config
)
{
return new S3InputSource(
S3InputSourceTest.SERVICE,
S3InputSourceTest.SERVER_SIDE_ENCRYPTING_AMAZON_S3_BUILDER,
S3InputSourceTest.INPUT_DATA_CONFIG,
null,
CatalogUtils.stringListToUriList(uris),
CatalogUtils.stringListToUriList(prefixes),
objects,
objectGlob,
config,
null,
null,
null
);
}
@Test
public void testValidateNoFormat()
{
// No format: valid. Format can be provided at run time.
S3InputSource s3InputSource = s3InputSource(
Collections.singletonList("s3://foo/bar/file.csv"), null, null, null);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(s3InputSource))
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
}
@Test
public void testValidateNoFormatWithColumns()
{
// No format, but columns: not valid
S3InputSource s3InputSource = s3InputSource(
Collections.singletonList("s3://foo/bar/file.csv"), null, null, null);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(s3InputSource))
.column("x", Columns.VARCHAR)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
private Map<String, Object> toMap(Object obj)
{
try {
return mapper.convertValue(obj, ExternalTableDefn.MAP_TYPE_REF);
}
catch (Exception e) {
throw new ISE(e, "bad conversion");
}
}
@Test
public void testValidateNoColumns()
{
// If a format is provided, then columns must also be provided.
S3InputSource s3InputSource = s3InputSource(
Collections.singletonList("s3://foo/bar/file.csv"), null, null, null);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(s3InputSource))
.inputFormat(CSV_FORMAT)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testValidateGood()
{
// Minimum possible configuration that passes validation.
S3InputSource s3InputSource = s3InputSource(
Collections.singletonList("s3://foo/bar/file.csv"), null, null, null);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(s3InputSource))
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
}
@Test
public void testBucketOnly()
{
TableMetadata table = TableBuilder.external("foo")
.inputSource(ImmutableMap.of("type", S3StorageDruidModule.SCHEME))
.inputFormat(CSV_FORMAT)
.property(S3InputSourceDefn.BUCKET_PROPERTY, "s3://foo.com")
.column("x", Columns.VARCHAR)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
}
@Test
public void testBucketAndUri()
{
S3InputSource s3InputSource = s3InputSource(
Collections.singletonList("s3://foo/bar/file.csv"), null, null, null);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(s3InputSource))
.inputFormat(CSV_FORMAT)
.property(S3InputSourceDefn.BUCKET_PROPERTY, "foo.com")
.column("x", Columns.VARCHAR)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testBucketAndPrefix()
{
S3InputSource s3InputSource = s3InputSource(
null,
Collections.singletonList("s3://foo/bar/"),
null,
null
);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(s3InputSource))
.inputFormat(CSV_FORMAT)
.property(S3InputSourceDefn.BUCKET_PROPERTY, "foo.com")
.column("x", Columns.VARCHAR)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testBucketAndObject()
{
S3InputSource s3InputSource = s3InputSource(
null,
null,
Collections.singletonList(new CloudObjectLocation("foo.com", "bar/file.csv")),
null
);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(s3InputSource))
.inputFormat(CSV_FORMAT)
.property(S3InputSourceDefn.BUCKET_PROPERTY, "foo.com")
.column("x", Columns.VARCHAR)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testBucketAndGlob()
{
TableMetadata table = TableBuilder.external("foo")
.inputSource(
ImmutableMap.of(
"type", S3StorageDruidModule.SCHEME,
"objectGlob", "*.csv")
)
.inputFormat(CSV_FORMAT)
.property(S3InputSourceDefn.BUCKET_PROPERTY, "foo.com")
.column("x", Columns.VARCHAR)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testAdHocParameters()
{
TableFunction fn = s3Defn.adHocTableFn();
assertTrue(hasParam(fn, S3InputSourceDefn.URIS_PARAMETER));
assertTrue(hasParam(fn, S3InputSourceDefn.OBJECT_GLOB_PARAMETER));
assertTrue(hasParam(fn, S3InputSourceDefn.PREFIXES_PARAMETER));
assertTrue(hasParam(fn, S3InputSourceDefn.BUCKET_PARAMETER));
assertTrue(hasParam(fn, S3InputSourceDefn.PATHS_PARAMETER));
assertTrue(hasParam(fn, S3InputSourceDefn.ACCESS_KEY_ID_PARAMETER));
assertTrue(hasParam(fn, S3InputSourceDefn.SECRET_ACCESS_KEY_PARAMETER));
assertTrue(hasParam(fn, S3InputSourceDefn.ASSUME_ROLE_ARN_PARAMETER));
assertTrue(hasParam(fn, FormattedInputSourceDefn.FORMAT_PARAMETER));
}
@Test
public void testAdHocNoArgs()
{
TableFunction fn = s3Defn.adHocTableFn();
Map<String, Object> args = new HashMap<>();
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
assertThrows(IAE.class, () -> fn.apply("x", args, COLUMNS, mapper));
}
@Test
public void testAdHocUri()
{
TableFunction fn = s3Defn.adHocTableFn();
// Convert to an external table. Must provide the URIs plus format and columns.
Map<String, Object> args = new HashMap<>();
List<String> uris = Collections.singletonList("s3://foo/bar/file.csv");
args.put(S3InputSourceDefn.URIS_PARAMETER, uris);
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
S3InputSource s3InputSource = (S3InputSource) externSpec.inputSource;
assertEquals(
CatalogUtils.stringListToUriList(uris),
s3InputSource.getUris()
);
// But, it fails if there are no columns.
assertThrows(IAE.class, () -> fn.apply("x", args, Collections.emptyList(), mapper));
}
@Test
public void testMultipleAdHocUris()
{
TableFunction fn = s3Defn.adHocTableFn();
Map<String, Object> args = new HashMap<>();
List<String> uris = Arrays.asList("s3://foo/bar/file1.csv", "s3://foo/mumble/file2.csv");
args.put(S3InputSourceDefn.URIS_PARAMETER, uris);
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
S3InputSource s3InputSource = (S3InputSource) externSpec.inputSource;
assertEquals(
CatalogUtils.stringListToUriList(uris),
s3InputSource.getUris()
);
}
@Test
public void testAdHocEmptyUri()
{
TableFunction fn = s3Defn.adHocTableFn();
Map<String, Object> args = new HashMap<>();
args.put(S3InputSourceDefn.URIS_PARAMETER, Collections.emptyList());
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
assertThrows(IAE.class, () -> fn.apply("x", args, COLUMNS, mapper));
}
@Test
public void testAdHocUriWithGlob()
{
TableFunction fn = s3Defn.adHocTableFn();
// Convert to an external table. Must provide the URIs plus format and columns.
Map<String, Object> args = new HashMap<>();
List<String> uris = Collections.singletonList("s3://foo/bar/");
args.put(S3InputSourceDefn.URIS_PARAMETER, uris);
args.put(S3InputSourceDefn.OBJECT_GLOB_PARAMETER, "*.csv");
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
S3InputSource s3InputSource = (S3InputSource) externSpec.inputSource;
assertEquals(
CatalogUtils.stringListToUriList(uris),
s3InputSource.getUris()
);
assertEquals("*.csv", s3InputSource.getObjectGlob());
}
@Test
public void testAdHocPrefix()
{
TableFunction fn = s3Defn.adHocTableFn();
// Convert to an external table. Must provide the URIs plus format and columns.
Map<String, Object> args = new HashMap<>();
List<String> prefixes = Collections.singletonList("s3://foo/bar/data");
args.put(S3InputSourceDefn.PREFIXES_PARAMETER, prefixes);
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
S3InputSource s3InputSource = (S3InputSource) externSpec.inputSource;
assertEquals(
CatalogUtils.stringListToUriList(prefixes),
s3InputSource.getPrefixes()
);
// But, it fails if there are no columns.
assertThrows(IAE.class, () -> fn.apply("x", args, Collections.emptyList(), mapper));
}
@Test
public void testMultipleAdHocPrefixes()
{
TableFunction fn = s3Defn.adHocTableFn();
// Convert to an external table. Must provide the URIs plus format and columns.
Map<String, Object> args = new HashMap<>();
List<String> prefixes = Arrays.asList("s3://foo/bar/", "s3://foo/mumble/");
args.put(S3InputSourceDefn.PREFIXES_PARAMETER, prefixes);
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
S3InputSource s3InputSource = (S3InputSource) externSpec.inputSource;
assertEquals(
CatalogUtils.stringListToUriList(prefixes),
s3InputSource.getPrefixes()
);
}
@Test
public void testAdHocBucketAndPaths()
{
TableFunction fn = s3Defn.adHocTableFn();
// Convert to an external table. Must provide the URIs plus format and columns.
Map<String, Object> args = new HashMap<>();
args.put(S3InputSourceDefn.BUCKET_PARAMETER, "foo.com");
args.put(S3InputSourceDefn.PATHS_PARAMETER, Collections.singletonList("bar/file.csv"));
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
S3InputSource s3InputSource = (S3InputSource) externSpec.inputSource;
assertEquals(1, s3InputSource.getObjects().size());
CloudObjectLocation obj = s3InputSource.getObjects().get(0);
assertEquals("foo.com", obj.getBucket());
assertEquals("bar/file.csv", obj.getPath());
// But, it fails if there are no columns.
assertThrows(IAE.class, () -> fn.apply("x", args, Collections.emptyList(), mapper));
}
@Test
public void testAdHocPathWithoutBucket()
{
TableFunction fn = s3Defn.adHocTableFn();
Map<String, Object> args = new HashMap<>();
args.put(S3InputSourceDefn.PATHS_PARAMETER, Collections.singletonList("bar/file.csv"));
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
assertThrows(IAE.class, () -> fn.apply("x", args, COLUMNS, mapper));
}
@Test
public void testAdHocBucketWithoutPaths()
{
TableFunction fn = s3Defn.adHocTableFn();
Map<String, Object> args = new HashMap<>();
args.put(S3InputSourceDefn.BUCKET_PARAMETER, "foo.com");
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
assertThrows(IAE.class, () -> fn.apply("x", args, COLUMNS, mapper));
}
@Test
public void testMultipleAdHocObjects()
{
TableFunction fn = s3Defn.adHocTableFn();
// Convert to an external table. Must provide the URIs plus format and columns.
Map<String, Object> args = new HashMap<>();
args.put(S3InputSourceDefn.BUCKET_PARAMETER, "foo.com");
args.put(S3InputSourceDefn.PATHS_PARAMETER, Arrays.asList("bar/file1.csv", "mumble/file2.csv"));
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
S3InputSource s3InputSource = (S3InputSource) externSpec.inputSource;
assertEquals(2, s3InputSource.getObjects().size());
CloudObjectLocation obj = s3InputSource.getObjects().get(0);
assertEquals("foo.com", obj.getBucket());
assertEquals("bar/file1.csv", obj.getPath());
obj = s3InputSource.getObjects().get(1);
assertEquals("foo.com", obj.getBucket());
assertEquals("mumble/file2.csv", obj.getPath());
}
@Test
public void testAdHocUriPrefixConflict()
{
TableFunction fn = s3Defn.adHocTableFn();
// Convert to an external table. Must provide the URIs plus format and columns.
Map<String, Object> args = new HashMap<>();
args.put(S3InputSourceDefn.URIS_PARAMETER, Collections.singletonList("s3://foo/bar/file.csv"));
args.put(S3InputSourceDefn.PREFIXES_PARAMETER, Collections.singletonList("s3://foo/bar/data"));
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
assertThrows(IAE.class, () -> fn.apply("x", args, COLUMNS, mapper));
}
@Test
public void testAdHocUriPathConflict()
{
TableFunction fn = s3Defn.adHocTableFn();
// Convert to an external table. Must provide the URIs plus format and columns.
Map<String, Object> args = new HashMap<>();
args.put(S3InputSourceDefn.URIS_PARAMETER, Collections.singletonList("s3://foo/bar/file.csv"));
args.put(S3InputSourceDefn.BUCKET_PARAMETER, "foo.com");
args.put(S3InputSourceDefn.PATHS_PARAMETER, Collections.singletonList("bar/file.csv"));
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
assertThrows(IAE.class, () -> fn.apply("x", args, COLUMNS, mapper));
}
@Test
public void testAdHocPrefixPathConflict()
{
TableFunction fn = s3Defn.adHocTableFn();
// Convert to an external table. Must provide the URIs plus format and columns.
Map<String, Object> args = new HashMap<>();
args.put(S3InputSourceDefn.PREFIXES_PARAMETER, Collections.singletonList("s3://foo/bar/data"));
args.put(S3InputSourceDefn.BUCKET_PARAMETER, "foo.com");
args.put(S3InputSourceDefn.PATHS_PARAMETER, Collections.singletonList("bar/file.csv"));
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
assertThrows(IAE.class, () -> fn.apply("x", args, COLUMNS, mapper));
}
@Test
public void testFullTableSpecHappyPath()
{
S3InputSourceConfig config = new S3InputSourceConfig(null, null, "foo", null);
S3InputSource s3InputSource = s3InputSource(
Arrays.asList("s3://foo/bar/", "s3://mumble/"), null, null, "*.csv", config);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(s3InputSource))
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
// Check validation
table.validate();
// Check registry
ResolvedTable resolved = registry.resolve(table.spec());
assertNotNull(resolved);
// Convert to an external spec
ExternalTableDefn externDefn = (ExternalTableDefn) resolved.defn();
ExternalTableSpec externSpec = externDefn.convert(resolved);
assertEquals(s3InputSource, externSpec.inputSource);
// Get the partial table function
TableFunction fn = externDefn.tableFn(resolved);
assertTrue(fn.parameters().isEmpty());
// Convert to an external table.
externSpec = fn.apply("x", Collections.emptyMap(), Collections.emptyList(), mapper);
assertEquals(s3InputSource, externSpec.inputSource);
// But, it fails columns are provided since the table already has them.
assertThrows(IAE.class, () -> fn.apply("x", Collections.emptyMap(), COLUMNS, mapper));
}
@Test
public void testTableSpecWithoutConfig()
{
S3InputSource s3InputSource = s3InputSource(
Arrays.asList("s3://foo/bar/", "s3://mumble/"), null, null, "*.csv");
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(s3InputSource))
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
// Check validation
table.validate();
// Convert to an external spec
ResolvedTable resolved = registry.resolve(table.spec());
ExternalTableDefn externDefn = (ExternalTableDefn) resolved.defn();
ExternalTableSpec externSpec = externDefn.convert(resolved);
assertEquals(s3InputSource, externSpec.inputSource);
// Get the partial table function
TableFunction fn = externDefn.tableFn(resolved);
assertTrue(hasParam(fn, S3InputSourceDefn.ACCESS_KEY_ID_PARAMETER));
assertTrue(hasParam(fn, S3InputSourceDefn.SECRET_ACCESS_KEY_PARAMETER));
assertTrue(hasParam(fn, S3InputSourceDefn.ASSUME_ROLE_ARN_PARAMETER));
// Convert to an external table.
Map<String, Object> args = new HashMap<>();
String accessKey = "KLJ78979SDFdS2";
String secretAccessKey = "KLS89s98sKJHKJKJH8721lljkd";
String assumeRoleArn = "arn:aws:iam::2981002874992:role/role-s3";
args.put(S3InputSourceDefn.ACCESS_KEY_ID_PARAMETER, accessKey);
args.put(S3InputSourceDefn.SECRET_ACCESS_KEY_PARAMETER, secretAccessKey);
args.put(S3InputSourceDefn.ASSUME_ROLE_ARN_PARAMETER, assumeRoleArn);
externSpec = fn.apply("x", args, Collections.emptyList(), mapper);
S3InputSource actual = (S3InputSource) externSpec.inputSource;
S3InputSourceConfig actualConfig = actual.getS3InputSourceConfig();
assertNotNull(actualConfig);
assertEquals(accessKey, actualConfig.getAccessKeyId().getPassword());
assertEquals(secretAccessKey, actualConfig.getSecretAccessKey().getPassword());
assertEquals(assumeRoleArn, actualConfig.getAssumeRoleArn());
}
@Test
public void testTableSpecWithBucketAndFormat()
{
TableMetadata table = TableBuilder.external("foo")
.inputSource(ImmutableMap.of("type", S3StorageDruidModule.SCHEME))
.inputFormat(CSV_FORMAT)
.property(S3InputSourceDefn.BUCKET_PROPERTY, "foo.com")
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
// Check validation
table.validate();
// Convert to an external spec fails, because the table is partial
ResolvedTable resolved = registry.resolve(table.spec());
ExternalTableDefn externDefn = (ExternalTableDefn) resolved.defn();
assertThrows(IAE.class, () -> externDefn.convert(resolved));
// Get the partial table function
TableFunction fn = externDefn.tableFn(resolved);
assertTrue(hasParam(fn, S3InputSourceDefn.PATHS_PARAMETER));
assertFalse(hasParam(fn, FormattedInputSourceDefn.FORMAT_PARAMETER));
// Convert to an external table.
Map<String, Object> args = new HashMap<>();
args.put(S3InputSourceDefn.PATHS_PARAMETER, Collections.singletonList("bar/file.csv"));
ExternalTableSpec externSpec = fn.apply("x", args, Collections.emptyList(), mapper);
S3InputSource s3InputSource = (S3InputSource) externSpec.inputSource;
assertEquals(1, s3InputSource.getObjects().size());
CloudObjectLocation obj = s3InputSource.getObjects().get(0);
assertEquals("foo.com", obj.getBucket());
assertEquals("bar/file.csv", obj.getPath());
// But, it fails columns are provided since the table already has them.
assertThrows(IAE.class, () -> fn.apply("x", args, COLUMNS, mapper));
// Also fails if the user omits the objects argument
assertThrows(IAE.class, () -> fn.apply("x", Collections.emptyMap(), Collections.emptyList(), mapper));
}
@Test
public void testTableSpecAsConnection()
{
TableMetadata table = TableBuilder.external("foo")
.inputSource(ImmutableMap.of("type", S3StorageDruidModule.SCHEME))
.property(S3InputSourceDefn.BUCKET_PROPERTY, "foo.com")
.build();
// Check validation
table.validate();
// Convert to an external spec fails, because the table is partial
ResolvedTable resolved = registry.resolve(table.spec());
ExternalTableDefn externDefn = (ExternalTableDefn) resolved.defn();
assertThrows(IAE.class, () -> externDefn.convert(resolved));
// Get the partial table function
TableFunction fn = externDefn.tableFn(resolved);
assertTrue(hasParam(fn, S3InputSourceDefn.PATHS_PARAMETER));
assertTrue(hasParam(fn, FormattedInputSourceDefn.FORMAT_PARAMETER));
// Convert to an external table.
Map<String, Object> args = new HashMap<>();
args.put(S3InputSourceDefn.PATHS_PARAMETER, Collections.singletonList("bar/file.csv"));
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
S3InputSource s3InputSource = (S3InputSource) externSpec.inputSource;
assertEquals(1, s3InputSource.getObjects().size());
CloudObjectLocation obj = s3InputSource.getObjects().get(0);
assertEquals("foo.com", obj.getBucket());
assertEquals("bar/file.csv", obj.getPath());
assertTrue(externSpec.inputFormat instanceof CsvInputFormat);
// But, it fails columns are not provided since the table does not have them.
assertThrows(IAE.class, () -> fn.apply("x", args, Collections.emptyList(), mapper));
// Also fails if the user omits the objects argument
assertThrows(IAE.class, () -> fn.apply("x", Collections.emptyMap(), Collections.emptyList(), mapper));
// Also fails if the user omits the format argument
args.remove(FormattedInputSourceDefn.FORMAT_PARAMETER);
assertThrows(IAE.class, () -> fn.apply("x", args, COLUMNS, mapper));
}
protected boolean hasParam(TableFunction fn, String key)
{
for (ParameterDefn param : fn.parameters()) {
if (param.name().equals(key)) {
return true;
}
}
return false;
}
}

View File

@ -53,12 +53,10 @@ import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.InputRowSchema;
import org.apache.druid.data.input.InputSourceReader;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.InputStats;
import org.apache.druid.data.input.MaxSizeSplitHintSpec;
import org.apache.druid.data.input.impl.CloudObjectLocation;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.InputStatsImpl;
import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.initialization.DruidModule;
@ -101,16 +99,16 @@ import static org.easymock.EasyMock.expectLastCall;
public class S3InputSourceTest extends InitializedNullHandlingTest
{
private static final ObjectMapper MAPPER = createS3ObjectMapper();
private static final AmazonS3Client S3_CLIENT = EasyMock.createMock(AmazonS3Client.class);
public static final AmazonS3Client S3_CLIENT = EasyMock.createMock(AmazonS3Client.class);
private static final ClientConfiguration CLIENT_CONFIGURATION = EasyMock.createMock(ClientConfiguration.class);
private static final ServerSideEncryptingAmazonS3.Builder SERVER_SIDE_ENCRYPTING_AMAZON_S3_BUILDER =
public static final ServerSideEncryptingAmazonS3.Builder SERVER_SIDE_ENCRYPTING_AMAZON_S3_BUILDER =
EasyMock.createMock(ServerSideEncryptingAmazonS3.Builder.class);
private static final AmazonS3ClientBuilder AMAZON_S3_CLIENT_BUILDER = AmazonS3Client.builder();
private static final ServerSideEncryptingAmazonS3 SERVICE = new ServerSideEncryptingAmazonS3(
public static final AmazonS3ClientBuilder AMAZON_S3_CLIENT_BUILDER = AmazonS3Client.builder();
public static final ServerSideEncryptingAmazonS3 SERVICE = new ServerSideEncryptingAmazonS3(
S3_CLIENT,
new NoopServerSideEncryption()
);
private static final S3InputDataConfig INPUT_DATA_CONFIG;
public static final S3InputDataConfig INPUT_DATA_CONFIG;
private static final int MAX_LISTING_LENGTH = 10;
private static final List<CloudObjectLocation> EXPECTED_OBJECTS = Arrays.asList(
@ -944,10 +942,8 @@ public class S3InputSourceTest extends InitializedNullHandlingTest
temporaryFolder.newFolder()
);
final InputStats inputStats = new InputStatsImpl();
CloseableIterator<InputRow> iterator = reader.read(inputStats);
CloseableIterator<InputRow> iterator = reader.read();
Assert.assertEquals(CONTENT.length, inputStats.getProcessedBytes());
while (iterator.hasNext()) {
InputRow nextRow = iterator.next();
Assert.assertEquals(NOW, nextRow.getTimestamp());
@ -1040,10 +1036,8 @@ public class S3InputSourceTest extends InitializedNullHandlingTest
temporaryFolder.newFolder()
);
final InputStats inputStats = new InputStatsImpl();
CloseableIterator<InputRow> iterator = reader.read(inputStats);
CloseableIterator<InputRow> iterator = reader.read();
Assert.assertEquals(CONTENT.length, inputStats.getProcessedBytes());
while (iterator.hasNext()) {
InputRow nextRow = iterator.next();
Assert.assertEquals(NOW, nextRow.getTimestamp());
@ -1096,7 +1090,7 @@ public class S3InputSourceTest extends InitializedNullHandlingTest
}
// Setup mocks for invoquing the resettable condition for the S3Entity:
// Setup mocks for invoking the resetable condition for the S3Entity
private static void expectSdkClientException(URI uri) throws IOException
{
final String s3Bucket = uri.getAuthority();

View File

@ -28,7 +28,7 @@ import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.TableId;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.TableSpec;
import org.apache.druid.catalog.model.table.AbstractDatasourceDefn;
import org.apache.druid.catalog.model.table.DatasourceDefn;
import org.apache.druid.catalog.model.table.TableBuilder;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.testsEx.categories.Catalog;
@ -71,8 +71,10 @@ public class ITCatalogRestTest
// Bogus schema
{
final TableMetadata table = new TableBuilder()
.id(TableId.of("bogus", "foo"))
final TableMetadata table = new TableBuilder(
TableId.of("bogus", "foo"),
DatasourceDefn.TABLE_TYPE
)
.build();
assertThrows(
@ -83,9 +85,11 @@ public class ITCatalogRestTest
// Read-only schema
{
final TableMetadata table = new TableBuilder()
.id(TableId.of(TableId.SYSTEM_SCHEMA, "foo"))
.property(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D")
final TableMetadata table = new TableBuilder(
TableId.of(TableId.SYSTEM_SCHEMA, "foo"),
DatasourceDefn.TABLE_TYPE
)
.property(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D")
.build();
assertThrows(
Exception.class,
@ -126,7 +130,7 @@ public class ITCatalogRestTest
// Update the datasource
TableSpec dsSpec2 = TableBuilder.copyOf(table)
.property(AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 3_000_000)
.property(DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 3_000_000)
.column("d", "DOUBLE")
.buildSpec();
@ -159,7 +163,7 @@ public class ITCatalogRestTest
read = client.readTable(table.id());
assertEquals(
Arrays.asList("e", "f"),
read.spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
read.spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
);
// Unhide
@ -170,7 +174,7 @@ public class ITCatalogRestTest
read = client.readTable(table.id());
assertEquals(
Collections.singletonList("f"),
read.spec().properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
read.spec().properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
);
// List schemas

View File

@ -20,20 +20,27 @@
package org.apache.druid.catalog.model;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Strings;
import org.apache.druid.catalog.model.ModelProperties.PropertyDefn;
import org.apache.druid.catalog.model.table.DatasourceDefn;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.granularity.GranularityType;
import org.apache.druid.java.util.common.granularity.PeriodGranularity;
import org.joda.time.Period;
import javax.annotation.Nullable;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
@ -58,14 +65,14 @@ public class CatalogUtils
*/
public static Granularity asDruidGranularity(String value)
{
if (Strings.isNullOrEmpty(value)) {
if (Strings.isNullOrEmpty(value) || value.equalsIgnoreCase(DatasourceDefn.ALL_GRANULARITY)) {
return Granularities.ALL;
}
try {
return new PeriodGranularity(new Period(value), null, null);
}
catch (IllegalArgumentException e) {
throw new IAE(StringUtils.format("%s is an invalid period string", value));
throw new IAE(StringUtils.format("'%s' is an invalid period string", value));
}
}
@ -82,7 +89,7 @@ public class CatalogUtils
return Arrays.asList(value.split(",\\s*"));
}
public static <T> T safeCast(Object value, Class<T> type, String propertyName)
public static <T> T safeCast(Object value, Class<T> type, String key)
{
if (value == null) {
return null;
@ -93,15 +100,54 @@ public class CatalogUtils
catch (ClassCastException e) {
throw new IAE("Value [%s] is not valid for property %s, expected type %s",
value,
propertyName,
key,
type.getSimpleName()
);
}
}
public static <T> T safeGet(Map<String, Object> map, String propertyName, Class<T> type)
public static <T> T safeGet(Map<String, Object> map, String key, Class<T> type)
{
return safeCast(map.get(propertyName), type, propertyName);
return safeCast(map.get(key), type, key);
}
public static long getLong(Map<String, Object> map, String key)
{
Object value = map.get(key);
if (value == null) {
return 0;
}
// Jackson may deserialize the value as either Integer or Long.
if (value instanceof Integer) {
return (Integer) value;
}
return safeCast(value, Long.class, key);
}
public static String getString(Map<String, Object> map, String key)
{
return safeGet(map, key, String.class);
}
public static List<String> getStringList(Map<String, Object> map, String key)
{
return stringToList(getString(map, key));
}
/**
* Get the value of a {@code VARCHAR ARRAY} parameter. Though the type is
* called {@code ARRAY}, Calcite provides the actual value as a {@link List}
* of {@code String}s.
*/
@SuppressWarnings("unchecked")
public static List<String> getStringArray(Map<String, Object> map, String key)
{
final Object value = map.get(key);
if (value == null) {
return null;
}
return (List<String>) safeCast(value, List.class, key);
}
public static String stringListToLines(List<String> lines)
@ -113,20 +159,18 @@ public class CatalogUtils
}
/**
* Catalog-specific 1uick & easy implementation of {@code toString()} for objects
* Catalog-specific quick & easy implementation of {@code toString()} for objects
* which are primarily representations of JSON objects. Use only for cases where the
* {@code toString()} is for debugging: the cost of creating an object mapper
* every time is undesirable for production code. Also, assumes that the
* type can serialized using the default mapper: doesn't work for types that
* {@code toString()} is for debugging. Also, assumes that the
* type can serialized using the default mapper: this trick doesn't work for types that
* require custom Jackson extensions. The catalog, however, has a simple type
* hierarchy, which is not extended via extensions, and so the default object mapper is
* fine.
*/
public static String toString(Object obj)
{
ObjectMapper jsonMapper = new ObjectMapper();
try {
return jsonMapper.writeValueAsString(obj);
return DefaultObjectMapper.INSTANCE.writerWithDefaultPrettyPrinter().writeValueAsString(obj);
}
catch (JsonProcessingException e) {
throw new ISE("Failed to serialize TableDefn");
@ -144,4 +188,125 @@ public class CatalogUtils
.flatMap(Collection::stream)
.collect(Collectors.toList());
}
/**
* Get a string parameter that can either be null or non-blank.
*/
public static String getNonBlankString(Map<String, Object> args, String parameter)
{
String value = CatalogUtils.getString(args, parameter);
if (value != null) {
value = value.trim();
if (value.isEmpty()) {
throw new IAE("%s parameter cannot be a blank string", parameter);
}
}
return value;
}
public static List<String> getUriListArg(Map<String, Object> args, String parameter)
{
String urisString = CatalogUtils.getString(args, parameter);
if (Strings.isNullOrEmpty(urisString)) {
throw new IAE("One or more values are required for parameter %s", parameter);
}
return stringToList(urisString);
}
public static List<URI> stringToUriList(String uris)
{
return stringListToUriList(stringToList(uris));
}
/**
* Convert a list of strings to a list of {@link URI} objects.
*/
public static List<URI> stringListToUriList(List<String> list)
{
if (list == null) {
return null;
}
List<URI> uris = new ArrayList<>();
for (String strValue : list) {
try {
uris.add(new URI(strValue));
}
catch (URISyntaxException e) {
throw new IAE(StringUtils.format("Argument [%s] is not a valid URI", strValue));
}
}
return uris;
}
/**
* Merge the properties for an object using a set of updates in a map. If the
* update value is {@code null}, then remove the property in the revised set. If the
* property is known, use the column definition to merge the values. Else, the
* update replaces any existing value.
* <p>
* This method does not validate the properties, except as needed to do a
* merge. A separate validation step is done on the final, merged object.
*/
public static Map<String, Object> mergeProperties(
final Map<String, PropertyDefn<?>> properties,
final Map<String, Object> source,
final Map<String, Object> update
)
{
if (update == null) {
return source;
}
if (source == null) {
return update;
}
final Map<String, Object> merged = new HashMap<>(source);
for (Map.Entry<String, Object> entry : update.entrySet()) {
if (entry.getValue() == null) {
merged.remove(entry.getKey());
} else {
Object value = entry.getValue();
final PropertyDefn<?> propDefn = properties.get(entry.getKey());
if (propDefn != null) {
value = propDefn.merge(merged.get(entry.getKey()), entry.getValue());
}
merged.put(entry.getKey(), value);
}
}
return merged;
}
public static void validateGranularity(String value)
{
if (value == null) {
return;
}
Granularity granularity;
try {
granularity = new PeriodGranularity(new Period(value), null, null);
}
catch (IllegalArgumentException e) {
throw new IAE(StringUtils.format("[%s] is an invalid granularity string", value));
}
if (!GranularityType.isStandard(granularity)) {
throw new IAE(
"Unsupported segment graularity. "
+ "Please use an equivalent of these granularities: %s.",
Arrays.stream(GranularityType.values())
.filter(granularityType -> !granularityType.equals(GranularityType.NONE))
.map(Enum::name)
.map(StringUtils::toLowerCase)
.collect(Collectors.joining(", "))
);
}
}
public static int findColumn(List<ColumnSpec> columns, String colName)
{
for (int i = 0; i < columns.size(); i++) {
if (columns.get(i).name().equals(colName)) {
return i;
}
}
return -1;
}
}

View File

@ -1,118 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.catalog.model.ModelProperties.PropertyDefn;
import org.apache.druid.java.util.common.IAE;
import java.util.List;
import java.util.Map;
/**
* The definition (meta-meta-data) for a column. Defines the set of allowed
* column types and the set of known properties (of which there are none
* at present.)
*/
public class ColumnDefn extends ObjectDefn
{
/**
* Convenience class that holds a column specification and its corresponding
* definition. This allows the spec to be a pure "data object" without knowledge
* of the metadata representation given by the column definition.
*/
public static class ResolvedColumn
{
private final ColumnDefn defn;
private final ColumnSpec spec;
public ResolvedColumn(ColumnDefn defn, ColumnSpec spec)
{
this.defn = defn;
this.spec = spec;
}
public ColumnDefn defn()
{
return defn;
}
public ColumnSpec spec()
{
return spec;
}
/**
* Merges an updated version of this column with an existing version.
*/
public ResolvedColumn merge(ColumnSpec update)
{
return new ResolvedColumn(defn, defn.merge(spec, update));
}
public void validate(ObjectMapper jsonMapper)
{
defn.validate(spec, jsonMapper);
}
}
public ColumnDefn(
final String name,
final String typeValue,
final List<PropertyDefn<?>> fields
)
{
super(name, typeValue, fields);
}
/**
* Merges an updated version of this column with an existing version.
* <p>
* The name cannot be changed (it is what links the existing column and the
* update). The SQL type will be that provided in the update, if non-null, else
* the original type. Properties are merged using standard rules: those in the
* update take precedence. Null values in the update remove the existing property,
* non-null values update the property. Any properties in the update but not in
* the existing set, are inserted (if non-null).
*/
public ColumnSpec merge(ColumnSpec spec, ColumnSpec update)
{
String updateType = update.type();
if (updateType != null && !spec.type().equals(updateType)) {
throw new IAE("The update type must be null or [%s]", spec.type());
}
String revisedType = update.sqlType() == null ? spec.sqlType() : update.sqlType();
Map<String, Object> revisedProps = mergeProperties(
spec.properties(),
update.properties()
);
return new ColumnSpec(spec.type(), spec.name(), revisedType, revisedProps);
}
public void validate(ColumnSpec spec, ObjectMapper jsonMapper)
{
spec.validate();
}
public void validateScalarColumn(ColumnSpec spec)
{
Columns.validateScalarColumn(spec.name(), spec.sqlType());
}
}

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonInclude.Include;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Strings;
import org.apache.druid.catalog.model.ModelProperties.PropertyDefn;
import org.apache.druid.guice.annotations.UnstableApi;
import org.apache.druid.java.util.common.IAE;
@ -40,12 +41,6 @@ import java.util.Objects;
@UnstableApi
public class ColumnSpec
{
/**
* The type of column spec (not the column data type). For example, in a
* cube, dimensions and measures have distinct sets of properties.
*/
private final String type;
/**
* The name of the column as known to the SQL layer. At present, there is no
* support for column aliases, so this is also the column name as physically
@ -73,22 +68,19 @@ public class ColumnSpec
@JsonCreator
public ColumnSpec(
@JsonProperty("type")final String type,
@JsonProperty("name")final String name,
@JsonProperty("sqlType") @Nullable final String sqlType,
@JsonProperty("properties") @Nullable final Map<String, Object> properties
)
{
this.type = type;
this.name = name;
this.sqlType = sqlType;
this.properties = properties == null ? Collections.emptyMap() : properties;
}
@JsonProperty("type")
public String type()
public ColumnSpec(ColumnSpec from)
{
return type;
this(from.name, from.sqlType, from.properties);
}
@JsonProperty("name")
@ -113,12 +105,34 @@ public class ColumnSpec
public void validate()
{
if (Strings.isNullOrEmpty(type)) {
throw new IAE("Column type is required");
}
if (Strings.isNullOrEmpty(name)) {
throw new IAE("Column name is required");
}
// Validate type in the next PR
}
/**
* Merges an updated version of this column with an existing version.
* <p>
* The name cannot be changed (it is what links the existing column and the
* update). The SQL type will be that provided in the update, if non-null, else
* the original type. Properties are merged using standard rules: those in the
* update take precedence. Null values in the update remove the existing property,
* non-null values update the property. Any properties in the update but not in
* the existing set, are inserted (if non-null).
*/
public ColumnSpec merge(
final Map<String, PropertyDefn<?>> columnProperties,
final ColumnSpec update
)
{
String revisedType = update.sqlType() == null ? sqlType() : update.sqlType();
Map<String, Object> revisedProps = CatalogUtils.mergeProperties(
columnProperties,
properties(),
update.properties()
);
return new ColumnSpec(name(), revisedType, revisedProps);
}
@Override
@ -137,8 +151,7 @@ public class ColumnSpec
return false;
}
ColumnSpec other = (ColumnSpec) o;
return Objects.equals(this.type, other.type)
&& Objects.equals(this.name, other.name)
return Objects.equals(this.name, other.name)
&& Objects.equals(this.sqlType, other.sqlType)
&& Objects.equals(this.properties, other.properties);
}
@ -147,7 +160,6 @@ public class ColumnSpec
public int hashCode()
{
return Objects.hash(
type,
name,
sqlType,
properties

View File

@ -54,6 +54,14 @@ public class Columns
.put(VARCHAR, ColumnType.STRING)
.build();
public static final Map<ColumnType, String> DRUID_TO_SQL_TYPES =
new ImmutableMap.Builder<ColumnType, String>()
.put(ColumnType.LONG, BIGINT)
.put(ColumnType.FLOAT, FLOAT)
.put(ColumnType.DOUBLE, DOUBLE)
.put(ColumnType.STRING, VARCHAR)
.build();
private Columns()
{
}
@ -91,12 +99,14 @@ public class Columns
return TIME_COLUMN.equals(name);
}
public static RowSignature convertSignature(TableSpec spec)
public static RowSignature convertSignature(List<ColumnSpec> columns)
{
List<ColumnSpec> columns = spec.columns();
RowSignature.Builder builder = RowSignature.builder();
for (ColumnSpec col : columns) {
ColumnType druidType = Columns.SQL_TO_DRUID_TYPES.get(StringUtils.toUpperCase(col.sqlType()));
ColumnType druidType = null;
if (col.sqlType() != null) {
druidType = Columns.SQL_TO_DRUID_TYPES.get(StringUtils.toUpperCase(col.sqlType()));
}
if (druidType == null) {
druidType = ColumnType.STRING;
}
@ -104,4 +114,9 @@ public class Columns
}
return builder.build();
}
public static String sqlType(ColumnType druidType)
{
return DRUID_TO_SQL_TYPES.get(druidType);
}
}

View File

@ -21,17 +21,12 @@ package org.apache.druid.catalog.model;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import com.google.common.base.Preconditions;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.PeriodGranularity;
import org.joda.time.Period;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
@ -54,13 +49,6 @@ public interface ModelProperties
*/
String name();
/**
* Metadata about properties, such as how they apply to SQL table functions.
*
* @see {@link PropertyAttributes} for details.
*/
Map<String, Object> attributes();
/**
* The name of the type of this property to be displayed in error messages.
*/
@ -84,27 +72,15 @@ public interface ModelProperties
* Decodes a JSON-encoded value into a corresponding Java value.
*/
T decode(Object value, ObjectMapper jsonMapper);
/**
* Decodes a SQL-encoded value into a corresponding Java value.
*/
T decodeSqlValue(Object value, ObjectMapper jsonMapper);
}
abstract class BasePropertyDefn<T> implements PropertyDefn<T>
{
protected final String name;
protected final Map<String, Object> attributes;
public BasePropertyDefn(final String name, Map<String, Object> attributes)
{
this.name = name;
this.attributes = attributes == null ? ImmutableMap.of() : attributes;
}
public BasePropertyDefn(final String name)
{
this(name, null);
this.name = name;
}
@Override
@ -113,63 +89,41 @@ public interface ModelProperties
return name;
}
@Override
public Map<String, Object> attributes()
{
return attributes;
}
@Override
public String typeName()
{
return PropertyAttributes.typeName(this);
}
@Override
public Object merge(Object existing, Object update)
{
return update == null ? existing : update;
}
@Override
public T decodeSqlValue(Object value, ObjectMapper jsonMapper)
{
return decode(value, jsonMapper);
}
@Override
public String toString()
{
return getClass().getSimpleName() + "{"
+ "name: " + name
+ ", attributes: " + attributes()
+ ", type: " + typeName()
+ "}";
}
}
abstract class SimplePropertyDefn<T> extends BasePropertyDefn<T>
class SimplePropertyDefn<T> extends BasePropertyDefn<T>
{
public final Class<T> valueClass;
public SimplePropertyDefn(
final String name,
final Class<T> valueClass,
final Map<String, Object> attribs
final Class<T> valueClass
)
{
super(
name,
PropertyAttributes.merge(
ImmutableMap.of(
PropertyAttributes.TYPE_NAME,
valueClass.getSimpleName()
),
attribs
)
);
super(name);
this.valueClass = valueClass;
}
@Override
public String typeName()
{
return valueClass.getSimpleName();
}
/**
* Convert the value from the deserialized JSON format to the type
* required by this field data type. Also used to decode values from
@ -205,13 +159,13 @@ public interface ModelProperties
decode(value, jsonMapper);
}
protected T decodeJson(Object value, ObjectMapper jsonMapper)
public T decodeJson(String value, ObjectMapper jsonMapper)
{
if (value == null) {
return null;
}
try {
return jsonMapper.readValue((String) value, valueClass);
return jsonMapper.readValue(value, valueClass);
}
catch (Exception e) {
throw new IAE(
@ -223,30 +177,70 @@ public interface ModelProperties
}
}
class ObjectPropertyDefn<T> extends BasePropertyDefn<T>
{
public final Class<T> valueClass;
public ObjectPropertyDefn(
final String name,
final Class<T> valueClass
)
{
super(name);
this.valueClass = valueClass;
}
@Override
public String typeName()
{
return valueClass.getSimpleName();
}
/**
* Convert the value from the deserialized JSON format to the type
* required by this field data type. Also used to decode values from
* SQL parameters. As a side effect, verifies that the value is of
* the correct type.
*/
@Override
public T decode(Object value, ObjectMapper jsonMapper)
{
return CatalogUtils.safeCast(value, valueClass, "JSON object");
}
/**
* Validate that the given value is valid for this property.
* By default, does a value conversion and discards the value.
*/
@Override
public void validate(Object value, ObjectMapper jsonMapper)
{
}
}
class TypeRefPropertyDefn<T> extends BasePropertyDefn<T>
{
public final String typeName;
public final TypeReference<T> valueType;
public TypeRefPropertyDefn(
final String name,
final String typeName,
final TypeReference<T> valueType,
final Map<String, Object> attribs
final TypeReference<T> valueType
)
{
super(
name,
PropertyAttributes.merge(
ImmutableMap.of(
PropertyAttributes.TYPE_NAME,
typeName
),
attribs
)
);
super(name);
this.typeName = Preconditions.checkNotNull(typeName);
this.valueType = valueType;
}
@Override
public String typeName()
{
return typeName;
}
@Override
public T decode(Object value, ObjectMapper jsonMapper)
{
@ -281,84 +275,40 @@ public interface ModelProperties
class StringPropertyDefn extends SimplePropertyDefn<String>
{
public StringPropertyDefn(String name, Map<String, Object> attribs)
public StringPropertyDefn(String name)
{
super(
name,
String.class,
PropertyAttributes.merge(
ImmutableMap.of(
PropertyAttributes.SQL_JAVA_TYPE,
String.class
),
attribs
)
);
super(name, String.class);
}
}
class GranularityPropertyDefn extends StringPropertyDefn
{
public GranularityPropertyDefn(String name, Map<String, Object> attribs)
public GranularityPropertyDefn(String name)
{
super(name, attribs);
super(name);
}
@Override
public void validate(Object value, ObjectMapper jsonMapper)
{
String gran = decode(value, jsonMapper);
validateGranularity(gran);
}
public void validateGranularity(String value)
{
if (value == null) {
return;
}
try {
//noinspection ResultOfObjectAllocationIgnored
new PeriodGranularity(new Period(value), null, null);
}
catch (IllegalArgumentException e) {
throw new IAE(StringUtils.format("[%s] is an invalid granularity string", value));
}
CatalogUtils.validateGranularity(gran);
}
}
class IntPropertyDefn extends SimplePropertyDefn<Integer>
{
public IntPropertyDefn(String name, Map<String, Object> attribs)
public IntPropertyDefn(String name)
{
super(
name,
Integer.class,
PropertyAttributes.merge(
ImmutableMap.of(
PropertyAttributes.SQL_JAVA_TYPE,
Integer.class
),
attribs
)
);
super(name, Integer.class);
}
}
class BooleanPropertyDefn extends SimplePropertyDefn<Boolean>
{
public BooleanPropertyDefn(String name, Map<String, Object> attribs)
public BooleanPropertyDefn(String name)
{
super(
name,
Boolean.class,
PropertyAttributes.merge(
ImmutableMap.of(
PropertyAttributes.SQL_JAVA_TYPE,
Boolean.class
),
attribs
)
);
super(name, Boolean.class);
}
}
@ -367,11 +317,10 @@ public interface ModelProperties
public ListPropertyDefn(
final String name,
final String typeName,
final TypeReference<List<T>> valueType,
final Map<String, Object> attribs
final TypeReference<List<T>> valueType
)
{
super(name, typeName, valueType, attribs);
super(name, typeName, valueType);
}
@SuppressWarnings("unchecked")
@ -410,33 +359,13 @@ public interface ModelProperties
class StringListPropertyDefn extends ListPropertyDefn<String>
{
public StringListPropertyDefn(
final String name,
final Map<String, Object> attribs
)
public StringListPropertyDefn(String name)
{
super(
name,
"string list",
new TypeReference<List<String>>() {},
PropertyAttributes.merge(
ImmutableMap.of(
PropertyAttributes.SQL_JAVA_TYPE,
String.class
),
attribs
)
new TypeReference<List<String>>() {}
);
}
@Override
public List<String> decodeSqlValue(Object value, ObjectMapper jsonMapper)
{
if (!(value instanceof String)) {
throw new IAE(StringUtils.format("Argument [%s] is not a VARCHAR", value));
}
String[] values = ((String) value).split(",\\s*");
return Arrays.asList(values);
}
}
}

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.catalog.model.ModelProperties.PropertyDefn;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -102,26 +101,7 @@ public class ObjectDefn
final Map<String, Object> update
)
{
if (update == null) {
return source;
}
if (source == null) {
return update;
}
Map<String, Object> merged = new HashMap<>(source);
for (Map.Entry<String, Object> entry : update.entrySet()) {
if (entry.getValue() == null) {
merged.remove(entry.getKey());
} else {
PropertyDefn<?> propDefn = property(entry.getKey());
Object value = entry.getValue();
if (propDefn != null) {
value = propDefn.merge(merged.get(entry.getKey()), entry.getValue());
}
merged.put(entry.getKey(), value);
}
}
return merged;
return CatalogUtils.mergeProperties(properties, source, update);
}
/**

View File

@ -41,22 +41,29 @@ public abstract class ObjectFacade
public boolean booleanProperty(String key)
{
return (Boolean) property(key);
Boolean value = CatalogUtils.safeCast(property(key), Boolean.class, key);
return value == null ? false : value;
}
public String stringProperty(String key)
{
return (String) property(key);
return CatalogUtils.safeCast(property(key), String.class, key);
}
public Integer intProperty(String key)
{
return (Integer) property(key);
return CatalogUtils.safeCast(property(key), Integer.class, key);
}
@SuppressWarnings("unchecked")
public List<String> stringListProperty(String key)
{
return (List<String>) property(key);
return (List<String>) CatalogUtils.safeCast(property(key), List.class, key);
}
@SuppressWarnings("unchecked")
public Map<String, Object> mapProperty(String key)
{
return (Map<String, Object>) CatalogUtils.safeCast(property(key), Map.class, key);
}
}

View File

@ -1,110 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.catalog.model.ModelProperties.PropertyDefn;
import java.util.HashMap;
import java.util.Map;
/**
* Definition and access of attributes of table definition properties. These
* are meta-attributes: attributes of attributes. These are primarily used to
* indicate the role of each table property when used in a SQL table function.
*/
public class PropertyAttributes
{
/**
* If set to {@code true}, then the property is also a SQL function parameter.
*/
public static final String IS_SQL_FN_PARAM_KEY = "sqlFnArg";
/**
* If set to {@code true}, then this SQL function parameter is optional. That is,
* it can take a SQL {@code NULL} value if parameters are listed in order, or can
* be ommited if parameters are provided by name.
*/
public static final String IS_SQL_FN_OPTIONAL = "optional";
public static final String IS_PARAMETER = "param";
/**
* The type name to display in error messages.
*/
public static final String TYPE_NAME = "typeName";
/**
* The type to use when creating a SQL function parameter.
*/
public static final String SQL_JAVA_TYPE = "sqlJavaType";
public static final Map<String, Object> SQL_FN_PARAM =
ImmutableMap.of(IS_SQL_FN_PARAM_KEY, true);
public static final Map<String, Object> OPTIONAL_SQL_FN_PARAM =
ImmutableMap.of(IS_SQL_FN_PARAM_KEY, true, IS_SQL_FN_OPTIONAL, true);
public static final Map<String, Object> TABLE_PARAM =
ImmutableMap.of(IS_PARAMETER, true);
public static final Map<String, Object> SQL_AND_TABLE_PARAM =
ImmutableMap.of(IS_SQL_FN_PARAM_KEY, true, IS_PARAMETER, true);
private static boolean getBoolean(PropertyDefn<?> defn, String key)
{
Object value = defn.attributes().get(key);
return value != null && (Boolean) value;
}
public static boolean isSqlFunctionParameter(PropertyDefn<?> defn)
{
return getBoolean(defn, IS_SQL_FN_PARAM_KEY);
}
public static boolean isOptional(PropertyDefn<?> defn)
{
return getBoolean(defn, IS_SQL_FN_OPTIONAL);
}
public static String typeName(PropertyDefn<?> defn)
{
return (String) defn.attributes().get(TYPE_NAME);
}
public static Class<?> sqlParameterType(PropertyDefn<?> defn)
{
return (Class<?>) defn.attributes().get(SQL_JAVA_TYPE);
}
public static boolean isExternTableParameter(PropertyDefn<?> defn)
{
return getBoolean(defn, IS_PARAMETER);
}
public static Map<String, Object> merge(Map<String, Object> attribs1, Map<String, Object> attribs2)
{
if (attribs1 == null) {
return attribs2;
}
if (attribs2 == null) {
return attribs1;
}
Map<String, Object> merged = new HashMap<>(attribs1);
merged.putAll(attribs2);
return ImmutableMap.copyOf(merged);
}
}

View File

@ -19,8 +19,9 @@
package org.apache.druid.catalog.model;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import org.apache.druid.catalog.model.table.AbstractDatasourceDefn;
import org.apache.druid.catalog.model.table.DatasourceDefn;
import org.apache.druid.catalog.model.table.ExternalTableDefn;
import org.apache.druid.server.security.ResourceType;
@ -95,7 +96,7 @@ public class SchemaRegistryImpl implements SchemaRegistry
register(new SchemaDefnImpl(
TableId.DRUID_SCHEMA,
ResourceType.DATASOURCE,
AbstractDatasourceDefn.tableTypes()
ImmutableSet.of(DatasourceDefn.TABLE_TYPE)
));
register(new SchemaDefnImpl(
TableId.LOOKUP_SCHEMA,
@ -115,7 +116,7 @@ public class SchemaRegistryImpl implements SchemaRegistry
register(new SchemaDefnImpl(
TableId.EXTERNAL_SCHEMA,
EXTERNAL_RESOURCE,
ExternalTableDefn.tableTypes()
ImmutableSet.of(ExternalTableDefn.TABLE_TYPE)
));
register(new SchemaDefnImpl(
TableId.VIEW_SCHEMA,

View File

@ -21,7 +21,6 @@ package org.apache.druid.catalog.model;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.catalog.model.ModelProperties.PropertyDefn;
import org.apache.druid.java.util.common.IAE;
@ -47,13 +46,13 @@ public class TableDefn extends ObjectDefn
*/
public static final String DESCRIPTION_PROPERTY = "description";
private final Map<String, ColumnDefn> columnDefns;
private final Map<String, PropertyDefn<?>> columnProperties;
public TableDefn(
final String name,
final String typeValue,
final List<PropertyDefn<?>> properties,
final List<ColumnDefn> columnDefns
final List<PropertyDefn<?>> columnProperties
)
{
super(
@ -61,21 +60,21 @@ public class TableDefn extends ObjectDefn
typeValue,
CatalogUtils.concatLists(
Collections.singletonList(
new ModelProperties.StringPropertyDefn(DESCRIPTION_PROPERTY, null)
new ModelProperties.StringPropertyDefn(DESCRIPTION_PROPERTY)
),
properties
)
);
this.columnDefns = columnDefns == null ? Collections.emptyMap() : toColumnMap(columnDefns);
this.columnProperties = toPropertyMap(columnProperties);
}
public static Map<String, ColumnDefn> toColumnMap(final List<ColumnDefn> colTypes)
/**
* Called after the table definition is added to the registry, along with all
* other definitions. Allows external tables to look up additional information,
* such as the set of input formats.
*/
public void bind(TableDefnRegistry registry)
{
ImmutableMap.Builder<String, ColumnDefn> builder = ImmutableMap.builder();
for (ColumnDefn colType : colTypes) {
builder.put(colType.typeValue(), colType);
}
return builder.build();
}
/**
@ -86,10 +85,10 @@ public class TableDefn extends ObjectDefn
public void validate(ResolvedTable table)
{
validate(table.properties(), table.jsonMapper());
validateColumns(table.spec().columns(), table.jsonMapper());
validateColumns(table.spec().columns());
}
public void validateColumns(List<ColumnSpec> columns, ObjectMapper jsonMapper)
public void validateColumns(List<ColumnSpec> columns)
{
if (columns == null) {
return;
@ -99,26 +98,17 @@ public class TableDefn extends ObjectDefn
if (!names.add(colSpec.name())) {
throw new IAE("Duplicate column name: " + colSpec.name());
}
ColumnDefn.ResolvedColumn resolvedCol = resolveColumn(colSpec);
resolvedCol.validate(jsonMapper);
colSpec.validate();
validateColumn(colSpec);
}
}
/**
* Resolve the column type to produce a composite object that holds
* both the definition and the column spec.
* Table-specific validation of a column spec. Override for table definitions
* that need table-specific validation rules.
*/
public ColumnDefn.ResolvedColumn resolveColumn(ColumnSpec spec)
protected void validateColumn(ColumnSpec colSpec)
{
String type = spec.type();
if (Strings.isNullOrEmpty(type)) {
throw new IAE("The column type is required.");
}
ColumnDefn defn = columnDefns.get(type);
if (defn == null) {
throw new IAE("Column type [%s] is not valid for tables of type [%s].", type, typeValue());
}
return new ColumnDefn.ResolvedColumn(defn, spec);
}
/**
@ -162,15 +152,19 @@ public class TableDefn extends ObjectDefn
}
Integer index = original.get(col.name());
if (index == null) {
if (Strings.isNullOrEmpty(col.type())) {
throw new IAE("Column %d must have a type", i + 1);
}
merged.add(col);
} else {
ColumnDefn.ResolvedColumn resolvedCol = resolveColumn(columns.get(index));
merged.set(index, resolvedCol.merge(col).spec());
merged.set(index, mergeColumn(columns.get(index), col));
}
}
return merged;
}
private ColumnSpec mergeColumn(ColumnSpec existingCol, ColumnSpec update)
{
ColumnSpec revised = existingCol.merge(columnProperties, update);
revised.validate();
validateColumn(revised);
return revised;
}
}

View File

@ -23,14 +23,20 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.catalog.model.table.DatasourceDefn;
import org.apache.druid.catalog.model.table.HttpTableDefn;
import org.apache.druid.catalog.model.table.InlineTableDefn;
import org.apache.druid.catalog.model.table.LocalTableDefn;
import org.apache.druid.catalog.model.table.ExternalTableDefn;
import org.apache.druid.catalog.model.table.HttpInputSourceDefn;
import org.apache.druid.catalog.model.table.InlineInputSourceDefn;
import org.apache.druid.catalog.model.table.InputFormatDefn;
import org.apache.druid.catalog.model.table.InputFormats;
import org.apache.druid.catalog.model.table.InputSourceDefn;
import org.apache.druid.catalog.model.table.LocalInputSourceDefn;
import org.apache.druid.guice.annotations.Json;
import org.apache.druid.java.util.common.IAE;
import javax.annotation.Nullable;
import javax.inject.Inject;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
/**
@ -55,27 +61,63 @@ public class TableDefnRegistry
{
// Temporary list of Druid-define table definitions. This should come from
// Guice later to allow extensions to define table types.
private static final TableDefn[] TABLE_DEFNS = {
private static final List<TableDefn> BUILTIN_TABLE_DEFNS = Arrays.asList(
new DatasourceDefn(),
new InlineTableDefn(),
new HttpTableDefn(),
new LocalTableDefn()
};
new ExternalTableDefn()
);
private static final List<InputSourceDefn> BUILTIN_INPUT_SOURCE_DEFNS = Arrays.asList(
new InlineInputSourceDefn(),
new HttpInputSourceDefn(),
new LocalInputSourceDefn()
);
private static final List<InputFormatDefn> BUILTIN_INPUT_FORMAT_DEFNS = Arrays.asList(
new InputFormats.CsvFormatDefn(),
new InputFormats.DelimitedFormatDefn(),
new InputFormats.JsonFormatDefn()
);
private final Map<String, TableDefn> defns;
private final Map<String, TableDefn> tableDefns;
private final Map<String, InputSourceDefn> inputSourceDefns;
private final Map<String, InputFormatDefn> inputFormatDefns;
private final ObjectMapper jsonMapper;
public TableDefnRegistry(
final TableDefn[] defns,
@Nullable final List<TableDefn> tableDefnExtns,
@Nullable final List<InputSourceDefn> inputSourceDefnExtns,
@Nullable final List<InputFormatDefn> inputFormatDefnExtns,
final ObjectMapper jsonMapper
)
{
ImmutableMap.Builder<String, TableDefn> builder = ImmutableMap.builder();
for (TableDefn defn : defns) {
builder.put(defn.typeValue(), defn);
}
this.defns = builder.build();
this.jsonMapper = jsonMapper;
final List<TableDefn> tableDefns = CatalogUtils.concatLists(tableDefnExtns, BUILTIN_TABLE_DEFNS);
final List<InputSourceDefn> inputSourceDefns = CatalogUtils.concatLists(inputSourceDefnExtns, BUILTIN_INPUT_SOURCE_DEFNS);
final List<InputFormatDefn> inputFormatDefns = CatalogUtils.concatLists(inputFormatDefnExtns, BUILTIN_INPUT_FORMAT_DEFNS);
ImmutableMap.Builder<String, TableDefn> tableBuilder = ImmutableMap.builder();
for (TableDefn defn : tableDefns) {
tableBuilder.put(defn.typeValue(), defn);
}
this.tableDefns = tableBuilder.build();
ImmutableMap.Builder<String, InputSourceDefn> sourceBuilder = ImmutableMap.builder();
for (InputSourceDefn defn : inputSourceDefns) {
sourceBuilder.put(defn.typeValue(), defn);
}
this.inputSourceDefns = sourceBuilder.build();
ImmutableMap.Builder<String, InputFormatDefn> formatBuilder = ImmutableMap.builder();
for (InputFormatDefn defn : inputFormatDefns) {
formatBuilder.put(defn.typeValue(), defn);
}
this.inputFormatDefns = formatBuilder.build();
// Initialize all items once the entire set of bindings is defined.
for (InputSourceDefn defn : inputSourceDefns) {
defn.bind(this);
}
for (TableDefn defn : tableDefns) {
defn.bind(this);
}
}
@Inject
@ -83,12 +125,12 @@ public class TableDefnRegistry
@Json ObjectMapper jsonMapper
)
{
this(TABLE_DEFNS, jsonMapper);
this(null, null, null, jsonMapper);
}
public TableDefn defnFor(String type)
public TableDefn tableDefnFor(String type)
{
return defns.get(type);
return tableDefns.get(type);
}
public ObjectMapper jsonMapper()
@ -102,10 +144,33 @@ public class TableDefnRegistry
if (Strings.isNullOrEmpty(type)) {
throw new IAE("The table type is required.");
}
TableDefn defn = defns.get(type);
TableDefn defn = tableDefns.get(type);
if (defn == null) {
throw new IAE("Table type [%s] is not valid.", type);
}
return new ResolvedTable(defn, spec, jsonMapper);
}
/**
* Return input source definition for the given input source type name, or
* {@code null} if there is no such definition.
*/
public InputSourceDefn inputSourceDefnFor(String type)
{
return inputSourceDefns.get(type);
}
/**
* Return input format definition for the given input format type name, or
* {@code null} if there is no such definition.
*/
public InputFormatDefn inputFormatDefnFor(String type)
{
return inputFormatDefns.get(type);
}
public Map<String, InputFormatDefn> formats()
{
return inputFormatDefns;
}
}

View File

@ -36,7 +36,7 @@ public class TableId
public static final String CATALOG_SCHEMA = "INFORMATION_SCHEMA";
// Extra for MSQE
public static final String EXTERNAL_SCHEMA = "extern";
public static final String EXTERNAL_SCHEMA = "ext";
// Extra for views
public static final String VIEW_SCHEMA = "view";

View File

@ -1,57 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.facade;
import org.apache.druid.catalog.model.ColumnDefn.ResolvedColumn;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.segment.column.ColumnType;
public class ColumnFacade
{
public static class DatasourceColumnFacade extends ColumnFacade
{
public DatasourceColumnFacade(ResolvedColumn column)
{
super(column);
}
@Override
public ColumnType druidType()
{
if (Columns.isTimeColumn(column.spec().name())) {
return ColumnType.LONG;
}
return super.druidType();
}
}
protected final ResolvedColumn column;
public ColumnFacade(ResolvedColumn column)
{
this.column = column;
}
public ColumnType druidType()
{
String sqlType = column.spec().sqlType();
return sqlType == null ? null : Columns.druidType(sqlType);
}
}

View File

@ -21,9 +21,10 @@ package org.apache.druid.catalog.model.facade;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.table.AbstractDatasourceDefn;
import org.apache.druid.catalog.model.table.ClusterKeySpec;
import org.apache.druid.catalog.model.table.DatasourceDefn;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.logger.Logger;
import java.util.Collections;
import java.util.List;
@ -35,6 +36,9 @@ import java.util.List;
*/
public class DatasourceFacade extends TableFacade
{
private static final Logger LOG = new Logger(DatasourceFacade.class);
public DatasourceFacade(ResolvedTable resolved)
{
super(resolved);
@ -42,30 +46,47 @@ public class DatasourceFacade extends TableFacade
public String segmentGranularityString()
{
return stringProperty(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY);
return stringProperty(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY);
}
public Granularity segmentGranularity()
{
String value = stringProperty(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY);
return value == null ? null : CatalogUtils.asDruidGranularity(value);
String definedGranularity = segmentGranularityString();
return definedGranularity == null ? null : CatalogUtils.asDruidGranularity(definedGranularity);
}
public Integer targetSegmentRows()
{
return intProperty(AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY);
return intProperty(DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY);
}
@SuppressWarnings("unchecked")
public List<ClusterKeySpec> clusterKeys()
{
return (List<ClusterKeySpec>) property(AbstractDatasourceDefn.CLUSTER_KEYS_PROPERTY);
Object value = property(DatasourceDefn.CLUSTER_KEYS_PROPERTY);
if (value == null) {
return Collections.emptyList();
}
try {
return jsonMapper().convertValue(value, ClusterKeySpec.CLUSTER_KEY_LIST_TYPE_REF);
}
catch (Exception e) {
LOG.error("Failed to convert a catalog %s property of value [%s]",
DatasourceDefn.CLUSTER_KEYS_PROPERTY,
value
);
return Collections.emptyList();
}
}
@SuppressWarnings("unchecked")
public List<String> hiddenColumns()
{
Object value = property(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY);
Object value = property(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY);
return value == null ? Collections.emptyList() : (List<String>) value;
}
public boolean isSealed()
{
return booleanProperty(DatasourceDefn.SEALED_PROPERTY);
}
}

View File

@ -19,10 +19,13 @@
package org.apache.druid.catalog.model.facade;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ObjectFacade;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableSpec;
import org.apache.druid.segment.column.ColumnType;
import java.util.List;
import java.util.Map;
@ -55,4 +58,18 @@ public class TableFacade extends ObjectFacade
{
return spec().columns();
}
public static ColumnType druidType(ColumnSpec col)
{
if (Columns.isTimeColumn(col.name())) {
return ColumnType.LONG;
}
final String sqlType = col.sqlType();
return sqlType == null ? null : Columns.druidType(sqlType);
}
public ObjectMapper jsonMapper()
{
return resolved.jsonMapper();
}
}

View File

@ -1,155 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.ColumnDefn;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ModelProperties;
import org.apache.druid.catalog.model.ModelProperties.GranularityPropertyDefn;
import org.apache.druid.catalog.model.ModelProperties.PropertyDefn;
import org.apache.druid.catalog.model.ModelProperties.StringListPropertyDefn;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableDefn;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.StringUtils;
import java.util.Arrays;
import java.util.List;
import java.util.Set;
public class AbstractDatasourceDefn extends TableDefn
{
/**
* Segment grain at ingestion and initial compaction. Aging rules
* may override the value as segments age. If not provided here,
* then it must be provided at ingestion time.
*/
public static final String SEGMENT_GRANULARITY_PROPERTY = "segmentGranularity";
/**
* The target segment size at ingestion and initial compaction.
* If unset, then the system setting is used.
*/
public static final String TARGET_SEGMENT_ROWS_PROPERTY = "targetSegmentRows";
/**
* The clustering column names and sort order for each new segment.
*/
public static final String CLUSTER_KEYS_PROPERTY = "clusterKeys";
/**
* The set of existing columns to "delete" (actually, just hide) from the
* SQL layer. Used to "remove" unwanted columns to avoid the need to rewrite
* existing segments to accomplish the task.
*/
public static final String HIDDEN_COLUMNS_PROPERTY = "hiddenColumns";
public static class SegmentGranularityFieldDefn extends GranularityPropertyDefn
{
public SegmentGranularityFieldDefn()
{
super(SEGMENT_GRANULARITY_PROPERTY, null);
}
@Override
public void validate(Object value, ObjectMapper jsonMapper)
{
String gran = decode(value, jsonMapper);
if (Strings.isNullOrEmpty(gran)) {
throw new IAE("Segment granularity is required.");
}
validateGranularity(gran);
}
}
public static class HiddenColumnsDefn extends StringListPropertyDefn
{
public HiddenColumnsDefn()
{
super(HIDDEN_COLUMNS_PROPERTY, null);
}
@Override
public void validate(Object value, ObjectMapper jsonMapper)
{
if (value == null) {
return;
}
List<String> hiddenColumns = decode(value, jsonMapper);
for (String col : hiddenColumns) {
if (Columns.TIME_COLUMN.equals(col)) {
throw new IAE(
StringUtils.format("Cannot hide column %s", col)
);
}
}
}
}
public AbstractDatasourceDefn(
final String name,
final String typeValue,
final List<PropertyDefn<?>> properties,
final List<ColumnDefn> columnDefns
)
{
super(
name,
typeValue,
CatalogUtils.concatLists(
Arrays.asList(
new SegmentGranularityFieldDefn(),
new ModelProperties.IntPropertyDefn(TARGET_SEGMENT_ROWS_PROPERTY, null),
new ModelProperties.ListPropertyDefn<ClusterKeySpec>(
CLUSTER_KEYS_PROPERTY,
"cluster keys",
new TypeReference<List<ClusterKeySpec>>() { },
null
),
new HiddenColumnsDefn()
),
properties
),
columnDefns
);
}
public static boolean isDatasource(String tableType)
{
return DatasourceDefn.TABLE_TYPE.equals(tableType);
}
public static boolean isDatasource(ResolvedTable table)
{
return table.defn() instanceof AbstractDatasourceDefn;
}
public static Set<String> tableTypes()
{
return ImmutableSet.of(
DatasourceDefn.TABLE_TYPE
);
}
}

View File

@ -0,0 +1,278 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.TableDefnRegistry;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.utils.CollectionUtils;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* Base class for input source definitions.
*
* @see {@link FormattedInputSourceDefn} for the base class for (most) input formats
* which take an input format.
*/
public abstract class BaseInputSourceDefn implements InputSourceDefn
{
/**
* The "from-scratch" table function for this input source. The parameters
* are those defined by the subclass, and the apply simply turns around and
* asks the input source definition to do the conversion.
*/
public class AdHocTableFunction extends BaseTableFunction
{
public AdHocTableFunction(List<ParameterDefn> parameters)
{
super(parameters);
}
@Override
public ExternalTableSpec apply(
final String fnName,
final Map<String, Object> args,
final List<ColumnSpec> columns,
final ObjectMapper jsonMapper
)
{
requireSchema(fnName, columns);
return convertArgsToTable(args, columns, jsonMapper);
}
}
/**
* The "partial" table function that starts with a catalog external table spec, then
* uses SQL function arguments to "complete" (i.e. fill in) the missing properties to
* produce a complete table which is then converted to an external table which Calcite
* can use.
* <p>
* The set of parameters depends on the input source and on whether or not the catalog
* spec provides a format.
*/
public class PartialTableFunction extends BaseTableFunction
{
private final ResolvedExternalTable table;
public PartialTableFunction(final ResolvedExternalTable table, List<ParameterDefn> params)
{
super(params);
this.table = table;
}
@Override
public ExternalTableSpec apply(
final String fnName,
final Map<String, Object> args,
final List<ColumnSpec> columns,
final ObjectMapper jsonMapper
)
{
if (CollectionUtils.isNullOrEmpty(table.resolvedTable().spec().columns())) {
requireSchema(fnName, columns);
}
return convertCompletedTable(table, args, columns);
}
}
/**
* The one and only from-scratch table function for this input source. The
* function is defined a bind time, not construction time, since it typically
* needs visibility to the set of available input formats.
*/
private AdHocTableFunction adHocTableFn;
/**
* Overridden by each subclass to return the input source class to be
* used for JSON conversions.
*/
protected abstract Class<? extends InputSource> inputSourceClass();
@Override
public void bind(TableDefnRegistry registry)
{
this.adHocTableFn = defineAdHocTableFunction();
}
@Override
public void validate(ResolvedExternalTable table)
{
convertTableToSource(table);
}
/**
* Overridden by each subclass to define the parameters needed by each
* input source.
*/
protected abstract AdHocTableFunction defineAdHocTableFunction();
@Override
public TableFunction adHocTableFn()
{
return adHocTableFn;
}
/**
* Define a table "from scratch" using SQL function arguments.
*/
protected ExternalTableSpec convertArgsToTable(
final Map<String, Object> args,
final List<ColumnSpec> columns,
final ObjectMapper jsonMapper
)
{
return new ExternalTableSpec(
convertArgsToSource(args, jsonMapper),
convertArgsToFormat(args, columns, jsonMapper),
Columns.convertSignature(columns)
);
}
/**
* Convert the input source using arguments to a "from scratch" table function.
*/
protected InputSource convertArgsToSource(Map<String, Object> args, ObjectMapper jsonMapper)
{
final Map<String, Object> jsonMap = new HashMap<>();
auditInputSource(jsonMap);
convertArgsToSourceMap(jsonMap, args);
return convertSource(jsonMap, jsonMapper);
}
/**
* Convert SQL arguments to the corresponding "generic JSON" form in the given map.
* The map will then be adjusted and converted to the actual input source.
*/
protected abstract void convertArgsToSourceMap(Map<String, Object> jsonMap, Map<String, Object> args);
/**
* Convert SQL arguments, and the column schema, to an input format, if required.
*/
protected InputFormat convertArgsToFormat(Map<String, Object> args, List<ColumnSpec> columns, ObjectMapper jsonMapper)
{
return null;
}
/**
* Complete a partial table using the table function arguments and columns provided.
* The arguments match the set of parameters used for the function. The columns are
* provided if the SQL included an {@code EXTENDS} clause: the implementation should decide
* if columns are required (or allowed) depending on whether the partial spec already
* defines columns.
*
* @param table the partial table spec, with input source and format parsed into a
* generic Java map
* @param args the argument values provided in the SQL table function call. The arguments
* use the Java types defined in the parameter definitions.
* @param columns the set of columns (if any) from the SQL {@code EXTEND} clause
*
* @return an external table spec which Calcite can consume
*/
protected abstract ExternalTableSpec convertCompletedTable(
ResolvedExternalTable table,
Map<String, Object> args,
List<ColumnSpec> columns
);
@Override
public ExternalTableSpec convertTable(ResolvedExternalTable table)
{
return new ExternalTableSpec(
convertTableToSource(table),
convertTableToFormat(table),
Columns.convertSignature(table.resolvedTable().spec().columns())
);
}
/**
* Converts the input source given in a table spec. Since Druid input sources
* were not designed for the use by the catalog or SQL, some cleanup is done to
* simplify the parameters which the user provides.
*
* @param table the resolved external table spec
* @return the input source converted from the spec
*/
protected InputSource convertTableToSource(ResolvedExternalTable table)
{
return convertSource(
new HashMap<>(table.inputSourceMap),
table.resolvedTable().jsonMapper()
);
}
/**
* Convert from a generic Java map to the target input source using the object
* mapper provided. Translates Jackson errors into a generic unchecked error.
*/
protected InputSource convertSource(
final Map<String, Object> jsonMap,
final ObjectMapper jsonMapper
)
{
try {
auditInputSource(jsonMap);
return jsonMapper.convertValue(jsonMap, inputSourceClass());
}
catch (Exception e) {
throw new IAE(e, "Invalid input source specification");
}
}
/**
* Optional step to audit or adjust the input source properties prior to
* conversion via Jackson. Changes are made directly in the {@code jsonMap}.
*/
protected void auditInputSource(Map<String, Object> jsonMap)
{
}
/**
* Convert the format spec, if any, to an input format.
*/
protected abstract InputFormat convertTableToFormat(ResolvedExternalTable table);
/**
* Choose table or SQL-provided columns: table takes precedence.
*/
protected List<ColumnSpec> selectPartialTableColumns(
final ResolvedExternalTable table,
final List<ColumnSpec> columns
)
{
final List<ColumnSpec> tableCols = table.resolvedTable().spec().columns();
if (CollectionUtils.isNullOrEmpty(tableCols)) {
return columns;
} else if (!CollectionUtils.isNullOrEmpty(columns)) {
throw new IAE(
"Catalog definition for the %s input source already contains column definitions",
typeValue()
);
} else {
return tableCols;
}
}
}

View File

@ -0,0 +1,88 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.java.util.common.IAE;
import java.util.List;
/**
* Base implementation for a table function definition.
*
* @see {@link TableFunction}
*/
public abstract class BaseTableFunction implements TableFunction
{
public static class Parameter implements ParameterDefn
{
private final String name;
private final ParameterType type;
private final boolean optional;
public Parameter(String name, ParameterType type, boolean optional)
{
this.name = name;
this.type = type;
this.optional = optional;
}
@Override
public String name()
{
return name;
}
@Override
public ParameterType type()
{
return type;
}
@Override
public boolean isOptional()
{
return optional;
}
}
private final List<ParameterDefn> parameters;
public BaseTableFunction(List<ParameterDefn> parameters)
{
this.parameters = parameters;
}
@Override
public List<ParameterDefn> parameters()
{
return parameters;
}
protected void requireSchema(String fnName, List<ColumnSpec> columns)
{
if (columns == null) {
throw new IAE(
"The %s table function requires an EXTEND clause with a schema",
fnName
);
}
}
}

View File

@ -20,10 +20,14 @@
package org.apache.druid.catalog.model.table;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonInclude.Include;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Objects;
/**
@ -31,11 +35,14 @@ import java.util.Objects;
* the process of physically sorting data by a sort key. This class represents
* one column of that sort key. The key consists of a name and a sort direction.
* Sort direction is optional: omitted, ascending is assumed.
* (In Druid, clustering is always {@code NULLS LOW} in SQL parlance, so that attribute
* does not appear here.
* (In Druid, clustering is always {@code NULLS LOW} in SQL terms, so that attribute
* does not appear here.)
*/
public class ClusterKeySpec
{
public static final TypeReference<List<ClusterKeySpec>> CLUSTER_KEY_LIST_TYPE_REF =
new TypeReference<List<ClusterKeySpec>>() { };
private final String expr;
private final boolean desc;
@ -56,6 +63,7 @@ public class ClusterKeySpec
}
@JsonProperty("desc")
@JsonInclude(Include.NON_DEFAULT)
public boolean desc()
{
return desc;

View File

@ -19,65 +19,147 @@
package org.apache.druid.catalog.model.table;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.catalog.model.ColumnDefn;
import com.google.common.base.Strings;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ModelProperties;
import org.apache.druid.catalog.model.ModelProperties.GranularityPropertyDefn;
import org.apache.druid.catalog.model.ModelProperties.StringListPropertyDefn;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableDefn;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.StringUtils;
import java.util.Collections;
import java.util.Arrays;
import java.util.List;
/**
* Definition of a Druid datasource. The datasource may use rollup, but rollup
* is not represented in the catalog: it is just a way that applications store
* data into a table.
*/
public class DatasourceDefn extends AbstractDatasourceDefn
public class DatasourceDefn extends TableDefn
{
/**
* Definition of a column in a datasource.
* Segment grain at ingestion and initial compaction. Aging rules
* may override the value as segments age. If not provided here,
* then it must be provided at ingestion time.
*/
public static class DatasourceColumnDefn extends ColumnDefn
{
public static final String COLUMN_TYPE = "column";
public static final String SEGMENT_GRANULARITY_PROPERTY = "segmentGranularity";
public DatasourceColumnDefn()
/**
* Catalog property value for the "all time" granularity.
*/
public static final String ALL_GRANULARITY = "ALL";
/**
* The target segment size at ingestion and initial compaction.
* If unset, then the system setting is used.
*/
public static final String TARGET_SEGMENT_ROWS_PROPERTY = "targetSegmentRows";
/**
* The clustering column names and sort order for each new segment.
*/
public static final String CLUSTER_KEYS_PROPERTY = "clusterKeys";
/**
* The set of existing columns to "delete" (actually, just hide) from the
* SQL layer. Used to "remove" unwanted columns to avoid the need to rewrite
* existing segments to accomplish the task.
*/
public static final String HIDDEN_COLUMNS_PROPERTY = "hiddenColumns";
/**
* By default: columns are optional hints. If a datasource has columns defined,
* well validate them, but MSQ and other tools are free to create additional columns.
* That is, we assume "auto-discovered" columns by default. However, in some use cases,
* the schema may be carefully designed. This is especially true for ETL use cases in
* which multiple input schemas are mapped into a single datasource schema designed for
* ease of end user use. In this second use case, we may want to reject an attempt to
* ingest columns other than those in the schema. To do that, set {@code sealed = true}.
* In other words, "sealed" mode works like a traditional RDBMS.
*/
public static final String SEALED_PROPERTY = "sealed";
public static final String TABLE_TYPE = "datasource";
public static class SegmentGranularityFieldDefn extends GranularityPropertyDefn
{
public SegmentGranularityFieldDefn()
{
super(
"Column",
COLUMN_TYPE,
null
);
super(SEGMENT_GRANULARITY_PROPERTY);
}
@Override
public void validate(ColumnSpec spec, ObjectMapper jsonMapper)
public void validate(Object value, ObjectMapper jsonMapper)
{
super.validate(spec, jsonMapper);
validateScalarColumn(spec);
if (Columns.isTimeColumn(spec.name()) &&
spec.sqlType() != null &&
!Columns.TIMESTAMP.equalsIgnoreCase(spec.sqlType())) {
throw new IAE(StringUtils.format(
"%s column must have no SQL type or SQL type %s",
Columns.TIME_COLUMN,
Columns.TIMESTAMP
)
);
String gran = decode(value, jsonMapper);
if (Strings.isNullOrEmpty(gran)) {
throw new IAE("Segment granularity is required.");
}
CatalogUtils.validateGranularity(gran);
}
}
public static final String TABLE_TYPE = "datasource";
public static class HiddenColumnsDefn extends StringListPropertyDefn
{
public HiddenColumnsDefn()
{
super(HIDDEN_COLUMNS_PROPERTY);
}
@Override
public void validate(Object value, ObjectMapper jsonMapper)
{
if (value == null) {
return;
}
List<String> hiddenColumns = decode(value, jsonMapper);
for (String col : hiddenColumns) {
if (Columns.TIME_COLUMN.equals(col)) {
throw new IAE(
StringUtils.format("Cannot hide column %s", col)
);
}
}
}
}
public DatasourceDefn()
{
super(
"Datasource",
TABLE_TYPE,
null,
Collections.singletonList(new DatasourceDefn.DatasourceColumnDefn())
Arrays.asList(
new SegmentGranularityFieldDefn(),
new ModelProperties.IntPropertyDefn(TARGET_SEGMENT_ROWS_PROPERTY),
new ModelProperties.ListPropertyDefn<ClusterKeySpec>(
CLUSTER_KEYS_PROPERTY,
"cluster keys",
new TypeReference<List<ClusterKeySpec>>() { }
),
new HiddenColumnsDefn(),
new ModelProperties.BooleanPropertyDefn(SEALED_PROPERTY)
),
null
);
}
@Override
protected void validateColumn(ColumnSpec spec)
{
super.validateColumn(spec);
if (Columns.isTimeColumn(spec.name()) && spec.sqlType() != null) {
// Validate type in next PR
}
}
public static boolean isDatasource(String tableType)
{
return DatasourceDefn.TABLE_TYPE.equals(tableType);
}
public static boolean isDatasource(ResolvedTable table)
{
return table.defn() instanceof DatasourceDefn;
}
}

View File

@ -19,254 +19,286 @@
package org.apache.druid.catalog.model.table;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.ColumnDefn;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.annotations.VisibleForTesting;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ModelProperties;
import org.apache.druid.catalog.model.ModelProperties.ObjectPropertyDefn;
import org.apache.druid.catalog.model.ModelProperties.PropertyDefn;
import org.apache.druid.catalog.model.ParameterizedDefn;
import org.apache.druid.catalog.model.PropertyAttributes;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableDefn;
import org.apache.druid.catalog.model.table.InputFormats.InputFormatDefn;
import org.apache.druid.catalog.model.TableDefnRegistry;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Arrays;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
/**
* Definition of an external input source, primarily for ingestion.
* Definition of an external table, primarily for ingestion.
* The components are derived from those for Druid ingestion: an
* input source, a format and a set of columns. Also provides
* properties, as do all table definitions.
*
* <h4>Partial Tables and Connections</h4>
*
* An input source is a template for an external table. The input
* source says how to get data, and optionally the format and structure of that data.
* Since Druid never ingests the same data twice, the actual external table needs
* details that says which data to read on any specific ingestion. Thus, an external
* table is usually a "partial table": all the information that remains constant
* across ingestions, but without the information that changes. The changing
* information is typically the list of files (or objects or URLs) to ingest.
* <p>
* The external table implements the mechanism for parameterized tables,
* but does not implement the {@link ParameterizedDefn} interface itself.
* Tables which are parameterized implement that interface to expose
* methods defined here.
* The pattern is:<br>
* {@code external table spec + parameters --> external table}
* <p>
* Since an input source is a parameterized (partial) external table, we can reuse
* the table metadata structures and APIs, avoiding the need to have a separate (but
* otherwise identical) structure for external tables.
*
* An external table can be thought of as a "connection", though Druid does not
* use that term. When used as a connection, the external table spec will omit the
* format. Instead, the format will also be provided at ingest time, along with the
* list of tables (or objects.)
* <p>
* To keep all this straight, we adopt the following terms:
* <dl>
* <dt>External table spec</dt>
* <dd>The JSON serialized version of an external table which can be partial or
* complete. The spec is a named entry in the Druid catalog</dd>
* <dt>Complete spec</dt>
* <dd>An external table spec that provides all information needed to access an
* external table. Each use identifies the same set of data. Useful if MSQ is used
* to query an external data source. A complete spec can be referenced as a
* first-class table in a {@code FROM} clause in an MSQ query.</dd>
* <dt>Partial spec</dt>
* <dd>An external table spec that omits some information. That information must
* be provided at query time in the form of a {@code TABLE} function. If the partial
* spec includes a format, then it is essentially a <i>partial table</i>. If it
* omits the format, then it is essentially a <i>connection</i>.</dd>
* <dt>Completed table</dt>
* <dd>The full external table that results from a partial spec and a set of SQL
* table function parameters.</dd>
* <dt>Ad-hoc table</dt>
* <dd>Users can define an external table using the generic {@code EXTERN} function
* or one of the input-source-specific functions. In this case, there is no
* catalog entry: all information comes from the SQL table function</dd>
* <dt>Partial table function</dt>
* <dd>The SQL table function used to "complete" a partial spec. The function
* defines parameters to fill in the missing information. The function is generated
* on demand and has the same name as the catalog entry for the partial spec.
* The function will include parameters for format if the catalog spec does not
* specify a format. Else, the format parameters are omitted and the completed
* table uses the format provided in the catalog spec.</dd>
* <dt>Ad-hoc table function</dt>
* <dd>The SQL table function used to create an ad-hoc external table. The function
* as a name defined by the {@link InputFormatDefn}, and has parameters for all
* support formats: the user must specify all input source and format properties.</dd>
* </dl>
*
* <h4>External Table Structure</h4>
*
* The external table is generic: it represents all valid combinations of input
* sources and formats. Rather than have a different table definition for each, we
* instead split out input sources and formats into their own definitions, and those
* definitions are integrated and used by this external table class. As a result,
* the {@code properties} field will contain the {@code source} property which has
* the JSON serialized form of the input source (minus items to be parameterized.)
* <p>
* Similarly, if the external table also defines a format (rather than requiring the
* format at ingest time), then the {@code format} property holds the JSON-serialized
* form of the input format, minus columns. The columns can be provided in the spec,
* in the {@code columns} field. The {@link InputFormatDefn} converts the columns to
* the form needed by the input format.
* <p>
* Druid's input sources all require formats. However, some sources may not actually
* need the format. A JDBC input source for example, needs no format. In other cases,
* there may be a subset of formats. Each {@link InputSourceDefn} is responsible for
* working out which formats (if any) are required. This class is agnostic about whether
* the format is supplied. (Remember that, when used as a connection, the external table
* will provide no format until ingest time.)
* <p>
* By contrast, the input source is always required.
*
* <h4>Data Formats and Conversions</h4>
*
* Much of the code here handles conversion of an external table specification to
* the form needed by SQL. Since SQL is not visible here, we instead create an
* instance of {@link ExternalTableSpec} which holds the input source, input format
* and row signature in the form required by SQL.
* <p>
* This class handles table specifications in three forms:
* <ol>
* <li>From a fully-defined table specification, converted to a {@code ExternalTableSpec}
* by the {@link #convert(ResolvedTable)} function.</li>
* <li>From a fully-defined set of arguments to a SQL table function. The
* {@link InputSourceDefn#adHocTableFn()} method provides the function definition which
* handles the conversion.</li>
* <li>From a partially-defined table specification in the catalog, augmented by
* parameters passed from a SQL function. The {@link #tableFn(ResolvedTable)} method
* creates the required function by caching the table spec. That function then combines
* the parameters to produce the required {@code ExternalTableSpec}.</li>
* </ol>
* <p>
* To handle these formats, and the need to adjust JSON, conversion to an
* {@code ExternalTableSpec} occurs in multiple steps:
* <ul>
* <li>When using a table spec, the serialized JSON is first converted to a generic
* Java map: one for the input source, another for the format.</li>
* <li>When using a SQL function, the SQL arguments are converted (if needed) and
* written into a Java map. If the function references an existing table spec: then
* the JSON map is first populated with the deserialized spec.</li>
* <li>Validation and/or adjustments are made to the Java map. Adjustments are
* those described elsewhere in this Javadoc.</li>
* <li>The column specifications from either SQL or the table spec are converted to
* a list of column names, and placed into the Java map for the input format.</li>
* <li>The maps are converted to the {@link InputSource} or {@link InputFormat}
* objects using a Jackson conversion.</li>
* </ul>
* The actual conversions are handled in the {@link InputFormatDefn} and
* {@link InputFormatDefn} classes, either directly (for a fully-defined table
* function) or starting here (for other use cases).
*
* <h4>Property and Parameter Names</h4>
*
* Pay careful attention to names: the names may be different in each of the above
* cases:
* <ul>
* <li>The table specification stores the input source and input format specs using
* the names defined by the classes themselves. That is, the table spec holds a string
* that represents the Jackson-serialized form of those classes. In some cases, the
* JSON can be a subset: some sources and formats have obscure checks, or options which
* are not available via this path. The code that does conversions will adjust the JSON
* prior to conversion. Each JSON object has a type field: the value of that type
* field must match that defined in the Jackson annotations for the corresponding
* class.</li>
* <li>SQL table functions use argument names that are typically selected for user
* convenience, and may not be the same as the JSON field name. For example, a field
* name may be a SQL reserved word, or may be overly long, or may be obscure. The code
* for each input source and input format definition does the needed conversion.</li>
* <li>Each input source and input format has a type. The input format type is given,
* in SQL by the {@code format} property. The format type name is typically the same
* as the JSON type name, but need not be.</li>
* </ul>
*
* <h4>Extensions</h4>
*
* This class is designed to work both with "well known" Druid input sources and formats,
* and those defined in an extension. For extension-defined sources and formats to work,
* the extension must define an {@link InputSourceDefn} or {@link InputFormatDefn} which
* are put into the {@link TableDefnRegistry} and thus available to this class. The result
* is that this class is ignorant of the actual details of sources and formats: it instead
* delegates to the input source and input format definitions for that work.
* <p>
* Input sources and input formats defined in an extension are considered "ephemeral":
* they can go away if the corresponding extension is removed from the system. In that
* case, any table functions defined by those extensions are no longer available, and
* any SQL statements that use those functions will no longer work. The catalog may contain
* an external table spec that references those definitions. Such specs will continue to
* reside in the catalog, and can be retrieved, but they will fail any query that attempts
* to reference them.
*/
public abstract class ExternalTableDefn extends TableDefn
public class ExternalTableDefn extends TableDefn
{
public static final String EXTERNAL_COLUMN_TYPE = "extern";
public abstract static class FormattedExternalTableDefn extends ExternalTableDefn
{
public static final String FORMAT_PROPERTY = "format";
private final Map<String, InputFormatDefn> formats;
public FormattedExternalTableDefn(
final String name,
final String typeValue,
final List<PropertyDefn<?>> properties,
final List<ColumnDefn> columnDefns,
final List<InputFormatDefn> formats
)
{
super(
name,
typeValue,
addFormatProperties(properties, formats),
columnDefns
);
ImmutableMap.Builder<String, InputFormatDefn> builder = ImmutableMap.builder();
for (InputFormatDefn format : formats) {
builder.put(format.typeTag(), format);
}
this.formats = builder.build();
}
/**
* Add format properties to the base set, in the order of the formats,
* in the order defined by the format. Allow same-named properties across
* formats, as long as the types are the same.
*/
private static List<PropertyDefn<?>> addFormatProperties(
final List<PropertyDefn<?>> properties,
final List<InputFormatDefn> formats
)
{
List<PropertyDefn<?>> toAdd = new ArrayList<>();
PropertyDefn<?> formatProp = new ModelProperties.StringPropertyDefn(FORMAT_PROPERTY, PropertyAttributes.SQL_FN_PARAM);
toAdd.add(formatProp);
Map<String, PropertyDefn<?>> formatProps = new HashMap<>();
for (InputFormatDefn format : formats) {
for (PropertyDefn<?> prop : format.properties()) {
PropertyDefn<?> existing = formatProps.putIfAbsent(prop.name(), prop);
if (existing == null) {
toAdd.add(prop);
} else if (existing.getClass() != prop.getClass()) {
throw new ISE(
"Format %s, property %s of class %s conflicts with another format property of class %s",
format.name(),
prop.name(),
prop.getClass().getSimpleName(),
existing.getClass().getSimpleName()
);
}
}
}
return CatalogUtils.concatLists(properties, toAdd);
}
@Override
protected InputFormat convertFormat(ResolvedTable table)
{
return formatDefn(table).convert(table);
}
protected InputFormatDefn formatDefn(ResolvedTable table)
{
String formatTag = table.stringProperty(FORMAT_PROPERTY);
if (formatTag == null) {
throw new IAE("%s property must be set", FORMAT_PROPERTY);
}
InputFormatDefn formatDefn = formats.get(formatTag);
if (formatDefn == null) {
throw new IAE(
"Format type [%s] for property %s is not valid",
formatTag,
FORMAT_PROPERTY
);
}
return formatDefn;
}
@Override
public void validate(ResolvedTable table)
{
super.validate(table);
formatDefn(table).validate(table);
List<ColumnSpec> columns = table.spec().columns();
if (columns == null || columns.isEmpty()) {
throw new IAE(
"An external table of type %s must specify one or more columns",
table.spec().type()
);
}
}
}
/**
* Identifier for external tables.
*/
public static final String TABLE_TYPE = "extern";
/**
* Definition of a column in a detail (non-rollup) datasource.
* Property which holds the input source specification as serialized as JSON.
*/
public static class ExternalColumnDefn extends ColumnDefn
{
public ExternalColumnDefn()
{
super(
"Column",
EXTERNAL_COLUMN_TYPE,
null
);
}
@Override
public void validate(ColumnSpec spec, ObjectMapper jsonMapper)
{
super.validate(spec, jsonMapper);
validateScalarColumn(spec);
}
}
protected static final ExternalColumnDefn INPUT_COLUMN_DEFN = new ExternalColumnDefn();
private final List<PropertyDefn<?>> fields;
public ExternalTableDefn(
final String name,
final String typeValue,
final List<PropertyDefn<?>> fields,
final List<ColumnDefn> columnDefns
)
{
super(name, typeValue, fields, columnDefns);
this.fields = fields;
}
public List<PropertyDefn<?>> parameters()
{
return fields.stream()
.filter(f -> PropertyAttributes.isExternTableParameter(f))
.collect(Collectors.toList());
}
public List<PropertyDefn<?>> tableFunctionParameters()
{
return fields.stream()
.filter(f -> PropertyAttributes.isSqlFunctionParameter(f))
.collect(Collectors.toList());
}
public static final String SOURCE_PROPERTY = "source";
/**
* Merge parameters provided by a SQL table function with the catalog information
* provided in the resolved table to produce a new resolved table used for a
* specific query.
* Property which holds the optional input format specification, serialized as JSON.
*/
public abstract ResolvedTable mergeParameters(ResolvedTable table, Map<String, Object> values);
public static final String FORMAT_PROPERTY = "format";
public ExternalTableSpec convertToExtern(ResolvedTable table)
/**
* Definition of the input source property.
*/
private static final PropertyDefn<InputSource> SOURCE_PROPERTY_DEFN =
new ObjectPropertyDefn<InputSource>(SOURCE_PROPERTY, InputSource.class);
/**
* Definition of the input format property.
*/
private static final PropertyDefn<InputFormat> FORMAT_PROPERTY_DEFN =
new ObjectPropertyDefn<InputFormat>(FORMAT_PROPERTY, InputFormat.class);
/**
* Type reference used to deserialize JSON to a generic map.
*/
@VisibleForTesting
public static final TypeReference<Map<String, Object>> MAP_TYPE_REF = new TypeReference<Map<String, Object>>() { };
private TableDefnRegistry registry;
public ExternalTableDefn()
{
return new ExternalTableSpec(
convertSource(table),
convertFormat(table),
Columns.convertSignature(table.spec())
super(
"External table",
TABLE_TYPE,
Arrays.asList(
SOURCE_PROPERTY_DEFN,
FORMAT_PROPERTY_DEFN
),
null
);
}
/**
* Convert a resolved table to the Druid internal {@link InputSource}
* object required by an MSQ query.
*/
protected abstract InputSource convertSource(ResolvedTable table);
@Override
public void bind(TableDefnRegistry registry)
{
this.registry = registry;
}
@Override
public void validate(ResolvedTable table)
{
for (PropertyDefn<?> propDefn : properties().values()) {
// Validate everything except the input source and input format: those
// are done elsewhere since they are complex and require context.
if (propDefn != SOURCE_PROPERTY_DEFN && propDefn != FORMAT_PROPERTY_DEFN) {
propDefn.validate(table.property(propDefn.name()), table.jsonMapper());
}
}
validateColumns(table.spec().columns());
new ResolvedExternalTable(table).validate(registry);
}
/**
* Convert a resolved table to the Druid internal {@link InputFormat}
* object required by an MSQ query. Not all input sources require a format.
* Return a table function definition for a partial table as given by
* the catalog table spec. The function defines parameters to gather the
* values needed to convert the partial table into a fully-defined table
* which can be converted to an {@link ExternalTableSpec}.
*/
protected InputFormat convertFormat(ResolvedTable table)
public TableFunction tableFn(ResolvedTable table)
{
return null;
return new ResolvedExternalTable(table).resolve(registry).tableFn();
}
protected InputSource convertObject(
final ObjectMapper jsonMapper,
final Map<String, Object> jsonMap,
final Class<? extends InputSource> targetClass
)
@Override
protected void validateColumn(ColumnSpec colSpec)
{
try {
return jsonMapper.convertValue(jsonMap, targetClass);
}
catch (Exception e) {
throw new IAE(e, "Invalid table specification");
}
// Validate type in next PR
}
public ExternalTableSpec applyParameters(ResolvedTable table, Map<String, Object> parameters)
/**
* Return the {@link ExternalTableSpec} for a catalog entry for a
* fully-defined table. This form exists for completeness, since ingestion never
* reads the same data twice. This form is handy for tests, and will become
* generally useful when MSQ fully supports queries and those queries can
* read from external tables.
*/
public ExternalTableSpec convert(ResolvedTable table)
{
ResolvedTable revised = mergeParameters(table, parameters);
return convertToExtern(revised);
return new ResolvedExternalTable(table).resolve(registry).convert();
}
public static Set<String> tableTypes()
public static boolean isExternalTable(ResolvedTable table)
{
// Known input tables. Get this from a registry later.
return ImmutableSet.of(
InlineTableDefn.TABLE_TYPE,
HttpTableDefn.TABLE_TYPE,
LocalTableDefn.TABLE_TYPE
);
return TABLE_TYPE.equals(table.spec().type());
}
}

View File

@ -28,13 +28,13 @@ import javax.annotation.Nullable;
/**
* Catalog form of an external table specification used to pass along the three
* components needed for an external table in MSQ ingest. Just like
* {@code ExternalTableSource}, except that the parameters are not required
* to be non-null.
* {@code ExternalTableSource}, except that the class is visible in this
* module.
*/
public class ExternalTableSpec
{
@Nullable public final InputSource inputSource;
@Nullable public final InputFormat inputFormat;
public final InputSource inputSource;
public final InputFormat inputFormat;
@Nullable public final RowSignature signature;
public ExternalTableSpec(

View File

@ -0,0 +1,196 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.TableDefnRegistry;
import org.apache.druid.catalog.model.table.BaseTableFunction.Parameter;
import org.apache.druid.catalog.model.table.TableFunction.ParameterDefn;
import org.apache.druid.catalog.model.table.TableFunction.ParameterType;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.utils.CollectionUtils;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* Base class for input formats that require an input format (which is most of them.)
* By default, an input source supports all formats defined in the table registry, but
* specific input sources can be more restrictive. The list of formats defines the list
* of SQL function arguments available when defining a table from scratch.
*/
public abstract class FormattedInputSourceDefn extends BaseInputSourceDefn
{
public static final String FORMAT_PARAMETER = "format";
private Map<String, InputFormatDefn> formats;
@Override
public void bind(TableDefnRegistry registry)
{
formats = registry.formats();
super.bind(registry);
}
@Override
public void validate(ResolvedExternalTable table)
{
final boolean hasColumns = !CollectionUtils.isNullOrEmpty(table.resolvedTable().spec().columns());
final boolean hasFormat = table.inputFormatMap != null;
if (hasColumns && !hasFormat) {
throw new IAE("If an external table provides columns, it must also provide a format");
}
if (!hasColumns && hasFormat) {
throw new IAE("If an external table provides a format, it must also provide columns");
}
super.validate(table);
}
@Override
protected AdHocTableFunction defineAdHocTableFunction()
{
List<ParameterDefn> fullTableParams = adHocTableFnParameters();
List<ParameterDefn> allParams = addFormatParameters(fullTableParams);
return new AdHocTableFunction(allParams);
}
/**
* Overridden by subclasses to provide the list of table function parameters for
* this specific input format. This list is combined with parameters for input
* formats. The method is called only once per run.
*/
protected abstract List<ParameterDefn> adHocTableFnParameters();
/**
* Add format properties to the base set, in the order of the formats,
* in the order defined by the format. Allow same-named properties across
* formats, as long as the types are the same.
*/
protected List<ParameterDefn> addFormatParameters(
final List<ParameterDefn> properties
)
{
final List<ParameterDefn> toAdd = new ArrayList<>();
final ParameterDefn formatProp = new Parameter(FORMAT_PARAMETER, ParameterType.VARCHAR, false);
toAdd.add(formatProp);
final Map<String, ParameterDefn> formatProps = new HashMap<>();
for (InputFormatDefn format : formats.values()) {
for (ParameterDefn prop : format.parameters()) {
final ParameterDefn existing = formatProps.putIfAbsent(prop.name(), prop);
if (existing == null) {
toAdd.add(prop);
} else if (existing.type() != prop.type()) {
throw new ISE(
"Format %s, property %s of class %s conflicts with another format property of class %s",
format.typeValue(),
prop.name(),
prop.type().sqlName(),
existing.type().sqlName()
);
}
}
}
return CatalogUtils.concatLists(properties, toAdd);
}
@Override
protected InputFormat convertTableToFormat(ResolvedExternalTable table)
{
final String formatTag = CatalogUtils.getString(table.inputFormatMap, InputFormat.TYPE_PROPERTY);
if (formatTag == null) {
throw new IAE("%s property must be set", InputFormat.TYPE_PROPERTY);
}
final InputFormatDefn formatDefn = formats.get(formatTag);
if (formatDefn == null) {
throw new IAE(
"Format type [%s] for property %s is not valid",
formatTag,
InputFormat.TYPE_PROPERTY
);
}
return formatDefn.convertFromTable(table);
}
@Override
protected InputFormat convertArgsToFormat(Map<String, Object> args, List<ColumnSpec> columns, ObjectMapper jsonMapper)
{
final String formatTag = CatalogUtils.getString(args, FORMAT_PARAMETER);
if (formatTag == null) {
throw new IAE("%s parameter must be set", FORMAT_PARAMETER);
}
final InputFormatDefn formatDefn = formats.get(formatTag);
if (formatDefn == null) {
throw new IAE(
"Format type [%s] for property %s is not valid",
formatTag,
FORMAT_PARAMETER
);
}
return formatDefn.convertFromArgs(args, columns, jsonMapper);
}
/**
* Converted a formatted external table given the table definition, function
* args, columns and the merged generic JSON map representing the input source.
*
* @param table the resolved external table from the catalog
* @param args values of arguments from an SQL table function. Here we consider
* only the format arguments; input source arguments should already
* have been handled
* @param columns the set of columns provided by the SQL table function
* @param sourceMap the generic JSON map for the input source with function
* parameters merged into the definition in the catalog
*
* @return an external table spec to be used to create a Calcite table
*/
protected ExternalTableSpec convertPartialFormattedTable(
final ResolvedExternalTable table,
final Map<String, Object> args,
final List<ColumnSpec> columns,
final Map<String, Object> sourceMap
)
{
final ObjectMapper jsonMapper = table.resolvedTable().jsonMapper();
// Choose table or SQL-provided columns: table takes precedence.
final List<ColumnSpec> completedCols = selectPartialTableColumns(table, columns);
// Get the format from the table, if defined, else from arguments.
final InputFormat inputFormat;
if (table.inputFormatMap == null) {
inputFormat = convertArgsToFormat(args, completedCols, jsonMapper);
} else {
inputFormat = convertTableToFormat(table);
}
return new ExternalTableSpec(
convertSource(sourceMap, jsonMapper),
inputFormat,
Columns.convertSignature(completedCols)
);
}
}

View File

@ -0,0 +1,307 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.table.BaseTableFunction.Parameter;
import org.apache.druid.catalog.model.table.TableFunction.ParameterDefn;
import org.apache.druid.catalog.model.table.TableFunction.ParameterType;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.impl.HttpInputSource;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.metadata.DefaultPasswordProvider;
import org.apache.druid.metadata.EnvironmentVariablePasswordProvider;
import org.apache.druid.utils.CollectionUtils;
import java.net.URI;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
/**
* Definition of an HTTP input source source.
* <p>
* Provides a parameterized form where the user defines a value for the
* {@code uriTemplate} table property in the table spec, then provides the partial URLs
* in a table function to use for that one query. The final URIs are created by combining
* the template and the arguments. Example:
* <li>{@code uriTemplate} property): "http://example.com/data/kttm-{}.json"</li>
* <li>{@code uris} function argument: "22-Nov-21, 22-Nov-22"</li>
* </ul>
* <p>
* When the template is used, the format is optional: it can be provided either with
* the table spec or at runtime, depending on what the user wants to accomplish. In the
* above, where the ".json" is encoded in the template, it makes sense to include the format
* with the spec. If the template was "http://example.com/data/{}", and the data comes in
* multiple formats, it might make sense to specify the format in the query. In this case,
* the table spec acts more like a connection.
* <p>
* If the template is not used, then the {@code uris} property must be provided in the
* table spec, along with the corresponding format.
* <p>
* The above semantics make a bit more sense when we realize that the spec can also
* provide a user name and password. When those are provided, then the input source must
* name a single site: the one for which the credentials are valid. Given this, the only
* table spec that makes sense is one where the URI is defined: either as a template or
* explicitly.
* <p>
* When used as an ad-hoc function, the user specifies the uris and optional user name
* and password: the template is not available (or useful) in the ad-hoc case.
* <p>
* Table function parameters are cleaned up relative to the input source field names to
* make them a bit easier to use.
*/
public class HttpInputSourceDefn extends FormattedInputSourceDefn
{
public static final String TYPE_KEY = HttpInputSource.TYPE_KEY;
// Catalog properties that map to fields in the HttpInputSource. See
// that class for the meaning of these properties.
public static final String URI_TEMPLATE_PROPERTY = "uriTemplate";
public static final String URIS_PARAMETER = "uris";
// Note, cannot be the simpler "user" since USER is a reserved word in SQL
// and we don't want to require users to quote "user" each time it is used.
public static final String USER_PARAMETER = "userName";
public static final String PASSWORD_PARAMETER = "password";
public static final String PASSWORD_ENV_VAR_PARAMETER = "passwordEnvVar";
private static final List<ParameterDefn> URI_PARAMS = Collections.singletonList(
new Parameter(URIS_PARAMETER, ParameterType.VARCHAR_ARRAY, true)
);
private static final List<ParameterDefn> USER_PWD_PARAMS = Arrays.asList(
new Parameter(USER_PARAMETER, ParameterType.VARCHAR, true),
new Parameter(PASSWORD_PARAMETER, ParameterType.VARCHAR, true),
new Parameter(PASSWORD_ENV_VAR_PARAMETER, ParameterType.VARCHAR, true)
);
// Field names in the HttpInputSource
private static final String URIS_FIELD = "uris";
private static final String PASSWORD_FIELD = "httpAuthenticationPassword";
private static final String USERNAME_FIELD = "httpAuthenticationUsername";
@Override
public String typeValue()
{
return TYPE_KEY;
}
@Override
protected Class<? extends InputSource> inputSourceClass()
{
return HttpInputSource.class;
}
@Override
public void validate(ResolvedExternalTable table)
{
final Map<String, Object> sourceMap = table.inputSourceMap;
final boolean hasUri = sourceMap.containsKey(URIS_FIELD);
final String uriTemplate = table.resolvedTable().stringProperty(URI_TEMPLATE_PROPERTY);
final boolean hasTemplate = uriTemplate != null;
final boolean hasFormat = table.inputFormatMap != null;
final boolean hasColumns = !CollectionUtils.isNullOrEmpty(table.resolvedTable().spec().columns());
if (!hasUri && !hasTemplate) {
throw new IAE(
"External HTTP tables must provide either a URI or a %s property",
URI_TEMPLATE_PROPERTY
);
}
if (hasUri && hasTemplate) {
throw new IAE(
"External HTTP tables must provide only one of a URI or a %s property",
URI_TEMPLATE_PROPERTY
);
}
if (hasUri && !hasFormat) {
throw new IAE(
"An external HTTP table with a URI must also provide the corresponding format"
);
}
if (hasUri && !hasColumns) {
throw new IAE(
"An external HTTP table with a URI must also provide the corresponding columns"
);
}
if (hasTemplate) {
// Verify the template
templateMatcher(uriTemplate);
// Patch in a dummy URI so that validation of the rest of the fields
// will pass.
try {
sourceMap.put(
URIS_FIELD,
Collections.singletonList(new URI("https://bogus.com/file"))
);
}
catch (Exception e) {
throw new ISE(e, "URI parse failed");
}
}
super.validate(table);
}
private Matcher templateMatcher(String uriTemplate)
{
Pattern p = Pattern.compile("\\{}");
Matcher m = p.matcher(uriTemplate);
if (!m.find()) {
throw new IAE(
"Value [%s] for property %s must include a '{}' placeholder",
uriTemplate,
URI_TEMPLATE_PROPERTY
);
}
return m;
}
@Override
protected List<ParameterDefn> adHocTableFnParameters()
{
return CatalogUtils.concatLists(URI_PARAMS, USER_PWD_PARAMS);
}
@Override
protected void convertArgsToSourceMap(Map<String, Object> jsonMap, Map<String, Object> args)
{
jsonMap.put(InputSource.TYPE_PROPERTY, HttpInputSource.TYPE_KEY);
convertUriArg(jsonMap, args);
convertUserPasswordArgs(jsonMap, args);
}
@Override
public TableFunction partialTableFn(ResolvedExternalTable table)
{
List<ParameterDefn> params = Collections.emptyList();
// Does the table define URIs?
Map<String, Object> sourceMap = table.inputSourceMap;
if (!sourceMap.containsKey(URIS_FIELD)) {
params = CatalogUtils.concatLists(params, URI_PARAMS);
}
// Does the table define a user or password?
if (!sourceMap.containsKey(USERNAME_FIELD) && !sourceMap.containsKey(PASSWORD_FIELD)) {
params = CatalogUtils.concatLists(params, USER_PWD_PARAMS);
}
// Does the table define a format?
if (table.inputFormatMap == null) {
params = addFormatParameters(params);
}
return new PartialTableFunction(table, params);
}
@Override
protected ExternalTableSpec convertCompletedTable(
final ResolvedExternalTable table,
final Map<String, Object> args,
final List<ColumnSpec> columns
)
{
// Get URIs from table if defined, else from arguments.
final Map<String, Object> sourceMap = new HashMap<>(table.inputSourceMap);
final String uriTemplate = table.resolvedTable().stringProperty(URI_TEMPLATE_PROPERTY);
if (uriTemplate != null) {
convertUriTemplateArgs(sourceMap, uriTemplate, args);
} else if (!sourceMap.containsKey(URIS_FIELD)) {
convertUriArg(sourceMap, args);
}
// Get user and password from the table if defined, else from arguments.
if (!sourceMap.containsKey(USERNAME_FIELD) && !sourceMap.containsKey(PASSWORD_FIELD)) {
convertUserPasswordArgs(sourceMap, args);
}
return convertPartialFormattedTable(table, args, columns, sourceMap);
}
private void convertUriTemplateArgs(Map<String, Object> jsonMap, String uriTemplate, Map<String, Object> args)
{
List<String> uriStrings = CatalogUtils.getStringArray(args, URIS_PARAMETER);
if (CollectionUtils.isNullOrEmpty(uriStrings)) {
throw new IAE("One or more URIs is required in parameter %s", URIS_PARAMETER);
}
final Matcher m = templateMatcher(uriTemplate);
final List<String> uris = uriStrings.stream()
.map(uri -> m.replaceFirst(uri))
.collect(Collectors.toList());
jsonMap.put(URIS_FIELD, CatalogUtils.stringListToUriList(uris));
}
/**
* URIs in SQL is in the form of a string that contains a comma-delimited
* set of URIs. Done since SQL doesn't support array scalars.
*/
private void convertUriArg(Map<String, Object> jsonMap, Map<String, Object> args)
{
List<String> uris = CatalogUtils.getStringArray(args, URIS_PARAMETER);
if (uris != null) {
jsonMap.put(URIS_FIELD, CatalogUtils.stringListToUriList(uris));
}
}
/**
* Convert the user name and password. All are SQL strings. Passwords must be in
* the form of a password provider, so do the needed conversion. HTTP provides
* two kinds of passwords (plain test an reference to an env var), but at most
* one can be provided.
*/
private void convertUserPasswordArgs(Map<String, Object> jsonMap, Map<String, Object> args)
{
String user = CatalogUtils.getString(args, USER_PARAMETER);
if (user != null) {
jsonMap.put(USERNAME_FIELD, user);
}
String password = CatalogUtils.getString(args, PASSWORD_PARAMETER);
String passwordEnvVar = CatalogUtils.getString(args, PASSWORD_ENV_VAR_PARAMETER);
if (password != null && passwordEnvVar != null) {
throw new ISE(
"Specify only one of %s or %s",
PASSWORD_PARAMETER,
PASSWORD_ENV_VAR_PARAMETER
);
}
if (password != null) {
jsonMap.put(
PASSWORD_FIELD,
ImmutableMap.of("type", DefaultPasswordProvider.TYPE_KEY, "password", password)
);
} else if (passwordEnvVar != null) {
jsonMap.put(
PASSWORD_FIELD,
ImmutableMap.of("type", EnvironmentVariablePasswordProvider.TYPE_KEY, "variable", passwordEnvVar)
);
}
}
}

View File

@ -1,194 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.ModelProperties.StringListPropertyDefn;
import org.apache.druid.catalog.model.ModelProperties.StringPropertyDefn;
import org.apache.druid.catalog.model.ParameterizedDefn;
import org.apache.druid.catalog.model.PropertyAttributes;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.table.ExternalTableDefn.FormattedExternalTableDefn;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.impl.HttpInputSource;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.metadata.DefaultPasswordProvider;
import org.apache.druid.metadata.EnvironmentVariablePasswordProvider;
import org.apache.druid.utils.CollectionUtils;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
/**
* Definition of an input table for an HTTP data source. Provides the same
* properties as the {@link HttpInputSource}, but as top-level properties
* that can be mapped to SQL function parameters. Property names are
* cleaned up for ease-of-use. The HTTP input source has multiple quirks,
* the conversion method smoothes over those quirks for a simpler catalog
* experience. Provides a parameterized
* form where the user provides the partial URLs to use for a particular
* query.
*/
public class HttpTableDefn extends FormattedExternalTableDefn implements ParameterizedDefn
{
public static final String TABLE_TYPE = HttpInputSource.TYPE_KEY;
// Catalog properties that map to fields in the HttpInputSource. See
// that class for the meaning of these properties.
public static final String URI_TEMPLATE_PROPERTY = "uriTemplate";
// Note, cannot be the simpler "user" since USER is a reserved word in SQL
// and we don't want to require users to quote "user" each time it is used.
public static final String USER_PROPERTY = "userName";
public static final String PASSWORD_PROPERTY = "password";
public static final String PASSWORD_ENV_VAR_PROPERTY = "passwordEnvVar";
public static final String URIS_PROPERTY = "uris";
public HttpTableDefn()
{
super(
"HTTP input table",
TABLE_TYPE,
Arrays.asList(
new StringListPropertyDefn(URIS_PROPERTY, PropertyAttributes.SQL_AND_TABLE_PARAM),
new StringPropertyDefn(USER_PROPERTY, PropertyAttributes.OPTIONAL_SQL_FN_PARAM),
new StringPropertyDefn(PASSWORD_PROPERTY, PropertyAttributes.OPTIONAL_SQL_FN_PARAM),
new StringPropertyDefn(PASSWORD_ENV_VAR_PROPERTY, PropertyAttributes.OPTIONAL_SQL_FN_PARAM),
new StringPropertyDefn(URI_TEMPLATE_PROPERTY, null)
),
Collections.singletonList(INPUT_COLUMN_DEFN),
InputFormats.ALL_FORMATS
);
}
@Override
public ResolvedTable mergeParameters(ResolvedTable table, Map<String, Object> values)
{
String urisValue = CatalogUtils.safeGet(values, URIS_PROPERTY, String.class);
List<String> uriValues = CatalogUtils.stringToList(urisValue);
if (CollectionUtils.isNullOrEmpty(uriValues)) {
throw new IAE("One or more values are required for parameter %s", URIS_PROPERTY);
}
String uriTemplate = table.stringProperty(URI_TEMPLATE_PROPERTY);
if (Strings.isNullOrEmpty(uriTemplate)) {
throw new IAE("Property %s must provide a URI template.", URI_TEMPLATE_PROPERTY);
}
Pattern p = Pattern.compile("\\{}");
Matcher m = p.matcher(uriTemplate);
if (!m.find()) {
throw new IAE(
"Value [%s] for property %s must include a '{}' placeholder.",
uriTemplate,
URI_TEMPLATE_PROPERTY
);
}
List<String> uris = new ArrayList<>();
for (String uri : uriValues) {
uris.add(m.replaceFirst(uri));
}
Map<String, Object> revisedProps = new HashMap<>(table.properties());
revisedProps.remove(URI_TEMPLATE_PROPERTY);
revisedProps.put("uris", uris);
return table.withProperties(revisedProps);
}
@Override
protected InputSource convertSource(ResolvedTable table)
{
Map<String, Object> jsonMap = new HashMap<>();
jsonMap.put(InputSource.TYPE_PROPERTY, HttpInputSource.TYPE_KEY);
jsonMap.put("httpAuthenticationUsername", table.stringProperty(USER_PROPERTY));
String password = table.stringProperty(PASSWORD_PROPERTY);
String passwordEnvVar = table.stringProperty(PASSWORD_ENV_VAR_PROPERTY);
if (password != null && passwordEnvVar != null) {
throw new ISE(
"Specify only one of %s or %s",
PASSWORD_PROPERTY,
PASSWORD_ENV_VAR_PROPERTY
);
}
if (password != null) {
jsonMap.put(
"httpAuthenticationPassword",
ImmutableMap.of("type", DefaultPasswordProvider.TYPE_KEY, "password", password)
);
} else if (passwordEnvVar != null) {
jsonMap.put(
"httpAuthenticationPassword",
ImmutableMap.of("type", EnvironmentVariablePasswordProvider.TYPE_KEY, "variable", passwordEnvVar)
);
}
jsonMap.put("uris", convertUriList(table.stringListProperty(URIS_PROPERTY)));
return convertObject(table.jsonMapper(), jsonMap, HttpInputSource.class);
}
@SuppressWarnings("unchecked")
public static List<URI> convertUriList(Object value)
{
if (value == null) {
return null;
}
List<String> list;
try {
list = (List<String>) value;
}
catch (ClassCastException e) {
throw new IAE("Value [%s] must be a list of strings", value);
}
List<URI> uris = new ArrayList<>();
for (String strValue : list) {
try {
uris.add(new URI(strValue));
}
catch (URISyntaxException e) {
throw new IAE(StringUtils.format("Argument [%s] is not a valid URI", value));
}
}
return uris;
}
@Override
public void validate(ResolvedTable table)
{
super.validate(table);
// Validate the HTTP properties only if we don't have a template.
// If we do have a template, then we don't know how to form
// a valid parameter for that template.
// TODO: plug in a dummy URL so we can validate other properties.
if (!table.hasProperty(URI_TEMPLATE_PROPERTY)) {
convertSource(table);
}
}
}

View File

@ -0,0 +1,134 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.table.BaseTableFunction.Parameter;
import org.apache.druid.catalog.model.table.TableFunction.ParameterDefn;
import org.apache.druid.catalog.model.table.TableFunction.ParameterType;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.impl.InlineInputSource;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.utils.CollectionUtils;
import java.util.Collections;
import java.util.List;
import java.util.Map;
/**
* Describes an inline input source: one where the data is provided in the
* table spec as a series of text lines. Since the data is provided, the input
* format is required in the table spec: it cannot be provided at ingest time.
* Primarily for testing.
*/
public class InlineInputSourceDefn extends FormattedInputSourceDefn
{
public static final String TYPE_KEY = InlineInputSource.TYPE_KEY;
public static final String DATA_PROPERTY = "data";
public static final String DATA_FIELD = "data";
@Override
public String typeValue()
{
return TYPE_KEY;
}
@Override
protected Class<? extends InputSource> inputSourceClass()
{
return InlineInputSource.class;
}
@Override
protected List<ParameterDefn> adHocTableFnParameters()
{
return Collections.singletonList(
new Parameter(DATA_PROPERTY, ParameterType.VARCHAR_ARRAY, false)
);
}
@Override
public TableFunction partialTableFn(ResolvedExternalTable table)
{
return new PartialTableFunction(table, Collections.emptyList());
}
@Override
public void validate(ResolvedExternalTable table)
{
// For inline, format is required to match the data
if (table.inputFormatMap == null) {
throw new IAE("An inline input source must provide a format.");
}
if (CollectionUtils.isNullOrEmpty(table.resolvedTable().spec().columns())) {
throw new IAE("An inline input source must provide one or more columns");
}
super.validate(table);
}
@Override
protected void convertArgsToSourceMap(Map<String, Object> jsonMap, Map<String, Object> args)
{
jsonMap.put(InputSource.TYPE_PROPERTY, InlineInputSource.TYPE_KEY);
List<String> data = CatalogUtils.getStringArray(args, DATA_PROPERTY);
// Would be nice, from a completeness perspective, for the inline data
// source to allow zero rows of data. However, such is not the case.
if (CollectionUtils.isNullOrEmpty(data)) {
throw new IAE(
"An inline table requires one or more rows of data in the '%s' property",
DATA_PROPERTY
);
}
jsonMap.put(DATA_FIELD, String.join("\n", data));
}
@Override
protected ExternalTableSpec convertCompletedTable(
final ResolvedExternalTable table,
final Map<String, Object> args,
final List<ColumnSpec> columns
)
{
if (!args.isEmpty()) {
throw new ISE("Cannot provide arguments for an inline table");
}
if (!CollectionUtils.isNullOrEmpty(columns)) {
throw new IAE("Cannot provide columns for an inline table");
}
return convertTable(table);
}
@Override
protected void auditInputSource(Map<String, Object> jsonMap)
{
// Special handling of the data property which, in SQL, is a null-delimited
// list of rows. The user will usually provide a trailing newline which should
// not be interpreted as an empty data row. That is, if the data ends with
// a newline, the inline input source will interpret that as a blank line, oddly.
String data = CatalogUtils.getString(jsonMap, DATA_FIELD);
if (data != null && !data.endsWith("\n")) {
jsonMap.put(DATA_FIELD, data + "\n");
}
}
}

View File

@ -1,119 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.ModelProperties.StringListPropertyDefn;
import org.apache.druid.catalog.model.PropertyAttributes;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.table.ExternalTableDefn.FormattedExternalTableDefn;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.impl.InlineInputSource;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.utils.CollectionUtils;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* Describes an inline table: one where the data is provided in the
* table spec as a series of text lines.
*/
public class InlineTableDefn extends FormattedExternalTableDefn
{
public static final String TABLE_TYPE = InlineInputSource.TYPE_KEY;
public static final String DATA_PROPERTY = "data";
/**
* Special handling of the data property which, in SQL, is a null-delimited
* list of rows. The user will usually provide a trailing newline which should
* not be interpreted as an empty data row.
*/
private static class DataPropertyDefn extends StringListPropertyDefn
{
public DataPropertyDefn(
final Map<String, Object> attribs
)
{
super(DATA_PROPERTY, attribs);
}
@Override
public List<String> decodeSqlValue(Object value, ObjectMapper jsonMapper)
{
if (!(value instanceof String)) {
throw new IAE(StringUtils.format("Argument [%s] is not a VARCHAR", value));
}
String[] values = ((String) value).trim().split("\n");
return Arrays.asList(values);
}
}
public InlineTableDefn()
{
super(
"Inline input table",
TABLE_TYPE,
Collections.singletonList(
new DataPropertyDefn(PropertyAttributes.SQL_FN_PARAM)
),
Collections.singletonList(INPUT_COLUMN_DEFN),
InputFormats.ALL_FORMATS
);
}
@Override
protected InputSource convertSource(ResolvedTable table)
{
Map<String, Object> jsonMap = new HashMap<>();
jsonMap.put(InputSource.TYPE_PROPERTY, InlineInputSource.TYPE_KEY);
List<String> dataList = table.stringListProperty(DATA_PROPERTY);
// Would be nice, from a completeness perspective, for the inline data
// source to allow zero rows of data. However, such is not the case.
if (CollectionUtils.isNullOrEmpty(dataList)) {
throw new IAE(
"An inline table requires one or more rows of data in the '%s' property",
DATA_PROPERTY
);
}
jsonMap.put("data", CatalogUtils.stringListToLines(dataList));
return convertObject(table.jsonMapper(), jsonMap, InlineInputSource.class);
}
@Override
public ResolvedTable mergeParameters(ResolvedTable spec, Map<String, Object> values)
{
throw new UOE("Inline table does not support parameters");
}
@Override
public void validate(ResolvedTable table)
{
super.validate(table);
convertSource(table);
}
}

View File

@ -0,0 +1,98 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.table.TableFunction.ParameterDefn;
import org.apache.druid.data.input.InputFormat;
import java.util.List;
import java.util.Map;
/**
* Metadata about a Druid {@link InputFormat}. Provides the logic to
* convert from a table spec or SQL function arguments to a specific
* form of {@code InputFormat}. There is one instance of this interface for each
* supported {@code InputFormat}.
*/
public interface InputFormatDefn
{
/**
* @return the string used to identify the input format type in the serialized
* JSON for the input format. This is also the value used in the {@code format}
* property for SQL functions.
*/
String typeValue();
/**
* Given a resolved table that has the serialized JSON converted to a Java map,
* validate the values of that map, typically by converting that map the target
* input format object (after adjustments and filling in dummy columns.) THe
* goal is to validate the information the user has provided in the table spec.
* The final format information is validated elsewhere.
*/
void validate(ResolvedExternalTable table);
/**
* Obtain the parameters used to fully define an input format in a SQL function
* that defines an external table from scratch. Note that the final list of table
* function arguments combines parameters from all the various input sources. It
* is legal for multiple formats to define the same parameter, as long as both
* definitions are of the same type.
* @return
*/
List<ParameterDefn> parameters();
/**
* Create an input format instance from the values provided as arguments that
* correspond to the defined parameters. The map provided is guaranteed to have
* a value for each parameter defined as non-optional, and those values will be
* of the type defined in the parameter. The map will also contain values for
* the input source: the format should ignore values that don't correspond to
* parameters it defined.
*
* @param args the actual arguments for the defined parameters
* @param columns the columns provided in SQL, typically via the `EXTEND` clause,
* but perhaps from an the table spec, if the function is for
* a partial table
* @param jsonMapper the mapper to use to perform conversions
*
* @return an input format as defined by the arguments
* @throws org.apache.druid.java.util.common.IAE if the arguments are not valid
*/
InputFormat convertFromArgs(
Map<String, Object> args,
List<ColumnSpec> columns,
ObjectMapper jsonMapper
);
/**
* Create an input format from a resolved catalog table spec. The format is given
* by the Java map within the given object.
*
* @param table resolved form of a table spec, with the format JSON parsed into
* a JSON map
*
* @return an input format as defined by the table spec
* @throws org.apache.druid.java.util.common.IAE if the spec is not valid
*/
InputFormat convertFromTable(ResolvedExternalTable table);
}

View File

@ -20,22 +20,19 @@
package org.apache.druid.catalog.model.table;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import org.apache.curator.shaded.com.google.common.collect.ImmutableList;
import com.google.common.annotations.VisibleForTesting;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.ModelProperties.BooleanPropertyDefn;
import org.apache.druid.catalog.model.ModelProperties.IntPropertyDefn;
import org.apache.druid.catalog.model.ModelProperties.PropertyDefn;
import org.apache.druid.catalog.model.ModelProperties.SimplePropertyDefn;
import org.apache.druid.catalog.model.ModelProperties.StringPropertyDefn;
import org.apache.druid.catalog.model.PropertyAttributes;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.table.BaseTableFunction.Parameter;
import org.apache.druid.catalog.model.table.TableFunction.ParameterDefn;
import org.apache.druid.catalog.model.table.TableFunction.ParameterType;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.DelimitedInputFormat;
import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.utils.CollectionUtils;
import java.util.Arrays;
import java.util.Collections;
@ -45,59 +42,85 @@ import java.util.Map;
import java.util.stream.Collectors;
/**
* Definition of the input formats which converts from property
* lists in table specs to subclasses of {@link InputFormat}.
* Catalog definitions for the Druid input formats. Maps from catalog and SQL
* functions to the Druid input format implementations.
*/
public class InputFormats
{
public interface InputFormatDefn
{
String name();
String typeTag();
List<PropertyDefn<?>> properties();
void validate(ResolvedTable table);
InputFormat convert(ResolvedTable table);
}
/**
* Base class for input format definitions.
*/
public abstract static class BaseFormatDefn implements InputFormatDefn
{
private final String name;
private final String typeTag;
private final List<PropertyDefn<?>> properties;
/**
* The set of SQL function parameters available when the format is
* specified via a SQL function. The parameters correspond to input format
* properties, but typically have simpler names and must require only simple
* scalar types of the kind that SQL can provide. Each subclass must perform
* conversion to the type required for Jackson conversion.
*/
private final List<ParameterDefn> parameters;
public BaseFormatDefn(
final String name,
final String typeTag,
final List<PropertyDefn<?>> properties
public BaseFormatDefn(List<ParameterDefn> parameters)
{
this.parameters = parameters == null ? Collections.emptyList() : parameters;
}
@Override
public List<ParameterDefn> parameters()
{
return parameters;
}
/**
* The target input format class for Jackson conversions.
*/
protected abstract Class<? extends InputFormat> inputFormatClass();
@Override
public void validate(ResolvedExternalTable table)
{
// Bare-bones validation: the format has to convert to the proper object.
// Subclasses should replace this with something fancier where needed.
if (table.inputFormatMap != null) {
convertFromTable(table);
}
}
/**
* Convert columns from the {@link ColumnSpec} format used by the catalog to the
* list of names form requires by input formats.
*/
protected void convertColumns(Map<String, Object> jsonMap, List<ColumnSpec> columns)
{
List<String> cols = columns
.stream()
.map(col -> col.name())
.collect(Collectors.toList());
jsonMap.put("columns", cols);
}
/**
* Convert a generic Java map of input format properties to an input format object.
*/
public InputFormat convert(
final Map<String, Object> jsonMap,
final ObjectMapper jsonMapper
)
{
this.name = name;
this.typeTag = typeTag;
this.properties = properties;
try {
return jsonMapper.convertValue(jsonMap, inputFormatClass());
}
catch (Exception e) {
throw new IAE(e, "Invalid format specification");
}
}
@Override
public String name()
public InputFormat convertFromTable(ResolvedExternalTable table)
{
return name;
}
@Override
public String typeTag()
{
return typeTag;
}
@Override
public List<PropertyDefn<?>> properties()
{
return properties;
}
@Override
public void validate(ResolvedTable table)
{
convert(table);
return convert(table.inputFormatMap, table.resolvedTable().jsonMapper());
}
}
@ -116,213 +139,193 @@ public class InputFormats
*/
public abstract static class FlatTextFormatDefn extends BaseFormatDefn
{
public static final String LIST_DELIMITER_PROPERTY = "listDelimiter";
public static final String SKIP_ROWS_PROPERTY = "skipRows";
public static final String LIST_DELIMITER_PARAMETER = "listDelimiter";
public static final String SKIP_ROWS_PARAMETER = "skipHeaderRows";
public FlatTextFormatDefn(
final String name,
final String typeTag,
final List<PropertyDefn<?>> properties
)
private static final String COLUMNS_FIELD = "columns";
private static final String FIND_COLUMNS_FIELD = "findColumnsFromHeader";
private static final String SKIP_HEADERS_FIELD = "skipHeaderRows";
private static final String LIST_DELIMITER_FIELD = "listDelimiter";
public FlatTextFormatDefn(List<ParameterDefn> parameters)
{
super(
name,
typeTag,
CatalogUtils.concatLists(
Arrays.asList(
new StringPropertyDefn(LIST_DELIMITER_PROPERTY, PropertyAttributes.OPTIONAL_SQL_FN_PARAM),
new IntPropertyDefn(SKIP_ROWS_PROPERTY, PropertyAttributes.OPTIONAL_SQL_FN_PARAM)
new Parameter(LIST_DELIMITER_PARAMETER, ParameterType.VARCHAR, true),
new Parameter(SKIP_ROWS_PARAMETER, ParameterType.BOOLEAN, true)
),
properties
parameters
)
);
}
protected Map<String, Object> gatherFields(ResolvedTable table)
@Override
public void validate(ResolvedExternalTable table)
{
if (table.inputFormatMap == null) {
return;
}
ResolvedTable resolvedTable = table.resolvedTable();
Map<String, Object> jsonMap = toMap(table);
// Make up a column just so that validation will pass.
jsonMap.putIfAbsent(COLUMNS_FIELD, Collections.singletonList("a"));
convert(jsonMap, resolvedTable.jsonMapper());
}
protected Map<String, Object> toMap(ResolvedExternalTable table)
{
ResolvedTable resolvedTable = table.resolvedTable();
Map<String, Object> jsonMap = new HashMap<>(table.inputFormatMap);
if (!CollectionUtils.isNullOrEmpty(resolvedTable.spec().columns())) {
convertColumns(jsonMap, resolvedTable.spec().columns());
}
adjustValues(jsonMap);
return jsonMap;
}
protected void adjustValues(Map<String, Object> jsonMap)
{
// findColumnsFromHeader is required, even though we don't infer headers.
jsonMap.put(FIND_COLUMNS_FIELD, false);
jsonMap.computeIfAbsent(SKIP_HEADERS_FIELD, key -> 0);
}
protected Map<String, Object> mapFromArgs(Map<String, Object> args, List<ColumnSpec> columns)
{
Map<String, Object> jsonMap = new HashMap<>();
jsonMap.put(InputFormat.TYPE_PROPERTY, CsvInputFormat.TYPE_KEY);
jsonMap.put("listDelimiter", table.property(LIST_DELIMITER_PROPERTY));
// hasHeaderRow is required, even though we don't infer headers.
jsonMap.put("hasHeaderRow", false);
jsonMap.put("findColumnsFromHeader", false);
// Column list is required. Infer from schema.
List<String> cols = table.spec().columns()
.stream()
.map(col -> col.name())
.collect(Collectors.toList());
jsonMap.put("columns", cols);
Object value = table.property(SKIP_ROWS_PROPERTY);
jsonMap.put("skipHeaderRows", value == null ? 0 : value);
return jsonMap;
}
}
public static final String CSV_FORMAT_TYPE = CsvInputFormat.TYPE_KEY;
public static class CsvFormatDefn extends FlatTextFormatDefn
{
public CsvFormatDefn()
{
super(
"CSV",
CSV_FORMAT_TYPE,
null
);
}
@Override
protected Map<String, Object> gatherFields(ResolvedTable table)
{
Map<String, Object> jsonMap = super.gatherFields(table);
jsonMap.put(InputFormat.TYPE_PROPERTY, CsvInputFormat.TYPE_KEY);
jsonMap.put(LIST_DELIMITER_FIELD, args.get(LIST_DELIMITER_PARAMETER));
Object value = args.get(SKIP_ROWS_PARAMETER);
jsonMap.put(SKIP_HEADERS_FIELD, value == null ? 0 : value);
convertColumns(jsonMap, columns);
adjustValues(jsonMap);
return jsonMap;
}
@Override
public InputFormat convert(ResolvedTable table)
public InputFormat convertFromTable(ResolvedExternalTable table)
{
try {
return table.jsonMapper().convertValue(gatherFields(table), CsvInputFormat.class);
}
catch (Exception e) {
throw new IAE(e, "Invalid format specification");
}
}
}
public static final String DELIMITED_FORMAT_TYPE = DelimitedInputFormat.TYPE_KEY;
public static class DelimitedFormatDefn extends FlatTextFormatDefn
{
public static final String DELIMITER_PROPERTY = "delimiter";
public DelimitedFormatDefn()
{
super(
"Delimited Text",
DELIMITED_FORMAT_TYPE,
Collections.singletonList(
new StringPropertyDefn(DELIMITER_PROPERTY, PropertyAttributes.OPTIONAL_SQL_FN_PARAM)
)
);
}
@Override
protected Map<String, Object> gatherFields(ResolvedTable table)
{
Map<String, Object> jsonMap = super.gatherFields(table);
jsonMap.put(InputFormat.TYPE_PROPERTY, DelimitedInputFormat.TYPE_KEY);
Object value = table.property(DELIMITER_PROPERTY);
if (value != null) {
jsonMap.put("delimiter", value);
}
return jsonMap;
}
@Override
public InputFormat convert(ResolvedTable table)
{
return table.jsonMapper().convertValue(gatherFields(table), DelimitedInputFormat.class);
}
}
public static final String JSON_FORMAT_TYPE = JsonInputFormat.TYPE_KEY;
public static class JsonFormatDefn extends BaseFormatDefn
{
public static final String KEEP_NULLS_PROPERTY = "keepNulls";
public JsonFormatDefn()
{
super(
"JSON",
JSON_FORMAT_TYPE,
Collections.singletonList(
new BooleanPropertyDefn(KEEP_NULLS_PROPERTY, PropertyAttributes.OPTIONAL_SQL_FN_PARAM)
)
);
}
@Override
public InputFormat convert(ResolvedTable table)
{
// TODO flatten & feature specs
Map<String, Object> jsonMap = new HashMap<>();
jsonMap.put(InputFormat.TYPE_PROPERTY, JsonInputFormat.TYPE_KEY);
jsonMap.put("keepNullColumns", table.property(KEEP_NULLS_PROPERTY));
return table.jsonMapper().convertValue(jsonMap, JsonInputFormat.class);
return convert(toMap(table), table.resolvedTable().jsonMapper());
}
}
/**
* Generic format which allows a literal input spec. Allows the user to
* specify any input format and any options directly as JSON. The
* drawback is that the user must repeat the columns.
* Definition for the CSV input format. Designed so that, in most cases, the
* user only need specify the format as CSV: the definition fills in the common
* "boiler plate" properties.
*/
public static class GenericFormatDefn extends BaseFormatDefn
public static class CsvFormatDefn extends FlatTextFormatDefn
{
public static final String INPUT_FORMAT_SPEC_PROPERTY = "inputFormatSpec";
public static final String FORMAT_KEY = "generic";
public static final String TYPE_KEY = CsvInputFormat.TYPE_KEY;
private static class FormatPropertyDefn extends SimplePropertyDefn<InputFormat>
public CsvFormatDefn()
{
public FormatPropertyDefn()
{
super(
INPUT_FORMAT_SPEC_PROPERTY,
InputFormat.class,
PropertyAttributes.merge(
ImmutableMap.of(
PropertyAttributes.SQL_JAVA_TYPE,
String.class
),
PropertyAttributes.OPTIONAL_SQL_FN_PARAM
)
);
}
@Override
public InputFormat decodeSqlValue(Object value, ObjectMapper jsonMapper)
{
return decodeJson(value, jsonMapper);
}
super(null);
}
public GenericFormatDefn()
@Override
public String typeValue()
{
return TYPE_KEY;
}
@Override
protected Class<? extends InputFormat> inputFormatClass()
{
return CsvInputFormat.class;
}
@Override
public InputFormat convertFromArgs(
Map<String, Object> args,
List<ColumnSpec> columns,
ObjectMapper jsonMapper
)
{
Map<String, Object> jsonMap = mapFromArgs(args, columns);
jsonMap.put(InputFormat.TYPE_PROPERTY, CsvInputFormat.TYPE_KEY);
return convert(jsonMap, jsonMapper);
}
}
public static class DelimitedFormatDefn extends FlatTextFormatDefn
{
public static final String TYPE_KEY = DelimitedInputFormat.TYPE_KEY;
public static final String DELIMITER_PARAMETER = "delimiter";
@VisibleForTesting
public static final String DELIMITER_FIELD = "delimiter";
public DelimitedFormatDefn()
{
super(
"Generic",
FORMAT_KEY,
Collections.singletonList(
new FormatPropertyDefn()
new Parameter(DELIMITER_PARAMETER, ParameterType.VARCHAR, true)
)
);
}
@Override
public InputFormat convert(ResolvedTable table)
public String typeValue()
{
Object value = table.property(INPUT_FORMAT_SPEC_PROPERTY);
if (value == null) {
throw new ISE(
"An input format must be provided in the %s property when input type is %s",
INPUT_FORMAT_SPEC_PROPERTY,
name()
);
}
return table.jsonMapper().convertValue(value, InputFormat.class);
return TYPE_KEY;
}
@Override
protected Class<? extends InputFormat> inputFormatClass()
{
return DelimitedInputFormat.class;
}
@Override
public InputFormat convertFromArgs(
Map<String, Object> args,
List<ColumnSpec> columns,
ObjectMapper jsonMapper
)
{
Map<String, Object> jsonMap = mapFromArgs(args, columns);
jsonMap.put(InputFormat.TYPE_PROPERTY, DelimitedInputFormat.TYPE_KEY);
jsonMap.put(DELIMITER_FIELD, CatalogUtils.getString(args, DELIMITER_PARAMETER));
return convert(jsonMap, jsonMapper);
}
}
public static final InputFormatDefn CSV_FORMAT_DEFN = new CsvFormatDefn();
public static final InputFormatDefn DELIMITED_FORMAT_DEFN = new DelimitedFormatDefn();
public static final InputFormatDefn JSON_FORMAT_DEFN = new JsonFormatDefn();
public static final GenericFormatDefn GENERIC_FORMAT_DEFN = new GenericFormatDefn();
public static final List<InputFormatDefn> ALL_FORMATS = ImmutableList.of(
CSV_FORMAT_DEFN,
DELIMITED_FORMAT_DEFN,
JSON_FORMAT_DEFN,
GENERIC_FORMAT_DEFN
);
/**
* JSON format definition. For now, we only expose the "keep nulls" attribute via a table
* function argument. We can easily add more later as the JSON format evolves.
*/
public static class JsonFormatDefn extends BaseFormatDefn
{
public static final String TYPE_KEY = JsonInputFormat.TYPE_KEY;
public JsonFormatDefn()
{
super(null);
}
@Override
public String typeValue()
{
return TYPE_KEY;
}
@Override
protected Class<? extends InputFormat> inputFormatClass()
{
return JsonInputFormat.class;
}
@Override
public InputFormat convertFromArgs(
Map<String, Object> args,
List<ColumnSpec> columns,
ObjectMapper jsonMapper
)
{
Map<String, Object> jsonMap = new HashMap<>();
jsonMap.put(InputFormat.TYPE_PROPERTY, JsonInputFormat.TYPE_KEY);
return convert(jsonMap, jsonMapper);
}
}
}

View File

@ -0,0 +1,128 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import org.apache.druid.catalog.model.TableDefnRegistry;
/**
* Metadata definition for one Druid input source.
* <p>
* This class models the merger of a catalog definition (where the input source is stored
* as serialized JSON), and table function parameters to produce a Druid input source.
* We use the following names:
* <dl>
* <dt>Property</dt>
* <dd>a top-level entry in the properties map of an (external) table specification.
* The entire JSON-serialized input source spec is a property. There may be others.
* Property names are defined by constants that end with {@code _PROPERTY}.</dd>
* <dt>Field</dt>
* <dd>The code often has the work with individual entries within the input source. To do that,
* we convert the input source to a Java map. Each entry in the map is a "field". Field names
* are implicit in the input source code. For convenience here, we declare constants for each
* field name, each ends with {@code _FIELD}.</dd>
* <dt>Parameter<dt>
* <dd>A SQL table function provides one or more named parameters. Each tends to map,
* directly or indirectly, to an input source field. Table parameter name constants end
* with {@code _PARAMETER}.</dd>
* <dt>Argument<dt>
* <dd>The actual value passed for a table function parameter for a specific query.
* Arguments correspond to a parameter. In Calcite, arguments are ordered. For convenience
* in the merging code, this layer converts the ordered parameter/argument lists into a
* map to allow simply by (parameter) name access.</dd>
* </dl>
* You will see constants that have the same value, but have each of the three suffixes,
* depending on how we've chosen to do the mapping for each input source. Since the input
* sources were not designed for this use case, we can't do a simple mapping: some creativity
* is required in each case.
*
* @see {@link ExternalTableDefn} for a detailed explanation.
*/
public interface InputSourceDefn
{
/**
* Gather information about the set of format definitions.
*/
void bind(TableDefnRegistry registry);
/**
* Type value for this format: same as the type string used in the serialized
* JSON for this input source. Used as the key for this definition within the
* table registry, and associates the serialized JSON with the corresponding
* input source definition.
*/
String typeValue();
/**
* Given a external table catalog spec, with the JSON input source and format
* properties parsed to generic Java maps, validate that the properties are
* valid prior to saving the spec into the catalog.
*
* @param table a catalog table spec with the input source and input format
* properties parsed into generic Java maps
*
* @throws org.apache.druid.java.util.common.IAE if the spec
* properties are invalid
*/
void validate(ResolvedExternalTable table);
/**
* Provide a definition for a SQL table function that defines an ad-hoc external
* table "from scratch" for this input source. Typically defines parameters for the
* input source and all allowed input formats.
*
* @return a fully-defined external table to be handed off to the Calcite planner.
*
* @throws org.apache.druid.java.util.common.IAE if the function arguments
* are invalid
*/
TableFunction adHocTableFn();
/**
* Provide a definition for a SQL table function that completes a partial table
* spec from the catalog. Used when the spec represents a "partial table" or a
* "connection". The function provides parameters needed to complete the table
* (typically the set of input files, objects, etc.) If the catalog table spec
* does not provide a format, and this input source requires a format, then the
* parameters also include parameters for all supported input formats, same as
* for {@link InputSourceDefn#adHocTableFn()}.
*
* @param table a catalog table spec with the input source and input format
* properties parsed into generic Java maps
*
* @return a fully-defined external table to be handed off to the Calcite planner.
*
* @throws org.apache.druid.java.util.common.IAE if the function arguments
* are invalid
*/
TableFunction partialTableFn(ResolvedExternalTable table);
/**
* Convert a complete (that is, fully-defined) table spec to an external table spec.
* Used when SQL references the catalog table directly by name in the {@code FROM}
* clause without using a table function.
*
* @return a fully-defined external table to be handed off to the Calcite planner.
*
* @throws org.apache.druid.java.util.common.IAE if the spec properties are invalid,
* or if the spec is partial and requires the use of a table function to
* complete
*/
ExternalTableSpec convertTable(ResolvedExternalTable table);
}

View File

@ -0,0 +1,279 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.google.common.base.Strings;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.table.BaseTableFunction.Parameter;
import org.apache.druid.catalog.model.table.TableFunction.ParameterDefn;
import org.apache.druid.catalog.model.table.TableFunction.ParameterType;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.utils.CollectionUtils;
import java.io.File;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
* Definition for a {@link LocalInputSource}.
*/
public class LocalInputSourceDefn extends FormattedInputSourceDefn
{
public static final String TYPE_KEY = LocalInputSource.TYPE_KEY;
/**
* Base directory for file or filter operations. If not provided,
* then the servers current working directory is assumed, which is
* typically valid only for sample data.
*/
public static final String BASE_DIR_PARAMETER = "baseDir";
public static final String FILTER_PARAMETER = "filter";
public static final String FILES_PARAMETER = "files";
protected static final String BASE_DIR_FIELD = "baseDir";
protected static final String FILES_FIELD = "files";
protected static final String FILTER_FIELD = "filter";
private static final ParameterDefn FILTER_PARAM_DEFN =
new Parameter(FILTER_PARAMETER, ParameterType.VARCHAR, true);
private static final ParameterDefn FILES_PARAM_DEFN =
new Parameter(FILES_PARAMETER, ParameterType.VARCHAR_ARRAY, true);
@Override
public String typeValue()
{
return LocalInputSource.TYPE_KEY;
}
@Override
protected Class<? extends InputSource> inputSourceClass()
{
return LocalInputSource.class;
}
@Override
public void validate(ResolvedExternalTable table)
{
final Map<String, Object> sourceMap = new HashMap<>(table.inputSourceMap);
final boolean hasBaseDir = sourceMap.containsKey(BASE_DIR_FIELD);
final boolean hasFiles = !CollectionUtils.isNullOrEmpty(CatalogUtils.safeGet(sourceMap, FILES_FIELD, List.class));
final boolean hasFilter = !Strings.isNullOrEmpty(CatalogUtils.getString(sourceMap, FILTER_FIELD));
if (!hasBaseDir && !hasFiles) {
throw new IAE(
"A local input source requires one property of %s or %s",
BASE_DIR_FIELD,
FILES_FIELD
);
}
if (!hasBaseDir && hasFilter) {
throw new IAE(
"If a local input source sets property %s, it must also set property %s",
FILTER_FIELD,
BASE_DIR_FIELD
);
}
if (hasBaseDir && hasFiles) {
throw new IAE(
"A local input source accepts only one of %s or %s",
BASE_DIR_FIELD,
FILES_FIELD
);
}
super.validate(table);
}
@Override
protected List<ParameterDefn> adHocTableFnParameters()
{
return Arrays.asList(
new Parameter(BASE_DIR_PARAMETER, ParameterType.VARCHAR, true),
FILTER_PARAM_DEFN,
FILES_PARAM_DEFN
);
}
@Override
protected void convertArgsToSourceMap(Map<String, Object> jsonMap, Map<String, Object> args)
{
jsonMap.put(InputSource.TYPE_PROPERTY, LocalInputSource.TYPE_KEY);
final String baseDirParam = CatalogUtils.getString(args, BASE_DIR_PARAMETER);
final List<String> filesParam = CatalogUtils.getStringArray(args, FILES_PARAMETER);
final String filterParam = CatalogUtils.getString(args, FILTER_PARAMETER);
final boolean hasBaseDir = !Strings.isNullOrEmpty(baseDirParam);
final boolean hasFiles = !CollectionUtils.isNullOrEmpty(filesParam);
final boolean hasFilter = !Strings.isNullOrEmpty(filterParam);
if (!hasBaseDir && !hasFiles) {
throw new IAE(
"A local input source requires one parameter of %s or %s",
BASE_DIR_PARAMETER,
FILES_PARAMETER
);
}
if (hasBaseDir && hasFiles) {
if (hasFilter) {
throw new IAE(
"A local input source can set parameter %s or %s, but not both.",
FILES_PARAMETER,
FILTER_PARAMETER
);
}
// Special workaround: if the user provides a base dir, and files, convert
// the files to be absolute paths relative to the base dir. Then, remove
// the baseDir, since the Druid input source does not allow both a baseDir
// and a list of files.
jsonMap.put(FILES_FIELD, absolutePath(baseDirParam, filesParam));
return;
}
if (!hasBaseDir && !Strings.isNullOrEmpty(filterParam)) {
throw new IAE(
"If a local input source sets parameter %s, it must also set parameter %s",
FILTER_PARAMETER,
BASE_DIR_PARAMETER
);
}
if (hasBaseDir) {
jsonMap.put(BASE_DIR_FIELD, baseDirParam);
}
if (hasFiles) {
jsonMap.put(FILES_FIELD, filesParam);
}
if (filterParam != null) {
jsonMap.put(FILTER_FIELD, filterParam);
}
}
private List<String> absolutePath(String baseDirPath, List<String> files)
{
final File baseDir = new File(baseDirPath);
return files.stream()
.map(f -> new File(baseDir, f).toString())
.collect(Collectors.toList());
}
@Override
public TableFunction partialTableFn(ResolvedExternalTable table)
{
final Map<String, Object> sourceMap = new HashMap<>(table.inputSourceMap);
final boolean hasFiles = !CollectionUtils.isNullOrEmpty(CatalogUtils.safeGet(sourceMap, FILES_FIELD, List.class));
final boolean hasFilter = !Strings.isNullOrEmpty(CatalogUtils.getString(sourceMap, FILTER_FIELD));
List<ParameterDefn> params = new ArrayList<>();
if (!hasFiles && !hasFilter) {
params.add(FILES_PARAM_DEFN);
params.add(FILTER_PARAM_DEFN);
}
// Does the table define a format?
if (table.inputFormatMap == null) {
params = addFormatParameters(params);
}
return new PartialTableFunction(table, params);
}
@Override
protected ExternalTableSpec convertCompletedTable(
final ResolvedExternalTable table,
final Map<String, Object> args,
final List<ColumnSpec> columns
)
{
final Map<String, Object> sourceMap = new HashMap<>(table.inputSourceMap);
final boolean hasFiles = !CollectionUtils.isNullOrEmpty(CatalogUtils.safeGet(sourceMap, FILES_FIELD, List.class));
if (hasFiles) {
if (!args.isEmpty()) {
throw new IAE("The local input source has a file list: do not provide other arguments");
}
} else {
final String baseDir = CatalogUtils.getString(sourceMap, BASE_DIR_FIELD);
if (Strings.isNullOrEmpty(baseDir)) {
throw new IAE(
"When a local external table is used with a table function, %s must be set",
BASE_DIR_FIELD
);
}
final boolean hasFilter = !Strings.isNullOrEmpty(CatalogUtils.getString(sourceMap, FILTER_FIELD));
final List<String> filesParam = CatalogUtils.getStringArray(args, FILES_PARAMETER);
final String filterParam = CatalogUtils.getString(args, FILTER_PARAMETER);
final boolean hasFilesParam = !CollectionUtils.isNullOrEmpty(filesParam);
final boolean hasFilterParam = !Strings.isNullOrEmpty(filterParam);
if (!hasFilter && !hasFilesParam && !hasFilterParam) {
throw new IAE(
"For a local input source, set either %s or %s",
FILES_PARAMETER,
FILTER_PARAMETER
);
}
if (hasFilesParam) {
// Special workaround: if the user provides a base dir, and files, convert
// the files to be absolute paths relative to the base dir. Then, remove
// the baseDir, since the Druid input source does not allow both a baseDir
// and a list of files.
sourceMap.remove(FILTER_FIELD);
sourceMap.remove(BASE_DIR_FIELD);
sourceMap.put(FILES_FIELD, absolutePath(baseDir, filesParam));
} else if (filterParam != null) {
sourceMap.put(FILTER_FIELD, filterParam);
}
}
return convertPartialFormattedTable(table, args, columns, sourceMap);
}
@Override
protected void auditInputSource(Map<String, Object> jsonMap)
{
// Note the odd semantics of this class.
// If we give a base directory, and explicitly state files, we must
// also provide a file filter which presumably matches the very files
// we list. Take pity on the user and provide a filter in this case.
String filter = CatalogUtils.getString(jsonMap, FILTER_FIELD);
if (filter != null) {
return;
}
String baseDir = CatalogUtils.getString(jsonMap, BASE_DIR_FIELD);
if (baseDir != null) {
jsonMap.put(FILTER_FIELD, "*");
}
}
@Override
public ExternalTableSpec convertTable(ResolvedExternalTable table)
{
final Map<String, Object> sourceMap = new HashMap<>(table.inputSourceMap);
final boolean hasFiles = !CollectionUtils.isNullOrEmpty(CatalogUtils.safeGet(sourceMap, FILES_FIELD, List.class));
final boolean hasFilter = !Strings.isNullOrEmpty(CatalogUtils.getString(sourceMap, FILTER_FIELD));
if (!hasFiles && !hasFilter) {
throw new IAE(
"Use a table function to set either %s or %s",
FILES_PARAMETER,
FILTER_PARAMETER
);
}
return super.convertTable(table);
}
}

View File

@ -1,124 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.ModelProperties.StringListPropertyDefn;
import org.apache.druid.catalog.model.ModelProperties.StringPropertyDefn;
import org.apache.druid.catalog.model.ParameterizedDefn;
import org.apache.druid.catalog.model.PropertyAttributes;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.table.ExternalTableDefn.FormattedExternalTableDefn;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.utils.CollectionUtils;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* Definition for a catalog table object that represents a Druid
* {@link LocalInputSource}.
*/
public class LocalTableDefn extends FormattedExternalTableDefn implements ParameterizedDefn
{
public static final String TABLE_TYPE = LocalInputSource.TYPE_KEY;
/**
* Base directory for file or filter operations. If not provided,
* then the servers current working directory is assumed, which is
* typically valid only for sample data.
*/
public static final String BASE_DIR_PROPERTY = "baseDir";
// Note name "fileFilter", not "filter". These properties mix in with
// others and "filter" is a bit too generic in that context.
public static final String FILE_FILTER_PROPERTY = "fileFilter";
public static final String FILES_PROPERTY = "files";
public LocalTableDefn()
{
super(
"Local file input table",
TABLE_TYPE,
Arrays.asList(
new StringPropertyDefn(BASE_DIR_PROPERTY, PropertyAttributes.OPTIONAL_SQL_FN_PARAM),
new StringPropertyDefn(FILE_FILTER_PROPERTY, PropertyAttributes.OPTIONAL_SQL_FN_PARAM),
new StringListPropertyDefn(FILES_PROPERTY, PropertyAttributes.SQL_AND_TABLE_PARAM)
),
Collections.singletonList(INPUT_COLUMN_DEFN),
InputFormats.ALL_FORMATS
);
}
@Override
public ResolvedTable mergeParameters(ResolvedTable table, Map<String, Object> values)
{
// The safe get can only check
final String filesParam = CatalogUtils.safeGet(values, FILES_PROPERTY, String.class);
final String filterParam = CatalogUtils.safeGet(values, FILE_FILTER_PROPERTY, String.class);
final Map<String, Object> revisedProps = new HashMap<>(table.properties());
if (filesParam != null) {
revisedProps.put(FILES_PROPERTY, CatalogUtils.stringToList(filesParam));
}
if (filterParam != null) {
revisedProps.put(FILE_FILTER_PROPERTY, filterParam);
}
return table.withProperties(revisedProps);
}
@Override
protected InputSource convertSource(ResolvedTable table)
{
final Map<String, Object> jsonMap = new HashMap<>();
jsonMap.put(InputSource.TYPE_PROPERTY, LocalInputSource.TYPE_KEY);
final String baseDir = table.stringProperty(BASE_DIR_PROPERTY);
jsonMap.put("baseDir", baseDir);
final List<String> files = table.stringListProperty(FILES_PROPERTY);
jsonMap.put("files", files);
// Note the odd semantics of this class.
// If we give a base directory, and explicitly state files, we must
// also provide a file filter which presumably matches the very files
// we list. Take pity on the user and provide a filter in this case.
String filter = table.stringProperty(FILE_FILTER_PROPERTY);
if (baseDir != null && !CollectionUtils.isNullOrEmpty(files) && filter == null) {
filter = "*";
}
jsonMap.put("filter", filter);
return convertObject(table.jsonMapper(), jsonMap, LocalInputSource.class);
}
@Override
public void validate(ResolvedTable table)
{
super.validate(table);
formatDefn(table).validate(table);
// Validate the source if it is complete enough; else we need
// parameters later.
if (table.hasProperty(BASE_DIR_PROPERTY) || table.hasProperty(FILES_PROPERTY)) {
convertSource(table);
}
}
}

View File

@ -0,0 +1,141 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableDefnRegistry;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.java.util.common.IAE;
import java.util.HashMap;
import java.util.Map;
/**
* Internal class to hold the intermediate form of an external table: the
* input source and input format properties converted to Java maps, and the
* types of each resolved to the corresponding definitions. Used to validate
* a table specification, and to convert a table specification to an
* {@link ExternalTableSpec} when used in SQL.
*/
public class ResolvedExternalTable
{
private final ResolvedTable table;
protected final Map<String, Object> inputSourceMap;
protected final Map<String, Object> inputFormatMap;
private InputSourceDefn inputSourceDefn;
private InputFormatDefn inputFormatDefn;
/**
* Construct a resolved external table by extracting the input source
* and input format properties, and converting each to a Java map.
* Validates that the input source is present: the format is optional.
* <p>
* Note: does <i>not</i> resolve the input source and input format
* definitions: that is done as a separate step when needed.
*
* @see {@link #resolve(TableDefnRegistry)}.
*/
public ResolvedExternalTable(final ResolvedTable table)
{
this.table = table;
Map<String, Object> map = table.mapProperty(ExternalTableDefn.SOURCE_PROPERTY);
if (map == null || map.isEmpty()) {
throw new IAE("%s property is required", ExternalTableDefn.SOURCE_PROPERTY);
}
this.inputSourceMap = new HashMap<>(map);
map = table.mapProperty(ExternalTableDefn.FORMAT_PROPERTY);
this.inputFormatMap = map == null ? null : new HashMap<>(map);
}
public ResolvedTable resolvedTable()
{
return table;
}
/**
* Look up the input source type and input format type to find the corresponding
* definitions in the table registry. Throws an exception if the types are not
* defined. The input source is required, the format is optional.
* <p>
* Note, for resolution to work, the name of each definition must be the same as
* that used as the type key in the serialized JSON.
*/
public ResolvedExternalTable resolve(TableDefnRegistry registry)
{
String inputSourceType = CatalogUtils.getString(inputSourceMap, InputSource.TYPE_PROPERTY);
if (inputSourceType == null) {
throw new IAE("Input source type %s is required", InputSource.TYPE_PROPERTY);
}
inputSourceDefn = registry.inputSourceDefnFor(inputSourceType);
if (inputSourceDefn == null) {
throw new IAE("Input source type %s is not registered", inputSourceType);
}
if (inputFormatMap != null) {
String inputFormatType = CatalogUtils.getString(inputFormatMap, InputFormat.TYPE_PROPERTY);
if (inputFormatType != null) {
inputFormatDefn = registry.inputFormatDefnFor(inputFormatType);
if (inputFormatDefn == null) {
throw new IAE("Input format type %s is not registered", inputFormatType);
}
}
}
return this;
}
/**
* Validate that the table spec is correct by resolving the definitions, then
* converting the JSON to the desired object type. Note that this path requires
* special handling: the table spec may be partial, which means it is missing information
* needed to create a complete input source. The input source definition defines which
* values can be omitted, and defined later in SQL via function parameters. If those
* values are missing, then the input source defn should provide dummy values so that
* the validation will succeed (assuming that the properties that are provided are valid.)
*/
public void validate(TableDefnRegistry registry)
{
resolve(registry);
inputSourceDefn.validate(this);
if (inputFormatDefn != null) {
inputFormatDefn.validate(this);
}
}
/**
* Return a table function definition for a partial table as given by
* this object. The function defines parameters to gather the
* values needed to convert the partial table into a fully-defined table
* which can be converted to an {@link ExternalTableSpec}.
*/
public TableFunction tableFn()
{
return inputSourceDefn.partialTableFn(this);
}
/**
* Return the {@link ExternalTableSpec} for a catalog entry for a this object
* which must be a fully-defined table.
*/
public ExternalTableSpec convert()
{
return inputSourceDefn.convertTable(this);
}
}

View File

@ -29,9 +29,7 @@ import org.apache.druid.catalog.model.TableDefn;
import org.apache.druid.catalog.model.TableId;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.TableSpec;
import org.apache.druid.catalog.model.table.ExternalTableDefn.FormattedExternalTableDefn;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import java.util.ArrayList;
import java.util.Arrays;
@ -47,32 +45,40 @@ import java.util.Map;
*/
public class TableBuilder
{
private TableId id;
private final TableId id;
private final String tableType;
private TableDefn defn;
private String tableType;
private Map<String, Object> properties = new HashMap<>();
private List<ColumnSpec> columns = new ArrayList<>();
public static TableBuilder datasource(String name, String granularity)
public TableBuilder(TableId id, String tableType)
{
return new TableBuilder()
.datasource(name)
.type(DatasourceDefn.TABLE_TYPE)
.segmentGranularity(granularity);
this.id = id;
this.tableType = tableType;
}
public static TableBuilder external(String type, String name)
public static TableBuilder datasource(String name, String granularity)
{
return new TableBuilder()
.external(name)
.type(type);
return new TableBuilder(
TableId.datasource(name),
DatasourceDefn.TABLE_TYPE
).segmentGranularity(granularity);
}
public static TableBuilder external(String name)
{
return new TableBuilder(
TableId.of(TableId.EXTERNAL_SCHEMA, name),
ExternalTableDefn.TABLE_TYPE
);
}
public static TableBuilder updateFor(TableMetadata table)
{
return new TableBuilder()
.id(table.id())
.type(table.spec().type());
return new TableBuilder(
table.id(),
table.spec().type()
);
}
public static TableBuilder copyOf(TableMetadata table)
@ -82,62 +88,27 @@ public class TableBuilder
public static TableBuilder copyOf(TableId newId, TableSpec from)
{
return new TableBuilder()
.id(newId)
.type(from.type())
return new TableBuilder(newId, from.type())
.properties(new HashMap<>(from.properties()))
.columns(new ArrayList<>(from.columns()));
}
public static TableBuilder of(TableDefn defn)
public static TableBuilder of(TableId id, TableDefn defn)
{
TableBuilder builder = new TableBuilder();
TableBuilder builder = new TableBuilder(id, defn.typeValue());
builder.defn = defn;
builder.tableType = defn.typeValue();
return builder;
}
public TableBuilder copy()
{
TableBuilder builder = new TableBuilder();
TableBuilder builder = new TableBuilder(id, tableType);
builder.defn = defn;
builder.tableType = tableType;
builder.id = id;
builder.properties.putAll(properties);
builder.columns.addAll(columns);
return builder;
}
public TableBuilder id(TableId id)
{
this.id = id;
return this;
}
public TableBuilder datasource(String name)
{
this.id = TableId.datasource(name);
return this;
}
public TableBuilder external(String name)
{
this.id = TableId.of(TableId.EXTERNAL_SCHEMA, name);
return this;
}
public TableBuilder path(String schema, String name)
{
this.id = TableId.of(schema, name);
return this;
}
public TableBuilder type(String tableType)
{
this.tableType = tableType;
return this;
}
public TableBuilder properties(Map<String, Object> properties)
{
this.properties = properties;
@ -162,12 +133,37 @@ public class TableBuilder
public TableBuilder segmentGranularity(String segmentGranularity)
{
return property(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, segmentGranularity);
return property(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, segmentGranularity);
}
public TableBuilder clusterColumns(ClusterKeySpec...clusterKeys)
{
return property(AbstractDatasourceDefn.CLUSTER_KEYS_PROPERTY, Arrays.asList(clusterKeys));
return property(DatasourceDefn.CLUSTER_KEYS_PROPERTY, Arrays.asList(clusterKeys));
}
public TableBuilder hiddenColumns(List<String> hiddenColumns)
{
return property(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, hiddenColumns);
}
public TableBuilder sealed(boolean sealed)
{
return property(DatasourceDefn.SEALED_PROPERTY, sealed);
}
public TableBuilder hiddenColumns(String...hiddenColumns)
{
return hiddenColumns(Arrays.asList(hiddenColumns));
}
public TableBuilder inputSource(Map<String, Object> inputSource)
{
return property(ExternalTableDefn.SOURCE_PROPERTY, inputSource);
}
public TableBuilder inputFormat(Map<String, Object> format)
{
return property(ExternalTableDefn.FORMAT_PROPERTY, format);
}
public TableBuilder columns(List<ColumnSpec> columns)
@ -196,59 +192,14 @@ public class TableBuilder
}
public TableBuilder column(String name, String sqlType)
{
return column(name, sqlType, null);
}
public TableBuilder column(String name, String sqlType, Map<String, Object> properties)
{
Preconditions.checkNotNull(tableType);
String colType;
if (isInputTable(tableType)) {
colType = ExternalTableDefn.EXTERNAL_COLUMN_TYPE;
} else if (DatasourceDefn.TABLE_TYPE.equals(tableType)) {
colType = DatasourceDefn.DatasourceColumnDefn.COLUMN_TYPE;
} else {
throw new ISE("Unknown table type: %s", tableType);
}
return column(colType, name, sqlType);
}
public static boolean isInputTable(String tableType)
{
switch (tableType) {
case InlineTableDefn.TABLE_TYPE:
case HttpTableDefn.TABLE_TYPE:
case LocalTableDefn.TABLE_TYPE:
return true;
default:
return false;
}
}
public TableBuilder column(String colType, String name, String sqlType)
{
return column(new ColumnSpec(colType, name, sqlType, null));
}
public TableBuilder hiddenColumns(List<String> hiddenColumns)
{
return property(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, hiddenColumns);
}
public TableBuilder hiddenColumns(String...hiddenColumns)
{
return hiddenColumns(Arrays.asList(hiddenColumns));
}
public TableBuilder format(String format)
{
return property(FormattedExternalTableDefn.FORMAT_PROPERTY, format);
}
public TableBuilder data(List<String> data)
{
return property(InlineTableDefn.DATA_PROPERTY, data);
}
public TableBuilder data(String...data)
{
return data(Arrays.asList(data));
return column(new ColumnSpec(name, sqlType, properties));
}
public TableSpec buildSpec()

View File

@ -0,0 +1,85 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.catalog.model.ColumnSpec;
import java.util.List;
import java.util.Map;
/**
* Representation of a SQL table function. The SQL module is not visible here.
* To avoid unnecessary dependencies, this class provides the information that
* Calcite needs, but without Calcite dependencies. Each function defines some
* number of arguments needed to create an external table. This same class
* can define a "from scratch" table, or to convert a partial table definition
* in the catalog into a full-defined external table.
* <p>
* The parameters tell Calcite the allowed parameters. Functions created from
* this class are most useful when used with named arguments:<code><pre>
* SELECT ... FROM TABLE(thisFn(foo -> "bar", answer -> 42))
* </pre</code>
* <p>
* Calcite provides actual arguments as an array, with null values for arguments
* which the user did not provide. That form is not helpful for this use case.
* The caller converts those a map, with only the actual arguments set. The
* call also provides a row schema, obtained from the Calcite {@code EXTEND}
* extension. The result is an {@link ExternalTableSpec} which the caller uses
* to create the Calcite form of an external table.
*/
public interface TableFunction
{
enum ParameterType
{
VARCHAR("VARCHAR"),
BIGINT("BIGINT"),
BOOLEAN("BOOLEAN"),
VARCHAR_ARRAY("VARCHAR array");
private final String name;
ParameterType(String name)
{
this.name = name;
}
public String sqlName()
{
return name;
}
}
interface ParameterDefn
{
String name();
ParameterType type();
boolean isOptional();
}
List<ParameterDefn> parameters();
ExternalTableSpec apply(
String fnName,
Map<String, Object> args,
List<ColumnSpec> columns,
ObjectMapper jsonMapper
);
}

View File

@ -36,6 +36,7 @@ import javax.ws.rs.Path;
import javax.ws.rs.Produces;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import java.util.Collections;
import java.util.List;
import java.util.Set;

View File

@ -39,7 +39,6 @@ import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
@ -51,11 +50,9 @@ public class PropertyDefnTest
@Test
public void testString()
{
StringPropertyDefn prop = new StringPropertyDefn("prop", ImmutableMap.of("foo", "bar"));
StringPropertyDefn prop = new StringPropertyDefn("prop");
assertEquals("prop", prop.name());
assertEquals("String", prop.typeName());
assertEquals("bar", prop.attributes.get("foo"));
assertEquals(String.class, PropertyAttributes.sqlParameterType(prop));
assertNull(prop.decode(null, mapper));
assertEquals("value", prop.decode("value", mapper));
@ -70,21 +67,12 @@ public class PropertyDefnTest
assertThrows(Exception.class, () -> prop.validate(Arrays.asList("a", "b"), mapper));
}
@Test
public void testCustomTypeName()
{
// Custom type name
StringPropertyDefn prop = new StringPropertyDefn("prop", ImmutableMap.of(PropertyAttributes.TYPE_NAME, "MyType"));
assertEquals("MyType", prop.typeName());
}
@Test
public void testBoolean()
{
BooleanPropertyDefn prop = new BooleanPropertyDefn("prop", null);
BooleanPropertyDefn prop = new BooleanPropertyDefn("prop");
assertEquals("prop", prop.name());
assertEquals("Boolean", prop.typeName());
assertEquals(Boolean.class, PropertyAttributes.sqlParameterType(prop));
assertNull(prop.decode(null, mapper));
assertTrue(prop.decode("true", mapper));
@ -98,10 +86,9 @@ public class PropertyDefnTest
@Test
public void testInt()
{
IntPropertyDefn prop = new IntPropertyDefn("prop", null);
IntPropertyDefn prop = new IntPropertyDefn("prop");
assertEquals("prop", prop.name());
assertEquals("Integer", prop.typeName());
assertEquals(Integer.class, PropertyAttributes.sqlParameterType(prop));
assertNull(prop.decode(null, mapper));
assertEquals((Integer) 0, prop.decode(0, mapper));
@ -114,10 +101,9 @@ public class PropertyDefnTest
@Test
public void testStringList()
{
StringListPropertyDefn prop = new StringListPropertyDefn("prop", null);
StringListPropertyDefn prop = new StringListPropertyDefn("prop");
assertEquals("prop", prop.name());
assertEquals("string list", prop.typeName());
assertSame(String.class, PropertyAttributes.sqlParameterType(prop));
assertNull(prop.decode(null, mapper));
prop.validate(null, mapper);
@ -134,12 +120,10 @@ public class PropertyDefnTest
ListPropertyDefn<ClusterKeySpec> prop = new ListPropertyDefn<ClusterKeySpec>(
"prop",
"cluster key list",
new TypeReference<List<ClusterKeySpec>>() { },
null
new TypeReference<List<ClusterKeySpec>>() { }
);
assertEquals("prop", prop.name());
assertEquals("cluster key list", prop.typeName());
assertNull(PropertyAttributes.sqlParameterType(prop));
assertNull(prop.decode(null, mapper));
List<Map<String, Object>> value = Arrays.asList(

View File

@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableMap;
import nl.jqno.equalsverifier.EqualsVerifier;
import org.apache.druid.catalog.CatalogTest;
import org.apache.druid.catalog.model.TableMetadata.TableState;
import org.apache.druid.catalog.model.table.AbstractDatasourceDefn;
import org.apache.druid.catalog.model.table.DatasourceDefn;
import org.apache.druid.java.util.common.IAE;
import org.junit.Test;
@ -66,7 +65,7 @@ public class TableMetadataTest
public void testTableMetadata()
{
Map<String, Object> props = ImmutableMap.of(
AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D"
DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D"
);
TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null);
{
@ -87,6 +86,7 @@ public class TableMetadataTest
}
{
// Missing schema
TableMetadata table = TableMetadata.newTable(
TableId.of(null, "foo"),
spec
@ -95,6 +95,7 @@ public class TableMetadataTest
}
{
// Missing table name
TableMetadata table = TableMetadata.newTable(
TableId.of(TableId.DRUID_SCHEMA, null),
spec
@ -106,7 +107,7 @@ public class TableMetadataTest
public void testConversions()
{
Map<String, Object> props = ImmutableMap.of(
AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D"
DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D"
);
TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null);
TableMetadata table = TableMetadata.newTable(

View File

@ -20,13 +20,31 @@
package org.apache.druid.catalog.model.table;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ModelProperties.PropertyDefn;
import org.apache.druid.catalog.model.TableDefnRegistry;
import org.apache.druid.catalog.model.table.TableFunction.ParameterDefn;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.ISE;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
public class BaseExternTableTest
{
protected final ObjectMapper mapper = new ObjectMapper();
public static final Map<String, Object> CSV_FORMAT = ImmutableMap.of("type", CsvInputFormat.TYPE_KEY);
protected static final List<ColumnSpec> COLUMNS = Arrays.asList(
new ColumnSpec("x", Columns.VARCHAR, null),
new ColumnSpec("y", Columns.BIGINT, null)
);
protected final ObjectMapper mapper = DefaultObjectMapper.INSTANCE;
protected final TableDefnRegistry registry = new TableDefnRegistry(mapper);
protected PropertyDefn<?> findProperty(List<PropertyDefn<?>> props, String name)
{
@ -37,4 +55,36 @@ public class BaseExternTableTest
}
return null;
}
protected Map<String, Object> toMap(Object obj)
{
try {
return mapper.convertValue(obj, ExternalTableDefn.MAP_TYPE_REF);
}
catch (Exception e) {
throw new ISE(e, "bad conversion");
}
}
protected Map<String, Object> formatToMap(InputFormat format)
{
Map<String, Object> formatMap = toMap(format);
formatMap.remove("columns");
return formatMap;
}
protected boolean hasParam(TableFunction fn, String key)
{
return hasParam(fn.parameters(), key);
}
protected boolean hasParam(List<ParameterDefn> params, String key)
{
for (ParameterDefn param : params) {
if (param.name().equals(key)) {
return true;
}
}
return false;
}
}

View File

@ -0,0 +1,111 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.table.InputFormats.FlatTextFormatDefn;
import org.apache.druid.catalog.model.table.TableFunction.ParameterDefn;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.InlineInputSource;
import org.junit.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
public class CsvInputFormatTest extends BaseExternTableTest
{
@Test
public void testDefaults()
{
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(new InlineInputSource("a\n")))
.inputFormat(CSV_FORMAT)
.column("a", Columns.VARCHAR)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
InputFormatDefn defn = registry.inputFormatDefnFor(CsvInputFormat.TYPE_KEY);
InputFormat inputFormat = defn.convertFromTable(new ResolvedExternalTable(resolved));
CsvInputFormat csvFormat = (CsvInputFormat) inputFormat;
assertEquals(0, csvFormat.getSkipHeaderRows());
assertFalse(csvFormat.isFindColumnsFromHeader());
assertNull(csvFormat.getListDelimiter());
assertEquals(Collections.singletonList("a"), csvFormat.getColumns());
}
@Test
public void testConversion()
{
CsvInputFormat format = new CsvInputFormat(
Collections.singletonList("a"), ";", false, false, 1);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(new InlineInputSource("a\n")))
.inputFormat(formatToMap(format))
.column("a", Columns.VARCHAR)
.column("b", Columns.BIGINT)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
InputFormatDefn defn = registry.inputFormatDefnFor(CsvInputFormat.TYPE_KEY);
InputFormat inputFormat = defn.convertFromTable(new ResolvedExternalTable(resolved));
CsvInputFormat csvFormat = (CsvInputFormat) inputFormat;
assertEquals(1, csvFormat.getSkipHeaderRows());
assertFalse(csvFormat.isFindColumnsFromHeader());
assertEquals(";", csvFormat.getListDelimiter());
assertEquals(Arrays.asList("a", "b"), csvFormat.getColumns());
}
@Test
public void testFunctionParams()
{
InputFormatDefn defn = registry.inputFormatDefnFor(CsvInputFormat.TYPE_KEY);
List<ParameterDefn> params = defn.parameters();
assertEquals(2, params.size());
}
@Test
public void testCreateFromArgs()
{
Map<String, Object> args = new HashMap<>();
args.put(FlatTextFormatDefn.LIST_DELIMITER_PARAMETER, ";");
args.put(FlatTextFormatDefn.SKIP_ROWS_PARAMETER, 1);
InputFormatDefn defn = registry.inputFormatDefnFor(CsvInputFormat.TYPE_KEY);
List<ColumnSpec> columns = Collections.singletonList(new ColumnSpec("a", null, null));
InputFormat inputFormat = defn.convertFromArgs(args, columns, mapper);
CsvInputFormat csvFormat = (CsvInputFormat) inputFormat;
assertEquals(1, csvFormat.getSkipHeaderRows());
assertFalse(csvFormat.isFindColumnsFromHeader());
assertEquals(";", csvFormat.getListDelimiter());
assertEquals(Collections.singletonList("a"), csvFormat.getColumns());
}
}

View File

@ -28,9 +28,13 @@ import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableDefn;
import org.apache.druid.catalog.model.TableDefnRegistry;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.TableSpec;
import org.apache.druid.catalog.model.table.DatasourceDefn.DatasourceColumnDefn;
import org.apache.druid.catalog.model.facade.DatasourceFacade;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.logger.Logger;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
@ -44,6 +48,7 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
@ -53,9 +58,9 @@ import static org.junit.Assert.assertTrue;
@Category(CatalogTest.class)
public class DatasourceTableTest
{
private static final String SUM_BIGINT = "SUM(BIGINT)";
private static final Logger LOG = new Logger(DatasourceTableTest.class);
private final ObjectMapper mapper = new ObjectMapper();
private final ObjectMapper mapper = DefaultObjectMapper.INSTANCE;
private final TableDefnRegistry registry = new TableDefnRegistry(mapper);
@Test
@ -63,23 +68,19 @@ public class DatasourceTableTest
{
// Minimum possible definition
Map<String, Object> props = ImmutableMap.of(
AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D"
DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D"
);
{
TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null);
ResolvedTable table = registry.resolve(spec);
assertNotNull(table);
assertTrue(table.defn() instanceof DatasourceDefn);
table.validate();
}
{
TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null);
ResolvedTable table = registry.resolve(spec);
assertNotNull(table);
assertTrue(table.defn() instanceof DatasourceDefn);
table.validate();
}
TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null);
ResolvedTable table = registry.resolve(spec);
assertNotNull(table);
assertTrue(table.defn() instanceof DatasourceDefn);
table.validate();
DatasourceFacade facade = new DatasourceFacade(registry.resolve(table.spec()));
assertEquals("P1D", facade.segmentGranularityString());
assertNull(facade.targetSegmentRows());
assertTrue(facade.hiddenColumns().isEmpty());
assertFalse(facade.isSealed());
}
private void expectValidationFails(final ResolvedTable table)
@ -124,20 +125,18 @@ public class DatasourceTableTest
{
Map<String, Object> props = ImmutableMap.<String, Object>builder()
.put(TableDefn.DESCRIPTION_PROPERTY, "My table")
.put(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D")
.put(AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000)
.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("foo", "bar"))
.put(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D")
.put(DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000)
.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Arrays.asList("foo", "bar"))
.put(DatasourceDefn.SEALED_PROPERTY, true)
.build();
{
TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null);
expectValidationSucceeds(spec);
}
{
TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null);
expectValidationSucceeds(spec);
}
TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null);
DatasourceFacade facade = new DatasourceFacade(registry.resolve(spec));
assertEquals("P1D", facade.segmentGranularityString());
assertEquals(1_000_000, (int) facade.targetSegmentRows());
assertEquals(Arrays.asList("foo", "bar"), facade.hiddenColumns());
assertTrue(facade.isSealed());
}
@Test
@ -162,7 +161,7 @@ public class DatasourceTableTest
// Target segment rows
{
TableSpec spec = TableBuilder.datasource("foo", "P1D")
.property(AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, "bogus")
.property(DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, "bogus")
.buildSpec();
expectValidationFails(spec);
}
@ -170,7 +169,7 @@ public class DatasourceTableTest
// Hidden columns
{
TableSpec spec = TableBuilder.datasource("foo", "P1D")
.property(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, "bogus")
.property(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, "bogus")
.buildSpec();
expectValidationFails(spec);
}
@ -180,6 +179,14 @@ public class DatasourceTableTest
.buildSpec();
expectValidationFails(spec);
}
// Sealed
{
TableSpec spec = TableBuilder.datasource("foo", "P1D")
.property(DatasourceDefn.SEALED_PROPERTY, "bogus")
.buildSpec();
expectValidationFails(spec);
}
}
@Test
@ -195,31 +202,25 @@ public class DatasourceTableTest
@Test
public void testColumnSpec()
{
// Type is required
{
ColumnSpec spec = new ColumnSpec(null, null, null, null);
assertThrows(IAE.class, () -> spec.validate());
}
// Name is required
{
ColumnSpec spec = new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, null, null, null);
ColumnSpec spec = new ColumnSpec(null, null, null);
assertThrows(IAE.class, () -> spec.validate());
}
{
ColumnSpec spec = new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "foo", null, null);
ColumnSpec spec = new ColumnSpec("foo", null, null);
spec.validate();
}
// Type is optional
{
ColumnSpec spec = new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "foo", "VARCHAR", null);
ColumnSpec spec = new ColumnSpec("foo", "VARCHAR", null);
spec.validate();
}
}
@Test
public void testDetailTableColumns()
public void testColumns()
{
TableBuilder builder = TableBuilder.datasource("foo", "P1D");
@ -227,7 +228,8 @@ public class DatasourceTableTest
{
TableSpec spec = builder.copy()
.buildSpec();
expectValidationSucceeds(spec);
ResolvedTable table = registry.resolve(spec);
table.validate();
}
// OK to have no column type
@ -235,29 +237,12 @@ public class DatasourceTableTest
TableSpec spec = builder.copy()
.column("foo", null)
.buildSpec();
expectValidationSucceeds(spec);
}
ResolvedTable table = registry.resolve(spec);
table.validate();
// Time column can have no type
{
TableSpec spec = builder.copy()
.column(Columns.TIME_COLUMN, null)
.buildSpec();
expectValidationSucceeds(spec);
}
// Time column can only have TIMESTAMP type
{
TableSpec spec = builder.copy()
.timeColumn()
.buildSpec();
expectValidationSucceeds(spec);
}
{
TableSpec spec = builder.copy()
.column(Columns.TIME_COLUMN, Columns.VARCHAR)
.buildSpec();
expectValidationFails(spec);
DatasourceFacade facade = new DatasourceFacade(registry.resolve(table.spec()));
assertNotNull(facade.jsonMapper());
assertEquals(1, facade.properties().size());
}
// Can have a legal scalar type
@ -265,23 +250,8 @@ public class DatasourceTableTest
TableSpec spec = builder.copy()
.column("foo", Columns.VARCHAR)
.buildSpec();
expectValidationSucceeds(spec);
}
// Reject an unknown SQL type
{
TableSpec spec = builder.copy()
.column("foo", "BOGUS")
.buildSpec();
expectValidationFails(spec);
}
// Cannot use a measure type
{
TableSpec spec = builder.copy()
.column("foo", SUM_BIGINT)
.buildSpec();
expectValidationFails(spec);
ResolvedTable table = registry.resolve(spec);
table.validate();
}
// Reject duplicate columns
@ -299,6 +269,76 @@ public class DatasourceTableTest
.buildSpec();
expectValidationFails(spec);
}
{
TableSpec spec = builder.copy()
.column(Columns.TIME_COLUMN, null)
.column("s", Columns.VARCHAR)
.column("bi", Columns.BIGINT)
.column("f", Columns.FLOAT)
.column("d", Columns.DOUBLE)
.buildSpec();
ResolvedTable table = registry.resolve(spec);
table.validate();
}
}
@Test
public void testRollup()
{
TableMetadata table = TableBuilder.datasource("foo", "P1D")
.column(Columns.TIME_COLUMN, "TIMESTAMP('PT1M')")
.column("a", null)
.column("b", Columns.VARCHAR)
.column("c", "SUM(BIGINT)")
.build();
table.validate();
List<ColumnSpec> columns = table.spec().columns();
assertEquals(4, columns.size());
assertEquals(Columns.TIME_COLUMN, columns.get(0).name());
assertEquals("TIMESTAMP('PT1M')", columns.get(0).sqlType());
assertEquals("a", columns.get(1).name());
assertNull(columns.get(1).sqlType());
assertEquals("b", columns.get(2).name());
assertEquals(Columns.VARCHAR, columns.get(2).sqlType());
assertEquals("c", columns.get(3).name());
assertEquals("SUM(BIGINT)", columns.get(3).sqlType());
}
@Test
public void testTimeColumn()
{
TableBuilder builder = TableBuilder.datasource("foo", "P1D");
// Time column can have no type
{
TableSpec spec = builder.copy()
.column(Columns.TIME_COLUMN, null)
.buildSpec();
ResolvedTable table = registry.resolve(spec);
table.validate();
}
// Time column can only have TIMESTAMP type
{
TableSpec spec = builder.copy()
.timeColumn()
.buildSpec();
ResolvedTable table = registry.resolve(spec);
table.validate();
}
{
TableSpec spec = builder.copy()
.column(Columns.TIME_COLUMN, "TIMESTAMP('PT5M')")
.buildSpec();
ResolvedTable table = registry.resolve(spec);
table.validate();
}
}
@Test
@ -320,11 +360,11 @@ public class DatasourceTableTest
.build();
TableSpec spec = TableBuilder.datasource("foo", "PT1H")
.description("My table")
.property(AbstractDatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000)
.property(DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000)
.hiddenColumns("foo", "bar")
.property("tag1", "some value")
.property("tag2", "second value")
.column(new ColumnSpec(DatasourceColumnDefn.COLUMN_TYPE, "a", null, colProps))
.column(new ColumnSpec("a", null, colProps))
.column("b", Columns.VARCHAR)
.buildSpec();
@ -381,7 +421,7 @@ public class DatasourceTableTest
// such values to indicate which properties to remove.
Map<String, Object> updatedProps = new HashMap<>();
// Update a property
updatedProps.put(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D");
updatedProps.put(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D");
// Remove a property
updatedProps.put("tag1", null);
// Add a property
@ -396,8 +436,8 @@ public class DatasourceTableTest
// changed.
assertNotEquals(spec, merged);
assertEquals(
updatedProps.get(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY),
merged.properties().get(AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY)
updatedProps.get(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY),
merged.properties().get(DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY)
);
assertFalse(merged.properties().containsKey("tag1"));
assertEquals(
@ -413,24 +453,24 @@ public class DatasourceTableTest
// Remove all hidden columns
Map<String, Object> updatedProps = new HashMap<>();
updatedProps.put(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, null);
updatedProps.put(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, null);
TableSpec update = new TableSpec(null, updatedProps, null);
TableSpec merged = mergeTables(spec, update);
expectValidationSucceeds(merged);
assertFalse(
merged.properties().containsKey(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
merged.properties().containsKey(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
);
// Wrong type
updatedProps = ImmutableMap.of(
AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, "mumble"
DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, "mumble"
);
update = new TableSpec(null, updatedProps, null);
assertMergeFails(spec, update);
// Merge
updatedProps = ImmutableMap.of(
AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.singletonList("mumble")
DatasourceDefn.HIDDEN_COLUMNS_PROPERTY, Collections.singletonList("mumble")
);
update = new TableSpec(null, updatedProps, null);
merged = mergeTables(spec, update);
@ -438,7 +478,7 @@ public class DatasourceTableTest
assertEquals(
Arrays.asList("foo", "bar", "mumble"),
merged.properties().get(AbstractDatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
merged.properties().get(DatasourceDefn.HIDDEN_COLUMNS_PROPERTY)
);
}
@ -446,13 +486,12 @@ public class DatasourceTableTest
public void testMergeColsWithEmptyList()
{
Map<String, Object> props = ImmutableMap.of(
AbstractDatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D"
DatasourceDefn.SEGMENT_GRANULARITY_PROPERTY, "P1D"
);
TableSpec spec = new TableSpec(DatasourceDefn.TABLE_TYPE, props, null);
List<ColumnSpec> colUpdates = Collections.singletonList(
new ColumnSpec(
DatasourceColumnDefn.COLUMN_TYPE,
"a",
Columns.BIGINT,
null
@ -481,13 +520,11 @@ public class DatasourceTableTest
List<ColumnSpec> colUpdates = Arrays.asList(
new ColumnSpec(
DatasourceColumnDefn.COLUMN_TYPE,
"a",
Columns.BIGINT,
updatedProps
),
new ColumnSpec(
DatasourceColumnDefn.COLUMN_TYPE,
"c",
Columns.VARCHAR,
null
@ -509,4 +546,26 @@ public class DatasourceTableTest
assertEquals("c", columns.get(2).name());
assertEquals(Columns.VARCHAR, columns.get(2).sqlType());
}
/**
* Test case for multiple of the {@code datasource.md} examples. To use this, enable the
* test, run it, then copy the JSON from the console. The examples pull out bits
* and pieces in multiple places.
*/
@Test
@Ignore
public void docExample()
{
TableSpec spec = TableBuilder.datasource("foo", "PT1H")
.description("Web server performance metrics")
.property(DatasourceDefn.TARGET_SEGMENT_ROWS_PROPERTY, 1_000_000)
.hiddenColumns("foo", "bar")
.column("__time", Columns.TIMESTAMP)
.column("host", Columns.VARCHAR, ImmutableMap.of(TableDefn.DESCRIPTION_PROPERTY, "The web server host"))
.column("bytesSent", Columns.BIGINT, ImmutableMap.of(TableDefn.DESCRIPTION_PROPERTY, "Number of response bytes sent"))
.clusterColumns(new ClusterKeySpec("a", false), new ClusterKeySpec("b", true))
.sealed(true)
.buildSpec();
LOG.info(spec.toString());
}
}

View File

@ -0,0 +1,123 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.table.InputFormats.DelimitedFormatDefn;
import org.apache.druid.catalog.model.table.InputFormats.FlatTextFormatDefn;
import org.apache.druid.catalog.model.table.TableFunction.ParameterDefn;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.impl.DelimitedInputFormat;
import org.apache.druid.data.input.impl.InlineInputSource;
import org.junit.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
public class DelimitedInputFormatTest extends BaseExternTableTest
{
@Test
public void testDefaults()
{
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(new InlineInputSource("a\n")))
.inputFormat(ImmutableMap.of(
"type", DelimitedInputFormat.TYPE_KEY,
DelimitedFormatDefn.DELIMITER_FIELD, "|"
)
)
.column("a", Columns.VARCHAR)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
InputFormatDefn defn = registry.inputFormatDefnFor(DelimitedInputFormat.TYPE_KEY);
InputFormat inputFormat = defn.convertFromTable(new ResolvedExternalTable(resolved));
DelimitedInputFormat delmited = (DelimitedInputFormat) inputFormat;
assertEquals(0, delmited.getSkipHeaderRows());
assertFalse(delmited.isFindColumnsFromHeader());
assertNull(delmited.getListDelimiter());
assertEquals("|", delmited.getDelimiter());
assertEquals(Collections.singletonList("a"), delmited.getColumns());
}
@Test
public void testConversion()
{
DelimitedInputFormat format = new DelimitedInputFormat(
Collections.singletonList("a"), ";", "|", false, false, 1);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(new InlineInputSource("a\n")))
.inputFormat(formatToMap(format))
.column("a", Columns.VARCHAR)
.column("b", Columns.BIGINT)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
InputFormatDefn defn = registry.inputFormatDefnFor(DelimitedInputFormat.TYPE_KEY);
InputFormat inputFormat = defn.convertFromTable(new ResolvedExternalTable(resolved));
DelimitedInputFormat delmited = (DelimitedInputFormat) inputFormat;
assertEquals(1, delmited.getSkipHeaderRows());
assertFalse(delmited.isFindColumnsFromHeader());
assertEquals(";", delmited.getListDelimiter());
assertEquals("|", delmited.getDelimiter());
assertEquals(Arrays.asList("a", "b"), delmited.getColumns());
}
@Test
public void testFunctionParams()
{
InputFormatDefn defn = registry.inputFormatDefnFor(DelimitedInputFormat.TYPE_KEY);
List<ParameterDefn> params = defn.parameters();
assertEquals(3, params.size());
assertTrue(hasParam(params, DelimitedFormatDefn.DELIMITER_PARAMETER));
}
@Test
public void testCreateFromArgs()
{
Map<String, Object> args = new HashMap<>();
args.put(DelimitedFormatDefn.DELIMITER_PARAMETER, "|");
args.put(FlatTextFormatDefn.LIST_DELIMITER_PARAMETER, ";");
args.put(FlatTextFormatDefn.SKIP_ROWS_PARAMETER, 1);
InputFormatDefn defn = registry.inputFormatDefnFor(DelimitedInputFormat.TYPE_KEY);
List<ColumnSpec> columns = Collections.singletonList(new ColumnSpec("a", null, null));
InputFormat inputFormat = defn.convertFromArgs(args, columns, mapper);
DelimitedInputFormat delmited = (DelimitedInputFormat) inputFormat;
assertEquals(1, delmited.getSkipHeaderRows());
assertFalse(delmited.isFindColumnsFromHeader());
assertEquals(";", delmited.getListDelimiter());
assertEquals("|", delmited.getDelimiter());
assertEquals(Collections.singletonList("a"), delmited.getColumns());
}
}

View File

@ -0,0 +1,218 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableDefnRegistry;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.HttpInputSource;
import org.apache.druid.data.input.impl.HttpInputSourceConfig;
import org.apache.druid.data.input.impl.InlineInputSource;
import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.metadata.DefaultPasswordProvider;
import org.junit.Ignore;
import org.junit.Test;
import java.io.File;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Collections;
import java.util.Map;
import static org.junit.Assert.assertThrows;
public class ExternalTableTest extends BaseExternTableTest
{
private static final Logger LOG = new Logger(ExternalTableTest.class);
private final TableDefnRegistry registry = new TableDefnRegistry(mapper);
@Test
public void testValidateEmptyTable()
{
// Empty table: not valid
TableMetadata table = TableBuilder.external("foo").build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testValidateMissingSourceType()
{
// Empty table: not valid
TableMetadata table = TableBuilder.external("foo")
.inputSource(ImmutableMap.of())
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testValidateUnknownSourceType()
{
// Empty table: not valid
TableMetadata table = TableBuilder.external("foo")
.inputSource(ImmutableMap.of("type", "unknown"))
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testValidateSourceOnly()
{
// Input source only: valid, assumes the format is given later
LocalInputSource inputSource = new LocalInputSource(new File("/tmp"), "*");
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(inputSource))
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
}
@Test
public void testValidateMissingFormatType()
{
// Input source only: valid, assumes the format is given later
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(new InlineInputSource("a\n")))
.inputFormat(ImmutableMap.of())
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testValidateUnknownFormatType()
{
// Input source only: valid, assumes the format is given later
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(new InlineInputSource("a\n")))
.inputFormat(ImmutableMap.of("type", "unknown"))
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testValidateSourceAndFormat()
{
// Format is given without columns: it is validated
CsvInputFormat format = new CsvInputFormat(
Collections.singletonList("a"), ";", false, false, 0);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(new InlineInputSource("a\n")))
.inputFormat(formatToMap(format))
.column("a", Columns.VARCHAR)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
}
/**
* Test case for multiple of the {@code ext.md} examples. To use this, enable the
* test, run it, then copy the JSON from the console. The examples pull out bits
* and pieces in multiple places.
*/
@Test
@Ignore
public void wikipediaDocExample()
{
JsonInputFormat format = new JsonInputFormat(null, null, true, true, false);
LocalInputSource inputSource = new LocalInputSource(new File("/Users/bob/druid/quickstart/tutorial"), "wikiticker-*-sampled.json");
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(inputSource))
.inputFormat(formatToMap(format))
.description("Sample Wikipedia data")
.column("timetamp", Columns.VARCHAR)
.column("page", Columns.VARCHAR)
.column("language", Columns.VARCHAR)
.column("unpatrolled", Columns.VARCHAR)
.column("newPage", Columns.VARCHAR)
.column("robot", Columns.VARCHAR)
.column("added", Columns.VARCHAR)
.column("namespace", Columns.BIGINT)
.column("deleted", Columns.BIGINT)
.column("delta", Columns.BIGINT)
.build();
LOG.info(table.spec().toString());
}
@Test
@Ignore
public void httpDocExample() throws URISyntaxException
{
HttpInputSource inputSource = new HttpInputSource(
Collections.singletonList(new URI("https://example.com/my.csv")), // removed
"bob",
new DefaultPasswordProvider("secret"),
new HttpInputSourceConfig(null)
);
Map<String, Object> sourceMap = toMap(inputSource);
sourceMap.remove("uris");
TableMetadata table = TableBuilder.external("koala")
.inputSource(sourceMap)
.inputFormat(CSV_FORMAT)
.property(HttpInputSourceDefn.URI_TEMPLATE_PROPERTY, "https://example.com/{}")
.description("Example parameterized external table")
.column("timetamp", Columns.VARCHAR)
.column("metric", Columns.VARCHAR)
.column("value", Columns.BIGINT)
.build();
LOG.info(table.spec().toString());
}
@Test
@Ignore
public void httpConnDocExample() throws URISyntaxException
{
HttpInputSource inputSource = new HttpInputSource(
Collections.singletonList(new URI("https://example.com/")),
"bob",
new DefaultPasswordProvider("secret"),
new HttpInputSourceConfig(null)
);
TableMetadata table = TableBuilder.external("koala")
.inputSource(toMap(inputSource))
.description("Example connection")
.build();
LOG.info(table.spec().toString());
}
@Test
@Ignore
public void localDocExample()
{
Map<String, Object> sourceMap = ImmutableMap.of(
"type", LocalInputSource.TYPE_KEY,
"baseDir", "/var/data"
);
TableMetadata table = TableBuilder.external("koala")
.inputSource(sourceMap)
.build();
LOG.info(table.spec().toString());
}
}

View File

@ -0,0 +1,481 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.fasterxml.jackson.databind.InjectableValues;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.table.InputFormats.CsvFormatDefn;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.HttpInputSource;
import org.apache.druid.data.input.impl.HttpInputSourceConfig;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.metadata.DefaultPasswordProvider;
import org.apache.druid.metadata.EnvironmentVariablePasswordProvider;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.junit.Before;
import org.junit.Test;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
public class HttpInputSourceDefnTest extends BaseExternTableTest
{
@Before
public void setup()
{
mapper.setInjectableValues(new InjectableValues.Std().addValue(
HttpInputSourceConfig.class,
new HttpInputSourceConfig(HttpInputSourceConfig.DEFAULT_ALLOWED_PROTOCOLS)
));
}
@Test
public void testEmptyInputSource()
{
// No URIs property or template: not valid. Need at least a URI or template
TableMetadata table = TableBuilder.external("foo")
.inputSource(ImmutableMap.of("type", HttpInputSource.TYPE_KEY))
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testInvalidTemplate()
{
// No format: valid. Format can be provided via a table function
TableMetadata table = TableBuilder.external("foo")
.inputSource(ImmutableMap.of("type", HttpInputSource.TYPE_KEY))
.property(HttpInputSourceDefn.URI_TEMPLATE_PROPERTY, "http://example.com/")
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testNoFormatWithURI() throws URISyntaxException
{
// No format: not valid if URI is provided
HttpInputSource inputSource = new HttpInputSource(
Collections.singletonList(new URI("http://example.com/file.csv")),
null,
null,
new HttpInputSourceConfig(null)
);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(inputSource))
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testNoColumnsWithUri() throws URISyntaxException
{
// No format: not valid if URI is provided
HttpInputSource inputSource = new HttpInputSource(
Collections.singletonList(new URI("http://example.com/file.csv")),
null,
null,
new HttpInputSourceConfig(null)
);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(inputSource))
.inputFormat(CSV_FORMAT)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testNoFormatWithTemplate()
{
// No format: valid. Format can be provided via a table function
TableMetadata table = TableBuilder.external("foo")
.inputSource(ImmutableMap.of("type", HttpInputSource.TYPE_KEY))
.property(HttpInputSourceDefn.URI_TEMPLATE_PROPERTY, "http://example.com/{}")
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
}
@Test
public void testURIAndTemplate() throws URISyntaxException
{
// No format: not valid if URI is provided
HttpInputSource inputSource = new HttpInputSource(
Collections.singletonList(new URI("http://example.com/file.csv")),
null,
null,
new HttpInputSourceConfig(null)
);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(inputSource))
.inputFormat(CSV_FORMAT)
.property(HttpInputSourceDefn.URI_TEMPLATE_PROPERTY, "http://example.com/{}")
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testAdHocBadFormat()
{
InputSourceDefn httpDefn = registry.inputSourceDefnFor(HttpInputSourceDefn.TYPE_KEY);
TableFunction fn = httpDefn.adHocTableFn();
Map<String, Object> args = new HashMap<>();
args.put(HttpInputSourceDefn.URIS_PARAMETER, new String[] {"http://foo.com/my.csv"});
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, "bogus");
assertThrows(IAE.class, () -> fn.apply("x", args, COLUMNS, mapper));
}
@Test
public void testAdHocBadUri()
{
InputSourceDefn httpDefn = registry.inputSourceDefnFor(HttpInputSourceDefn.TYPE_KEY);
TableFunction fn = httpDefn.adHocTableFn();
Map<String, Object> args = new HashMap<>();
args.put(HttpInputSourceDefn.URIS_PARAMETER, new String[] {"bogus"});
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
assertThrows(IAE.class, () -> fn.apply("x", args, COLUMNS, mapper));
}
@Test
public void testAdHocHappyPath()
{
InputSourceDefn httpDefn = registry.inputSourceDefnFor(HttpInputSourceDefn.TYPE_KEY);
TableFunction fn = httpDefn.adHocTableFn();
assertTrue(hasParam(fn, HttpInputSourceDefn.URIS_PARAMETER));
assertTrue(hasParam(fn, FormattedInputSourceDefn.FORMAT_PARAMETER));
assertTrue(hasParam(fn, HttpInputSourceDefn.USER_PARAMETER));
// Convert to an external table. Must provide the URIs plus format and columns.
Map<String, Object> args = new HashMap<>();
args.put(HttpInputSourceDefn.URIS_PARAMETER, Collections.singletonList("http://foo.com/my.csv"));
args.put(HttpInputSourceDefn.USER_PARAMETER, "bob");
args.put(HttpInputSourceDefn.PASSWORD_PARAMETER, "secret");
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
validateHappyPath(externSpec);
// But, it fails if there are no columns.
assertThrows(IAE.class, () -> fn.apply("x", args, Collections.emptyList(), mapper));
}
@Test
public void testFullTableSpecHappyPath() throws URISyntaxException
{
HttpInputSource inputSource = new HttpInputSource(
Collections.singletonList(new URI("http://foo.com/my.csv")),
"bob",
new DefaultPasswordProvider("secret"),
new HttpInputSourceConfig(null)
);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(inputSource))
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
// Check validation
table.validate();
// Check registry
ResolvedTable resolved = registry.resolve(table.spec());
assertNotNull(resolved);
// Convert to an external spec
ExternalTableDefn externDefn = (ExternalTableDefn) resolved.defn();
ExternalTableSpec externSpec = externDefn.convert(resolved);
validateHappyPath(externSpec);
// Get the partial table function
TableFunction fn = externDefn.tableFn(resolved);
assertTrue(fn.parameters().isEmpty());
// Convert to an external table.
externSpec = fn.apply("x", Collections.emptyMap(), Collections.emptyList(), mapper);
validateHappyPath(externSpec);
// But, it fails columns are provided since the table already has them.
assertThrows(IAE.class, () -> fn.apply("x", Collections.emptyMap(), COLUMNS, mapper));
}
@Test
public void testTemplateSpecWithFormatHappyPath() throws URISyntaxException
{
HttpInputSource inputSource = new HttpInputSource(
Collections.singletonList(new URI("http://foo.com/my.csv")), // removed
"bob",
new DefaultPasswordProvider("secret"),
new HttpInputSourceConfig(null)
);
TableMetadata table = TableBuilder.external("foo")
.inputSource(httpToMap(inputSource))
.inputFormat(CSV_FORMAT)
.property(HttpInputSourceDefn.URI_TEMPLATE_PROPERTY, "http://foo.com/{}")
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
// Check validation
table.validate();
// Check registry
ResolvedTable resolved = registry.resolve(table.spec());
assertNotNull(resolved);
// Not a full table, can't directly convert
// Convert to an external spec
ExternalTableDefn externDefn = (ExternalTableDefn) resolved.defn();
assertThrows(IAE.class, () -> externDefn.convert(resolved));
// Get the partial table function
TableFunction fn = externDefn.tableFn(resolved);
assertEquals(1, fn.parameters().size());
assertTrue(hasParam(fn, HttpInputSourceDefn.URIS_PARAMETER));
// Convert to an external table.
ExternalTableSpec externSpec = fn.apply(
"x",
ImmutableMap.of(
HttpInputSourceDefn.URIS_PARAMETER,
Collections.singletonList("my.csv")
),
Collections.emptyList(),
mapper
);
validateHappyPath(externSpec);
}
@Test
public void testTemplateSpecWithoutFormatHappyPath() throws URISyntaxException
{
HttpInputSource inputSource = new HttpInputSource(
Collections.singletonList(new URI("http://foo.com/my.csv")), // removed
"bob",
new DefaultPasswordProvider("secret"),
new HttpInputSourceConfig(null)
);
TableMetadata table = TableBuilder.external("foo")
.inputSource(httpToMap(inputSource))
.property(HttpInputSourceDefn.URI_TEMPLATE_PROPERTY, "http://foo.com/{}")
.build();
// Check validation
table.validate();
// Not a full table, can't directly convert
// Convert to an external spec
ResolvedTable resolved = registry.resolve(table.spec());
ExternalTableDefn externDefn = (ExternalTableDefn) resolved.defn();
assertThrows(IAE.class, () -> externDefn.convert(resolved));
// Get the partial table function
TableFunction fn = externDefn.tableFn(resolved);
assertTrue(hasParam(fn, HttpInputSourceDefn.URIS_PARAMETER));
assertTrue(hasParam(fn, FormattedInputSourceDefn.FORMAT_PARAMETER));
assertFalse(hasParam(fn, HttpInputSourceDefn.USER_PARAMETER));
// Convert to an external table. Must provide the URIs plus format and columns.
Map<String, Object> args = new HashMap<>();
args.put(HttpInputSourceDefn.URIS_PARAMETER, Collections.singletonList("my.csv"));
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
validateHappyPath(externSpec);
}
@Test
public void testMultipleURIsInTableSpec() throws URISyntaxException
{
HttpInputSource inputSource = new HttpInputSource(
Arrays.asList(new URI("http://foo.com/foo.csv"), new URI("http://foo.com/bar.csv")),
"bob",
new EnvironmentVariablePasswordProvider("SECRET"),
new HttpInputSourceConfig(null)
);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(inputSource))
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
// Check validation
table.validate();
// Convert to an external spec
ResolvedTable resolved = registry.resolve(table.spec());
ExternalTableDefn externDefn = (ExternalTableDefn) resolved.defn();
ExternalTableSpec externSpec = externDefn.convert(resolved);
HttpInputSource sourceSpec = (HttpInputSource) externSpec.inputSource;
assertEquals(
CatalogUtils.stringListToUriList(Arrays.asList("http://foo.com/foo.csv", "http://foo.com/bar.csv")),
sourceSpec.getUris()
);
}
@Test
public void testMultipleURIsWithTemplate() throws URISyntaxException
{
HttpInputSource inputSource = new HttpInputSource(
Collections.singletonList(new URI("http://foo.com/my.csv")), // removed
"bob",
new DefaultPasswordProvider("secret"),
new HttpInputSourceConfig(null)
);
TableMetadata table = TableBuilder.external("foo")
.inputSource(httpToMap(inputSource))
.inputFormat(CSV_FORMAT)
.property(HttpInputSourceDefn.URI_TEMPLATE_PROPERTY, "http://foo.com/{}")
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
// Check validation
table.validate();
// Get the partial table function
ResolvedTable resolved = registry.resolve(table.spec());
ExternalTableDefn externDefn = (ExternalTableDefn) resolved.defn();
TableFunction fn = externDefn.tableFn(resolved);
assertEquals(1, fn.parameters().size());
assertTrue(hasParam(fn, HttpInputSourceDefn.URIS_PARAMETER));
// Convert to an external table.
ExternalTableSpec externSpec = fn.apply(
"x",
ImmutableMap.of(
HttpInputSourceDefn.URIS_PARAMETER, Arrays.asList("my.csv", "bar.csv")),
Collections.emptyList(),
mapper
);
HttpInputSource sourceSpec = (HttpInputSource) externSpec.inputSource;
assertEquals(
CatalogUtils.stringListToUriList(Arrays.asList("http://foo.com/my.csv", "http://foo.com/bar.csv")),
sourceSpec.getUris()
);
}
@Test
public void testMultipleURIsAdHoc()
{
InputSourceDefn httpDefn = registry.inputSourceDefnFor(HttpInputSourceDefn.TYPE_KEY);
TableFunction fn = httpDefn.adHocTableFn();
assertTrue(hasParam(fn, HttpInputSourceDefn.URIS_PARAMETER));
assertTrue(hasParam(fn, FormattedInputSourceDefn.FORMAT_PARAMETER));
assertTrue(hasParam(fn, HttpInputSourceDefn.USER_PARAMETER));
// Convert to an external table. Must provide the URIs plus format and columns.
Map<String, Object> args = new HashMap<>();
args.put(HttpInputSourceDefn.URIS_PARAMETER, Arrays.asList("http://foo.com/foo.csv", "http://foo.com/bar.csv"));
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
HttpInputSource sourceSpec = (HttpInputSource) externSpec.inputSource;
assertEquals(
CatalogUtils.stringListToUriList(Arrays.asList("http://foo.com/foo.csv", "http://foo.com/bar.csv")),
sourceSpec.getUris()
);
}
@Test
public void testEnvPassword() throws URISyntaxException
{
HttpInputSource inputSource = new HttpInputSource(
Collections.singletonList(new URI("http://foo.com/my.csv")),
"bob",
new EnvironmentVariablePasswordProvider("SECRET"),
new HttpInputSourceConfig(null)
);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(inputSource))
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
// Check validation
table.validate();
// Convert to an external spec
ResolvedTable resolved = registry.resolve(table.spec());
ExternalTableDefn externDefn = (ExternalTableDefn) resolved.defn();
ExternalTableSpec externSpec = externDefn.convert(resolved);
HttpInputSource sourceSpec = (HttpInputSource) externSpec.inputSource;
assertEquals("bob", sourceSpec.getHttpAuthenticationUsername());
assertEquals("SECRET", ((EnvironmentVariablePasswordProvider) sourceSpec.getHttpAuthenticationPasswordProvider()).getVariable());
}
private void validateHappyPath(ExternalTableSpec externSpec)
{
HttpInputSource sourceSpec = (HttpInputSource) externSpec.inputSource;
assertEquals("bob", sourceSpec.getHttpAuthenticationUsername());
assertEquals("secret", ((DefaultPasswordProvider) sourceSpec.getHttpAuthenticationPasswordProvider()).getPassword());
assertEquals("http://foo.com/my.csv", sourceSpec.getUris().get(0).toString());
// Just a sanity check: details of CSV conversion are tested elsewhere.
CsvInputFormat csvFormat = (CsvInputFormat) externSpec.inputFormat;
assertEquals(Arrays.asList("x", "y"), csvFormat.getColumns());
RowSignature sig = externSpec.signature;
assertEquals(Arrays.asList("x", "y"), sig.getColumnNames());
assertEquals(ColumnType.STRING, sig.getColumnType(0).get());
assertEquals(ColumnType.LONG, sig.getColumnType(1).get());
}
private Map<String, Object> httpToMap(HttpInputSource source)
{
Map<String, Object> sourceMap = toMap(source);
sourceMap.remove("uris");
return sourceMap;
}
}

View File

@ -1,259 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.fasterxml.jackson.databind.InjectableValues;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.catalog.CatalogTest;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ModelProperties.PropertyDefn;
import org.apache.druid.catalog.model.ParameterizedDefn;
import org.apache.druid.catalog.model.PropertyAttributes;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableDefnRegistry;
import org.apache.druid.catalog.model.table.ExternalTableDefn.FormattedExternalTableDefn;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.HttpInputSource;
import org.apache.druid.data.input.impl.HttpInputSourceConfig;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.metadata.DefaultPasswordProvider;
import org.apache.druid.metadata.EnvironmentVariablePasswordProvider;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
@Category(CatalogTest.class)
public class HttpInputTableTest extends BaseExternTableTest
{
private final HttpTableDefn tableDefn = new HttpTableDefn();
private final TableBuilder baseBuilder = TableBuilder.of(tableDefn)
.description("http input")
.format(InputFormats.CSV_FORMAT_TYPE)
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT);
public HttpInputTableTest()
{
mapper.setInjectableValues(new InjectableValues.Std().addValue(
HttpInputSourceConfig.class,
new HttpInputSourceConfig(HttpInputSourceConfig.DEFAULT_ALLOWED_PROTOCOLS)
));
}
@Test
public void testHappyPath()
{
ResolvedTable table = baseBuilder.copy()
.property(HttpTableDefn.USER_PROPERTY, "bob")
.property(HttpTableDefn.PASSWORD_PROPERTY, "secret")
.property(HttpTableDefn.URIS_PROPERTY, Collections.singletonList("http://foo.com/my.csv"))
.buildResolved(mapper);
// Check validation
table.validate();
// Check registry
TableDefnRegistry registry = new TableDefnRegistry(mapper);
assertNotNull(registry.resolve(table.spec()));
// Convert to an external spec
ExternalTableSpec externSpec = tableDefn.convertToExtern(table);
HttpInputSource sourceSpec = (HttpInputSource) externSpec.inputSource;
assertEquals("bob", sourceSpec.getHttpAuthenticationUsername());
assertEquals("secret", ((DefaultPasswordProvider) sourceSpec.getHttpAuthenticationPasswordProvider()).getPassword());
assertEquals("http://foo.com/my.csv", sourceSpec.getUris().get(0).toString());
// Just a sanity check: details of CSV conversion are tested elsewhere.
CsvInputFormat csvFormat = (CsvInputFormat) externSpec.inputFormat;
assertEquals(Arrays.asList("x", "y"), csvFormat.getColumns());
RowSignature sig = externSpec.signature;
assertEquals(Arrays.asList("x", "y"), sig.getColumnNames());
assertEquals(ColumnType.STRING, sig.getColumnType(0).get());
assertEquals(ColumnType.LONG, sig.getColumnType(1).get());
}
@Test
public void testEnvPassword()
{
ResolvedTable table = baseBuilder.copy()
.property(HttpTableDefn.USER_PROPERTY, "bob")
.property(HttpTableDefn.PASSWORD_ENV_VAR_PROPERTY, "SECRET")
.property(HttpTableDefn.URIS_PROPERTY, Collections.singletonList("http://foo.com/my.csv"))
.buildResolved(mapper);
// Check validation
table.validate();
// Convert to an external spec
ExternalTableSpec externSpec = tableDefn.convertToExtern(table);
HttpInputSource sourceSpec = (HttpInputSource) externSpec.inputSource;
assertEquals("bob", sourceSpec.getHttpAuthenticationUsername());
assertEquals("SECRET", ((EnvironmentVariablePasswordProvider) sourceSpec.getHttpAuthenticationPasswordProvider()).getVariable());
}
@Test
public void testParameters()
{
ResolvedTable table = baseBuilder.copy()
.property(HttpTableDefn.USER_PROPERTY, "bob")
.property(HttpTableDefn.PASSWORD_ENV_VAR_PROPERTY, "SECRET")
.property(HttpTableDefn.URI_TEMPLATE_PROPERTY, "http://foo.com/{}")
.buildResolved(mapper);
// Check validation
table.validate();
// Parameters
ParameterizedDefn parameterizedTable = tableDefn;
assertEquals(1, parameterizedTable.parameters().size());
assertNotNull(findProperty(parameterizedTable.parameters(), HttpTableDefn.URIS_PROPERTY));
// Apply parameters
Map<String, Object> params = ImmutableMap.of(
HttpTableDefn.URIS_PROPERTY, "foo.csv,bar.csv"
);
// Convert to an external spec
ExternalTableSpec externSpec = parameterizedTable.applyParameters(table, params);
HttpInputSource sourceSpec = (HttpInputSource) externSpec.inputSource;
assertEquals("bob", sourceSpec.getHttpAuthenticationUsername());
assertEquals("SECRET", ((EnvironmentVariablePasswordProvider) sourceSpec.getHttpAuthenticationPasswordProvider()).getVariable());
assertEquals(
HttpTableDefn.convertUriList(Arrays.asList("http://foo.com/foo.csv", "http://foo.com/bar.csv")),
sourceSpec.getUris()
);
}
@Test
public void testNoTemplate()
{
ResolvedTable table = baseBuilder.copy()
.property(HttpTableDefn.URIS_PROPERTY, Collections.singletonList("http://foo.com/my.csv"))
.buildResolved(mapper);
// Check validation
table.validate();
// Apply parameters
Map<String, Object> params = ImmutableMap.of(
HttpTableDefn.URIS_PROPERTY, "foo.csv,bar.csv"
);
// Convert to an external spec
assertThrows(IAE.class, () -> tableDefn.applyParameters(table, params));
}
@Test
public void testNoParameters()
{
ResolvedTable table = baseBuilder.copy()
.property(HttpTableDefn.URI_TEMPLATE_PROPERTY, "http://foo.com/{}")
.buildResolved(mapper);
Map<String, Object> params = ImmutableMap.of();
assertThrows(IAE.class, () -> tableDefn.applyParameters(table, params));
}
@Test
public void testInvalidParameters()
{
// The URI parser is forgiving about items in the path, so
// screw up the head, where URI is particular.
ResolvedTable table = baseBuilder.copy()
.property(HttpTableDefn.URI_TEMPLATE_PROPERTY, "//foo.com/{}")
.buildResolved(mapper);
Map<String, Object> params = ImmutableMap.of(
HttpTableDefn.URIS_PROPERTY, "foo.csv"
);
assertThrows(IAE.class, () -> tableDefn.applyParameters(table, params));
}
@Test
public void testInvalidURI()
{
// The URI parser is forgiving about items in the path, so
// screw up the head, where URI is particular.
ResolvedTable table = baseBuilder.copy()
.property(HttpTableDefn.URIS_PROPERTY, Collections.singletonList("//foo.com"))
.buildResolved(mapper);
assertThrows(IAE.class, () -> table.validate());
}
@Test
public void testSqlFunction()
{
List<PropertyDefn<?>> params = tableDefn.tableFunctionParameters();
// Ensure the relevant properties are available as SQL function parameters
PropertyDefn<?> userProp = findProperty(params, HttpTableDefn.USER_PROPERTY);
assertNotNull(userProp);
assertEquals(String.class, PropertyAttributes.sqlParameterType(userProp));
PropertyDefn<?> pwdProp = findProperty(params, HttpTableDefn.PASSWORD_PROPERTY);
assertNotNull(pwdProp);
assertEquals(String.class, PropertyAttributes.sqlParameterType(pwdProp));
PropertyDefn<?> urisProp = findProperty(params, HttpTableDefn.URIS_PROPERTY);
assertNotNull(urisProp);
assertEquals(String.class, PropertyAttributes.sqlParameterType(urisProp));
assertNull(findProperty(params, HttpTableDefn.URI_TEMPLATE_PROPERTY));
PropertyDefn<?> formatProp = findProperty(params, FormattedExternalTableDefn.FORMAT_PROPERTY);
assertNotNull(formatProp);
assertEquals(String.class, PropertyAttributes.sqlParameterType(formatProp));
// Pretend to accept values for the SQL parameters.
final ResolvedTable table = TableBuilder.of(tableDefn)
.property(userProp.name(), userProp.decodeSqlValue("bob", mapper))
.property(pwdProp.name(), pwdProp.decodeSqlValue("secret", mapper))
.property(urisProp.name(), urisProp.decodeSqlValue("http://foo.com/foo.csv, http://foo.com/bar.csv", mapper))
.property(formatProp.name(), formatProp.decodeSqlValue(InputFormats.CSV_FORMAT_TYPE, mapper))
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.buildResolved(mapper);
ExternalTableSpec externSpec = tableDefn.convertToExtern(table);
HttpInputSource sourceSpec = (HttpInputSource) externSpec.inputSource;
assertEquals("bob", sourceSpec.getHttpAuthenticationUsername());
assertEquals("secret", ((DefaultPasswordProvider) sourceSpec.getHttpAuthenticationPasswordProvider()).getPassword());
assertEquals(
HttpTableDefn.convertUriList(Arrays.asList("http://foo.com/foo.csv", "http://foo.com/bar.csv")),
sourceSpec.getUris()
);
}
}

View File

@ -0,0 +1,217 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.table.InputFormats.CsvFormatDefn;
import org.apache.druid.catalog.model.table.InputFormats.FlatTextFormatDefn;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.InlineInputSource;
import org.apache.druid.java.util.common.IAE;
import org.junit.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
public class InlineInputSourceDefnTest extends BaseExternTableTest
{
@Test
public void testValidateEmptyInputSource()
{
// No data property: not valid
TableMetadata table = TableBuilder.external("foo")
.inputSource(ImmutableMap.of("type", InlineInputSource.TYPE_KEY))
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testValidateNoFormat()
{
// No format: not valid. For inline, format must be provided to match data
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(new InlineInputSource("a\n")))
.column("x", Columns.VARCHAR)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testValidateNoColumns()
{
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(new InlineInputSource("a\n")))
.inputFormat(CSV_FORMAT)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testValidateGood()
{
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(new InlineInputSource("a\n")))
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
}
@Test
public void testFullTableFnBasics()
{
InputSourceDefn defn = registry.inputSourceDefnFor(InlineInputSourceDefn.TYPE_KEY);
TableFunction fn = defn.adHocTableFn();
assertNotNull(fn);
assertTrue(hasParam(fn, InlineInputSourceDefn.DATA_PROPERTY));
assertTrue(hasParam(fn, FormattedInputSourceDefn.FORMAT_PARAMETER));
assertTrue(hasParam(fn, FlatTextFormatDefn.LIST_DELIMITER_PARAMETER));
}
@Test
public void testMissingArgs()
{
InputSourceDefn defn = registry.inputSourceDefnFor(InlineInputSourceDefn.TYPE_KEY);
TableFunction fn = defn.adHocTableFn();
assertThrows(IAE.class, () -> fn.apply("x", new HashMap<>(), Collections.emptyList(), mapper));
}
@Test
public void testMissingFormat()
{
InputSourceDefn defn = registry.inputSourceDefnFor(InlineInputSourceDefn.TYPE_KEY);
TableFunction fn = defn.adHocTableFn();
Map<String, Object> args = new HashMap<>();
args.put(InlineInputSourceDefn.DATA_PROPERTY, "a");
assertThrows(IAE.class, () -> fn.apply("x", args, Collections.emptyList(), mapper));
}
@Test
public void testValidAdHocFn()
{
// Simulate the information obtained from an SQL table function
final InputSourceDefn defn = registry.inputSourceDefnFor(InlineInputSourceDefn.TYPE_KEY);
final Map<String, Object> args = new HashMap<>();
args.put(InlineInputSourceDefn.DATA_PROPERTY, Arrays.asList("a,b", "c,d"));
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
final List<ColumnSpec> columns = Arrays.asList(
new ColumnSpec("a", Columns.VARCHAR, null),
new ColumnSpec("b", Columns.VARCHAR, null)
);
final TableFunction fn = defn.adHocTableFn();
ExternalTableSpec extern = fn.apply("x", args, columns, mapper);
assertTrue(extern.inputSource instanceof InlineInputSource);
InlineInputSource inputSource = (InlineInputSource) extern.inputSource;
assertEquals("a,b\nc,d\n", inputSource.getData());
assertTrue(extern.inputFormat instanceof CsvInputFormat);
CsvInputFormat format = (CsvInputFormat) extern.inputFormat;
assertEquals(Arrays.asList("a", "b"), format.getColumns());
assertEquals(2, extern.signature.size());
// Fails if no columns are provided.
assertThrows(IAE.class, () -> fn.apply("x", new HashMap<>(), Collections.emptyList(), mapper));
}
@Test
public void testPartialTable()
{
// Define an inline table
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(new InlineInputSource("a,b\nc,d\n")))
.inputFormat(CSV_FORMAT)
.column("a", Columns.VARCHAR)
.column("b", Columns.VARCHAR)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
// Get the partial table function
TableFunction fn = ((ExternalTableDefn) resolved.defn()).tableFn(resolved);
// Inline is always fully defined: no arguments needed
assertTrue(fn.parameters().isEmpty());
// Verify the conversion
ExternalTableSpec extern = fn.apply("x", new HashMap<>(), Collections.emptyList(), mapper);
assertTrue(extern.inputSource instanceof InlineInputSource);
InlineInputSource inputSource = (InlineInputSource) extern.inputSource;
assertEquals("a,b\nc,d\n", inputSource.getData());
assertTrue(extern.inputFormat instanceof CsvInputFormat);
CsvInputFormat actualFormat = (CsvInputFormat) extern.inputFormat;
assertEquals(Arrays.asList("a", "b"), actualFormat.getColumns());
assertEquals(2, extern.signature.size());
// Cannot supply columns with the function
List<ColumnSpec> columns = Arrays.asList(
new ColumnSpec("a", Columns.VARCHAR, null),
new ColumnSpec("b", Columns.VARCHAR, null)
);
assertThrows(IAE.class, () -> fn.apply("x", new HashMap<>(), columns, mapper));
}
@Test
public void testDefinedTable()
{
// Define an inline table
CsvInputFormat format = new CsvInputFormat(
Collections.singletonList("a"), ";", false, false, 0);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(new InlineInputSource("a,b\nc,d")))
.inputFormat(formatToMap(format))
.column("a", Columns.VARCHAR)
.column("b", Columns.VARCHAR)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
// Inline is always fully defined: can directly convert to a table
ExternalTableSpec extern = ((ExternalTableDefn) resolved.defn()).convert(resolved);
// Verify the conversion
assertTrue(extern.inputSource instanceof InlineInputSource);
InlineInputSource inputSource = (InlineInputSource) extern.inputSource;
assertEquals("a,b\nc,d\n", inputSource.getData());
assertTrue(extern.inputFormat instanceof CsvInputFormat);
CsvInputFormat actualFormat = (CsvInputFormat) extern.inputFormat;
assertEquals(Arrays.asList("a", "b"), actualFormat.getColumns());
assertEquals(2, extern.signature.size());
}
}

View File

@ -1,90 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.catalog.CatalogTest;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableDefnRegistry;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.InlineInputSource;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.util.Arrays;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThrows;
@Category(CatalogTest.class)
public class InlineTableTest
{
private final ObjectMapper mapper = new ObjectMapper();
private final InlineTableDefn tableDefn = new InlineTableDefn();
private final TableBuilder baseBuilder = TableBuilder.of(tableDefn)
.description("inline input")
.format(InputFormats.CSV_FORMAT_TYPE)
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT);
@Test
public void testEmptyData()
{
ResolvedTable table = baseBuilder.buildResolved(mapper);
// Check validation
assertThrows(IAE.class, () -> table.validate());
}
@Test
public void testValidData()
{
ResolvedTable table = baseBuilder.copy()
.data("a,b", "c,d")
.buildResolved(mapper);
// Check validation
table.validate();
// Check registry
TableDefnRegistry registry = new TableDefnRegistry(mapper);
assertNotNull(registry.resolve(table.spec()));
// Convert to an external spec
ExternalTableSpec externSpec = tableDefn.convertToExtern(table);
InlineInputSource inlineSpec = (InlineInputSource) externSpec.inputSource;
assertEquals("a,b\nc,d\n", inlineSpec.getData());
// Just a sanity check: details of CSV conversion are tested elsewhere.
CsvInputFormat csvFormat = (CsvInputFormat) externSpec.inputFormat;
assertEquals(Arrays.asList("x", "y"), csvFormat.getColumns());
RowSignature sig = externSpec.signature;
assertEquals(Arrays.asList("x", "y"), sig.getColumnNames());
assertEquals(ColumnType.STRING, sig.getColumnType(0).get());
assertEquals(ColumnType.LONG, sig.getColumnType(1).get());
}
}

View File

@ -1,284 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.catalog.CatalogTest;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableSpec;
import org.apache.druid.catalog.model.table.InputFormats.GenericFormatDefn;
import org.apache.druid.catalog.model.table.InputFormats.InputFormatDefn;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.DelimitedInputFormat;
import org.apache.druid.data.input.impl.JsonInputFormat;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThrows;
@Category(CatalogTest.class)
public class InputFormatTest
{
private final ObjectMapper mapper = new ObjectMapper();
@Test
public void testCsvFormat()
{
InputFormatDefn converter = InputFormats.CSV_FORMAT_DEFN;
List<ColumnSpec> cols = Arrays.asList(
new ColumnSpec("type", "x", Columns.VARCHAR, null),
new ColumnSpec("type", "y", Columns.BIGINT, null)
);
// Basic case
{
Map<String, Object> args = ImmutableMap.of(
"listDelimiter", "|", "skipRows", 1
);
TableSpec spec = new TableSpec("type", args, cols);
ResolvedTable table = new ResolvedTable(null, spec, mapper);
CsvInputFormat expectedFormat = new CsvInputFormat(
Arrays.asList("x", "y"),
"|",
false,
false,
1
);
InputFormat inputFormat = converter.convert(table);
assertEquals(expectedFormat, inputFormat);
}
// Minimal case. (However, though skipRows is required, JSON will handle
// a null value and set the value to 0.)
{
Map<String, Object> args = ImmutableMap.of();
TableSpec spec = new TableSpec("type", args, cols);
ResolvedTable table = new ResolvedTable(null, spec, mapper);
CsvInputFormat expectedFormat = new CsvInputFormat(
Arrays.asList("x", "y"),
null,
false,
false,
0
);
InputFormat inputFormat = converter.convert(table);
assertEquals(expectedFormat, inputFormat);
}
// Invalid format
{
Map<String, Object> args = ImmutableMap.of(
"skipRows", "bogus"
);
TableSpec spec = new TableSpec("type", args, cols);
ResolvedTable table = new ResolvedTable(null, spec, mapper);
assertThrows(Exception.class, () -> converter.convert(table));
}
// No columns
{
Map<String, Object> args = ImmutableMap.of(
"skipRows", 1
);
TableSpec spec = new TableSpec("type", args, null);
ResolvedTable table = new ResolvedTable(null, spec, mapper);
assertThrows(Exception.class, () -> converter.convert(table));
}
}
@Test
public void testDelimitedTextFormat()
{
InputFormatDefn converter = InputFormats.DELIMITED_FORMAT_DEFN;
List<ColumnSpec> cols = Arrays.asList(
new ColumnSpec("type", "x", Columns.VARCHAR, null),
new ColumnSpec("type", "y", Columns.BIGINT, null)
);
Map<String, Object> args = ImmutableMap.of(
"delimiter", ",", "listDelimiter", "|", "skipRows", 1
);
TableSpec spec = new TableSpec("type", args, cols);
ResolvedTable table = new ResolvedTable(null, spec, mapper);
DelimitedInputFormat expectedFormat = new DelimitedInputFormat(
Arrays.asList("x", "y"),
"|",
",",
false,
false,
1
);
InputFormat inputFormat = converter.convert(table);
assertEquals(expectedFormat, inputFormat);
}
@Test
public void testJsonFormat()
{
InputFormatDefn converter = InputFormats.JSON_FORMAT_DEFN;
List<ColumnSpec> cols = Arrays.asList(
new ColumnSpec("type", "x", Columns.VARCHAR, null),
new ColumnSpec("type", "y", Columns.BIGINT, null)
);
// The one supported property at present.
{
Map<String, Object> args = ImmutableMap.of(
"keepNulls", true
);
TableSpec spec = new TableSpec("type", args, cols);
ResolvedTable table = new ResolvedTable(null, spec, mapper);
InputFormat inputFormat = converter.convert(table);
assertEquals(new JsonInputFormat(null, null, true, null, null), inputFormat);
}
// Empty
{
TableSpec spec = new TableSpec("type", null, cols);
ResolvedTable table = new ResolvedTable(null, spec, mapper);
InputFormat inputFormat = converter.convert(table);
assertEquals(new JsonInputFormat(null, null, null, null, null), inputFormat);
}
}
/**
* Test the generic format which allows a literal input spec. The
* drawback is that the user must repeat the columns.
*/
@Test
public void testGenericFormat()
{
InputFormatDefn converter = InputFormats.GENERIC_FORMAT_DEFN;
List<ColumnSpec> cols = Arrays.asList(
new ColumnSpec("type", "x", Columns.VARCHAR, null),
new ColumnSpec("type", "y", Columns.BIGINT, null)
);
// No type
{
Map<String, Object> args = ImmutableMap.of(
"skipRows", 1
);
TableSpec spec = new TableSpec("type", args, null);
ResolvedTable table = new ResolvedTable(null, spec, mapper);
assertThrows(Exception.class, () -> converter.convert(table));
}
// CSV
{
Map<String, Object> args = ImmutableMap.of(
GenericFormatDefn.INPUT_FORMAT_SPEC_PROPERTY,
ImmutableMap.of(
"type", CsvInputFormat.TYPE_KEY,
"listDelimiter", "|",
"skipHeaderRows", 1,
"findColumnsFromHeader", false,
"columns", Arrays.asList("x", "y")
)
);
TableSpec spec = new TableSpec("type", args, cols);
ResolvedTable table = new ResolvedTable(null, spec, mapper);
CsvInputFormat expectedFormat = new CsvInputFormat(
Arrays.asList("x", "y"),
"|",
false,
false,
1
);
InputFormat inputFormat = converter.convert(table);
assertEquals(expectedFormat, inputFormat);
}
// No columns: when using generic, the columns must be in the
// JSON spec.
{
Map<String, Object> args = ImmutableMap.of(
"type", CsvInputFormat.TYPE_KEY,
"skipRows", 1
);
TableSpec spec = new TableSpec("type", args, cols);
ResolvedTable table = new ResolvedTable(null, spec, mapper);
assertThrows(Exception.class, () -> converter.convert(table));
}
// Delimited text
{
Map<String, Object> args = ImmutableMap.of(
GenericFormatDefn.INPUT_FORMAT_SPEC_PROPERTY,
ImmutableMap.builder()
.put("type", DelimitedInputFormat.TYPE_KEY)
.put("delimiter", ",")
.put("listDelimiter", "|")
.put("skipHeaderRows", 1)
.put("findColumnsFromHeader", false)
.put("columns", Arrays.asList("x", "y"))
.build()
);
TableSpec spec = new TableSpec("type", args, cols);
ResolvedTable table = new ResolvedTable(null, spec, mapper);
DelimitedInputFormat expectedFormat = new DelimitedInputFormat(
Arrays.asList("x", "y"),
"|",
",",
false,
false,
1
);
InputFormat inputFormat = converter.convert(table);
assertEquals(expectedFormat, inputFormat);
}
// JSON
{
Map<String, Object> args = ImmutableMap.of(
GenericFormatDefn.INPUT_FORMAT_SPEC_PROPERTY,
ImmutableMap.of(
"type", JsonInputFormat.TYPE_KEY,
"keepNullColumns", true
)
);
TableSpec spec = new TableSpec("type", args, cols);
ResolvedTable table = new ResolvedTable(null, spec, mapper);
InputFormat inputFormat = converter.convert(table);
assertEquals(new JsonInputFormat(null, null, true, null, null), inputFormat);
}
}
}

View File

@ -0,0 +1,111 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.table.TableFunction.ParameterDefn;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.impl.InlineInputSource;
import org.apache.druid.data.input.impl.JsonInputFormat;
import org.junit.Test;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
public class JsonInputFormatTest extends BaseExternTableTest
{
@Test
public void testDefaults()
{
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(new InlineInputSource("a\n")))
.inputFormat(ImmutableMap.of("type", JsonInputFormat.TYPE_KEY))
.column("a", Columns.VARCHAR)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
InputFormatDefn defn = registry.inputFormatDefnFor(JsonInputFormat.TYPE_KEY);
InputFormat inputFormat = defn.convertFromTable(new ResolvedExternalTable(resolved));
JsonInputFormat jsonFormat = (JsonInputFormat) inputFormat;
assertNull(jsonFormat.getFlattenSpec());
assertTrue(jsonFormat.getFeatureSpec().isEmpty());
assertFalse(jsonFormat.isKeepNullColumns());
assertFalse(jsonFormat.isAssumeNewlineDelimited());
assertFalse(jsonFormat.isUseJsonNodeReader());
}
@Test
public void testConversion()
{
JsonInputFormat format = new JsonInputFormat(null, null, true, true, false);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(new InlineInputSource("a\n")))
.inputFormat(formatToMap(format))
.column("a", Columns.VARCHAR)
.column("b", Columns.BIGINT)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
InputFormatDefn defn = registry.inputFormatDefnFor(JsonInputFormat.TYPE_KEY);
InputFormat inputFormat = defn.convertFromTable(new ResolvedExternalTable(resolved));
JsonInputFormat jsonFormat = (JsonInputFormat) inputFormat;
assertNull(jsonFormat.getFlattenSpec());
assertTrue(jsonFormat.getFeatureSpec().isEmpty());
assertTrue(jsonFormat.isKeepNullColumns());
assertTrue(jsonFormat.isAssumeNewlineDelimited());
assertFalse(jsonFormat.isUseJsonNodeReader());
}
@Test
public void testFunctionParams()
{
InputFormatDefn defn = registry.inputFormatDefnFor(JsonInputFormat.TYPE_KEY);
List<ParameterDefn> params = defn.parameters();
assertEquals(0, params.size());
}
@Test
public void testCreateFromArgs()
{
Map<String, Object> args = new HashMap<>();
InputFormatDefn defn = registry.inputFormatDefnFor(JsonInputFormat.TYPE_KEY);
List<ColumnSpec> columns = Collections.singletonList(new ColumnSpec("a", null, null));
InputFormat inputFormat = defn.convertFromArgs(args, columns, mapper);
JsonInputFormat jsonFormat = (JsonInputFormat) inputFormat;
assertNull(jsonFormat.getFlattenSpec());
assertTrue(jsonFormat.getFeatureSpec().isEmpty());
assertFalse(jsonFormat.isKeepNullColumns());
assertFalse(jsonFormat.isAssumeNewlineDelimited());
assertFalse(jsonFormat.isUseJsonNodeReader());
}
}

View File

@ -0,0 +1,540 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableDefnRegistry;
import org.apache.druid.catalog.model.TableMetadata;
import org.apache.druid.catalog.model.table.InputFormats.CsvFormatDefn;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.junit.Test;
import java.io.File;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
public class LocalInputSourceDefnTest extends BaseExternTableTest
{
private static final Map<String, Object> BASE_DIR_ONLY =
ImmutableMap.of("type", LocalInputSource.TYPE_KEY, "baseDir", "/tmp");
private final InputSourceDefn localDefn = registry.inputSourceDefnFor(LocalInputSourceDefn.TYPE_KEY);
@Test
public void testValidateEmptyInputSource()
{
// No data property: not valid
TableMetadata table = TableBuilder.external("foo")
.inputSource(ImmutableMap.of("type", LocalInputSource.TYPE_KEY))
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testValidateNoFormat()
{
// No format: Not valid. If columns are given, a format is required.
LocalInputSource inputSource = new LocalInputSource(new File("/tmp"), "*");
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(inputSource))
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testValidateNoColumns()
{
// No columns: Not valid. If a format is given, then columns required.
LocalInputSource inputSource = new LocalInputSource(new File("/tmp"), "*");
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(inputSource))
.inputFormat(CSV_FORMAT)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testValidateConnection()
{
// Valid if neither columns nor format are provided. This is a "connection"
// to some local directory.
LocalInputSource inputSource = new LocalInputSource(new File("/tmp"), "*");
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(inputSource))
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
}
@Test
public void testValidateConnectionNoFilter()
{
// Valid to provide an input source without a filter: we expect the filter
// to be provided later.
TableMetadata table = TableBuilder.external("foo")
.inputSource(BASE_DIR_ONLY)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
}
@Test
public void testValidateBaseDirWithFormat()
{
// Valid if neither columns nor format are provided. This is a "connection"
// to some local directory.
LocalInputSource inputSource = new LocalInputSource(new File("/tmp"), "*");
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(inputSource))
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
}
@Test
public void testValidateFilesWithFormat()
{
// Valid if neither columns nor format are provided. This is a "connection"
// to some local directory.
LocalInputSource inputSource = new LocalInputSource(
null,
null,
Collections.singletonList(new File("/tmp/myFile.csv"))
);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(inputSource))
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
resolved.validate();
}
@Test
public void testBaseDirAndFiles()
{
Map<String, Object> source = ImmutableMap.of(
InputSource.TYPE_PROPERTY, LocalInputSource.TYPE_KEY,
LocalInputSourceDefn.BASE_DIR_FIELD, "/tmp",
LocalInputSourceDefn.FILTER_FIELD, "*.csv",
LocalInputSourceDefn.FILES_FIELD, Collections.singletonList("foo.csv")
);
TableMetadata table = TableBuilder.external("foo")
.inputSource(source)
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
ResolvedTable resolved = registry.resolve(table.spec());
assertThrows(IAE.class, () -> resolved.validate());
}
@Test
public void testAdHocParameters()
{
TableFunction fn = localDefn.adHocTableFn();
assertTrue(hasParam(fn, LocalInputSourceDefn.BASE_DIR_PARAMETER));
assertTrue(hasParam(fn, LocalInputSourceDefn.FILES_PARAMETER));
assertTrue(hasParam(fn, LocalInputSourceDefn.FILTER_PARAMETER));
assertTrue(hasParam(fn, FormattedInputSourceDefn.FORMAT_PARAMETER));
}
@Test
public void testAdHocBaseDir()
{
TableFunction fn = localDefn.adHocTableFn();
Map<String, Object> args = new HashMap<>();
args.put(LocalInputSourceDefn.BASE_DIR_PARAMETER, "/tmp");
args.put(LocalInputSourceDefn.FILTER_PARAMETER, "*.csv");
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource;
assertEquals(new File("/tmp"), sourceSpec.getBaseDir());
assertEquals("*.csv", sourceSpec.getFilter());
assertTrue(sourceSpec.getFiles().isEmpty());
validateFormat(externSpec);
// But, it fails if there are no columns.
assertThrows(IAE.class, () -> fn.apply("x", args, Collections.emptyList(), mapper));
}
@Test
public void testAdHocBaseDirOnly()
{
TableFunction fn = localDefn.adHocTableFn();
Map<String, Object> args = new HashMap<>();
args.put(LocalInputSourceDefn.BASE_DIR_PARAMETER, "/tmp");
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource;
assertEquals(new File("/tmp"), sourceSpec.getBaseDir());
assertEquals("*", sourceSpec.getFilter());
assertTrue(sourceSpec.getFiles().isEmpty());
validateFormat(externSpec);
// But, it fails if there are no columns.
assertThrows(IAE.class, () -> fn.apply("x", args, Collections.emptyList(), mapper));
}
@Test
public void testAdHocFiles()
{
TableFunction fn = localDefn.adHocTableFn();
Map<String, Object> args = new HashMap<>();
args.put(LocalInputSourceDefn.FILES_PARAMETER, Arrays.asList("/tmp/foo.csv", "/tmp/bar.csv"));
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource;
assertNull(sourceSpec.getBaseDir());
assertNull(sourceSpec.getFilter());
assertEquals(
Arrays.asList(new File("/tmp/foo.csv"), new File("/tmp/bar.csv")),
sourceSpec.getFiles()
);
validateFormat(externSpec);
// But, it fails if there are no columns.
assertThrows(IAE.class, () -> fn.apply("x", args, Collections.emptyList(), mapper));
}
@Test
public void testAdHocBaseDirAndFiles()
{
TableFunction fn = localDefn.adHocTableFn();
Map<String, Object> args = new HashMap<>();
args.put(LocalInputSourceDefn.BASE_DIR_PARAMETER, "/tmp");
args.put(LocalInputSourceDefn.FILES_PARAMETER, Arrays.asList("foo.csv", "bar.csv"));
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource;
assertNull(sourceSpec.getBaseDir());
assertNull(sourceSpec.getFilter());
assertEquals(
Arrays.asList(new File("/tmp/foo.csv"), new File("/tmp/bar.csv")),
sourceSpec.getFiles()
);
validateFormat(externSpec);
// But, it fails if there are no columns.
assertThrows(IAE.class, () -> fn.apply("x", args, Collections.emptyList(), mapper));
}
@Test
public void testAdHocErrors()
{
TableFunction fn = localDefn.adHocTableFn();
{
// Empty arguments: not valid
Map<String, Object> args = new HashMap<>();
assertThrows(IAE.class, () -> fn.apply("x", args, COLUMNS, mapper));
}
{
// Base dir without filter: not valid.
Map<String, Object> args = new HashMap<>();
args.put(LocalInputSourceDefn.BASE_DIR_PARAMETER, "/tmp");
assertThrows(IAE.class, () -> fn.apply("x", args, COLUMNS, mapper));
}
{
// Filter without base dir: not valid
Map<String, Object> args = new HashMap<>();
args.put(LocalInputSourceDefn.FILTER_PARAMETER, "*.csv");
assertThrows(IAE.class, () -> fn.apply("x", args, COLUMNS, mapper));
}
{
// Cannot provide both a filter and a list of files.
Map<String, Object> args = new HashMap<>();
args.put(LocalInputSourceDefn.BASE_DIR_PARAMETER, "/tmp");
args.put(LocalInputSourceDefn.FILES_PARAMETER, "/tmp/foo.csv, /tmp/bar.csv");
args.put(LocalInputSourceDefn.FILTER_PARAMETER, "*.csv");
assertThrows(IAE.class, () -> fn.apply("x", args, COLUMNS, mapper));
}
}
@Test
public void testFullyDefinedBaseDirAndPattern()
{
LocalInputSource inputSource = new LocalInputSource(
new File("/tmp"),
"*.csv",
null
);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(inputSource))
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
// Check validation
table.validate();
// Check registry
TableDefnRegistry registry = new TableDefnRegistry(mapper);
assertNotNull(registry.resolve(table.spec()));
// Convert to an external spec
ResolvedTable resolved = registry.resolve(table.spec());
ExternalTableDefn externDefn = (ExternalTableDefn) resolved.defn();
ExternalTableSpec externSpec = externDefn.convert(resolved);
LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource;
assertEquals("/tmp", sourceSpec.getBaseDir().toString());
assertEquals("*.csv", sourceSpec.getFilter());
assertTrue(sourceSpec.getFiles().isEmpty());
validateFormat(externSpec);
// Get the partial table function. Since table is fully defined,
// no parameters available.
TableFunction fn = externDefn.tableFn(resolved);
assertEquals(0, fn.parameters().size());
// Apply the function with no arguments and no columns (since columns are already defined.)
externSpec = fn.apply("x", Collections.emptyMap(), Collections.emptyList(), mapper);
sourceSpec = (LocalInputSource) externSpec.inputSource;
assertEquals("/tmp", sourceSpec.getBaseDir().toString());
assertEquals("*.csv", sourceSpec.getFilter());
assertTrue(sourceSpec.getFiles().isEmpty());
validateFormat(externSpec);
// Fails if columns are provided.
assertThrows(IAE.class, () -> fn.apply("x", Collections.emptyMap(), COLUMNS, mapper));
}
@Test
public void testFullyDefinedFiles()
{
List<File> files = Collections.singletonList(new File("/tmp/my.csv"));
LocalInputSource inputSource = new LocalInputSource(
null,
null,
files
);
TableMetadata table = TableBuilder.external("foo")
.inputSource(toMap(inputSource))
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
// Check validation
table.validate();
// Check registry
TableDefnRegistry registry = new TableDefnRegistry(mapper);
assertNotNull(registry.resolve(table.spec()));
// Convert to an external spec
ResolvedTable resolved = registry.resolve(table.spec());
ExternalTableDefn externDefn = (ExternalTableDefn) resolved.defn();
ExternalTableSpec externSpec = externDefn.convert(resolved);
LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource;
assertNull(sourceSpec.getBaseDir());
assertNull(sourceSpec.getFilter());
assertEquals(files, sourceSpec.getFiles());
validateFormat(externSpec);
// Get the partial table function. Since table is fully defined,
// no parameters available.
TableFunction fn = externDefn.tableFn(resolved);
assertEquals(0, fn.parameters().size());
// Apply the function with no arguments and no columns (since columns are already defined.)
externSpec = fn.apply("x", Collections.emptyMap(), Collections.emptyList(), mapper);
sourceSpec = (LocalInputSource) externSpec.inputSource;
assertNull(sourceSpec.getBaseDir());
assertNull(sourceSpec.getFilter());
assertEquals(files, sourceSpec.getFiles());
validateFormat(externSpec);
// Fails if columns are provided.
assertThrows(IAE.class, () -> fn.apply("x", Collections.emptyMap(), COLUMNS, mapper));
}
@Test
public void testBaseDirAndFormat()
{
TableMetadata table = TableBuilder.external("foo")
.inputSource(BASE_DIR_ONLY)
.inputFormat(CSV_FORMAT)
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.build();
// Check validation
table.validate();
// Convert to an external spec. Fails because the table is partial.
TableDefnRegistry registry = new TableDefnRegistry(mapper);
ResolvedTable resolved = registry.resolve(table.spec());
ExternalTableDefn externDefn = (ExternalTableDefn) resolved.defn();
assertThrows(IAE.class, () -> externDefn.convert(resolved));
// Get the partial table function.
TableFunction fn = externDefn.tableFn(resolved);
assertTrue(hasParam(fn, LocalInputSourceDefn.FILES_PARAMETER));
assertTrue(hasParam(fn, LocalInputSourceDefn.FILTER_PARAMETER));
assertFalse(hasParam(fn, FormattedInputSourceDefn.FORMAT_PARAMETER));
// Must provide an additional parameter.
assertThrows(IAE.class, () -> fn.apply("x", Collections.emptyMap(), Collections.emptyList(), mapper));
{
// Create a table with a file pattern.
Map<String, Object> args = new HashMap<>();
args.put(LocalInputSourceDefn.FILTER_PARAMETER, "*.csv");
// Apply the function with no arguments and no columns (since columns are already defined.)
ExternalTableSpec externSpec = fn.apply("x", args, Collections.emptyList(), mapper);
LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource;
assertEquals("/tmp", sourceSpec.getBaseDir().toString());
assertEquals("*.csv", sourceSpec.getFilter());
validateFormat(externSpec);
}
{
// Create a table with a file list.
Map<String, Object> args = new HashMap<>();
args.put(LocalInputSourceDefn.FILES_PARAMETER, Arrays.asList("foo.csv", "bar.csv"));
ExternalTableSpec externSpec = fn.apply("x", args, Collections.emptyList(), mapper);
LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource;
assertNull(sourceSpec.getBaseDir());
assertNull(sourceSpec.getFilter());
assertEquals(Arrays.asList(new File("/tmp/foo.csv"), new File("/tmp/bar.csv")), sourceSpec.getFiles());
validateFormat(externSpec);
// Fails if columns are provided.
assertThrows(IAE.class, () -> fn.apply("x", args, COLUMNS, mapper));
}
}
@Test
public void testBaseDirOnly()
{
TableMetadata table = TableBuilder.external("foo")
.inputSource(BASE_DIR_ONLY)
.build();
// Check validation
table.validate();
// Convert to an external spec. Fails because the table is partial.
TableDefnRegistry registry = new TableDefnRegistry(mapper);
ResolvedTable resolved = registry.resolve(table.spec());
ExternalTableDefn externDefn = (ExternalTableDefn) resolved.defn();
assertThrows(IAE.class, () -> externDefn.convert(resolved));
// Get the partial table function.
TableFunction fn = externDefn.tableFn(resolved);
assertTrue(hasParam(fn, LocalInputSourceDefn.FILES_PARAMETER));
assertTrue(hasParam(fn, LocalInputSourceDefn.FILTER_PARAMETER));
assertTrue(hasParam(fn, FormattedInputSourceDefn.FORMAT_PARAMETER));
// Must provide an additional parameter.
assertThrows(IAE.class, () -> fn.apply("x", Collections.emptyMap(), Collections.emptyList(), mapper));
{
// Create a table with a file pattern and format.
Map<String, Object> args = new HashMap<>();
args.put(LocalInputSourceDefn.FILTER_PARAMETER, "*.csv");
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
// Function fails without columns, since the table has none.
assertThrows(IAE.class, () -> fn.apply("x", args, Collections.emptyList(), mapper));
// Apply the function with no arguments and columns
ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource;
assertEquals("/tmp", sourceSpec.getBaseDir().toString());
assertEquals("*.csv", sourceSpec.getFilter());
validateFormat(externSpec);
}
{
// Create a table with a file list.
Map<String, Object> args = new HashMap<>();
args.put(LocalInputSourceDefn.FILES_PARAMETER, Arrays.asList("foo.csv", "bar.csv"));
args.put(FormattedInputSourceDefn.FORMAT_PARAMETER, CsvFormatDefn.TYPE_KEY);
// Function fails without columns, since the table has none.
assertThrows(IAE.class, () -> fn.apply("x", args, Collections.emptyList(), mapper));
// Provide format and columns.
ExternalTableSpec externSpec = fn.apply("x", args, COLUMNS, mapper);
LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource;
assertNull(sourceSpec.getBaseDir());
assertNull(sourceSpec.getFilter());
assertEquals(Arrays.asList(new File("/tmp/foo.csv"), new File("/tmp/bar.csv")), sourceSpec.getFiles());
validateFormat(externSpec);
}
}
private void validateFormat(ExternalTableSpec externSpec)
{
// Just a sanity check: details of CSV conversion are tested elsewhere.
CsvInputFormat csvFormat = (CsvInputFormat) externSpec.inputFormat;
assertEquals(Arrays.asList("x", "y"), csvFormat.getColumns());
RowSignature sig = externSpec.signature;
assertEquals(Arrays.asList("x", "y"), sig.getColumnNames());
assertEquals(ColumnType.STRING, sig.getColumnType(0).get());
assertEquals(ColumnType.LONG, sig.getColumnType(1).get());
}
}

View File

@ -1,239 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.catalog.model.table;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.catalog.CatalogTest;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.ModelProperties.PropertyDefn;
import org.apache.druid.catalog.model.ParameterizedDefn;
import org.apache.druid.catalog.model.PropertyAttributes;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.TableDefnRegistry;
import org.apache.druid.catalog.model.table.ExternalTableDefn.FormattedExternalTableDefn;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.io.File;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThrows;
import static org.junit.Assert.assertTrue;
@Category(CatalogTest.class)
public class LocalTableTest extends BaseExternTableTest
{
private final LocalTableDefn tableDefn = new LocalTableDefn();
private final TableBuilder baseBuilder = TableBuilder.of(tableDefn)
.description("local file input")
.format(InputFormats.CSV_FORMAT_TYPE)
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT);
@Test
public void testFullyDefined()
{
ResolvedTable table = baseBuilder.copy()
.property(LocalTableDefn.BASE_DIR_PROPERTY, "/tmp")
.property(LocalTableDefn.FILE_FILTER_PROPERTY, "*.csv")
.property(LocalTableDefn.FILES_PROPERTY, Collections.singletonList("my.csv"))
.buildResolved(mapper);
// Check validation
table.validate();
// Check registry
TableDefnRegistry registry = new TableDefnRegistry(mapper);
assertNotNull(registry.resolve(table.spec()));
// Convert to an external spec
ExternalTableSpec externSpec = tableDefn.convertToExtern(table);
LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource;
assertEquals("/tmp", sourceSpec.getBaseDir().toString());
assertEquals("*.csv", sourceSpec.getFilter());
assertEquals("my.csv", sourceSpec.getFiles().get(0).toString());
// Just a sanity check: details of CSV conversion are tested elsewhere.
CsvInputFormat csvFormat = (CsvInputFormat) externSpec.inputFormat;
assertEquals(Arrays.asList("x", "y"), csvFormat.getColumns());
RowSignature sig = externSpec.signature;
assertEquals(Arrays.asList("x", "y"), sig.getColumnNames());
assertEquals(ColumnType.STRING, sig.getColumnType(0).get());
assertEquals(ColumnType.LONG, sig.getColumnType(1).get());
}
@Test
public void testNoFilter()
{
ResolvedTable table = baseBuilder.copy()
.property(LocalTableDefn.BASE_DIR_PROPERTY, "/tmp")
.property(LocalTableDefn.FILES_PROPERTY, Collections.singletonList("my.csv"))
.buildResolved(mapper);
// Check validation
table.validate();
// Convert to an external spec
ExternalTableSpec externSpec = tableDefn.convertToExtern(table);
LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource;
assertEquals("/tmp", sourceSpec.getBaseDir().toString());
assertEquals("*", sourceSpec.getFilter());
assertEquals("my.csv", sourceSpec.getFiles().get(0).toString());
}
@Test
public void testNoFiles()
{
ResolvedTable table = baseBuilder.copy()
.property(LocalTableDefn.BASE_DIR_PROPERTY, "/tmp")
.property(LocalTableDefn.FILE_FILTER_PROPERTY, "*.csv")
.buildResolved(mapper);
// Check validation
table.validate();
// Convert to an external spec
ExternalTableSpec externSpec = tableDefn.convertToExtern(table);
LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource;
assertEquals("/tmp", sourceSpec.getBaseDir().toString());
assertEquals("*.csv", sourceSpec.getFilter());
assertTrue(sourceSpec.getFiles().isEmpty());
}
@Test
public void testNoFilesOrFlter()
{
ResolvedTable table = baseBuilder.copy()
.property(LocalTableDefn.BASE_DIR_PROPERTY, "/tmp")
.buildResolved(mapper);
// Check validation
assertThrows(IAE.class, () -> table.validate());
}
@Test
public void testNoProperties()
{
ResolvedTable table = baseBuilder
.buildResolved(mapper);
// Check validation: is legal for storage, but needs
// paramters to be valid at runtime.
table.validate();
}
@Test
public void testFilesParameter()
{
ResolvedTable table = baseBuilder.copy()
.property(LocalTableDefn.BASE_DIR_PROPERTY, "/tmp")
.buildResolved(mapper);
ParameterizedDefn parameterizedTable = tableDefn;
assertEquals(1, parameterizedTable.parameters().size());
// Apply files parameter
Map<String, Object> params = ImmutableMap.of(
LocalTableDefn.FILES_PROPERTY, "foo.csv,bar.csv"
);
// Convert to an external spec
ExternalTableSpec externSpec = parameterizedTable.applyParameters(table, params);
LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource;
assertEquals("/tmp", sourceSpec.getBaseDir().toString());
assertEquals("*", sourceSpec.getFilter());
assertEquals(
Arrays.asList(new File("foo.csv"), new File("bar.csv")),
sourceSpec.getFiles()
);
}
@Test
public void testFilterParameter()
{
ResolvedTable table = baseBuilder.copy()
.property(LocalTableDefn.BASE_DIR_PROPERTY, "/tmp")
.buildResolved(mapper);
// Apply files parameter
Map<String, Object> params = ImmutableMap.of(
LocalTableDefn.FILE_FILTER_PROPERTY, "Oct*.csv"
);
// Convert to an external spec
ExternalTableSpec externSpec = tableDefn.applyParameters(table, params);
LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource;
assertEquals("/tmp", sourceSpec.getBaseDir().toString());
assertEquals("Oct*.csv", sourceSpec.getFilter());
assertTrue(sourceSpec.getFiles().isEmpty());
}
@Test
public void testSqlFunction()
{
List<PropertyDefn<?>> params = tableDefn.tableFunctionParameters();
// Ensure the relevant properties are available as SQL function parameters
PropertyDefn<?> fileDirProp = findProperty(params, LocalTableDefn.BASE_DIR_PROPERTY);
assertNotNull(fileDirProp);
assertEquals(String.class, PropertyAttributes.sqlParameterType(fileDirProp));
PropertyDefn<?> filesProp = findProperty(params, LocalTableDefn.FILES_PROPERTY);
assertNotNull(filesProp);
assertEquals(String.class, PropertyAttributes.sqlParameterType(fileDirProp));
PropertyDefn<?> formatProp = findProperty(params, FormattedExternalTableDefn.FORMAT_PROPERTY);
assertNotNull(formatProp);
assertEquals(String.class, PropertyAttributes.sqlParameterType(formatProp));
// Pretend to accept values for the SQL parameters.
final ResolvedTable table = TableBuilder.of(tableDefn)
.property(fileDirProp.name(), fileDirProp.decodeSqlValue("/tmp", mapper))
.property(filesProp.name(), filesProp.decodeSqlValue("Oct.csv, Nov.csv", mapper))
.property(formatProp.name(), formatProp.decodeSqlValue(InputFormats.CSV_FORMAT_TYPE, mapper))
.column("x", Columns.VARCHAR)
.column("y", Columns.BIGINT)
.buildResolved(mapper);
ExternalTableSpec externSpec = tableDefn.convertToExtern(table);
LocalInputSource sourceSpec = (LocalInputSource) externSpec.inputSource;
assertEquals("/tmp", sourceSpec.getBaseDir().toString());
assertEquals(
Arrays.asList(new File("Oct.csv"), new File("Nov.csv")),
sourceSpec.getFiles()
);
}
}

View File

@ -0,0 +1,222 @@
/*
* 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.external;
import com.google.common.collect.ImmutableMap;
import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
import org.apache.calcite.linq4j.tree.BlockBuilder;
import org.apache.calcite.linq4j.tree.Expression;
import org.apache.calcite.linq4j.tree.Expressions;
import org.apache.calcite.linq4j.tree.FunctionExpression;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
import org.apache.calcite.schema.Function;
import org.apache.calcite.schema.FunctionParameter;
import org.apache.calcite.schema.TableMacro;
import org.apache.calcite.schema.TranslatableTable;
import org.apache.calcite.sql.SqlCall;
import org.apache.calcite.sql.SqlIdentifier;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.SqlLiteral;
import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.SqlUtil;
import org.apache.calcite.sql.type.ArraySqlType;
import org.apache.calcite.sql.type.SqlOperandTypeChecker;
import org.apache.calcite.sql.type.SqlOperandTypeInference;
import org.apache.calcite.sql.type.SqlReturnTypeInference;
import org.apache.calcite.sql.validate.SqlUserDefinedTableMacro;
import org.apache.calcite.util.ImmutableNullableList;
import org.apache.calcite.util.NlsString;
import org.apache.calcite.util.Pair;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
/**
* Druid-specific version of {@link SqlUserDefinedTableMacro} which
* copies & overrides a bunch of code to handle string array arguments.
* Would be best if Calcite handled such argument: retire this class if
* we upgrade to a version of Calcite that handles this task.
*/
public class BaseUserDefinedTableMacro extends SqlUserDefinedTableMacro
{
protected final TableMacro macro;
public BaseUserDefinedTableMacro(SqlIdentifier opName, SqlReturnTypeInference returnTypeInference,
SqlOperandTypeInference operandTypeInference, SqlOperandTypeChecker operandTypeChecker,
List<RelDataType> paramTypes, TableMacro tableMacro)
{
super(opName, returnTypeInference, operandTypeInference, operandTypeChecker, paramTypes, tableMacro);
// Because Calcite's copy of the macro is private
this.macro = tableMacro;
}
// Copy of Calcite method to add array handling
@Override
public TranslatableTable getTable(
RelDataTypeFactory typeFactory,
List<SqlNode> operandList
)
{
List<Object> arguments = convertArguments(typeFactory, operandList,
macro, getNameAsId(), true);
return macro.apply(arguments);
}
// Copy of Calcite method to add array handling
public static List<Object> convertArguments(
RelDataTypeFactory typeFactory,
List<SqlNode> operandList,
Function function,
SqlIdentifier opName,
boolean failOnNonLiteral
)
{
List<Object> arguments = new ArrayList<>(operandList.size());
// Construct a list of arguments, if they are all constants.
for (Pair<FunctionParameter, SqlNode> pair
: Pair.zip(function.getParameters(), operandList)) {
try {
final Object o = getValue(pair.right);
final Object o2 = coerce(o, pair.left.getType(typeFactory));
arguments.add(o2);
}
catch (NonLiteralException e) {
if (failOnNonLiteral) {
throw new IllegalArgumentException("All arguments of call to macro "
+ opName + " should be literal of the correct type. Actual argument #"
+ pair.left.getOrdinal() + " (" + pair.left.getName()
+ ") is not literal: " + pair.right);
}
final RelDataType type = pair.left.getType(typeFactory);
final Object value;
if (type.isNullable()) {
value = null;
} else {
value = 0L;
}
arguments.add(value);
}
}
return arguments;
}
// Copy of Calcite method to add array handling
private static Object getValue(SqlNode right) throws NonLiteralException
{
switch (right.getKind()) {
case ARRAY_VALUE_CONSTRUCTOR:
final List<Object> list = new ArrayList<>();
for (SqlNode o : ((SqlCall) right).getOperandList()) {
list.add(getValue(o));
}
return ImmutableNullableList.copyOf(list);
case MAP_VALUE_CONSTRUCTOR:
final ImmutableMap.Builder<Object, Object> builder2 =
ImmutableMap.builder();
final List<SqlNode> operands = ((SqlCall) right).getOperandList();
for (int i = 0; i < operands.size(); i += 2) {
final SqlNode key = operands.get(i);
final SqlNode value = operands.get(i + 1);
builder2.put(getValue(key), getValue(value));
}
return builder2.build();
default:
if (SqlUtil.isNullLiteral(right, true)) {
return null;
}
if (SqlUtil.isLiteral(right)) {
return ((SqlLiteral) right).getValue();
}
if (right.getKind() == SqlKind.DEFAULT) {
return null; // currently NULL is the only default value
}
throw new NonLiteralException();
}
}
// Copy of Calcite method with Druid-specific code added
private static Object coerce(Object o, RelDataType type) throws NonLiteralException
{
if (o == null) {
return null;
}
// Druid-specific code to handle arrays. Although the type
// is called an ARRAY in SQL, the actual argument is a generic
// list, which we then convert to another list with the elements
// coerced to the declared element type.
if (type instanceof ArraySqlType) {
RelDataType elementType = ((ArraySqlType) type).getComponentType();
if (!(elementType instanceof RelDataTypeFactoryImpl.JavaType)) {
throw new NonLiteralException();
}
// If a list (ARRAY), then coerce each member.
if (!(o instanceof List)) {
throw new NonLiteralException();
}
List<?> arg = (List<?>) o;
List<Object> revised = new ArrayList<>(arg.size());
for (Object value : arg) {
Object element = coerce(value, elementType);
if (element == null) {
throw new NonLiteralException();
}
revised.add(element);
}
return revised;
}
if (!(type instanceof RelDataTypeFactoryImpl.JavaType)) {
// If the type can't be converted, raise an error. Calcite returns null
// which causes odd downstream failures that are hard to diagnose.
throw new NonLiteralException();
}
final RelDataTypeFactoryImpl.JavaType javaType =
(RelDataTypeFactoryImpl.JavaType) type;
final Class<?> clazz = javaType.getJavaClass();
//noinspection unchecked
if (clazz.isAssignableFrom(o.getClass())) {
return o;
}
if (clazz == String.class && o instanceof NlsString) {
return ((NlsString) o).getValue();
}
// We need optimization here for constant folding.
// Not all the expressions can be interpreted (e.g. ternary), so
// we rely on optimization capabilities to fold non-interpretable
// expressions.
BlockBuilder bb = new BlockBuilder();
final Expression expr =
RexToLixTranslator.convert(Expressions.constant(o), clazz);
bb.add(Expressions.return_(null, expr));
final FunctionExpression<?> convert =
Expressions.lambda(bb.toBlock(), Collections.emptyList());
return convert.compile().dynamicInvoke();
}
/** Thrown when a non-literal occurs in an argument to a user-defined
* table macro. */
private static class NonLiteralException extends Exception
{
}
}

View File

@ -31,9 +31,9 @@ import org.apache.calcite.sql.parser.SqlParserPos;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.validate.SqlUserDefinedTableMacro;
import org.apache.druid.catalog.model.TableDefnRegistry;
import org.apache.druid.catalog.model.table.ExternalTableDefn;
import org.apache.druid.catalog.model.table.ExternalTableSpec;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.catalog.model.table.InputSourceDefn;
import org.apache.druid.catalog.model.table.TableFunction;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.server.security.ResourceAction;
import org.apache.druid.sql.calcite.expression.AuthorizableOperator;
@ -49,7 +49,7 @@ import java.util.List;
import java.util.Set;
/**
* Base class for input-source-specfic table functions with arguments derived from
* Base class for input-source-specific table functions with arguments derived from
* a catalog external table definition. Such functions work in conjunction with the
* EXTERN key word to provide a schema. Example of the HTTP form:
* <code><pre>
@ -57,7 +57,7 @@ import java.util.Set;
* FROM TABLE(http(
* userName => 'bob',
* password => 'secret',
* uris => 'http:foo.com/bar.csv',
* uris => ARRAY['http:foo.com/bar.csv'],
* format => 'csv'))
* EXTEND (x VARCHAR, y VARCHAR, z BIGINT)
* PARTITIONED BY ...
@ -74,13 +74,21 @@ public abstract class CatalogExternalTableOperatorConversion implements SqlOpera
final ObjectMapper jsonMapper
)
{
ExternalTableDefn tableDefn = (ExternalTableDefn) registry.defnFor(tableType);
this(
name,
((InputSourceDefn) registry.inputSourceDefnFor(tableType)).adHocTableFn(),
jsonMapper
);
}
public CatalogExternalTableOperatorConversion(
final String name,
final TableFunction fn,
final ObjectMapper jsonMapper
)
{
this.operator = new CatalogExternalTableOperator(
new CatalogTableMacro(
name,
tableDefn,
jsonMapper
)
new CatalogTableMacro(name, fn, jsonMapper)
);
}
@ -116,7 +124,7 @@ public abstract class CatalogExternalTableOperatorConversion implements SqlOpera
@Override
public Set<ResourceAction> computeResources(final SqlCall call)
{
return Collections.singleton(ExternalOperatorConversion.EXTERNAL_RESOURCE_ACTION);
return Collections.singleton(Externals.EXTERNAL_RESOURCE_ACTION);
}
}
@ -124,38 +132,38 @@ public abstract class CatalogExternalTableOperatorConversion implements SqlOpera
{
private final String name;
private final List<FunctionParameter> parameters;
private final ExternalTableDefn tableDefn;
private final TableFunction fn;
private final ObjectMapper jsonMapper;
public CatalogTableMacro(
final String name,
final ExternalTableDefn tableDefn,
final TableFunction fn,
final ObjectMapper jsonMapper
)
{
this.name = name;
this.tableDefn = tableDefn;
this.jsonMapper = jsonMapper;
this.parameters = Externals.convertParameters(tableDefn);
this.fn = fn;
this.parameters = Externals.convertParameters(fn);
}
/**
* Called when the function is used without an {@code EXTEND} clause.
* {@code EXTERN} allows this, most others do not.
*/
@Override
public TranslatableTable apply(final List<Object> arguments)
{
throw new IAE(
"The %s table function requires an EXTEND clause with a schema.",
name
);
return apply(arguments, null);
}
@Override
public TranslatableTable apply(List<Object> arguments, SqlNodeList schema)
{
final ExternalTableSpec externSpec = Externals.convertArguments(
tableDefn,
parameters,
arguments,
schema,
final ExternalTableSpec externSpec = fn.apply(
name,
Externals.convertArguments(fn, arguments),
schema == null ? null : Externals.convertColumns(schema),
jsonMapper
);
return Externals.buildExternalTable(externSpec, jsonMapper);

View File

@ -39,7 +39,6 @@ import org.apache.druid.java.util.common.ISE;
*/
public class ExtendOperator extends SqlInternalOperator
{
// private static final TableMacro macro = new ExtendsMacroWrapper();
public static final ExtendOperator EXTEND = new ExtendOperator();
ExtendOperator()

View File

@ -19,95 +19,112 @@
package org.apache.druid.sql.calcite.external;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
import org.apache.calcite.rex.RexNode;
import org.apache.calcite.sql.SqlCall;
import org.apache.calcite.sql.SqlIdentifier;
import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.parser.SqlParserPos;
import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.validate.SqlUserDefinedTableMacro;
import org.apache.druid.catalog.model.CatalogUtils;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.catalog.model.table.BaseTableFunction;
import org.apache.druid.catalog.model.table.ExternalTableSpec;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.guice.annotations.Json;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.Resource;
import org.apache.druid.server.security.ResourceAction;
import org.apache.druid.server.security.ResourceType;
import org.apache.druid.sql.calcite.expression.AuthorizableOperator;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.expression.SqlOperatorConversion;
import org.apache.druid.sql.calcite.planner.DruidTypeSystem;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import javax.annotation.Nullable;
import java.util.Collections;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
/**
* Registers the "EXTERN" operator, which is used in queries like
* "INSERT INTO dst SELECT * FROM TABLE(EXTERN(...))".
* <pre>{@code
* INSERT INTO dst SELECT * FROM TABLE(EXTERN(
* "<input source>",
* "<input format>",
* "<signature>"))
*
* This class is exercised in CalciteInsertDmlTest but is not currently exposed to end users.
* INSERT INTO dst SELECT * FROM TABLE(EXTERN(
* inputSource => "<input source>",
* inputFormat => "<input format>"))
* EXTEND (<columns>)
* }</pre>
* Where either the by-position or by-name forms are usable with either
* a Druid JSON signature, or an SQL {@code EXTEND} list of columns.
* As with all table functions, the {@code EXTEND} is optional.
*/
public class ExternalOperatorConversion implements SqlOperatorConversion
public class ExternalOperatorConversion extends CatalogExternalTableOperatorConversion
{
public static final String FUNCTION_NAME = "EXTERN";
// Resource that allows reading external data via SQL.
public static final ResourceAction EXTERNAL_RESOURCE_ACTION =
new ResourceAction(new Resource("EXTERNAL", ResourceType.EXTERNAL), Action.READ);
public static final String INPUT_SOURCE_PARAM = "inputSource";
public static final String INPUT_FORMAT_PARAM = "inputFormat";
public static final String SIGNATURE_PARAM = "signature";
private final SqlUserDefinedTableMacro operator;
/**
* The use of a table function allows the use of optional arguments,
* so that the signature can be given either as the original-style
* serialized JSON signature, or the updated SQL-style EXTEND clause.
*/
private static class ExternFunction extends BaseTableFunction
{
public ExternFunction()
{
super(Arrays.asList(
new Parameter(INPUT_SOURCE_PARAM, ParameterType.VARCHAR, true),
new Parameter(INPUT_FORMAT_PARAM, ParameterType.VARCHAR, true),
// Not required: the user can either provide the signature OR
// an EXTEND clause. Checked in the implementation.
new Parameter(SIGNATURE_PARAM, ParameterType.VARCHAR, false)
));
}
@Override
public ExternalTableSpec apply(
final String fnName,
final Map<String, Object> args,
final List<ColumnSpec> columns,
final ObjectMapper jsonMapper
)
{
try {
final String sigValue = CatalogUtils.getString(args, SIGNATURE_PARAM);
if (sigValue == null && columns == null) {
throw new IAE(
"EXTERN requires either a %s value or an EXTEND clause",
SIGNATURE_PARAM
);
}
if (sigValue != null && columns != null) {
throw new IAE(
"EXTERN requires either a %s value or an EXTEND clause, but not both",
SIGNATURE_PARAM
);
}
final RowSignature rowSignature;
if (columns != null) {
rowSignature = Columns.convertSignature(columns);
} else {
rowSignature = jsonMapper.readValue(sigValue, RowSignature.class);
}
return new ExternalTableSpec(
jsonMapper.readValue(CatalogUtils.getString(args, INPUT_SOURCE_PARAM), InputSource.class),
jsonMapper.readValue(CatalogUtils.getString(args, INPUT_FORMAT_PARAM), InputFormat.class),
rowSignature
);
}
catch (JsonProcessingException e) {
throw new RuntimeException(e);
}
}
}
@Inject
public ExternalOperatorConversion(@Json final ObjectMapper jsonMapper)
{
this.operator = new ExternalOperator(new ExternalTableMacro(jsonMapper));
}
@Override
public SqlOperator calciteOperator()
{
return operator;
}
@Nullable
@Override
public DruidExpression toDruidExpression(PlannerContext plannerContext, RowSignature rowSignature, RexNode rexNode)
{
return null;
}
private static class ExternalOperator extends SqlUserDefinedTableMacro implements AuthorizableOperator
{
public ExternalOperator(final ExternalTableMacro macro)
{
super(
new SqlIdentifier(FUNCTION_NAME, SqlParserPos.ZERO),
ReturnTypes.CURSOR,
null,
OperandTypes.sequence(
macro.signature(),
OperandTypes.family(SqlTypeFamily.STRING),
OperandTypes.family(SqlTypeFamily.STRING),
OperandTypes.family(SqlTypeFamily.STRING)
),
macro.getParameters()
.stream()
.map(parameter -> parameter.getType(DruidTypeSystem.TYPE_FACTORY))
.collect(Collectors.toList()),
macro
);
}
@Override
public Set<ResourceAction> computeResources(final SqlCall call)
{
return Collections.singleton(EXTERNAL_RESOURCE_ACTION);
}
super(FUNCTION_NAME, new ExternFunction(), jsonMapper);
}
}

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import org.apache.calcite.avatica.SqlType;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.schema.FunctionParameter;
import org.apache.calcite.sql.SqlCallBinding;
import org.apache.calcite.sql.SqlDataTypeSpec;
@ -34,22 +35,25 @@ import org.apache.calcite.sql.SqlTypeNameSpec;
import org.apache.calcite.sql.type.SqlOperandCountRanges;
import org.apache.calcite.sql.type.SqlOperandTypeChecker;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.druid.catalog.model.ModelProperties;
import org.apache.druid.catalog.model.ModelProperties.PropertyDefn;
import org.apache.druid.catalog.model.PropertyAttributes;
import org.apache.druid.catalog.model.ResolvedTable;
import org.apache.druid.catalog.model.table.ExternalTableDefn;
import org.apache.druid.catalog.model.ColumnSpec;
import org.apache.druid.catalog.model.table.ExternalTableSpec;
import org.apache.druid.catalog.model.table.TableBuilder;
import org.apache.druid.catalog.model.table.TableFunction;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.Resource;
import org.apache.druid.server.security.ResourceAction;
import org.apache.druid.server.security.ResourceType;
import org.apache.druid.sql.calcite.planner.DruidTypeSystem;
import org.apache.druid.sql.calcite.table.ExternalTable;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
@ -63,17 +67,42 @@ public class Externals
* Convert parameters from Catalog external table definition form to the SQL form
* used for a table macro and its function.
*/
public static List<FunctionParameter> convertParameters(final ExternalTableDefn tableDefn)
public static List<FunctionParameter> convertParameters(TableFunction fn)
{
List<ModelProperties.PropertyDefn<?>> props = tableDefn.tableFunctionParameters();
return convertToCalciteParameters(fn.parameters());
}
private static List<FunctionParameter> convertToCalciteParameters(List<TableFunction.ParameterDefn> paramDefns)
{
final RelDataTypeFactory typeFactory = DruidTypeSystem.TYPE_FACTORY;
ImmutableList.Builder<FunctionParameter> params = ImmutableList.builder();
for (int i = 0; i < props.size(); i++) {
ModelProperties.PropertyDefn<?> prop = props.get(i);
for (int i = 0; i < paramDefns.size(); i++) {
TableFunction.ParameterDefn paramDefn = paramDefns.get(i);
RelDataType paramType;
switch (paramDefn.type()) {
case BIGINT:
paramType = typeFactory.createJavaType(Long.class);
break;
case BOOLEAN:
paramType = typeFactory.createJavaType(Boolean.class);
break;
case VARCHAR:
paramType = typeFactory.createJavaType(String.class);
break;
case VARCHAR_ARRAY:
paramType = typeFactory.createArrayType(
typeFactory.createJavaType(String.class),
-1
);
break;
default:
throw new ISE("Undefined parameter type: %s", paramDefn.type().sqlName());
}
params.add(new FunctionParameterImpl(
i,
prop.name(),
DruidTypeSystem.TYPE_FACTORY.createJavaType(PropertyAttributes.sqlParameterType(prop)),
PropertyAttributes.isOptional(prop)
paramDefn.name(),
paramType,
paramDefn.isOptional()
));
}
return params.build();
@ -145,48 +174,40 @@ public class Externals
}
/**
* Convert the actual arguments to SQL external table function into a catalog
* resolved table, then convert that to an external table spec usable by MSQ.
*
* @param tableDefn catalog definition of the kind of external table
* @param parameters the parameters to the SQL table macro
* @param arguments the arguments that match the parameters. Optional arguments
* may be null
* @param schema the external table schema provided by the EXTEND clause
* @param jsonMapper the JSON mapper to use for value conversions
* @return a spec with the three values that MSQ needs to create an external table
* Convert the list of Calcite function arguments to a map of non-null arguments.
* The resulting map must be mutable as processing may rewrite values.
*/
public static ExternalTableSpec convertArguments(
final ExternalTableDefn tableDefn,
final List<FunctionParameter> parameters,
final List<Object> arguments,
final SqlNodeList schema,
final ObjectMapper jsonMapper
public static Map<String, Object> convertArguments(
final TableFunction fn,
final List<Object> arguments
)
{
final TableBuilder builder = TableBuilder.of(tableDefn);
for (int i = 0; i < parameters.size(); i++) {
String name = parameters.get(i).getName();
Object value = arguments.get(i);
if (value == null) {
continue;
final List<TableFunction.ParameterDefn> params = fn.parameters();
final Map<String, Object> argMap = new HashMap<>();
for (int i = 0; i < arguments.size(); i++) {
final Object value = arguments.get(i);
if (value != null) {
argMap.put(params.get(i).name(), value);
}
PropertyDefn<?> prop = tableDefn.property(name);
builder.property(name, prop.decodeSqlValue(value, jsonMapper));
}
// Converts from a list of (identifier, type, ...) pairs to
// a Druid row signature. The schema itself comes from the
// Druid-specific EXTEND syntax added to the parser.
for (int i = 0; i < schema.size(); i += 2) {
final String name = convertName((SqlIdentifier) schema.get(i));
String sqlType = convertType(name, (SqlDataTypeSpec) schema.get(i + 1));
builder.column(name, sqlType);
}
ResolvedTable table = builder.buildResolved(jsonMapper);
return tableDefn.convertToExtern(table);
return argMap;
}
/**
* Converts from a list of (identifier, type, ...) pairs to
* list of column specs. The schema itself comes from the
* Druid-specific EXTEND syntax added to the parser.
*/
public static List<ColumnSpec> convertColumns(SqlNodeList schema)
{
final List<ColumnSpec> columns = new ArrayList<>();
for (int i = 0; i < schema.size(); i += 2) {
final String name = convertName((SqlIdentifier) schema.get(i));
final String sqlType = convertType(name, (SqlDataTypeSpec) schema.get(i + 1));
columns.add(new ColumnSpec(name, sqlType, null));
}
return columns;
}
/**
* Define the Druid input schema from a name provided in the EXTEND
@ -269,10 +290,28 @@ public class Externals
+ "Please change the column name to something other than __time");
}
return toExternalTable(spec, jsonMapper);
}
public static ResourceAction externalRead(String name)
{
return new ResourceAction(new Resource(name, ResourceType.EXTERNAL), Action.READ);
}
public static ExternalTable toExternalTable(ExternalTableSpec spec, ObjectMapper jsonMapper)
{
return new ExternalTable(
new ExternalDataSource(spec.inputSource, spec.inputFormat, spec.signature),
spec.signature,
jsonMapper
new ExternalDataSource(
spec.inputSource,
spec.inputFormat,
spec.signature
),
spec.signature,
jsonMapper
);
}
// Resource that allows reading external data via SQL.
public static final ResourceAction EXTERNAL_RESOURCE_ACTION =
new ResourceAction(new Resource(ResourceType.EXTERNAL, ResourceType.EXTERNAL), Action.READ);
}

View File

@ -21,17 +21,15 @@ package org.apache.druid.sql.calcite.external;
import com.google.inject.Inject;
import org.apache.druid.catalog.model.TableDefnRegistry;
import org.apache.druid.catalog.model.table.HttpTableDefn;
import org.apache.druid.catalog.model.table.HttpInputSourceDefn;
public class HttpOperatorConversion extends CatalogExternalTableOperatorConversion
{
public static final String FUNCTION_NAME = "http";
@Inject
public HttpOperatorConversion(
final TableDefnRegistry registry
)
public HttpOperatorConversion(final TableDefnRegistry registry)
{
super(FUNCTION_NAME, registry, HttpTableDefn.TABLE_TYPE, registry.jsonMapper());
super(FUNCTION_NAME, registry, HttpInputSourceDefn.TYPE_KEY, registry.jsonMapper());
}
}

View File

@ -21,7 +21,7 @@ package org.apache.druid.sql.calcite.external;
import com.google.inject.Inject;
import org.apache.druid.catalog.model.TableDefnRegistry;
import org.apache.druid.catalog.model.table.InlineTableDefn;
import org.apache.druid.catalog.model.table.InlineInputSourceDefn;
public class InlineOperatorConversion extends CatalogExternalTableOperatorConversion
{
@ -32,6 +32,6 @@ public class InlineOperatorConversion extends CatalogExternalTableOperatorConver
final TableDefnRegistry registry
)
{
super(FUNCTION_NAME, registry, InlineTableDefn.TABLE_TYPE, registry.jsonMapper());
super(FUNCTION_NAME, registry, InlineInputSourceDefn.TYPE_KEY, registry.jsonMapper());
}
}

View File

@ -21,7 +21,7 @@ package org.apache.druid.sql.calcite.external;
import com.google.inject.Inject;
import org.apache.druid.catalog.model.TableDefnRegistry;
import org.apache.druid.catalog.model.table.LocalTableDefn;
import org.apache.druid.catalog.model.table.LocalInputSourceDefn;
public class LocalOperatorConversion extends CatalogExternalTableOperatorConversion
{
@ -34,6 +34,6 @@ public class LocalOperatorConversion extends CatalogExternalTableOperatorConvers
final TableDefnRegistry registry
)
{
super(FUNCTION_NAME, registry, LocalTableDefn.TABLE_TYPE, registry.jsonMapper());
super(FUNCTION_NAME, registry, LocalInputSourceDefn.TYPE_KEY, registry.jsonMapper());
}
}

View File

@ -36,7 +36,6 @@ import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlOperandTypeChecker;
import org.apache.calcite.sql.type.SqlOperandTypeInference;
import org.apache.calcite.sql.type.SqlReturnTypeInference;
import org.apache.calcite.sql.validate.SqlUserDefinedTableMacro;
import org.apache.druid.server.security.ResourceAction;
import org.apache.druid.sql.calcite.expression.AuthorizableOperator;
@ -100,10 +99,8 @@ import java.util.Set;
* </pre></code>
* Since we seldom use unparse, we can perhaps live with this limitation for now.
*/
public abstract class UserDefinedTableMacroFunction extends SqlUserDefinedTableMacro implements AuthorizableOperator
public abstract class UserDefinedTableMacroFunction extends BaseUserDefinedTableMacro implements AuthorizableOperator
{
protected final ExtendedTableMacro macro;
public UserDefinedTableMacroFunction(
SqlIdentifier opName,
SqlReturnTypeInference returnTypeInference,
@ -114,9 +111,6 @@ public abstract class UserDefinedTableMacroFunction extends SqlUserDefinedTableM
)
{
super(opName, returnTypeInference, operandTypeInference, operandTypeChecker, paramTypes, tableMacro);
// Because Calcite's copy of the macro is private
this.macro = tableMacro;
}
/**
@ -128,7 +122,7 @@ public abstract class UserDefinedTableMacroFunction extends SqlUserDefinedTableM
return new ExtendedCall(oldCall, new ShimTableMacroFunction(this, schema));
}
private static class ShimTableMacroFunction extends SqlUserDefinedTableMacro implements AuthorizableOperator
private static class ShimTableMacroFunction extends BaseUserDefinedTableMacro implements AuthorizableOperator
{
protected final UserDefinedTableMacroFunction base;
protected final SqlNodeList schema;
@ -141,7 +135,7 @@ public abstract class UserDefinedTableMacroFunction extends SqlUserDefinedTableM
null,
base.getOperandTypeChecker(),
base.getParamTypes(),
new ShimTableMacro(base.macro, schema)
new ShimTableMacro((ExtendedTableMacro) base.macro, schema)
);
this.base = base;
this.schema = schema;

View File

@ -23,11 +23,17 @@ import org.apache.calcite.avatica.remote.TypedValue;
import org.apache.calcite.sql.SqlDynamicParam;
import org.apache.calcite.sql.SqlLiteral;
import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.calcite.sql.parser.SqlParserPos;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.sql.util.SqlShuttle;
import org.apache.calcite.util.TimestampString;
import org.apache.druid.java.util.common.IAE;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
/**
* Replaces all {@link SqlDynamicParam} encountered in an {@link SqlNode} tree
* with a {@link SqlLiteral} if a value binding exists for the parameter, if
@ -84,6 +90,9 @@ public class SqlParameterizerShuttle extends SqlShuttle
);
}
if (typeName == SqlTypeName.ARRAY) {
return createArrayLiteral(paramBinding.value);
}
try {
// This throws ClassCastException for a DATE parameter given as
// an Integer. The parameter is left in place and is replaced
@ -95,4 +104,42 @@ public class SqlParameterizerShuttle extends SqlShuttle
return param;
}
}
/**
* Convert an ARRAY parameter to the equivalent of the ARRAY[a, b, ...]
* syntax. This is not well-supported in the present version of Calcite,
* so we have to do a bit of roll-our-own code to create the required
* structure. Supports a limited set of member types. Does not attempt
* to enforce that all elements have the same type.
*/
private SqlNode createArrayLiteral(Object value)
{
List<?> list = Arrays.asList((Object[]) value);
List<SqlNode> args = new ArrayList<>(list.size());
for (Object element : list) {
if (element == null) {
throw new IAE("An array parameter cannot contain null values");
}
SqlNode node;
if (element instanceof String) {
node = SqlLiteral.createCharString((String) element, SqlParserPos.ZERO);
} else if (element instanceof Integer || element instanceof Long) {
// No direct way to create a literal from an Integer or Long, have
// to parse a string, sadly.
node = SqlLiteral.createExactNumeric(element.toString(), SqlParserPos.ZERO);
} else if (element instanceof Boolean) {
node = SqlLiteral.createBoolean((Boolean) value, SqlParserPos.ZERO);
} else {
throw new IAE(
"An array parameter does not allow values of type %s",
value.getClass().getSimpleName()
);
}
args.add(node);
}
return SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR.createCall(
SqlParserPos.ZERO,
args
);
}
}

View File

@ -23,9 +23,9 @@ import com.google.inject.Binder;
import com.google.inject.Module;
import com.google.inject.Provides;
import com.google.inject.Scopes;
import com.google.inject.Singleton;
import com.google.inject.name.Named;
import com.google.inject.name.Names;
import org.apache.druid.guice.LazySingleton;
import org.apache.druid.guice.LifecycleModule;
import org.apache.druid.sql.guice.SqlBindings;
@ -64,7 +64,7 @@ public class DruidCalciteSchemaModule implements Module
}
@Provides
@Singleton
@LazySingleton
private DruidSchemaCatalog getRootSchema(@Named(INCOMPLETE_SCHEMA) DruidSchemaCatalog rootSchema, InformationSchema informationSchema)
{
rootSchema.getRootSchema().add(INFORMATION_SCHEMA_NAME, informationSchema);

View File

@ -40,11 +40,8 @@ import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.AuthenticationResult;
import org.apache.druid.server.security.Resource;
import org.apache.druid.server.security.ResourceAction;
import org.apache.druid.server.security.ResourceType;
import org.apache.druid.sql.SqlQueryPlus;
import org.apache.druid.sql.calcite.external.ExternalDataSource;
import org.apache.druid.sql.calcite.external.ExternalOperatorConversion;
@ -56,6 +53,7 @@ import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.guice.SqlBindings;
import org.apache.druid.sql.http.SqlParameter;
import org.hamcrest.CoreMatchers;
import org.hamcrest.Matcher;
import org.hamcrest.MatcherAssert;
@ -75,6 +73,12 @@ public class CalciteIngestionDmlTest extends BaseCalciteQueryTest
.put(QueryContexts.CTX_SQL_QUERY_ID, DUMMY_SQL_ID)
.build();
public static final Map<String, Object> PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT = ImmutableMap.of(
DruidSqlInsert.SQL_INSERT_SEGMENT_GRANULARITY,
"{\"type\":\"all\"}"
);
protected static final RowSignature FOO_TABLE_SIGNATURE =
RowSignature.builder()
.addTimeColumn()
@ -211,6 +215,7 @@ public class CalciteIngestionDmlTest extends BaseCalciteQueryTest
private Query<?> expectedQuery;
private Matcher<Throwable> validationErrorMatcher;
private String expectedLogicalPlanResource;
private List<SqlParameter> parameters;
private IngestionDmlTester()
{
@ -244,6 +249,12 @@ public class CalciteIngestionDmlTest extends BaseCalciteQueryTest
return this;
}
public IngestionDmlTester parameters(List<SqlParameter> parameters)
{
this.parameters = parameters;
return this;
}
public IngestionDmlTester expectTarget(
final String expectedTargetDataSource,
final RowSignature expectedTargetSignature
@ -365,6 +376,7 @@ public class CalciteIngestionDmlTest extends BaseCalciteQueryTest
.sql(sql)
.queryContext(queryContext)
.authResult(authenticationResult)
.parameters(parameters)
.plannerConfig(plannerConfig)
.expectedResources(expectedResources)
.run();
@ -382,6 +394,7 @@ public class CalciteIngestionDmlTest extends BaseCalciteQueryTest
.sql(sql)
.queryContext(queryContext)
.authResult(authenticationResult)
.parameters(parameters)
.plannerConfig(plannerConfig)
.expectedQuery(expectedQuery)
.expectedResults(Collections.singletonList(new Object[]{expectedTargetDataSource, expectedTargetSignature}))
@ -397,19 +410,4 @@ public class CalciteIngestionDmlTest extends BaseCalciteQueryTest
.build();
}
}
protected static ResourceAction viewRead(final String viewName)
{
return new ResourceAction(new Resource(viewName, ResourceType.VIEW), Action.READ);
}
protected static ResourceAction dataSourceRead(final String dataSource)
{
return new ResourceAction(new Resource(dataSource, ResourceType.DATASOURCE), Action.READ);
}
protected static ResourceAction dataSourceWrite(final String dataSource)
{
return new ResourceAction(new Resource(dataSource, ResourceType.DATASOURCE), Action.WRITE);
}
}

View File

@ -41,7 +41,7 @@ import org.apache.druid.segment.join.JoinType;
import org.apache.druid.server.security.ForbiddenException;
import org.apache.druid.sql.SqlPlanningException;
import org.apache.druid.sql.calcite.external.ExternalDataSource;
import org.apache.druid.sql.calcite.external.ExternalOperatorConversion;
import org.apache.druid.sql.calcite.external.Externals;
import org.apache.druid.sql.calcite.filtration.Filtration;
import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
import org.apache.druid.sql.calcite.planner.IngestHandler;
@ -293,7 +293,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
.sql("INSERT INTO dst SELECT * FROM %s PARTITIONED BY ALL TIME", externSql(externalDataSource))
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("dst", externalDataSource.getSignature())
.expectResources(dataSourceWrite("dst"), ExternalOperatorConversion.EXTERNAL_RESOURCE_ACTION)
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
newScanQueryBuilder()
.dataSource(externalDataSource)
@ -391,7 +391,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
@Test
public void testInsertWithClusteredBy()
{
// Test correctness of the query when only CLUSTERED BY clause is present
// Test correctness of the query when only the CLUSTERED BY clause is present
RowSignature targetRowSignature = RowSignature.builder()
.add("__time", ColumnType.LONG)
.add("floor_m1", ColumnType.FLOAT)
@ -696,7 +696,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
)
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("dst", RowSignature.builder().add("xy", ColumnType.STRING).add("z", ColumnType.LONG).build())
.expectResources(dataSourceWrite("dst"), ExternalOperatorConversion.EXTERNAL_RESOURCE_ACTION)
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
newScanQueryBuilder()
.dataSource(externalDataSource)
@ -734,7 +734,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
.add("cnt", ColumnType.LONG)
.build()
)
.expectResources(dataSourceWrite("dst"), ExternalOperatorConversion.EXTERNAL_RESOURCE_ACTION)
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
GroupByQuery.builder()
.setDataSource(externalDataSource)
@ -768,7 +768,7 @@ public class CalciteInsertDmlTest extends CalciteIngestionDmlTest
.add("cnt", ColumnType.LONG)
.build()
)
.expectResources(dataSourceWrite("dst"), ExternalOperatorConversion.EXTERNAL_RESOURCE_ACTION)
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
GroupByQuery.builder()
.setDataSource(externalDataSource)

View File

@ -34,7 +34,7 @@ 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.SqlPlanningException;
import org.apache.druid.sql.calcite.external.ExternalOperatorConversion;
import org.apache.druid.sql.calcite.external.Externals;
import org.apache.druid.sql.calcite.filtration.Filtration;
import org.apache.druid.sql.calcite.parser.DruidSqlInsert;
import org.apache.druid.sql.calcite.parser.DruidSqlParserUtils;
@ -492,7 +492,7 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest
.sql("REPLACE INTO dst OVERWRITE ALL SELECT * FROM %s PARTITIONED BY ALL TIME", externSql(externalDataSource))
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("dst", externalDataSource.getSignature())
.expectResources(dataSourceWrite("dst"), ExternalOperatorConversion.EXTERNAL_RESOURCE_ACTION)
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
newScanQueryBuilder()
.dataSource(externalDataSource)
@ -686,7 +686,7 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest
)
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("dst", RowSignature.builder().add("xy", ColumnType.STRING).add("z", ColumnType.LONG).build())
.expectResources(dataSourceWrite("dst"), ExternalOperatorConversion.EXTERNAL_RESOURCE_ACTION)
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
newScanQueryBuilder()
.dataSource(externalDataSource)
@ -716,7 +716,7 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest
.add("cnt", ColumnType.LONG)
.build()
)
.expectResources(dataSourceWrite("dst"), ExternalOperatorConversion.EXTERNAL_RESOURCE_ACTION)
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
GroupByQuery.builder()
.setDataSource(externalDataSource)
@ -748,7 +748,7 @@ public class CalciteReplaceDmlTest extends CalciteIngestionDmlTest
.add("cnt", ColumnType.LONG)
.build()
)
.expectResources(dataSourceWrite("dst"), ExternalOperatorConversion.EXTERNAL_RESOURCE_ACTION)
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
GroupByQuery.builder()
.setDataSource(externalDataSource)

View File

@ -22,20 +22,24 @@ package org.apache.druid.sql.calcite;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import org.apache.calcite.avatica.SqlType;
import org.apache.druid.catalog.model.Columns;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.HttpInputSource;
import org.apache.druid.data.input.impl.HttpInputSourceConfig;
import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.metadata.DefaultPasswordProvider;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.sql.calcite.external.ExternalDataSource;
import org.apache.druid.sql.calcite.external.ExternalOperatorConversion;
import org.apache.druid.sql.calcite.external.Externals;
import org.apache.druid.sql.calcite.filtration.Filtration;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.http.SqlParameter;
import org.junit.Test;
import java.io.File;
@ -54,7 +58,7 @@ import java.util.Collections;
* query ensure that the resulting MSQ task is identical regardless of the path
* taken.
*/
public class CatalogIngestionTest extends CalciteIngestionDmlTest
public class IngestTableFunctionTest extends CalciteIngestionDmlTest
{
protected static URI toURI(String uri)
{
@ -91,7 +95,7 @@ public class CatalogIngestionTest extends CalciteIngestionDmlTest
.sql("INSERT INTO dst SELECT * FROM %s PARTITIONED BY ALL TIME", externSql(httpDataSource))
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("dst", httpDataSource.getSignature())
.expectResources(dataSourceWrite("dst"), ExternalOperatorConversion.EXTERNAL_RESOURCE_ACTION)
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
newScanQueryBuilder()
.dataSource(httpDataSource)
@ -99,7 +103,7 @@ public class CatalogIngestionTest extends CalciteIngestionDmlTest
.columns("x", "y", "z")
.context(CalciteInsertDmlTest.PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
.build()
)
)
.expectLogicalPlanFrom("httpExtern")
.verify();
}
@ -133,7 +137,7 @@ public class CatalogIngestionTest extends CalciteIngestionDmlTest
.sql("INSERT INTO dst SELECT *\nFROM %s\nPARTITIONED BY ALL TIME", externSqlByName(httpDataSource))
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("dst", httpDataSource.getSignature())
.expectResources(dataSourceWrite("dst"), ExternalOperatorConversion.EXTERNAL_RESOURCE_ACTION)
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
newScanQueryBuilder()
.dataSource(httpDataSource)
@ -141,7 +145,7 @@ public class CatalogIngestionTest extends CalciteIngestionDmlTest
.columns("x", "y", "z")
.context(CalciteInsertDmlTest.PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
.build()
)
)
.expectLogicalPlanFrom("httpExtern")
.verify();
}
@ -155,13 +159,15 @@ public class CatalogIngestionTest extends CalciteIngestionDmlTest
{
testIngestionQuery()
.sql("INSERT INTO dst SELECT *\n" +
"FROM TABLE(http(userName => 'bob', password => 'secret',\n" +
" uris => 'http:foo.com/bar.csv', format => 'csv'))\n" +
"FROM TABLE(http(userName => 'bob',\n" +
" password => 'secret',\n" +
" uris => ARRAY['http:foo.com/bar.csv'],\n" +
" format => 'csv'))\n" +
" EXTEND (x VARCHAR, y VARCHAR, z BIGINT)\n" +
"PARTITIONED BY ALL TIME")
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("dst", httpDataSource.getSignature())
.expectResources(dataSourceWrite("dst"), ExternalOperatorConversion.EXTERNAL_RESOURCE_ACTION)
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
newScanQueryBuilder()
.dataSource(httpDataSource)
@ -169,13 +175,40 @@ public class CatalogIngestionTest extends CalciteIngestionDmlTest
.columns("x", "y", "z")
.context(CalciteInsertDmlTest.PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
.build()
)
)
.expectLogicalPlanFrom("httpExtern")
.verify();
}
@Test
public void testHttpFnWithParameters()
{
testIngestionQuery()
.sql("INSERT INTO dst SELECT *\n" +
"FROM TABLE(http(userName => 'bob',\n" +
" password => 'secret',\n" +
" uris => ?,\n" +
" format => 'csv'))\n" +
" EXTEND (x VARCHAR, y VARCHAR, z BIGINT)\n" +
"PARTITIONED BY ALL TIME")
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.parameters(Collections.singletonList(new SqlParameter(SqlType.ARRAY, new String[] {"http:foo.com/bar.csv"})))
.expectTarget("dst", httpDataSource.getSignature())
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
newScanQueryBuilder()
.dataSource(httpDataSource)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("x", "y", "z")
.context(CalciteInsertDmlTest.PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
.build()
)
.expectLogicalPlanFrom("httpExtern")
.verify();
}
/**
* Basic use of INLINE
* Basic use of an inline input source via EXTERN
*/
@Test
public void testInlineExtern()
@ -184,7 +217,7 @@ public class CatalogIngestionTest extends CalciteIngestionDmlTest
.sql("INSERT INTO dst SELECT * FROM %s PARTITIONED BY ALL TIME", externSql(externalDataSource))
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("dst", externalDataSource.getSignature())
.expectResources(dataSourceWrite("dst"), ExternalOperatorConversion.EXTERNAL_RESOURCE_ACTION)
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
newScanQueryBuilder()
.dataSource(externalDataSource)
@ -192,7 +225,75 @@ public class CatalogIngestionTest extends CalciteIngestionDmlTest
.columns("x", "y", "z")
.context(CalciteInsertDmlTest.PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
.build()
)
)
.expectLogicalPlanFrom("insertFromExternal")
.verify();
}
protected String externSqlByNameNoSig(final ExternalDataSource externalDataSource)
{
ObjectMapper queryJsonMapper = queryFramework().queryJsonMapper();
try {
return StringUtils.format(
"TABLE(extern(inputSource => %s, inputFormat => %s))",
Calcites.escapeStringLiteral(queryJsonMapper.writeValueAsString(externalDataSource.getInputSource())),
Calcites.escapeStringLiteral(queryJsonMapper.writeValueAsString(externalDataSource.getInputFormat()))
);
}
catch (JsonProcessingException e) {
throw new RuntimeException(e);
}
}
protected String externClauseFromSig(final ExternalDataSource externalDataSource)
{
RowSignature sig = externalDataSource.getSignature();
StringBuilder buf = new StringBuilder("(");
for (int i = 0; i < sig.size(); i++) {
if (i > 0) {
buf.append(", ");
}
buf.append(sig.getColumnName(i)).append(" ");
ColumnType type = sig.getColumnType(i).get();
if (type == ColumnType.STRING) {
buf.append(Columns.VARCHAR);
} else if (type == ColumnType.LONG) {
buf.append(Columns.BIGINT);
} else if (type == ColumnType.DOUBLE) {
buf.append(Columns.DOUBLE);
} else if (type == ColumnType.FLOAT) {
buf.append(Columns.FLOAT);
} else {
throw new UOE("Unsupported native type %s", type);
}
}
return buf.append(")").toString();
}
/**
* Use an inline input source with EXTERN and EXTEND
*/
@Test
public void testInlineExternWithExtend()
{
testIngestionQuery()
.sql("INSERT INTO dst SELECT *\n" +
" FROM %s\n" +
" %s\n" +
" PARTITIONED BY ALL TIME",
externSqlByNameNoSig(externalDataSource),
externClauseFromSig(externalDataSource))
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("dst", externalDataSource.getSignature())
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
newScanQueryBuilder()
.dataSource(externalDataSource)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("x", "y", "z")
.context(CalciteInsertDmlTest.PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
.build()
)
.expectLogicalPlanFrom("insertFromExternal")
.verify();
}
@ -206,13 +307,13 @@ public class CatalogIngestionTest extends CalciteIngestionDmlTest
{
testIngestionQuery()
.sql("INSERT INTO dst SELECT *\n" +
"FROM TABLE(inline(data => 'a,b,1\nc,d,2\n',\n" +
"FROM TABLE(inline(data => ARRAY['a,b,1', 'c,d,2'],\n" +
" format => 'csv'))\n" +
" EXTEND (x VARCHAR, y VARCHAR, z BIGINT)\n" +
"PARTITIONED BY ALL TIME")
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("dst", externalDataSource.getSignature())
.expectResources(dataSourceWrite("dst"), ExternalOperatorConversion.EXTERNAL_RESOURCE_ACTION)
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
newScanQueryBuilder()
.dataSource(externalDataSource)
@ -220,19 +321,12 @@ public class CatalogIngestionTest extends CalciteIngestionDmlTest
.columns("x", "y", "z")
.context(CalciteInsertDmlTest.PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
.build()
)
)
.expectLogicalPlanFrom("insertFromExternal")
.verify();
}
protected final ExternalDataSource localDataSource = new ExternalDataSource(
// The preferred form for this test. But, does not work.
// See Apache Druid issue #13359.
//new LocalInputSource(
// new File("/tmp"),
// "*.csv",
// Arrays.asList(new File("foo.csv"), new File("bar.csv"))
//),
new LocalInputSource(
null,
null,
@ -247,7 +341,7 @@ public class CatalogIngestionTest extends CalciteIngestionDmlTest
);
/**
* Basic use of LOCAL
* Basic use of LOCALFILES
*/
@Test
public void testLocalExtern()
@ -256,7 +350,7 @@ public class CatalogIngestionTest extends CalciteIngestionDmlTest
.sql("INSERT INTO dst SELECT * FROM %s PARTITIONED BY ALL TIME", externSql(localDataSource))
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("dst", localDataSource.getSignature())
.expectResources(dataSourceWrite("dst"), ExternalOperatorConversion.EXTERNAL_RESOURCE_ACTION)
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
newScanQueryBuilder()
.dataSource(localDataSource)
@ -264,27 +358,27 @@ public class CatalogIngestionTest extends CalciteIngestionDmlTest
.columns("x", "y", "z")
.context(CalciteInsertDmlTest.PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
.build()
)
)
.expectLogicalPlanFrom("localExtern")
.verify();
}
/**
* Local with parameters by name. Logical plan and native query are identical
* Localfiles with parameters by name. Logical plan and native query are identical
* to the basic EXTERN.
*/
@Test
public void testLocalFn()
public void testLocalFilesFn()
{
testIngestionQuery()
.sql("INSERT INTO dst SELECT *\n" +
"FROM TABLE(localfiles(files => '/tmp/foo.csv, /tmp/bar.csv',\n" +
"FROM TABLE(localfiles(files => ARRAY['/tmp/foo.csv', '/tmp/bar.csv'],\n" +
" format => 'csv'))\n" +
" EXTEND (x VARCHAR, y VARCHAR, z BIGINT)\n" +
"PARTITIONED BY ALL TIME")
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("dst", localDataSource.getSignature())
.expectResources(dataSourceWrite("dst"), ExternalOperatorConversion.EXTERNAL_RESOURCE_ACTION)
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
newScanQueryBuilder()
.dataSource(localDataSource)
@ -292,7 +386,7 @@ public class CatalogIngestionTest extends CalciteIngestionDmlTest
.columns("x", "y", "z")
.context(CalciteInsertDmlTest.PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
.build()
)
)
.expectLogicalPlanFrom("localExtern")
.verify();
}
@ -306,13 +400,13 @@ public class CatalogIngestionTest extends CalciteIngestionDmlTest
{
testIngestionQuery()
.sql("INSERT INTO dst SELECT *\n" +
"FROM TABLE(localfiles(files => '/tmp/foo.csv, /tmp/bar.csv',\n" +
"FROM TABLE(localfiles(files => ARRAY['/tmp/foo.csv', '/tmp/bar.csv'],\n" +
" format => 'csv'))\n" +
" (x VARCHAR, y VARCHAR, z BIGINT)\n" +
"PARTITIONED BY ALL TIME")
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("dst", localDataSource.getSignature())
.expectResources(dataSourceWrite("dst"), ExternalOperatorConversion.EXTERNAL_RESOURCE_ACTION)
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
newScanQueryBuilder()
.dataSource(localDataSource)
@ -320,7 +414,67 @@ public class CatalogIngestionTest extends CalciteIngestionDmlTest
.columns("x", "y", "z")
.context(CalciteInsertDmlTest.PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
.build()
)
)
.expectLogicalPlanFrom("localExtern")
.verify();
}
/**
* Local with a table alias an explicit column references.
*/
@Test
public void testLocalFnWithAlias()
{
testIngestionQuery()
.sql("INSERT INTO dst\n" +
"SELECT myTable.x, myTable.y, myTable.z\n" +
"FROM TABLE(localfiles(files => ARRAY['/tmp/foo.csv', '/tmp/bar.csv'],\n" +
" format => 'csv'))\n" +
" (x VARCHAR, y VARCHAR, z BIGINT)\n" +
" As myTable\n" +
"PARTITIONED BY ALL TIME"
)
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("dst", localDataSource.getSignature())
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
newScanQueryBuilder()
.dataSource(localDataSource)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("x", "y", "z")
.context(CalciteInsertDmlTest.PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
.build()
)
.expectLogicalPlanFrom("localExtern")
.verify();
}
/**
* Local with NOT NULL on columns, which is ignored.
*/
@Test
public void testLocalFnNotNull()
{
testIngestionQuery()
.sql("INSERT INTO dst\n" +
"SELECT myTable.x, myTable.y, myTable.z\n" +
"FROM TABLE(localfiles(files => ARRAY['/tmp/foo.csv', '/tmp/bar.csv'],\n" +
" format => 'csv'))\n" +
" (x VARCHAR NOT NULL, y VARCHAR NOT NULL, z BIGINT NOT NULL)\n" +
" As myTable\n" +
"PARTITIONED BY ALL TIME"
)
.authentication(CalciteTests.SUPER_USER_AUTH_RESULT)
.expectTarget("dst", localDataSource.getSignature())
.expectResources(dataSourceWrite("dst"), Externals.EXTERNAL_RESOURCE_ACTION)
.expectQuery(
newScanQueryBuilder()
.dataSource(localDataSource)
.intervals(querySegmentSpec(Filtration.eternity()))
.columns("x", "y", "z")
.context(CalciteInsertDmlTest.PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT)
.build()
)
.expectLogicalPlanFrom("localExtern")
.verify();
}

View File

@ -23,6 +23,10 @@ import com.google.common.collect.ImmutableList;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.math.expr.ExpressionProcessing;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.Resource;
import org.apache.druid.server.security.ResourceAction;
import org.apache.druid.server.security.ResourceType;
import org.apache.druid.sql.calcite.expression.DruidExpression;
import org.apache.druid.sql.calcite.expression.SimpleExtraction;
import org.apache.druid.sql.http.SqlParameter;
@ -100,4 +104,19 @@ public abstract class CalciteTestBase
Collections.emptyList()
);
}
protected static ResourceAction viewRead(final String viewName)
{
return new ResourceAction(new Resource(viewName, ResourceType.VIEW), Action.READ);
}
protected static ResourceAction dataSourceRead(final String dataSource)
{
return new ResourceAction(new Resource(dataSource, ResourceType.DATASOURCE), Action.READ);
}
protected static ResourceAction dataSourceWrite(final String dataSource)
{
return new ResourceAction(new Resource(dataSource, ResourceType.DATASOURCE), Action.WRITE);
}
}

View File

@ -213,6 +213,7 @@ UI
UIs
URI
URIs
uris
UTF-16
UTF-8
UTF8
@ -1275,6 +1276,7 @@ kafka.timestamp
keyColumnName
keyFormat
listDelimiter
timestamp
timestampColumnName
timestampSpec
urls

View File

@ -108,6 +108,9 @@
"development/build": {
"title": "Build from source"
},
"development/experimental-features": {
"title": "Experimental features"
},
"development/experimental": {
"title": "Experimental features"
},
@ -141,6 +144,9 @@
"development/extensions-contrib/influxdb-emitter": {
"title": "InfluxDB Emitter"
},
"development/extensions-contrib/k8s-jobs": {
"title": "MM-less Druid in K8s"
},
"development/extensions-contrib/kafka-emitter": {
"title": "Kafka Emitter"
},
@ -328,8 +334,12 @@
"title": "Ingestion spec reference",
"sidebar_label": "Ingestion spec"
},
"ingestion/migrate-from-firehose": {
"title": "Migrate from firehose to input source ingestion",
"sidebar_label": "Migrate from firehose"
},
"ingestion/native-batch-firehose": {
"title": "Native batch ingestion with firehose",
"title": "Native batch ingestion with firehose (Deprecated)",
"sidebar_label": "Firehose (deprecated)"
},
"ingestion/native-batch-input-sources": {
@ -405,7 +415,8 @@
"title": "API reference"
},
"operations/auth-ldap": {
"title": "LDAP auth"
"title": "Configure LDAP authentication",
"sidebar_label": "LDAP auth"
},
"operations/basic-cluster-tuning": {
"title": "Basic cluster tuning"
@ -463,6 +474,9 @@
"operations/pull-deps": {
"title": "pull-deps tool"
},
"operations/python": {
"title": "Python Installation"
},
"operations/request-logging": {
"title": "Request logging",
"sidebar_label": "Request logging"
@ -474,7 +488,7 @@
"title": "Rolling updates"
},
"operations/rule-configuration": {
"title": "Retaining or automatically dropping data"
"title": "Using rules to drop and retain data"
},
"operations/security-overview": {
"title": "Security overview"
@ -693,6 +707,13 @@
"title": "Tutorial: Writing an ingestion spec",
"sidebar_label": "Writing an ingestion spec"
},
"tutorials/tutorial-jdbc": {
"title": "Tutorial: Using the JDBC driver to query Druid",
"sidebar_label": "JDBC connector"
},
"tutorials/tutorial-jupyter-index": {
"title": "Jupyter Notebook tutorials"
},
"tutorials/tutorial-kafka": {
"title": "Tutorial: Load streaming data from Apache Kafka",
"sidebar_label": "Load from Apache Kafka"
@ -729,6 +750,10 @@
"title": "Tutorial: Transforming input data",
"sidebar_label": "Transforming input data"
},
"tutorials/tutorial-unnest-datasource": {
"title": "Tutorial: Unnest data in a column",
"sidebar_label": "Using the unnest datasource"
},
"tutorials/tutorial-update-data": {
"title": "Tutorial: Updating existing data",
"sidebar_label": "Updating existing data"

View File

@ -2649,6 +2649,19 @@
"integrity": "sha512-cMlDqaLEqfSaW8Z7N5Jw+lyIW869EzT73/F5lhtY9cLGoVxSXznfgfXMO0Z5K0o0Q2TkTXq+0KFsdnSe3jDViA==",
"dev": true
},
"node_modules/available-typed-arrays": {
"version": "1.0.5",
"resolved": "https://registry.npmjs.org/available-typed-arrays/-/available-typed-arrays-1.0.5.tgz",
"integrity": "sha512-DMD0KiN46eipeziST1LPP/STfDU0sufISXmjSgvVsoU2tqxctQeASejWcfNtxYKqETM1UxQ8sp2OrSBWpHY6sw==",
"dev": true,
"peer": true,
"engines": {
"node": ">= 0.4"
},
"funding": {
"url": "https://github.com/sponsors/ljharb"
}
},
"node_modules/aws-sign2": {
"version": "0.7.0",
"resolved": "https://registry.npmjs.org/aws-sign2/-/aws-sign2-0.7.0.tgz",
@ -6472,6 +6485,16 @@
"node": ">=6"
}
},
"node_modules/for-each": {
"version": "0.3.3",
"resolved": "https://registry.npmjs.org/for-each/-/for-each-0.3.3.tgz",
"integrity": "sha512-jqYfLp7mo9vIyQf8ykW2v7A+2N4QjeCeI5+Dz9XraiO1ign81wjiH7Fb9vSOWvQfNtmSa4H2RoQTrrXivdUZmw==",
"dev": true,
"peer": true,
"dependencies": {
"is-callable": "^1.1.3"
}
},
"node_modules/for-in": {
"version": "1.0.2",
"resolved": "https://registry.npmjs.org/for-in/-/for-in-1.0.2.tgz",
@ -6544,6 +6567,17 @@
"node": ">= 0.12"
}
},
"node_modules/formatio": {
"version": "1.1.1",
"resolved": "https://registry.npmjs.org/formatio/-/formatio-1.1.1.tgz",
"integrity": "sha512-cPh7is6k3d8tIUh+pnXXuAbD/uhSXGgqLPw0UrYpv5lfdJ+MMMSjx40JNpqP7Top9Nt25YomWEiRmkHbOvkCaA==",
"deprecated": "This package is unmaintained. Use @sinonjs/formatio instead",
"dev": true,
"peer": true,
"dependencies": {
"samsam": "~1.1"
}
},
"node_modules/forwarded": {
"version": "0.2.0",
"resolved": "https://registry.npmjs.org/forwarded/-/forwarded-0.2.0.tgz",
@ -7069,6 +7103,19 @@
"node": ">= 0.10"
}
},
"node_modules/gopd": {
"version": "1.0.1",
"resolved": "https://registry.npmjs.org/gopd/-/gopd-1.0.1.tgz",
"integrity": "sha512-d65bNlIadxvpb/A2abVdlqKqV563juRnZ1Wtk6s1sIR8uNsXR70xqIzVqxVf1eTqDunwT2MkczEeaezCKTZhwA==",
"dev": true,
"peer": true,
"dependencies": {
"get-intrinsic": "^1.1.3"
},
"funding": {
"url": "https://github.com/sponsors/ljharb"
}
},
"node_modules/got": {
"version": "7.1.0",
"resolved": "https://registry.npmjs.org/got/-/got-7.1.0.tgz",
@ -7867,6 +7914,23 @@
"node": ">=0.10.0"
}
},
"node_modules/is-arguments": {
"version": "1.1.1",
"resolved": "https://registry.npmjs.org/is-arguments/-/is-arguments-1.1.1.tgz",
"integrity": "sha512-8Q7EARjzEnKpt/PCD7e1cgUS0a6X8u5tdSiMqXhojOdoV9TsMsiO+9VLC5vAmO8N7/GmXn7yjR8qnA6bVAEzfA==",
"dev": true,
"peer": true,
"dependencies": {
"call-bind": "^1.0.2",
"has-tostringtag": "^1.0.0"
},
"engines": {
"node": ">= 0.4"
},
"funding": {
"url": "https://github.com/sponsors/ljharb"
}
},
"node_modules/is-arrayish": {
"version": "0.2.1",
"resolved": "https://registry.npmjs.org/is-arrayish/-/is-arrayish-0.2.1.tgz",
@ -8069,6 +8133,22 @@
"node": ">=4"
}
},
"node_modules/is-generator-function": {
"version": "1.0.10",
"resolved": "https://registry.npmjs.org/is-generator-function/-/is-generator-function-1.0.10.tgz",
"integrity": "sha512-jsEjy9l3yiXEQ+PsXdmBwEPcOxaXWLspKdplFUVI9vq1iZgIekeC0L167qeu86czQaxed3q/Uzuw0swL0irL8A==",
"dev": true,
"peer": true,
"dependencies": {
"has-tostringtag": "^1.0.0"
},
"engines": {
"node": ">= 0.4"
},
"funding": {
"url": "https://github.com/sponsors/ljharb"
}
},
"node_modules/is-gif": {
"version": "3.0.0",
"resolved": "https://registry.npmjs.org/is-gif/-/is-gif-3.0.0.tgz",
@ -8330,6 +8410,26 @@
"url": "https://github.com/sponsors/ljharb"
}
},
"node_modules/is-typed-array": {
"version": "1.1.10",
"resolved": "https://registry.npmjs.org/is-typed-array/-/is-typed-array-1.1.10.tgz",
"integrity": "sha512-PJqgEHiWZvMpaFZ3uTc8kHPM4+4ADTlDniuQL7cU/UDA0Ql7F70yGfHph3cLNe+c9toaigv+DFzTJKhc2CtO6A==",
"dev": true,
"peer": true,
"dependencies": {
"available-typed-arrays": "^1.0.5",
"call-bind": "^1.0.2",
"for-each": "^0.3.3",
"gopd": "^1.0.1",
"has-tostringtag": "^1.0.0"
},
"engines": {
"node": ">= 0.4"
},
"funding": {
"url": "https://github.com/sponsors/ljharb"
}
},
"node_modules/is-typedarray": {
"version": "1.0.0",
"resolved": "https://registry.npmjs.org/is-typedarray/-/is-typedarray-1.0.0.tgz",
@ -8909,6 +9009,13 @@
"node": ">=0.10.0"
}
},
"node_modules/lolex": {
"version": "1.3.2",
"resolved": "https://registry.npmjs.org/lolex/-/lolex-1.3.2.tgz",
"integrity": "sha512-YYp8cqz7/8eruZ15L1mzcPkvLYxipfdsWIDESvNdNmQP9o7TsDitRhNuV2xb7aFu2ofZngao1jiVrVZ842x4BQ==",
"dev": true,
"peer": true
},
"node_modules/longest": {
"version": "1.0.1",
"resolved": "https://registry.npmjs.org/longest/-/longest-1.0.1.tgz",
@ -13213,6 +13320,14 @@
"integrity": "sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg==",
"dev": true
},
"node_modules/samsam": {
"version": "1.1.2",
"resolved": "https://registry.npmjs.org/samsam/-/samsam-1.1.2.tgz",
"integrity": "sha512-iVL7LibpM3tl4rQPweOXXrmjGegxx27flTOjQEZD3PXe4oZNFzuz6Si4mgleK/JWU/hyCvtV01RUovjvBEpDmw==",
"deprecated": "This package has been deprecated in favour of @sinonjs/samsam",
"dev": true,
"peer": true
},
"node_modules/sass-graph": {
"version": "4.0.1",
"resolved": "https://registry.npmjs.org/sass-graph/-/sass-graph-4.0.1.tgz",
@ -13611,6 +13726,22 @@
"integrity": "sha512-eVRqCvVlZbuw3GrM63ovNSNAeA1K16kaR/LRY/92w0zxQ5/1YzwblUX652i4Xs9RwAGjW9d9y6X88t8OaAJfWQ==",
"dev": true
},
"node_modules/sinon": {
"version": "1.17.7",
"resolved": "https://registry.npmjs.org/sinon/-/sinon-1.17.7.tgz",
"integrity": "sha512-M9rtyQxKfcTTdB64rpPSRaTzOvunb+HHPv/3PxvNPrEDnFSny95Pi6/3VoD471ody0ay0IHyzT3BErfcLXj6NA==",
"dev": true,
"peer": true,
"dependencies": {
"formatio": "1.1.1",
"lolex": "1.3.2",
"samsam": "1.1.2",
"util": ">=0.10.3 <1"
},
"engines": {
"node": ">=0.1.103"
}
},
"node_modules/sinon-as-promised": {
"version": "4.0.3",
"resolved": "https://registry.npmjs.org/sinon-as-promised/-/sinon-as-promised-4.0.3.tgz",
@ -15170,6 +15301,20 @@
"node": ">=0.10.0"
}
},
"node_modules/util": {
"version": "0.12.5",
"resolved": "https://registry.npmjs.org/util/-/util-0.12.5.tgz",
"integrity": "sha512-kZf/K6hEIrWHI6XqOFUiiMa+79wE/D8Q+NCNAWclkyg3b4d2k7s0QGepNjiABc+aR3N1PAyHL7p6UcLY6LmrnA==",
"dev": true,
"peer": true,
"dependencies": {
"inherits": "^2.0.3",
"is-arguments": "^1.0.4",
"is-generator-function": "^1.0.7",
"is-typed-array": "^1.1.3",
"which-typed-array": "^1.1.2"
}
},
"node_modules/util-deprecate": {
"version": "1.0.2",
"resolved": "https://registry.npmjs.org/util-deprecate/-/util-deprecate-1.0.2.tgz",
@ -15321,6 +15466,27 @@
"url": "https://github.com/sponsors/ljharb"
}
},
"node_modules/which-typed-array": {
"version": "1.1.9",
"resolved": "https://registry.npmjs.org/which-typed-array/-/which-typed-array-1.1.9.tgz",
"integrity": "sha512-w9c4xkx6mPidwp7180ckYWfMmvxpjlZuIudNtDf4N/tTAUB8VJbX25qZoAsrtGuYNnGw3pa0AXgbGKRB8/EceA==",
"dev": true,
"peer": true,
"dependencies": {
"available-typed-arrays": "^1.0.5",
"call-bind": "^1.0.2",
"for-each": "^0.3.3",
"gopd": "^1.0.1",
"has-tostringtag": "^1.0.0",
"is-typed-array": "^1.1.10"
},
"engines": {
"node": ">= 0.4"
},
"funding": {
"url": "https://github.com/sponsors/ljharb"
}
},
"node_modules/wide-align": {
"version": "1.1.5",
"resolved": "https://registry.npmjs.org/wide-align/-/wide-align-1.1.5.tgz",
@ -17447,6 +17613,13 @@
}
}
},
"available-typed-arrays": {
"version": "1.0.5",
"resolved": "https://registry.npmjs.org/available-typed-arrays/-/available-typed-arrays-1.0.5.tgz",
"integrity": "sha512-DMD0KiN46eipeziST1LPP/STfDU0sufISXmjSgvVsoU2tqxctQeASejWcfNtxYKqETM1UxQ8sp2OrSBWpHY6sw==",
"dev": true,
"peer": true
},
"aws-sign2": {
"version": "0.7.0",
"resolved": "https://registry.npmjs.org/aws-sign2/-/aws-sign2-0.7.0.tgz",
@ -20475,6 +20648,16 @@
"semver-regex": "^2.0.0"
}
},
"for-each": {
"version": "0.3.3",
"resolved": "https://registry.npmjs.org/for-each/-/for-each-0.3.3.tgz",
"integrity": "sha512-jqYfLp7mo9vIyQf8ykW2v7A+2N4QjeCeI5+Dz9XraiO1ign81wjiH7Fb9vSOWvQfNtmSa4H2RoQTrrXivdUZmw==",
"dev": true,
"peer": true,
"requires": {
"is-callable": "^1.1.3"
}
},
"for-in": {
"version": "1.0.2",
"resolved": "https://registry.npmjs.org/for-in/-/for-in-1.0.2.tgz",
@ -20530,6 +20713,16 @@
"mime-types": "^2.1.12"
}
},
"formatio": {
"version": "1.1.1",
"resolved": "https://registry.npmjs.org/formatio/-/formatio-1.1.1.tgz",
"integrity": "sha512-cPh7is6k3d8tIUh+pnXXuAbD/uhSXGgqLPw0UrYpv5lfdJ+MMMSjx40JNpqP7Top9Nt25YomWEiRmkHbOvkCaA==",
"dev": true,
"peer": true,
"requires": {
"samsam": "~1.1"
}
},
"forwarded": {
"version": "0.2.0",
"resolved": "https://registry.npmjs.org/forwarded/-/forwarded-0.2.0.tgz",
@ -20945,6 +21138,16 @@
"minimatch": "~3.0.2"
}
},
"gopd": {
"version": "1.0.1",
"resolved": "https://registry.npmjs.org/gopd/-/gopd-1.0.1.tgz",
"integrity": "sha512-d65bNlIadxvpb/A2abVdlqKqV563juRnZ1Wtk6s1sIR8uNsXR70xqIzVqxVf1eTqDunwT2MkczEeaezCKTZhwA==",
"dev": true,
"peer": true,
"requires": {
"get-intrinsic": "^1.1.3"
}
},
"got": {
"version": "7.1.0",
"resolved": "https://registry.npmjs.org/got/-/got-7.1.0.tgz",
@ -21558,6 +21761,17 @@
}
}
},
"is-arguments": {
"version": "1.1.1",
"resolved": "https://registry.npmjs.org/is-arguments/-/is-arguments-1.1.1.tgz",
"integrity": "sha512-8Q7EARjzEnKpt/PCD7e1cgUS0a6X8u5tdSiMqXhojOdoV9TsMsiO+9VLC5vAmO8N7/GmXn7yjR8qnA6bVAEzfA==",
"dev": true,
"peer": true,
"requires": {
"call-bind": "^1.0.2",
"has-tostringtag": "^1.0.0"
}
},
"is-arrayish": {
"version": "0.2.1",
"resolved": "https://registry.npmjs.org/is-arrayish/-/is-arrayish-0.2.1.tgz",
@ -21701,6 +21915,16 @@
"integrity": "sha1-o7MKXE8ZkYMWeqq5O+764937ZU8=",
"dev": true
},
"is-generator-function": {
"version": "1.0.10",
"resolved": "https://registry.npmjs.org/is-generator-function/-/is-generator-function-1.0.10.tgz",
"integrity": "sha512-jsEjy9l3yiXEQ+PsXdmBwEPcOxaXWLspKdplFUVI9vq1iZgIekeC0L167qeu86czQaxed3q/Uzuw0swL0irL8A==",
"dev": true,
"peer": true,
"requires": {
"has-tostringtag": "^1.0.0"
}
},
"is-gif": {
"version": "3.0.0",
"resolved": "https://registry.npmjs.org/is-gif/-/is-gif-3.0.0.tgz",
@ -21886,6 +22110,20 @@
"has-symbols": "^1.0.2"
}
},
"is-typed-array": {
"version": "1.1.10",
"resolved": "https://registry.npmjs.org/is-typed-array/-/is-typed-array-1.1.10.tgz",
"integrity": "sha512-PJqgEHiWZvMpaFZ3uTc8kHPM4+4ADTlDniuQL7cU/UDA0Ql7F70yGfHph3cLNe+c9toaigv+DFzTJKhc2CtO6A==",
"dev": true,
"peer": true,
"requires": {
"available-typed-arrays": "^1.0.5",
"call-bind": "^1.0.2",
"for-each": "^0.3.3",
"gopd": "^1.0.1",
"has-tostringtag": "^1.0.0"
}
},
"is-typedarray": {
"version": "1.0.0",
"resolved": "https://registry.npmjs.org/is-typedarray/-/is-typedarray-1.0.0.tgz",
@ -22387,6 +22625,13 @@
"squeak": "^1.0.0"
}
},
"lolex": {
"version": "1.3.2",
"resolved": "https://registry.npmjs.org/lolex/-/lolex-1.3.2.tgz",
"integrity": "sha512-YYp8cqz7/8eruZ15L1mzcPkvLYxipfdsWIDESvNdNmQP9o7TsDitRhNuV2xb7aFu2ofZngao1jiVrVZ842x4BQ==",
"dev": true,
"peer": true
},
"longest": {
"version": "1.0.1",
"resolved": "https://registry.npmjs.org/longest/-/longest-1.0.1.tgz",
@ -25776,6 +26021,13 @@
"integrity": "sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg==",
"dev": true
},
"samsam": {
"version": "1.1.2",
"resolved": "https://registry.npmjs.org/samsam/-/samsam-1.1.2.tgz",
"integrity": "sha512-iVL7LibpM3tl4rQPweOXXrmjGegxx27flTOjQEZD3PXe4oZNFzuz6Si4mgleK/JWU/hyCvtV01RUovjvBEpDmw==",
"dev": true,
"peer": true
},
"sass-graph": {
"version": "4.0.1",
"resolved": "https://registry.npmjs.org/sass-graph/-/sass-graph-4.0.1.tgz",
@ -26109,6 +26361,19 @@
}
}
},
"sinon": {
"version": "1.17.7",
"resolved": "https://registry.npmjs.org/sinon/-/sinon-1.17.7.tgz",
"integrity": "sha512-M9rtyQxKfcTTdB64rpPSRaTzOvunb+HHPv/3PxvNPrEDnFSny95Pi6/3VoD471ody0ay0IHyzT3BErfcLXj6NA==",
"dev": true,
"peer": true,
"requires": {
"formatio": "1.1.1",
"lolex": "1.3.2",
"samsam": "1.1.2",
"util": ">=0.10.3 <1"
}
},
"sinon-as-promised": {
"version": "4.0.3",
"resolved": "https://registry.npmjs.org/sinon-as-promised/-/sinon-as-promised-4.0.3.tgz",
@ -27377,6 +27642,20 @@
"integrity": "sha512-cwESVXlO3url9YWlFW/TA9cshCEhtu7IKJ/p5soJ/gGpj7vbvFrAY/eIioQ6Dw23KjZhYgiIo8HOs1nQ2vr/oQ==",
"dev": true
},
"util": {
"version": "0.12.5",
"resolved": "https://registry.npmjs.org/util/-/util-0.12.5.tgz",
"integrity": "sha512-kZf/K6hEIrWHI6XqOFUiiMa+79wE/D8Q+NCNAWclkyg3b4d2k7s0QGepNjiABc+aR3N1PAyHL7p6UcLY6LmrnA==",
"dev": true,
"peer": true,
"requires": {
"inherits": "^2.0.3",
"is-arguments": "^1.0.4",
"is-generator-function": "^1.0.7",
"is-typed-array": "^1.1.3",
"which-typed-array": "^1.1.2"
}
},
"util-deprecate": {
"version": "1.0.2",
"resolved": "https://registry.npmjs.org/util-deprecate/-/util-deprecate-1.0.2.tgz",
@ -27496,6 +27775,21 @@
"is-symbol": "^1.0.3"
}
},
"which-typed-array": {
"version": "1.1.9",
"resolved": "https://registry.npmjs.org/which-typed-array/-/which-typed-array-1.1.9.tgz",
"integrity": "sha512-w9c4xkx6mPidwp7180ckYWfMmvxpjlZuIudNtDf4N/tTAUB8VJbX25qZoAsrtGuYNnGw3pa0AXgbGKRB8/EceA==",
"dev": true,
"peer": true,
"requires": {
"available-typed-arrays": "^1.0.5",
"call-bind": "^1.0.2",
"for-each": "^0.3.3",
"gopd": "^1.0.1",
"has-tostringtag": "^1.0.0",
"is-typed-array": "^1.1.10"
}
},
"wide-align": {
"version": "1.1.5",
"resolved": "https://registry.npmjs.org/wide-align/-/wide-align-1.1.5.tgz",