mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-14 02:12:21 +00:00
195 lines
11 KiB
Markdown
195 lines
11 KiB
Markdown
---
|
|
slug: /en/engines/table-engines/integrations/s3
|
|
sidebar_position: 7
|
|
sidebar_label: S3
|
|
---
|
|
|
|
# S3 Table Engine
|
|
|
|
This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ecosystem. This engine is similar to the [HDFS](../../../engines/table-engines/special/file.md#table_engines-hdfs) engine, but provides S3-specific features.
|
|
|
|
## Create Table {#creating-a-table}
|
|
|
|
``` sql
|
|
CREATE TABLE s3_engine_table (name String, value UInt32)
|
|
ENGINE = S3(path [, NOSIGN | aws_access_key_id, aws_secret_access_key,] format, [compression])
|
|
[PARTITION BY expr]
|
|
[SETTINGS ...]
|
|
```
|
|
|
|
**Engine parameters**
|
|
|
|
- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path).
|
|
- `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed.
|
|
- `format` — The [format](../../../interfaces/formats.md#formats) of the file.
|
|
- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3).
|
|
- `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will autodetect compression by file extension.
|
|
|
|
### PARTITION BY
|
|
|
|
`PARTITION BY` — Optional. In most cases you don't need a partition key, and if it is needed you generally don't need a partition key more granular than by month. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead, make client identifier or name the first column in the ORDER BY expression).
|
|
|
|
For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](/docs/en/sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format.
|
|
|
|
**Example**
|
|
|
|
``` sql
|
|
CREATE TABLE s3_engine_table (name String, value UInt32)
|
|
ENGINE=S3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/test-data.csv.gz', 'CSV', 'gzip')
|
|
SETTINGS input_format_with_names_use_header = 0;
|
|
|
|
INSERT INTO s3_engine_table VALUES ('one', 1), ('two', 2), ('three', 3);
|
|
|
|
SELECT * FROM s3_engine_table LIMIT 2;
|
|
```
|
|
|
|
```text
|
|
┌─name─┬─value─┐
|
|
│ one │ 1 │
|
|
│ two │ 2 │
|
|
└──────┴───────┘
|
|
```
|
|
## Virtual columns {#virtual-columns}
|
|
|
|
- `_path` — Path to the file.
|
|
- `_file` — Name of the file.
|
|
|
|
For more information about virtual columns see [here](../../../engines/table-engines/index.md#table_engines-virtual_columns).
|
|
|
|
## Implementation Details {#implementation-details}
|
|
|
|
- Reads and writes can be parallel
|
|
- Not supported:
|
|
- `ALTER` and `SELECT...SAMPLE` operations.
|
|
- Indexes.
|
|
- [Zero-copy](../../../operations/storing-data.md#zero-copy) replication is possible, but not supported.
|
|
|
|
:::note Zero-copy replication is not ready for production
|
|
Zero-copy replication is disabled by default in ClickHouse version 22.8 and higher. This feature is not recommended for production use.
|
|
:::
|
|
|
|
## Wildcards In Path {#wildcards-in-path}
|
|
|
|
`path` argument can specify multiple files using bash-like wildcards. For being processed file should exist and match to the whole path pattern. Listing of files is determined during `SELECT` (not at `CREATE` moment).
|
|
|
|
- `*` — Substitutes any number of any characters except `/` including empty string.
|
|
- `?` — Substitutes any single character.
|
|
- `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`.
|
|
- `{N..M}` — Substitutes any number in range from N to M including both borders. N and M can have leading zeroes e.g. `000..078`.
|
|
|
|
Constructions with `{}` are similar to the [remote](../../../sql-reference/table-functions/remote.md) table function.
|
|
|
|
:::note
|
|
If the listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`.
|
|
:::
|
|
|
|
**Example with wildcards 1**
|
|
|
|
Create table with files named `file-000.csv`, `file-001.csv`, … , `file-999.csv`:
|
|
|
|
``` sql
|
|
CREATE TABLE big_table (name String, value UInt32)
|
|
ENGINE = S3('https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/my_folder/file-{000..999}.csv', 'CSV');
|
|
```
|
|
|
|
**Example with wildcards 2**
|
|
|
|
Suppose we have several files in CSV format with the following URIs on S3:
|
|
|
|
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/some_folder/some_file_1.csv'
|
|
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/some_folder/some_file_2.csv'
|
|
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/some_folder/some_file_3.csv'
|
|
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/another_folder/some_file_1.csv'
|
|
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/another_folder/some_file_2.csv'
|
|
- 'https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/another_folder/some_file_3.csv'
|
|
|
|
|
|
There are several ways to make a table consisting of all six files:
|
|
|
|
1. Specify the range of file postfixes:
|
|
|
|
``` sql
|
|
CREATE TABLE table_with_range (name String, value UInt32)
|
|
ENGINE = S3('https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/{some,another}_folder/some_file_{1..3}', 'CSV');
|
|
```
|
|
|
|
2. Take all files with `some_file_` prefix (there should be no extra files with such prefix in both folders):
|
|
|
|
``` sql
|
|
CREATE TABLE table_with_question_mark (name String, value UInt32)
|
|
ENGINE = S3('https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/{some,another}_folder/some_file_?', 'CSV');
|
|
```
|
|
|
|
3. Take all the files in both folders (all files should satisfy format and schema described in query):
|
|
|
|
``` sql
|
|
CREATE TABLE table_with_asterisk (name String, value UInt32)
|
|
ENGINE = S3('https://clickhouse-public-datasets.s3.amazonaws.com/my-bucket/{some,another}_folder/*', 'CSV');
|
|
```
|
|
|
|
## S3-related Settings {#settings}
|
|
|
|
The following settings can be set before query execution or placed into configuration file.
|
|
|
|
- `s3_max_single_part_upload_size` — The maximum size of object to upload using singlepart upload to S3. Default value is `64Mb`.
|
|
- `s3_min_upload_part_size` — The minimum size of part to upload during multipart upload to [S3 Multipart upload](https://docs.aws.amazon.com/AmazonS3/latest/dev/uploadobjusingmpu.html). Default value is `512Mb`.
|
|
- `s3_max_redirects` — Max number of S3 redirects hops allowed. Default value is `10`.
|
|
- `s3_single_read_retries` — The maximum number of attempts during single read. Default value is `4`.
|
|
- `s3_max_put_rps` — Maximum PUT requests per second rate before throttling. Default value is `0` (unlimited).
|
|
- `s3_max_put_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_put_rps`.
|
|
- `s3_max_get_rps` — Maximum GET requests per second rate before throttling. Default value is `0` (unlimited).
|
|
- `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`.
|
|
|
|
Security consideration: if malicious user can specify arbitrary S3 URLs, `s3_max_redirects` must be set to zero to avoid [SSRF](https://en.wikipedia.org/wiki/Server-side_request_forgery) attacks; or alternatively, `remote_host_filter` must be specified in server configuration.
|
|
|
|
## Endpoint-based Settings {#endpoint-settings}
|
|
|
|
The following settings can be specified in configuration file for given endpoint (which will be matched by exact prefix of a URL):
|
|
|
|
- `endpoint` — Specifies prefix of an endpoint. Mandatory.
|
|
- `access_key_id` and `secret_access_key` — Specifies credentials to use with given endpoint. Optional.
|
|
- `use_environment_credentials` — If set to `true`, S3 client will try to obtain credentials from environment variables and [Amazon EC2](https://en.wikipedia.org/wiki/Amazon_Elastic_Compute_Cloud) metadata for given endpoint. Optional, default value is `false`.
|
|
- `region` — Specifies S3 region name. Optional.
|
|
- `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Optional, default value is `false`.
|
|
- `expiration_window_seconds` — Grace period for checking if expiration-based credentials have expired. Optional, default value is `120`.
|
|
- `no_sign_request` - Ignore all the credentials so requests are not signed. Useful for accessing public buckets.
|
|
- `header` — Adds specified HTTP header to a request to given endpoint. Optional, can be specified multiple times.
|
|
- `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. Optional.
|
|
- `max_single_read_retries` — The maximum number of attempts during single read. Default value is `4`. Optional.
|
|
- `max_put_rps`, `max_put_burst`, `max_get_rps` and `max_get_burst` - Throttling settings (see description above) to use for specific endpoint instead of per query. Optional.
|
|
|
|
**Example:**
|
|
|
|
``` xml
|
|
<s3>
|
|
<endpoint-name>
|
|
<endpoint>https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/</endpoint>
|
|
<!-- <access_key_id>ACCESS_KEY_ID</access_key_id> -->
|
|
<!-- <secret_access_key>SECRET_ACCESS_KEY</secret_access_key> -->
|
|
<!-- <region>us-west-1</region> -->
|
|
<!-- <use_environment_credentials>false</use_environment_credentials> -->
|
|
<!-- <use_insecure_imds_request>false</use_insecure_imds_request> -->
|
|
<!-- <expiration_window_seconds>120</expiration_window_seconds> -->
|
|
<!-- <no_sign_request>false</no_sign_request> -->
|
|
<!-- <header>Authorization: Bearer SOME-TOKEN</header> -->
|
|
<!-- <server_side_encryption_customer_key_base64>BASE64-ENCODED-KEY</server_side_encryption_customer_key_base64> -->
|
|
<!-- <max_single_read_retries>4</max_single_read_retries> -->
|
|
</endpoint-name>
|
|
</s3>
|
|
```
|
|
|
|
## Accessing public buckets
|
|
|
|
ClickHouse tries to fetch credentials from many different types of sources.
|
|
Sometimes, it can produce problems when accessing some buckets that are public causing the client to return `403` error code.
|
|
This issue can be avoided by using `NOSIGN` keyword, forcing the client to ignore all the credentials, and not sign the requests.
|
|
|
|
``` sql
|
|
CREATE TABLE big_table (name String, value UInt32)
|
|
ENGINE = S3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/aapl_stock.csv', NOSIGN, 'CSVWithNames');
|
|
```
|
|
|
|
## See also
|
|
|
|
- [s3 table function](../../../sql-reference/table-functions/s3.md)
|